[ 
https://issues.apache.org/jira/browse/HDDS-2281?focusedWorklogId=330421&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-330421
 ]

ASF GitHub Bot logged work on HDDS-2281:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 18/Oct/19 10:49
            Start Date: 18/Oct/19 10:49
    Worklog Time Spent: 10m 
      Work Description: bshashikant commented on pull request #1641: HDDS-2281. 
ContainerStateMachine#handleWriteChunk should ignore close container exception.
URL: https://github.com/apache/hadoop/pull/1641#discussion_r336431510
 
 

 ##########
 File path: 
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachineFailures.java
 ##########
 @@ -418,6 +424,108 @@ public void 
testApplyTransactionIdempotencyWithClosedContainer()
     Assert.assertFalse(snapshot.getPath().equals(latestSnapshot.getPath()));
   }
 
+  // The test injects multiple write chunk requests along with closed container
+  // request thereby inducing a situation where a writeStateMachine call
+  // gets executed when the closed container apply completes thereby
+  // failing writeStateMachine call. In any case, our stateMachine should
+  // not be marked unhealthy and pipeline should not fail if container gets
+  // closed here.
+  @Test
+  public void testWriteStateMachineDataIdempotencyWithClosedContainer()
+      throws Exception {
+    OzoneOutputStream key =
+        objectStore.getVolume(volumeName).getBucket(bucketName)
+            .createKey("ratis-1", 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);
+    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();
+    Pipeline pipeline = cluster.getStorageContainerLocationClient()
+        .getContainerWithPipeline(containerID).getPipeline();
+    XceiverClientSpi xceiverClient =
+        xceiverClientManager.acquireClient(pipeline);
+    CountDownLatch latch = new CountDownLatch(100);
+    int count = 0;
+    AtomicInteger failCount = new AtomicInteger(0);
+    Runnable r1 = () -> {
+      try {
+        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());
+        xceiverClient.sendCommand(request.build());
+      } catch (IOException e) {
+        failCount.incrementAndGet();
+      }
+    };
+    Runnable r2 = () -> {
+      try {
+        xceiverClient.sendCommand(ContainerTestHelper
+            .getWriteChunkRequest(pipeline, omKeyLocationInfo.getBlockID(),
+                1024, new Random().nextInt()));
+        latch.countDown();
+      } catch (IOException e) {
+        latch.countDown();
+        if (!(HddsClientUtils
+            .checkForException(e) instanceof ContainerNotOpenException)) {
+          failCount.incrementAndGet();
+        }
+      }
+    };
+
+    for (int i=0 ; i < 100; i++) {
+      count++;
+      new Thread(r2).start();
+    }
+
+    new Thread(r1).start();
+    latch.await(600, TimeUnit.SECONDS);
+    if (failCount.get() > 0) {
+      fail("testWriteStateMachineDataIdempotencyWithClosedContainer failed");
+    }
+    Assert.assertTrue(
+        cluster.getHddsDatanodes().get(0).getDatanodeStateMachine()
+            .getContainer().getContainerSet().getContainer(containerID)
+            .getContainerState()
+            == ContainerProtos.ContainerDataProto.State.CLOSED);
+    Assert.assertTrue(stateMachine.isStateMachineHealthy());
+    try {
+      stateMachine.takeSnapshot();
+    } catch (IOException ioe) {
+      Assert.fail("Exception should not be thrown");
+    }
+    FileInfo latestSnapshot = storage.findLatestSnapshot().getFile();
+    Assert.assertFalse(snapshot.getPath().equals(latestSnapshot.getPath()));
 
 Review comment:
   The test is to not mark stateMachine unhealthy because of closed container 
exception and hence the latest snapshot should not match with that of previous 
one. The test of marking stateMachine unhealthy because of error is covered in 
TestContainerStateMachineFailures#testApplyTransactionFailure
 
----------------------------------------------------------------
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:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 330421)
    Time Spent: 1h  (was: 50m)

> ContainerStateMachine#handleWriteChunk should ignore close container 
> exception 
> -------------------------------------------------------------------------------
>
>                 Key: HDDS-2281
>                 URL: https://issues.apache.org/jira/browse/HDDS-2281
>             Project: Hadoop Distributed Data Store
>          Issue Type: Bug
>          Components: Ozone Datanode
>            Reporter: Shashikant Banerjee
>            Assignee: Shashikant Banerjee
>            Priority: Major
>              Labels: pull-request-available
>             Fix For: 0.5.0
>
>          Time Spent: 1h
>  Remaining Estimate: 0h
>
> Currently, ContainerStateMachine#applyTrannsaction ignores close container 
> exception.Similarly,ContainerStateMachine#handleWriteChunk call also should 
> ignore close container exception.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

---------------------------------------------------------------------
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org

Reply via email to