Uploaded image for project: 'Apache Ozone'
  1. Apache Ozone
  2. HDDS-1589

CloseContainer transaction on unhealthy replica should fail with CONTAINER_UNHEALTHY exception

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • None
    • 0.5.0
    • Ozone Datanode
    • None
    • Done

    Description

      Currently, while trying to close an unhealthy container over Ratis, it fails with INTERNAL_ERROR which leads to exception as follow:

      2019-05-19 22:00:48,386 ERROR commandhandler.CloseContainerCommandHandler (CloseContainerCommandHandler.java:handle(124)) - Can't close container #125
      org.apache.ratis.protocol.StateMachineException: java.util.concurrent.CompletionException from Server faea26b0-9c60-4b4c-a0df-bf7c67cc5b48: java.lang.IllegalStateException
              at org.apache.ratis.server.impl.RaftServerImpl.lambda$replyPendingRequest$24(RaftServerImpl.java:1221)
              at java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760)
              at java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:736)
              at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
              at java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1595)
              at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
              at java.lang.Thread.run(Thread.java:748)
      Caused by: java.util.concurrent.CompletionException: java.lang.IllegalStateException
              at java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:273)
              at java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:280)
              at java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1592)
              ... 3 more
      Caused by: java.lang.IllegalStateException
              at com.google.common.base.Preconditions.checkState(Preconditions.java:129)
              at org.apache.hadoop.ozone.container.common.impl.HddsDispatcher.dispatchRequest(HddsDispatcher.java:300)
              at org.apache.hadoop.ozone.container.common.impl.HddsDispatcher.dispatch(HddsDispatcher.java:149)
              at org.apache.hadoop.ozone.container.common.transport.server.ratis.ContainerStateMachine.dispatchCommand(ContainerStateMachine.java:347)
              at org.apache.hadoop.ozone.container.common.transport.server.ratis.ContainerStateMachine.runCommand(ContainerStateMachine.java:354)
              at org.apache.hadoop.ozone.container.common.transport.server.ratis.ContainerStateMachine.lambda$applyTransaction$5(ContainerStateMachine.java:613)
              at java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1590)
      

      This happens when , it tries to mark the container unhealthy as the transaction has failed and tries to mark the container unhealthy where it expects the container to be in OPEN or CLOSIG state ad hence asserts. It should ideally fail with CONTAINER_UNHEALTHY so as to not retry to not change the state to be UNHEALTHY.

      Attachments

        Issue Links

          Activity

            People

              shashikant Shashikant Banerjee
              shashikant Shashikant Banerjee
              Votes:
              0 Vote for this issue
              Watchers:
              1 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: