HDDS-638. Enable ratis snapshots for HDDS datanodes. Contributed by Mukul Kumar Singh.
This commit is contained in:
parent
52cb766ad0
commit
82919a1e7a
|
@ -104,6 +104,10 @@ public final class ScmConfigKeys {
|
|||
DFS_RATIS_LEADER_ELECTION_MINIMUM_TIMEOUT_DURATION_DEFAULT =
|
||||
TimeDuration.valueOf(1, TimeUnit.SECONDS);
|
||||
|
||||
public static final String DFS_RATIS_SNAPSHOT_THRESHOLD_KEY =
|
||||
"dfs.ratis.snapshot.threshold";
|
||||
public static final long DFS_RATIS_SNAPSHOT_THRESHOLD_DEFAULT = 10000;
|
||||
|
||||
public static final String DFS_RATIS_SERVER_FAILURE_DURATION_KEY =
|
||||
"dfs.ratis.server.failure.duration";
|
||||
public static final TimeDuration
|
||||
|
|
|
@ -266,6 +266,10 @@ public final class OzoneConfigKeys {
|
|||
public static final TimeDuration
|
||||
DFS_RATIS_LEADER_ELECTION_MINIMUM_TIMEOUT_DURATION_DEFAULT =
|
||||
ScmConfigKeys.DFS_RATIS_LEADER_ELECTION_MINIMUM_TIMEOUT_DURATION_DEFAULT;
|
||||
public static final String DFS_RATIS_SNAPSHOT_THRESHOLD_KEY =
|
||||
ScmConfigKeys.DFS_RATIS_SNAPSHOT_THRESHOLD_KEY;
|
||||
public static final long DFS_RATIS_SNAPSHOT_THRESHOLD_DEFAULT =
|
||||
ScmConfigKeys.DFS_RATIS_SNAPSHOT_THRESHOLD_DEFAULT;
|
||||
|
||||
public static final String DFS_RATIS_SERVER_FAILURE_DURATION_KEY =
|
||||
ScmConfigKeys.DFS_RATIS_SERVER_FAILURE_DURATION_KEY;
|
||||
|
|
|
@ -118,6 +118,14 @@
|
|||
etc. This picks one of those for this cluster.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>dfs.ratis.snapshot.threshold</name>
|
||||
<value>10000</value>
|
||||
<tag>OZONE, RATIS</tag>
|
||||
<description>Number of transactions after which a ratis snapshot should be
|
||||
taken.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>dfs.container.ratis.num.write.chunk.threads</name>
|
||||
<value>60</value>
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -130,6 +130,10 @@ public final class XceiverServerRatis implements XceiverServerSpi {
|
|||
.build();
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public ContainerStateMachine getStateMachine() {
|
||||
return stateMachine;
|
||||
}
|
||||
|
||||
private RaftProperties newRaftProperties(Configuration conf) {
|
||||
final RaftProperties properties = new RaftProperties();
|
||||
|
@ -254,6 +258,15 @@ public final class XceiverServerRatis implements XceiverServerSpi {
|
|||
} else if (rpc == SupportedRpcType.NETTY) {
|
||||
NettyConfigKeys.Server.setPort(properties, port);
|
||||
}
|
||||
|
||||
long snapshotThreshold =
|
||||
conf.getLong(OzoneConfigKeys.DFS_RATIS_SNAPSHOT_THRESHOLD_KEY,
|
||||
OzoneConfigKeys.DFS_RATIS_SNAPSHOT_THRESHOLD_DEFAULT);
|
||||
RaftServerConfigKeys.Snapshot.
|
||||
setAutoTriggerEnabled(properties, true);
|
||||
RaftServerConfigKeys.Snapshot.
|
||||
setAutoTriggerThreshold(properties, snapshotThreshold);
|
||||
|
||||
return properties;
|
||||
}
|
||||
|
||||
|
@ -298,7 +311,6 @@ public final class XceiverServerRatis implements XceiverServerSpi {
|
|||
public void stop() {
|
||||
try {
|
||||
chunkExecutor.shutdown();
|
||||
stateMachine.close();
|
||||
server.close();
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
|
|
|
@ -230,6 +230,11 @@ public class OzoneContainer {
|
|||
return this.hddsDispatcher;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public XceiverServerSpi getServer(ReplicationType replicationType) {
|
||||
return servers.get(replicationType);
|
||||
}
|
||||
|
||||
public VolumeSet getVolumeSet() {
|
||||
return volumeSet;
|
||||
}
|
||||
|
|
|
@ -0,0 +1,122 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.ozone.freon;
|
||||
|
||||
import org.apache.hadoop.hdds.client.ReplicationFactor;
|
||||
import org.apache.hadoop.hdds.client.ReplicationType;
|
||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||
import org.apache.hadoop.ozone.MiniOzoneCluster;
|
||||
import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerSpi;
|
||||
import org.apache.hadoop.ozone.container.common.transport.server.ratis.ContainerStateMachine;
|
||||
import org.apache.hadoop.ozone.container.common.transport.server.ratis.XceiverServerRatis;
|
||||
import org.apache.ratis.server.protocol.TermIndex;
|
||||
import org.apache.ratis.statemachine.impl.SimpleStateMachineStorage;
|
||||
import org.apache.ratis.statemachine.impl.SingleFileSnapshotInfo;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_STALENODE_INTERVAL;
|
||||
|
||||
/**
|
||||
* Tests Freon with Datanode restarts.
|
||||
*/
|
||||
public class TestFreonWithDatanodeRestart {
|
||||
|
||||
private static MiniOzoneCluster cluster;
|
||||
private static OzoneConfiguration conf;
|
||||
|
||||
/**
|
||||
* Create a MiniDFSCluster for testing.
|
||||
* <p>
|
||||
* Ozone is made active by setting OZONE_ENABLED = true
|
||||
*
|
||||
*/
|
||||
@BeforeClass
|
||||
public static void init() throws Exception {
|
||||
conf = new OzoneConfiguration();
|
||||
conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 5, TimeUnit.SECONDS);
|
||||
cluster = MiniOzoneCluster.newBuilder(conf)
|
||||
.setHbProcessorInterval(1000)
|
||||
.setHbInterval(1000)
|
||||
.setNumDatanodes(3)
|
||||
.build();
|
||||
cluster.waitForClusterToBeReady();
|
||||
}
|
||||
|
||||
/**
|
||||
* Shutdown MiniDFSCluster.
|
||||
*/
|
||||
@AfterClass
|
||||
public static void shutdown() {
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRestart() throws Exception {
|
||||
RandomKeyGenerator randomKeyGenerator =
|
||||
new RandomKeyGenerator((OzoneConfiguration) cluster.getConf());
|
||||
randomKeyGenerator.setNumOfVolumes(1);
|
||||
randomKeyGenerator.setNumOfBuckets(1);
|
||||
randomKeyGenerator.setNumOfKeys(1);
|
||||
randomKeyGenerator.setType(ReplicationType.RATIS);
|
||||
randomKeyGenerator.setFactor(ReplicationFactor.THREE);
|
||||
randomKeyGenerator.setKeySize(20971520);
|
||||
randomKeyGenerator.setValidateWrites(true);
|
||||
randomKeyGenerator.call();
|
||||
Assert.assertEquals(1, randomKeyGenerator.getNumberOfVolumesCreated());
|
||||
Assert.assertEquals(1, randomKeyGenerator.getNumberOfBucketsCreated());
|
||||
Assert.assertEquals(1, randomKeyGenerator.getNumberOfKeysAdded());
|
||||
Assert.assertEquals(0, randomKeyGenerator.getUnsuccessfulValidationCount());
|
||||
|
||||
ContainerStateMachine sm = getStateMachine();
|
||||
TermIndex termIndexBeforeRestart = sm.getLastAppliedTermIndex();
|
||||
cluster.restartHddsDatanode(0);
|
||||
sm = getStateMachine();
|
||||
SimpleStateMachineStorage storage =
|
||||
(SimpleStateMachineStorage)sm.getStateMachineStorage();
|
||||
SingleFileSnapshotInfo snapshotInfo = storage.getLatestSnapshot();
|
||||
TermIndex termInSnapshot = snapshotInfo.getTermIndex();
|
||||
String expectedSnapFile =
|
||||
storage.getSnapshotFile(termIndexBeforeRestart.getTerm(),
|
||||
termIndexBeforeRestart.getIndex()).getAbsolutePath();
|
||||
Assert.assertEquals(snapshotInfo.getFile().getPath().toString(),
|
||||
expectedSnapFile);
|
||||
Assert.assertEquals(termInSnapshot, termIndexBeforeRestart);
|
||||
|
||||
// After restart the term index might have progressed to apply pending
|
||||
// transactions.
|
||||
TermIndex termIndexAfterRestart = sm.getLastAppliedTermIndex();
|
||||
Assert.assertTrue(termIndexAfterRestart.getIndex() >=
|
||||
termIndexBeforeRestart.getIndex());
|
||||
}
|
||||
|
||||
private ContainerStateMachine getStateMachine() {
|
||||
XceiverServerSpi server =
|
||||
cluster.getHddsDatanodes().get(0).getDatanodeStateMachine().
|
||||
getContainer().getServer(HddsProtos.ReplicationType.RATIS);
|
||||
return ((XceiverServerRatis)server).getStateMachine();
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue