Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HDDS-1610. applyTransaction failure should not be lost on restart. #1226

Closed
wants to merge 5 commits into from
Closed
Show file tree
Hide file tree
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
import org.apache.hadoop.util.Time;
import org.apache.ratis.proto.RaftProtos.RaftPeerRole;
import org.apache.ratis.protocol.RaftGroupId;
import org.apache.ratis.protocol.StateMachineException;
import org.apache.ratis.server.RaftServer;
import org.apache.ratis.server.impl.RaftServerProxy;
import org.apache.ratis.server.protocol.TermIndex;
Expand Down Expand Up @@ -83,6 +84,7 @@
import java.util.concurrent.Semaphore;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.stream.Collectors;
import java.util.Set;
import java.util.concurrent.ConcurrentSkipListSet;
Expand Down Expand Up @@ -147,6 +149,7 @@ public class ContainerStateMachine extends BaseStateMachine {
private final Cache<Long, ByteString> stateMachineDataCache;
private final boolean isBlockTokenEnabled;
private final TokenVerifier tokenVerifier;
private final AtomicBoolean isStateMachineHealthy;

private final Semaphore applyTransactionSemaphore;
/**
Expand Down Expand Up @@ -184,6 +187,7 @@ public ContainerStateMachine(RaftGroupId gid, ContainerDispatcher dispatcher,
ScmConfigKeys.
DFS_CONTAINER_RATIS_STATEMACHINE_MAX_PENDING_APPLY_TXNS_DEFAULT);
applyTransactionSemaphore = new Semaphore(maxPendingApplyTransactions);
isStateMachineHealthy = new AtomicBoolean(true);
this.executors = new ExecutorService[numContainerOpExecutors];
for (int i = 0; i < numContainerOpExecutors; i++) {
final int index = i;
Expand Down Expand Up @@ -265,6 +269,13 @@ public void persistContainerSet(OutputStream out) throws IOException {
public long takeSnapshot() throws IOException {
TermIndex ti = getLastAppliedTermIndex();
long startTime = Time.monotonicNow();
if (!isStateMachineHealthy.get()) {
String msg =
"Failed to take snapshot " + " for " + gid + " as the stateMachine"
+ " is unhealthy. The last applied index is at " + ti;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

lets log this as well.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Addressed in the latest patch.

StateMachineException sme = new StateMachineException(msg);
throw sme;
}
if (ti != null && ti.getIndex() != RaftLog.INVALID_LOG_INDEX) {
final File snapshotFile =
storage.getSnapshotFile(ti.getTerm(), ti.getIndex());
Expand All @@ -275,12 +286,12 @@ public long takeSnapshot() throws IOException {
// make sure the snapshot file is synced
fos.getFD().sync();
} catch (IOException ioe) {
LOG.info("{}: Failed to write snapshot at:{} file {}", gid, ti,
LOG.error("{}: Failed to write snapshot at:{} file {}", gid, ti,
snapshotFile);
throw ioe;
}
LOG.info("{}: Finished taking a snapshot at:{} file:{} time:{}",
gid, ti, snapshotFile, (Time.monotonicNow() - startTime));
LOG.info("{}: Finished taking a snapshot at:{} file:{} time:{}", gid, ti,
snapshotFile, (Time.monotonicNow() - startTime));
return ti.getIndex();
}
return -1;
Expand Down Expand Up @@ -385,17 +396,12 @@ private ContainerCommandResponseProto dispatchCommand(
return response;
}

private ContainerCommandResponseProto runCommandGetResponse(
private ContainerCommandResponseProto runCommand(
ContainerCommandRequestProto requestProto,
DispatcherContext context) {
return dispatchCommand(requestProto, context);
}

private Message runCommand(ContainerCommandRequestProto requestProto,
DispatcherContext context) {
return runCommandGetResponse(requestProto, context)::toByteString;
}

private ExecutorService getCommandExecutor(
ContainerCommandRequestProto requestProto) {
int executorId = (int)(requestProto.getContainerID() % executors.length);
Expand Down Expand Up @@ -425,7 +431,7 @@ private CompletableFuture<Message> handleWriteChunk(
// thread.
CompletableFuture<ContainerCommandResponseProto> writeChunkFuture =
CompletableFuture.supplyAsync(() ->
runCommandGetResponse(requestProto, context), chunkExecutor);
runCommand(requestProto, context), chunkExecutor);

CompletableFuture<Message> raftFuture = new CompletableFuture<>();

Expand Down Expand Up @@ -502,7 +508,8 @@ public CompletableFuture<Message> query(Message request) {
metrics.incNumQueryStateMachineOps();
final ContainerCommandRequestProto requestProto =
getContainerCommandRequestProto(request.getContent());
return CompletableFuture.completedFuture(runCommand(requestProto, null));
return CompletableFuture
.completedFuture(runCommand(requestProto, null)::toByteString);
} catch (IOException e) {
metrics.incNumQueryStateMachineFails();
return completeExceptionally(e);
Expand Down Expand Up @@ -674,30 +681,60 @@ public CompletableFuture<Message> applyTransaction(TransactionContext trx) {
if (cmdType == Type.WriteChunk || cmdType ==Type.PutSmallFile) {
builder.setCreateContainerSet(createContainerSet);
}
CompletableFuture<Message> applyTransactionFuture =
new CompletableFuture<>();
// Ensure the command gets executed in a separate thread than
// stateMachineUpdater thread which is calling applyTransaction here.
CompletableFuture<Message> future = CompletableFuture
.supplyAsync(() -> runCommand(requestProto, builder.build()),
CompletableFuture<ContainerCommandResponseProto> future =
CompletableFuture.supplyAsync(
() -> runCommand(requestProto, builder.build()),
getCommandExecutor(requestProto));

future.thenAccept(m -> {
future.thenApply(r -> {
if (trx.getServerRole() == RaftPeerRole.LEADER) {
long startTime = (long) trx.getStateMachineContext();
metrics.incPipelineLatency(cmdType,
Time.monotonicNowNanos() - startTime);
}

final Long previous =
applyTransactionCompletionMap
.put(index, trx.getLogEntry().getTerm());
Preconditions.checkState(previous == null);
if (cmdType == Type.WriteChunk || cmdType == Type.PutSmallFile) {
metrics.incNumBytesCommittedCount(
if (r.getResult() != ContainerProtos.Result.SUCCESS) {
StorageContainerException sce =
new StorageContainerException(r.getMessage(), r.getResult());
LOG.error(
"gid {} : ApplyTransaction failed. cmd {} logIndex {} msg : "
+ "{} Container Result: {}", gid, r.getCmdType(), index,
r.getMessage(), r.getResult());
metrics.incNumApplyTransactionsFails();
ratisServer.handleApplyTransactionFailure(gid, trx.getServerRole());
// Since the applyTransaction now is completed exceptionally,
// before any further snapshot is taken , the exception will be
// caught in stateMachineUpdater in Ratis and ratis server will
// shutdown.
applyTransactionFuture.completeExceptionally(sce);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

lets move the ratisServer.handleApplyTransactionFailure(gid, trx.getServerRole()); as the last line in the if block.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Addressed in the latest patch.

isStateMachineHealthy.compareAndSet(true, false);
} else {
metrics.incNumBytesWrittenCount(
requestProto.getWriteChunk().getChunkData().getLen());
LOG.debug(
"gid {} : ApplyTransaction completed. cmd {} logIndex {} msg : "
+ "{} Container Result: {}", gid, r.getCmdType(), index,
r.getMessage(), r.getResult());
applyTransactionFuture.complete(r::toByteString);
if (cmdType == Type.WriteChunk || cmdType == Type.PutSmallFile) {
metrics.incNumBytesCommittedCount(
requestProto.getWriteChunk().getChunkData().getLen());
}
// add the entry to the applyTransactionCompletionMap only if the
// stateMachine is healthy i.e, there has been no applyTransaction
// failures before.
if (isStateMachineHealthy.get()) {
final Long previous = applyTransactionCompletionMap
.put(index, trx.getLogEntry().getTerm());
Preconditions.checkState(previous == null);
updateLastApplied();
}
}
updateLastApplied();
return applyTransactionFuture;
}).whenComplete((r, t) -> applyTransactionSemaphore.release());
return future;
return applyTransactionFuture;
} catch (IOException | InterruptedException e) {
metrics.incNumApplyTransactionsFails();
return completeExceptionally(e);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -609,6 +609,15 @@ void handleNoLeader(RaftGroupId groupId, RoleInfoProto roleInfoProto) {
handlePipelineFailure(groupId, roleInfoProto);
}

void handleApplyTransactionFailure(RaftGroupId groupId,
RaftProtos.RaftPeerRole role) {
UUID dnId = RatisHelper.toDatanodeId(getServer().getId());
String msg =
"Ratis Transaction failure in datanode " + dnId + " with role " + role
+ " .Triggering pipeline close action.";
triggerPipelineClose(groupId, msg,
ClosePipelineInfo.Reason.STATEMACHINE_TRANSACTION_FAILED, true);
}
/**
* The fact that the snapshot contents cannot be used to actually catch up
* the follower, it is the reason to initiate close pipeline and
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -214,6 +214,7 @@ message ClosePipelineInfo {
enum Reason {
PIPELINE_FAILED = 1;
PIPELINE_LOG_FAILED = 2;
STATEMACHINE_TRANSACTION_FAILED = 3;
}
required PipelineID pipelineID = 1;
optional Reason reason = 3;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,29 +23,41 @@
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.XceiverClientManager;
import org.apache.hadoop.hdds.scm.XceiverClientSpi;
import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.client.ObjectStore;
import org.apache.hadoop.ozone.client.OzoneClient;
import org.apache.hadoop.ozone.client.OzoneClientFactory;
import org.apache.hadoop.ozone.client.OzoneKeyDetails;
import org.apache.hadoop.ozone.client.io.KeyOutputStream;
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
import org.apache.hadoop.ozone.container.ContainerTestHelper;
import org.apache.hadoop.ozone.container.common.impl.ContainerData;
import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher;
import org.apache.hadoop.ozone.container.common.transport.server.ratis.ContainerStateMachine;
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.ratis.protocol.RaftRetryFailureException;
import org.apache.ratis.protocol.StateMachineException;
import org.apache.ratis.server.storage.FileInfo;
import org.apache.ratis.statemachine.impl.SimpleStateMachineStorage;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;

import java.io.File;
import java.io.IOException;
import java.nio.file.Path;
import java.util.HashMap;
import java.util.List;
import java.util.concurrent.TimeUnit;
Expand Down Expand Up @@ -77,7 +89,7 @@ public class TestContainerStateMachineFailures {
private static String volumeName;
private static String bucketName;
private static String path;
private static int chunkSize;
private static XceiverClientManager xceiverClientManager;

/**
* Create a MiniDFSCluster for testing.
Expand All @@ -102,13 +114,15 @@ public static void init() throws Exception {
conf.setTimeDuration(OZONE_SCM_PIPELINE_DESTROY_TIMEOUT, 10,
TimeUnit.SECONDS);
conf.setQuietMode(false);
conf.setLong(OzoneConfigKeys.DFS_RATIS_SNAPSHOT_THRESHOLD_KEY , 1);
cluster =
MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1).setHbInterval(200)
.build();
cluster.waitForClusterToBeReady();
//the easiest way to create an open container is creating a key
client = OzoneClientFactory.getClient(conf);
objectStore = client.getObjectStore();
xceiverClientManager = new XceiverClientManager(conf);
volumeName = "testcontainerstatemachinefailures";
bucketName = volumeName;
objectStore.createVolume(volumeName);
Expand Down Expand Up @@ -270,4 +284,88 @@ public void testUnhealthyContainer() throws Exception {
Assert.assertEquals(ContainerProtos.Result.CONTAINER_UNHEALTHY,
dispatcher.dispatch(request.build(), null).getResult());
}

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you speak to the validity of these generated assert statement for the above test method?

org.junit.Assert.assertTrue( ONE.readContainerFile( node ) );
org.junit.Assert.assertTrue( ONE.readContainerFile( node.toString() ));
org.junit.Assert.assertTrue( ONE.readContainerFile( node.toString(), KeyOutputStream ))

Thank you.


@Test
public void testApplyTransactionFailure() throws Exception {
OzoneOutputStream key =
objectStore.getVolume(volumeName).getBucket(bucketName)
.createKey("ratis", 1024, ReplicationType.RATIS,
ReplicationFactor.ONE, new HashMap<>());
// First write and flush creates a container in the datanode
key.write("ratis".getBytes());
key.flush();
key.write("ratis".getBytes());

//get the name of a valid container
OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(volumeName).
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is unused.

setBucketName(bucketName).setType(HddsProtos.ReplicationType.RATIS)
.setFactor(HddsProtos.ReplicationFactor.ONE).setKeyName("ratis")
.build();
KeyOutputStream groupOutputStream = (KeyOutputStream) key.getOutputStream();
List<OmKeyLocationInfo> locationInfoList =
groupOutputStream.getLocationInfoList();
Assert.assertEquals(1, locationInfoList.size());
OmKeyLocationInfo omKeyLocationInfo = locationInfoList.get(0);
ContainerData containerData =
cluster.getHddsDatanodes().get(0).getDatanodeStateMachine()
.getContainer().getContainerSet()
.getContainer(omKeyLocationInfo.getContainerID())
.getContainerData();
Assert.assertTrue(containerData instanceof KeyValueContainerData);
KeyValueContainerData keyValueContainerData =
(KeyValueContainerData) containerData;
key.close();
ContainerStateMachine stateMachine =
(ContainerStateMachine)ContainerTestHelper.getStateMachine(cluster);
SimpleStateMachineStorage storage =
(SimpleStateMachineStorage) stateMachine.getStateMachineStorage();
Path path = storage.findLatestSnapshot().getFile().getPath();
// Since the snapshot threshold is set to 1, since there are
// applyTransactions, we should see snapshots
Assert.assertTrue(path.getParent().toFile().listFiles().length > 0);
FileInfo snapshot = storage.findLatestSnapshot().getFile();
Assert.assertNotNull(snapshot);
long containerID = omKeyLocationInfo.getContainerID();
// delete the container db file
FileUtil.fullyDelete(new File(keyValueContainerData.getContainerPath()));
Pipeline pipeline = cluster.getStorageContainerLocationClient()
.getContainerWithPipeline(containerID).getPipeline();
XceiverClientSpi xceiverClient = xceiverClientManager.acquireClient(pipeline);
ContainerProtos.ContainerCommandRequestProto.Builder request =
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

How about we append some more data to the key and flush again ? in place of a close ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The idea is to execute a transaction on the same container. If we write more data , it can potentially go a new container altogether.

ContainerProtos.ContainerCommandRequestProto.newBuilder();
request.setDatanodeUuid(pipeline.getFirstNode().getUuidString());
request.setCmdType(ContainerProtos.Type.CloseContainer);
request.setContainerID(containerID);
request.setCloseContainer(
ContainerProtos.CloseContainerRequestProto.getDefaultInstance());
// close container transaction will fail over Ratis and will initiate
// a pipeline close action

// Since the applyTransaction failure is propagated to Ratis,
// stateMachineUpdater will it exception while taking the next snapshot
// and should shutdown the RaftServerImpl. The client request will fail
// with RaftRetryFailureException.
try {
xceiverClient.sendCommand(request.build());
Assert.fail("Expected exception not thrown");
} catch (IOException e) {
Assert.assertTrue(HddsClientUtils
.checkForException(e) instanceof RaftRetryFailureException);
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Perhaps a log message here to say that the test caught an IOException as expected by the test case.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Will address in the next patch..

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What exception are we expecting here ?

// Make sure the container is marked unhealthy
Assert.assertTrue(
cluster.getHddsDatanodes().get(0).getDatanodeStateMachine()
.getContainer().getContainerSet().getContainer(containerID)
.getContainerState()
== ContainerProtos.ContainerDataProto.State.UNHEALTHY);
try {
// try to take a new snapshot, ideally it should just fail
stateMachine.takeSnapshot();
} catch(IOException ioe) {
Assert.assertTrue(ioe instanceof StateMachineException);
}
// Make sure the latest snapshot is same as the previous one
FileInfo latestSnapshot = storage.findLatestSnapshot().getFile();
Assert.assertTrue(snapshot.getPath().equals(latestSnapshot.getPath()));
}
}

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you speak to the validity of these generated assert statement for the above test method?

org.junit.Assert.assertTrue( name, volumeName.isValid() )

Thank you.

Original file line number Diff line number Diff line change
Expand Up @@ -73,6 +73,10 @@

import com.google.common.base.Preconditions;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.ratis.protocol.RaftGroupId;
import org.apache.ratis.server.impl.RaftServerImpl;
import org.apache.ratis.server.impl.RaftServerProxy;
import org.apache.ratis.statemachine.StateMachine;
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
import org.junit.Assert;
import org.slf4j.Logger;
Expand Down Expand Up @@ -866,4 +870,16 @@ public static void waitForContainerClose(MiniOzoneCluster cluster,
index++;
}
}

public static StateMachine getStateMachine(MiniOzoneCluster cluster)
throws Exception {
XceiverServerSpi server =
cluster.getHddsDatanodes().get(0).getDatanodeStateMachine().
getContainer().getWriteChannel();
RaftServerProxy proxy =
(RaftServerProxy) (((XceiverServerRatis) server).getServer());
RaftGroupId groupId = proxy.getGroupIds().iterator().next();
RaftServerImpl impl = proxy.getImpl(groupId);
return impl.getStateMachine();
}
}
Loading