|
@@ -69,6 +69,7 @@ public class OzoneManagerStateMachine extends BaseStateMachine {
|
|
|
private final OzoneManagerServerProtocol ozoneManager;
|
|
|
private RequestHandler handler;
|
|
|
private RaftGroupId raftGroupId;
|
|
|
+ private long lastAppliedIndex = 0;
|
|
|
|
|
|
public OzoneManagerStateMachine(OzoneManagerRatisServer ratisServer) {
|
|
|
this.omRatisServer = ratisServer;
|
|
@@ -95,6 +96,7 @@ public class OzoneManagerStateMachine extends BaseStateMachine {
|
|
|
* should be rejected.
|
|
|
* @throws IOException thrown by the state machine while validating
|
|
|
*/
|
|
|
+ @Override
|
|
|
public TransactionContext startTransaction(
|
|
|
RaftClientRequest raftClientRequest) throws IOException {
|
|
|
ByteString messageContent = raftClientRequest.getMessage().getContent();
|
|
@@ -115,7 +117,63 @@ public class OzoneManagerStateMachine extends BaseStateMachine {
|
|
|
return ctxt;
|
|
|
}
|
|
|
return handleStartTransactionRequests(raftClientRequest, omRequest);
|
|
|
+ }
|
|
|
|
|
|
+ /*
|
|
|
+ * Apply a committed log entry to the state machine.
|
|
|
+ */
|
|
|
+ @Override
|
|
|
+ public CompletableFuture<Message> applyTransaction(TransactionContext trx) {
|
|
|
+ try {
|
|
|
+ OMRequest request = OMRatisHelper.convertByteStringToOMRequest(
|
|
|
+ trx.getStateMachineLogEntry().getLogData());
|
|
|
+ long trxLogIndex = trx.getLogEntry().getIndex();
|
|
|
+ CompletableFuture<Message> future = CompletableFuture
|
|
|
+ .supplyAsync(() -> runCommand(request, trxLogIndex));
|
|
|
+ return future;
|
|
|
+ } catch (IOException e) {
|
|
|
+ return completeExceptionally(e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Query the state machine. The request must be read-only.
|
|
|
+ */
|
|
|
+ @Override
|
|
|
+ public CompletableFuture<Message> query(Message request) {
|
|
|
+ try {
|
|
|
+ OMRequest omRequest = OMRatisHelper.convertByteStringToOMRequest(
|
|
|
+ request.getContent());
|
|
|
+ return CompletableFuture.completedFuture(queryCommand(omRequest));
|
|
|
+ } catch (IOException e) {
|
|
|
+ return completeExceptionally(e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Take OM Ratis snapshot. Write the snapshot index to file. Snapshot index
|
|
|
+ * is the log index corresponding to the last applied transaction on the OM
|
|
|
+ * State Machine.
|
|
|
+ *
|
|
|
+ * @return the last applied index on the state machine which has been
|
|
|
+ * stored in the snapshot file.
|
|
|
+ */
|
|
|
+ @Override
|
|
|
+ public long takeSnapshot() throws IOException {
|
|
|
+ LOG.info("Saving Ratis snapshot on the OM.");
|
|
|
+ if (ozoneManager != null) {
|
|
|
+ return ozoneManager.saveRatisSnapshot();
|
|
|
+ }
|
|
|
+ return 0;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Notifies the state machine that the raft peer is no longer leader.
|
|
|
+ */
|
|
|
+ @Override
|
|
|
+ public void notifyNotLeader(Collection<TransactionContext> pendingEntries)
|
|
|
+ throws IOException {
|
|
|
+ omRatisServer.updateServerRole();
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -142,10 +200,8 @@ public class OzoneManagerStateMachine extends BaseStateMachine {
|
|
|
.setLogData(raftClientRequest.getMessage().getContent())
|
|
|
.build();
|
|
|
}
|
|
|
-
|
|
|
}
|
|
|
|
|
|
-
|
|
|
private TransactionContext handleInitiateMultipartUpload(
|
|
|
RaftClientRequest raftClientRequest, OMRequest omRequest) {
|
|
|
|
|
@@ -237,7 +293,6 @@ public class OzoneManagerStateMachine extends BaseStateMachine {
|
|
|
.build();
|
|
|
}
|
|
|
|
|
|
-
|
|
|
/**
|
|
|
* Handle AllocateBlock Request, which needs a special handling. This
|
|
|
* request needs to be executed on the leader, where it connects to SCM and
|
|
@@ -250,7 +305,6 @@ public class OzoneManagerStateMachine extends BaseStateMachine {
|
|
|
RaftClientRequest raftClientRequest, OMRequest omRequest) {
|
|
|
OMResponse omResponse = handler.handle(omRequest);
|
|
|
|
|
|
-
|
|
|
// If request is failed, no need to proceed further.
|
|
|
// Setting the exception with omResponse message and code.
|
|
|
|
|
@@ -270,7 +324,6 @@ public class OzoneManagerStateMachine extends BaseStateMachine {
|
|
|
return transactionContext;
|
|
|
}
|
|
|
|
|
|
-
|
|
|
// Get original request
|
|
|
OzoneManagerProtocolProtos.AllocateBlockRequest allocateBlockRequest =
|
|
|
omRequest.getAllocateBlockRequest();
|
|
@@ -294,7 +347,6 @@ public class OzoneManagerStateMachine extends BaseStateMachine {
|
|
|
.setServerRole(RaftProtos.RaftPeerRole.LEADER)
|
|
|
.setLogData(messageContent)
|
|
|
.build();
|
|
|
-
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -308,56 +360,33 @@ public class OzoneManagerStateMachine extends BaseStateMachine {
|
|
|
STATUS_CODE + omResponse.getStatus());
|
|
|
}
|
|
|
|
|
|
- /*
|
|
|
- * Apply a committed log entry to the state machine.
|
|
|
- */
|
|
|
- @Override
|
|
|
- public CompletableFuture<Message> applyTransaction(TransactionContext trx) {
|
|
|
- try {
|
|
|
- OMRequest request = OMRatisHelper.convertByteStringToOMRequest(
|
|
|
- trx.getStateMachineLogEntry().getLogData());
|
|
|
- CompletableFuture<Message> future = CompletableFuture
|
|
|
- .supplyAsync(() -> runCommand(request));
|
|
|
- return future;
|
|
|
- } catch (IOException e) {
|
|
|
- return completeExceptionally(e);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Query the state machine. The request must be read-only.
|
|
|
- */
|
|
|
- @Override
|
|
|
- public CompletableFuture<Message> query(Message request) {
|
|
|
- try {
|
|
|
- OMRequest omRequest = OMRatisHelper.convertByteStringToOMRequest(
|
|
|
- request.getContent());
|
|
|
- return CompletableFuture.completedFuture(runCommand(omRequest));
|
|
|
- } catch (IOException e) {
|
|
|
- return completeExceptionally(e);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
- * Notifies the state machine that the raft peer is no longer leader.
|
|
|
+ * Submits write request to OM and returns the response Message.
|
|
|
+ * @param request OMRequest
|
|
|
+ * @return response from OM
|
|
|
+ * @throws ServiceException
|
|
|
*/
|
|
|
- @Override
|
|
|
- public void notifyNotLeader(Collection<TransactionContext> pendingEntries)
|
|
|
- throws IOException {
|
|
|
- omRatisServer.updateServerRole();
|
|
|
+ private Message runCommand(OMRequest request, long trxLogIndex) {
|
|
|
+ OMResponse response = handler.handle(request);
|
|
|
+ lastAppliedIndex = trxLogIndex;
|
|
|
+ return OMRatisHelper.convertResponseToMessage(response);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Submits request to OM and returns the response Message.
|
|
|
+ * Submits read request to OM and returns the response Message.
|
|
|
* @param request OMRequest
|
|
|
* @return response from OM
|
|
|
* @throws ServiceException
|
|
|
*/
|
|
|
- private Message runCommand(OMRequest request) {
|
|
|
+ private Message queryCommand(OMRequest request) {
|
|
|
OMResponse response = handler.handle(request);
|
|
|
return OMRatisHelper.convertResponseToMessage(response);
|
|
|
}
|
|
|
|
|
|
+ public long getLastAppliedIndex() {
|
|
|
+ return lastAppliedIndex;
|
|
|
+ }
|
|
|
+
|
|
|
private static <T> CompletableFuture<T> completeExceptionally(Exception e) {
|
|
|
final CompletableFuture<T> future = new CompletableFuture<>();
|
|
|
future.completeExceptionally(e);
|