-
Notifications
You must be signed in to change notification settings - Fork 9.2k
HDDS-1610. applyTransaction failure should not be lost on restart. #1226
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
Changes from 1 commit
a6f60eb
529a05b
412a7ff
0d301a0
d559ba0
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -674,30 +674,54 @@ 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( | ||
| () -> runCommandGetResponse(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 " + r.getCmdType() | ||
|
||
| + " logIndex " + index + " Error message: " + r.getMessage() | ||
| + " Container Result: " + 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); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Addressed in the latest patch. |
||
| } else { | ||
| metrics.incNumBytesWrittenCount( | ||
| requestProto.getWriteChunk().getChunkData().getLen()); | ||
| LOG.debug(gid + ": ApplyTransaction completed: cmd " + r.getCmdType() | ||
|
||
| + " logIndex " + index + " Error message: " + r.getMessage() | ||
| + " Container Result: " + r.getResult()); | ||
| applyTransactionFuture.complete(r::toByteString); | ||
| if (cmdType == Type.WriteChunk || cmdType == Type.PutSmallFile) { | ||
| metrics.incNumBytesCommittedCount( | ||
| requestProto.getWriteChunk().getChunkData().getLen()); | ||
| } | ||
| } | ||
|
|
||
| final Long previous = applyTransactionCompletionMap | ||
| .put(index, trx.getLogEntry().getTerm()); | ||
| Preconditions.checkState(previous == null); | ||
| updateLastApplied(); | ||
| }).whenComplete((r, t) -> applyTransactionSemaphore.release()); | ||
| return future; | ||
| applyTransactionSemaphore.release(); | ||
|
||
| return applyTransactionFuture; | ||
| }); | ||
| return applyTransactionFuture; | ||
| } catch (IOException | InterruptedException e) { | ||
| metrics.incNumApplyTransactionsFails(); | ||
| return completeExceptionally(e); | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -609,6 +609,16 @@ 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.PIPELINE_FAILED, | ||
|
||
| false); | ||
| stop(); | ||
|
||
| } | ||
| /** | ||
| * The fact that the snapshot contents cannot be used to actually catch up | ||
| * the follower, it is the reason to initiate close pipeline and | ||
|
|
@@ -630,6 +640,10 @@ void handleInstallSnapshotFromLeader(RaftGroupId groupId, | |
| handlePipelineFailure(groupId, roleInfoProto); | ||
| } | ||
|
|
||
| @VisibleForTesting | ||
| public boolean isClosed() { | ||
|
||
| return !isStarted; | ||
| } | ||
| /** | ||
| * Notify the Datanode Ratis endpoint of Ratis log failure. | ||
| * Expected to be invoked from the Container StateMachine | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -23,6 +23,11 @@ | |
| 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.container.common.helpers.StorageContainerException; | ||
|
||
| import org.apache.hadoop.hdds.scm.pipeline.Pipeline; | ||
| import org.apache.hadoop.hdds.scm.pipeline.PipelineNotFoundException; | ||
| import org.apache.hadoop.ozone.MiniOzoneCluster; | ||
| import org.apache.hadoop.ozone.OzoneConsts; | ||
| import org.apache.hadoop.ozone.client.ObjectStore; | ||
|
|
@@ -34,11 +39,14 @@ | |
| 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.XceiverServerRatis; | ||
| 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.StateMachineException; | ||
| import org.apache.ratis.util.LifeCycle; | ||
| import org.junit.AfterClass; | ||
| import org.junit.Assert; | ||
| import org.junit.BeforeClass; | ||
|
|
@@ -77,7 +85,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. | ||
|
|
@@ -109,6 +117,7 @@ public static void init() throws Exception { | |
| //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); | ||
|
|
@@ -270,4 +279,73 @@ public void testUnhealthyContainer() throws Exception { | |
| Assert.assertEquals(ContainerProtos.Result.CONTAINER_UNHEALTHY, | ||
| dispatcher.dispatch(request.build(), null).getResult()); | ||
| } | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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?
Thank you. |
||
|
|
||
| @Test | ||
| public void testAppyTransactionFailure() 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). | ||
|
||
| 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(); | ||
|
|
||
| long containerID = omKeyLocationInfo.getContainerID(); | ||
| // delete the container db file | ||
| FileUtil.fullyDelete(new File(keyValueContainerData.getContainerPath())); | ||
| Pipeline pipeline = cluster.getStorageContainerLocationClient() | ||
| .getContainerWithPipeline(containerID).getPipeline(); | ||
| XceiverClientSpi client = xceiverClientManager.acquireClient(pipeline); | ||
| ContainerProtos.ContainerCommandRequestProto.Builder request = | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 ?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 cause the raft | ||
|
||
| try { | ||
| client.sendCommand(request.build()); | ||
| Assert.fail("Expected exception not thrown"); | ||
| } catch (IOException e) { | ||
| } | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Will address in the next patch..
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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); | ||
| XceiverServerRatis raftServer = (XceiverServerRatis) | ||
| cluster.getHddsDatanodes().get(0).getDatanodeStateMachine() | ||
| .getContainer().getWriteChannel(); | ||
| Assert.assertTrue(raftServer.isClosed()); | ||
|
||
| try { | ||
| cluster.getStorageContainerManager().getPipelineManager() | ||
| .getPipeline(pipeline.getId()); | ||
| Assert.fail("Expected exception not thrown"); | ||
| } catch(PipelineNotFoundException e) { | ||
| } | ||
| } | ||
| } | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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?
Thank you. |
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
lets rename runCommandGetResponse and remove runCommand as all the existing caller of the earlier function runCommand can be removed.