[
https://issues.apache.org/jira/browse/HDDS-1610?focusedWorklogId=298689&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-298689
]
ASF GitHub Bot logged work on HDDS-1610:
----------------------------------------
Author: ASF GitHub Bot
Created on: 21/Aug/19 13:37
Start Date: 21/Aug/19 13:37
Worklog Time Spent: 10m
Work Description: cawatson commented on pull request #1226: HDDS-1610.
applyTransaction failure should not be lost on restart.
URL: https://github.com/apache/hadoop/pull/1226#discussion_r316188452
##########
File path:
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachineFailures.java
##########
@@ -270,4 +269,83 @@ public void testUnhealthyContainer() throws Exception {
Assert.assertEquals(ContainerProtos.Result.CONTAINER_UNHEALTHY,
dispatcher.dispatch(request.build(), null).getResult());
}
+
+ @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());
+ 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 parentPath = storage.findLatestSnapshot().getFile().getPath();
+ // Since the snapshot threshold is set to 1, since there are
+ // applyTransactions, we should see snapshots
+ Assert.assertTrue(parentPath.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 =
+ 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);
+ }
+ // 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()));
+ }
}
Review comment:
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.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 298689)
Time Spent: 7h (was: 6h 50m)
> applyTransaction failure should not be lost on restart
> ------------------------------------------------------
>
> Key: HDDS-1610
> URL: https://issues.apache.org/jira/browse/HDDS-1610
> Project: Hadoop Distributed Data Store
> Issue Type: Bug
> Reporter: Shashikant Banerjee
> Assignee: Shashikant Banerjee
> Priority: Blocker
> Labels: pull-request-available
> Time Spent: 7h
> Remaining Estimate: 0h
>
> If the applyTransaction fails in the containerStateMachine, then the
> container should not accept new writes on restart,.
> This can occur if
> # chunk write applyTransaction fails
> # container state update to UNHEALTHY also fails
> # Ratis snapshot is taken
> # Node restarts
> # container accepts new transactions
--
This message was sent by Atlassian Jira
(v8.3.2#803003)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]