|
@@ -25,6 +25,9 @@ import org.apache.hadoop.ozone.container.common.helpers.BlockData;
|
|
|
import org.apache.ratis.protocol.RaftGroup;
|
|
|
import org.apache.ratis.protocol.RaftGroupId;
|
|
|
import org.apache.ratis.server.RaftServer;
|
|
|
+import org.apache.ratis.server.impl.RaftServerConstants;
|
|
|
+import org.apache.ratis.server.protocol.TermIndex;
|
|
|
+import org.apache.ratis.statemachine.impl.SingleFileSnapshotInfo;
|
|
|
import org.apache.ratis.thirdparty.com.google.protobuf
|
|
|
.InvalidProtocolBufferException;
|
|
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type;
|
|
@@ -55,8 +58,10 @@ import org.apache.ratis.statemachine.impl.TransactionContextImpl;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
+import java.io.File;
|
|
|
import java.io.IOException;
|
|
|
import java.util.List;
|
|
|
+import java.util.Map;
|
|
|
import java.util.concurrent.CompletableFuture;
|
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
|
import java.util.concurrent.ThreadPoolExecutor;
|
|
@@ -115,6 +120,7 @@ public class ContainerStateMachine extends BaseStateMachine {
|
|
|
createContainerFutureMap;
|
|
|
private ExecutorService[] executors;
|
|
|
private final int numExecutors;
|
|
|
+ private final Map<Long, Long> containerCommandCompletionMap;
|
|
|
/**
|
|
|
* CSM metrics.
|
|
|
*/
|
|
@@ -131,6 +137,7 @@ public class ContainerStateMachine extends BaseStateMachine {
|
|
|
this.createContainerFutureMap = new ConcurrentHashMap<>();
|
|
|
this.numExecutors = numOfExecutors;
|
|
|
executors = new ExecutorService[numExecutors];
|
|
|
+ containerCommandCompletionMap = new ConcurrentHashMap<>();
|
|
|
for (int i = 0; i < numExecutors; i++) {
|
|
|
executors[i] = Executors.newSingleThreadExecutor();
|
|
|
}
|
|
@@ -151,10 +158,47 @@ public class ContainerStateMachine extends BaseStateMachine {
|
|
|
throws IOException {
|
|
|
super.initialize(server, id, raftStorage);
|
|
|
storage.init(raftStorage);
|
|
|
- // TODO handle snapshots
|
|
|
|
|
|
- // TODO: Add a flag that tells you that initialize has been called.
|
|
|
- // Check with Ratis if this feature is done in Ratis.
|
|
|
+ loadSnapshot(storage.getLatestSnapshot());
|
|
|
+ }
|
|
|
+
|
|
|
+ private long loadSnapshot(SingleFileSnapshotInfo snapshot) {
|
|
|
+ if (snapshot == null) {
|
|
|
+ TermIndex empty = TermIndex.newTermIndex(0, 0);
|
|
|
+ LOG.info("The snapshot info is null." +
|
|
|
+ "Setting the last applied index to:" + empty);
|
|
|
+ setLastAppliedTermIndex(empty);
|
|
|
+ return RaftServerConstants.INVALID_LOG_INDEX;
|
|
|
+ }
|
|
|
+
|
|
|
+ final TermIndex last =
|
|
|
+ SimpleStateMachineStorage.getTermIndexFromSnapshotFile(
|
|
|
+ snapshot.getFile().getPath().toFile());
|
|
|
+ LOG.info("Setting the last applied index to " + last);
|
|
|
+ setLastAppliedTermIndex(last);
|
|
|
+ return last.getIndex();
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public long takeSnapshot() throws IOException {
|
|
|
+ TermIndex ti = getLastAppliedTermIndex();
|
|
|
+ LOG.info("Taking snapshot at termIndex:" + ti);
|
|
|
+ if (ti != null) {
|
|
|
+ final File snapshotFile =
|
|
|
+ storage.getSnapshotFile(ti.getTerm(), ti.getIndex());
|
|
|
+ LOG.info("Taking a snapshot to file {}", snapshotFile);
|
|
|
+ try {
|
|
|
+ //TODO: For now, just create the file to save the term index,
|
|
|
+ //persist open container info to snapshot later.
|
|
|
+ snapshotFile.createNewFile();
|
|
|
+ } catch(IOException ioe) {
|
|
|
+ LOG.warn("Failed to write snapshot file \"" + snapshotFile
|
|
|
+ + "\", last applied index=" + ti);
|
|
|
+ throw ioe;
|
|
|
+ }
|
|
|
+ return ti.getIndex();
|
|
|
+ }
|
|
|
+ return -1;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -353,10 +397,9 @@ public class ContainerStateMachine extends BaseStateMachine {
|
|
|
public CompletableFuture<Void> flushStateMachineData(long index) {
|
|
|
List<CompletableFuture<Message>> futureList =
|
|
|
writeChunkFutureMap.entrySet().stream().filter(x -> x.getKey() <= index)
|
|
|
- .map(x -> x.getValue()).collect(Collectors.toList());
|
|
|
- CompletableFuture<Void> combinedFuture = CompletableFuture.allOf(
|
|
|
+ .map(Map.Entry::getValue).collect(Collectors.toList());
|
|
|
+ return CompletableFuture.allOf(
|
|
|
futureList.toArray(new CompletableFuture[futureList.size()]));
|
|
|
- return combinedFuture;
|
|
|
}
|
|
|
/*
|
|
|
* This api is used by the leader while appending logs to the follower
|
|
@@ -394,11 +437,28 @@ public class ContainerStateMachine extends BaseStateMachine {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ private void updateLastApplied() {
|
|
|
+ Long appliedTerm = null;
|
|
|
+ long appliedIndex = -1;
|
|
|
+ for(long i = getLastAppliedTermIndex().getIndex() + 1;; i++) {
|
|
|
+ final Long removed = containerCommandCompletionMap.remove(i);
|
|
|
+ if (removed == null) {
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ appliedTerm = removed;
|
|
|
+ appliedIndex = i;
|
|
|
+ }
|
|
|
+ if (appliedTerm != null) {
|
|
|
+ updateLastAppliedTermIndex(appliedIndex, appliedTerm);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/*
|
|
|
* ApplyTransaction calls in Ratis are sequential.
|
|
|
*/
|
|
|
@Override
|
|
|
public CompletableFuture<Message> applyTransaction(TransactionContext trx) {
|
|
|
+ long index = trx.getLogEntry().getIndex();
|
|
|
try {
|
|
|
metrics.incNumApplyTransactionsOps();
|
|
|
ContainerCommandRequestProto requestProto =
|
|
@@ -418,7 +478,7 @@ public class ContainerStateMachine extends BaseStateMachine {
|
|
|
blockDataProto.getBlockID());
|
|
|
return completeExceptionally(ioe);
|
|
|
}
|
|
|
- blockData.setBlockCommitSequenceId(trx.getLogEntry().getIndex());
|
|
|
+ blockData.setBlockCommitSequenceId(index);
|
|
|
final ContainerProtos.PutBlockRequestProto putBlockRequestProto =
|
|
|
ContainerProtos.PutBlockRequestProto
|
|
|
.newBuilder(requestProto.getPutBlock())
|
|
@@ -440,6 +500,14 @@ public class ContainerStateMachine extends BaseStateMachine {
|
|
|
future.thenApply(
|
|
|
r -> createContainerFutureMap.remove(containerID).complete(null));
|
|
|
}
|
|
|
+
|
|
|
+ future.thenAccept(m -> {
|
|
|
+ final Long previous =
|
|
|
+ containerCommandCompletionMap
|
|
|
+ .put(index, trx.getLogEntry().getTerm());
|
|
|
+ Preconditions.checkState(previous == null);
|
|
|
+ updateLastApplied();
|
|
|
+ });
|
|
|
return future;
|
|
|
} catch (IOException e) {
|
|
|
metrics.incNumApplyTransactionsFails();
|
|
@@ -466,7 +534,8 @@ public class ContainerStateMachine extends BaseStateMachine {
|
|
|
|
|
|
@Override
|
|
|
public void close() throws IOException {
|
|
|
- for (int i = 0; i < numExecutors; i++){
|
|
|
+ takeSnapshot();
|
|
|
+ for (int i = 0; i < numExecutors; i++) {
|
|
|
executors[i].shutdown();
|
|
|
}
|
|
|
}
|