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

[Hbase Ozone] Both HMaster down with "BlockOutputStream: Failed to commit BlockId"

    XMLWordPrintableJSON

Details

    Description

      DNs down in the cluster:
      vc0122.xyz, vc0130.xyz

      Both HMasters aborts because of 3 way commit failed on the pipeline, even though only one of the node in the pipeline is down.
      Can see below that its trying to commit into a pipeline with nodes: vc0124.xyz, vc0123.xyz and vc0122.xyz, and only one of these i.e. vc0122.xyz is down. 

      Error logs just before HMaster aborts:

      2024-04-15 07:08:40,002 WARN org.apache.hadoop.hdds.scm.XceiverClientRatis: 3 way commit failed on pipeline Pipeline[ Id: c3ac49c1-02c0-4d6c-aa2b-968ec9316380, Nodes: f7f57f95-7297-4a70-bc9a-bb2e8be573fc(vc0124.xyz/10.17.207.34)0246b7fe-2072-43f6-b2a3-aa6ca64cd94a(vc0123.xyz/10.17.207.33)0efdc82f-f19e-4bba-bc6b-ee61f973a24a(vc0122.xyz/10.17.207.32), ReplicationConfig: RATIS/THREE, State:OPEN, leaderId:0246b7fe-2072-43f6-b2a3-aa6ca64cd94a, CreationTimestamp2024-04-15T00:20:05.717-07:00[America/Los_Angeles]]
      java.util.concurrent.ExecutionException: org.apache.ratis.protocol.exceptions.TimeoutIOException: client-188734B8DEB9->0246b7fe-2072-43f6-b2a3-aa6ca64cd94a request #41258 timeout 180s
              at java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:357)
              at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1908)
              at org.apache.hadoop.hdds.scm.XceiverClientRatis.watchForCommit(XceiverClientRatis.java:279)
              at org.apache.hadoop.hdds.scm.storage.AbstractCommitWatcher.watchForCommit(AbstractCommitWatcher.java:142)
              at org.apache.hadoop.hdds.scm.storage.AbstractCommitWatcher.watchOnLastIndex(AbstractCommitWatcher.java:120)
              at org.apache.hadoop.hdds.scm.storage.RatisBlockOutputStream.sendWatchForCommit(RatisBlockOutputStream.java:107)
              at org.apache.hadoop.hdds.scm.storage.BlockOutputStream.watchForCommit(BlockOutputStream.java:454)
              at org.apache.hadoop.hdds.scm.storage.BlockOutputStream.handleFlushInternal(BlockOutputStream.java:634)
              at org.apache.hadoop.hdds.scm.storage.BlockOutputStream.handleFlush(BlockOutputStream.java:587)
              at org.apache.hadoop.hdds.scm.storage.RatisBlockOutputStream.hsync(RatisBlockOutputStream.java:139)
              at org.apache.hadoop.ozone.client.io.BlockOutputStreamEntry.hsync(BlockOutputStreamEntry.java:163)
              at org.apache.hadoop.ozone.client.io.KeyOutputStream.handleStreamAction(KeyOutputStream.java:524)
              at org.apache.hadoop.ozone.client.io.KeyOutputStream.handleFlushOrClose(KeyOutputStream.java:487)
              at org.apache.hadoop.ozone.client.io.KeyOutputStream.hsync(KeyOutputStream.java:457)
              at org.apache.hadoop.ozone.client.io.OzoneOutputStream.hsync(OzoneOutputStream.java:118)
              at org.apache.hadoop.fs.ozone.OzoneFSOutputStream.hsync(OzoneFSOutputStream.java:70)
              at org.apache.hadoop.fs.ozone.OzoneFSOutputStream.hflush(OzoneFSOutputStream.java:65)
              at org.apache.hadoop.fs.FSDataOutputStream.hflush(FSDataOutputStream.java:136)
              at org.apache.hadoop.hbase.io.asyncfs.WrapperAsyncFSOutput.flush0(WrapperAsyncFSOutput.java:92)
              at org.apache.hadoop.hbase.io.asyncfs.WrapperAsyncFSOutput.lambda$flush$0(WrapperAsyncFSOutput.java:113)
              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: org.apache.ratis.protocol.exceptions.TimeoutIOException: client-188734B8DEB9->0246b7fe-2072-43f6-b2a3-aa6ca64cd94a request #41258 timeout 180s
              at org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers.lambda$timeoutCheck$5(GrpcClientProtocolClient.java:374)
              at java.util.Optional.ifPresent(Optional.java:159)
              at org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers.handleReplyFuture(GrpcClientProtocolClient.java:378)
              at org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers.timeoutCheck(GrpcClientProtocolClient.java:373)
              at org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers.lambda$onNext$1(GrpcClientProtocolClient.java:362)
              at org.apache.ratis.util.TimeoutTimer.lambda$onTimeout$2(TimeoutTimer.java:101)
              at org.apache.ratis.util.LogUtils.runAndLog(LogUtils.java:38)
              at org.apache.ratis.util.LogUtils$1.run(LogUtils.java:78)
              at org.apache.ratis.util.TimeoutTimer$Task.run(TimeoutTimer.java:55)
              at java.util.TimerThread.mainLoop(Timer.java:555)
              at java.util.TimerThread.run(Timer.java:505)
      2024-04-15 07:08:40,018 INFO org.apache.hadoop.hdds.scm.XceiverClientRatis: Could not commit index 66131 on pipeline Pipeline[ Id: c3ac49c1-02c0-4d6c-aa2b-968ec9316380, Nodes: f7f57f95-7297-4a70-bc9a-bb2e8be573fc(vc0124.xyz/10.17.207.34)0246b7fe-2072-43f6-b2a3-aa6ca64cd94a(vc0123.xyz/10.17.207.33)0efdc82f-f19e-4bba-bc6b-ee61f973a24a(vc0122.xyz/10.17.207.32), ReplicationConfig: RATIS/THREE, State:OPEN, leaderId:0246b7fe-2072-43f6-b2a3-aa6ca64cd94a, CreationTimestamp2024-04-15T00:20:05.717-07:00[America/Los_Angeles]] to all the nodes. Server 0efdc82f-f19e-4bba-bc6b-ee61f973a24a has failed. Committed by majority.
      2024-04-15 07:08:40,019 WARN org.apache.hadoop.hdds.scm.storage.BlockOutputStream: Failed to commit BlockId conID: 2018 locID: 113750153625631788 bcsId: 66131 on Pipeline[ Id: c3ac49c1-02c0-4d6c-aa2b-968ec9316380, Nodes: f7f57f95-7297-4a70-bc9a-bb2e8be573fc(vc0124.xyz/10.17.207.34)0246b7fe-2072-43f6-b2a3-aa6ca64cd94a(vc0123.xyz/10.17.207.33)0efdc82f-f19e-4bba-bc6b-ee61f973a24a(vc0122.xyz/10.17.207.32), ReplicationConfig: RATIS/THREE, State:OPEN, leaderId:0246b7fe-2072-43f6-b2a3-aa6ca64cd94a, CreationTimestamp2024-04-15T00:20:05.717-07:00[America/Los_Angeles]]. Failed nodes: [0efdc82f-f19e-4bba-bc6b-ee61f973a24a(null/null)]
      2024-04-15 07:08:40,156 WARN org.apache.hadoop.ozone.client.io.KeyOutputStream: Encountered exception java.io.IOException: Unexpected Storage Container Exception: java.util.concurrent.CompletionException: java.util.concurrent.CompletionException: java.lang.NullPointerException on the pipeline Pipeline[ Id: c3ac49c1-02c0-4d6c-aa2b-968ec9316380, Nodes: f7f57f95-7297-4a70-bc9a-bb2e8be573fc(vc0124.xyz/10.17.207.34)0246b7fe-2072-43f6-b2a3-aa6ca64cd94a(vc0123.xyz/10.17.207.33)0efdc82f-f19e-4bba-bc6b-ee61f973a24a(vc0122.xyz/10.17.207.32), ReplicationConfig: RATIS/THREE, State:OPEN, leaderId:0246b7fe-2072-43f6-b2a3-aa6ca64cd94a, CreationTimestamp2024-04-15T00:20:05.717-07:00[America/Los_Angeles]]. The last committed block length is 83, uncommitted data length is 0 retry count 0
      2024-04-15 07:08:40,169 ERROR org.apache.hadoop.hbase.wal.AbstractWALRoller: Log rolling failed
      java.lang.RuntimeException
              at org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.writeWALMetadata(AsyncProtobufLogWriter.java:217)
              at org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.writeMagicAndWALHeader(AsyncProtobufLogWriter.java:223)
              at org.apache.hadoop.hbase.regionserver.wal.AbstractProtobufLogWriter.init(AbstractProtobufLogWriter.java:164)
              at org.apache.hadoop.hbase.wal.AsyncFSWALProvider.createAsyncWriter(AsyncFSWALProvider.java:116)
              at org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(AsyncFSWAL.java:726)
              at org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(AsyncFSWAL.java:129)
              at org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.rollWriter(AbstractFSWAL.java:886)
              at org.apache.hadoop.hbase.wal.AbstractWALRoller$RollController.rollWal(AbstractWALRoller.java:304)
              at org.apache.hadoop.hbase.wal.AbstractWALRoller.run(AbstractWALRoller.java:211)
      2024-04-15 07:08:40,234 ERROR org.apache.ratis.client.impl.OrderedAsync: client-546F8A3A61A4: Failed* RaftClientRequest:client-546F8A3A61A4->0efdc82f-f19e-4bba-bc6b-ee61f973a24a@group-534ED93B4EC4, cid=41261, seq=2768, RW, cmdType: WriteChunk
      traceID: ""
      containerID: 2008
      datanodeUuid: "0efdc82f-f19e-4bba-bc6b-ee61f973a24a"
      writeChunk {
        blockID {
          containerID: 2008
          localID: 113750153625630686
          blockCommitSequenceId: 256444
          replicaIndex: 0
        }
        chunkData {
          chunkName: "113750153625630686_chunk_1384"
          offset: 546232
          len: 8
          checksumData {
            type: CRC32
            bytesPerChecksum: 16384
            checksums: "[)E|"
          }
        }
      }
      encodedToken: "VwoFaGJhc2USJWNvbklEOiAyMDA4IGxvY0lEOiAxMTM3NTAxNTM2MjU2MzA2ODYYx9Gxue4xKAEoAjCAgICAAToWCIiEkcP56OT6iQEQ2c26m9bckYmpASBh2nTMHaIRg0MRIosCu98kApXmFzav-uYGKEQlKMWiIRBIRERTX0JMT0NLX1RPS0VOLGNvbklEOiAyMDA4IGxvY0lEOiAxMTM3NTAxNTM2MjU2MzA2ODYAAAAAAAAA"
      , data.size=8
      java.util.concurrent.CompletionException: org.apache.ratis.protocol.exceptions.AlreadyClosedException: client-546F8A3A61A4->0efdc82f-f19e-4bba-bc6b-ee61f973a24a is closed.
              at java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
              at java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
              at java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:607)
              at java.util.concurrent.CompletableFuture.uniApplyStage(CompletableFuture.java:628)
              at java.util.concurrent.CompletableFuture.thenApply(CompletableFuture.java:1996)
              at org.apache.ratis.client.impl.OrderedAsync.sendRequest(OrderedAsync.java:239)
              at org.apache.ratis.client.impl.OrderedAsync.sendRequestWithRetry(OrderedAsync.java:203)
              at org.apache.ratis.util.SlidingWindow$Client.sendOrDelayRequest(SlidingWindow.java:303)
              at org.apache.ratis.util.SlidingWindow$Client.submitNewRequest(SlidingWindow.java:291)
              at org.apache.ratis.client.impl.OrderedAsync.send(OrderedAsync.java:171)
              at org.apache.ratis.client.impl.AsyncImpl.send(AsyncImpl.java:41)
              at org.apache.ratis.client.impl.AsyncImpl.send(AsyncImpl.java:46)
              at org.apache.hadoop.hdds.scm.XceiverClientRatis.lambda$sendRequestAsync$2(XceiverClientRatis.java:237)
              at org.apache.hadoop.hdds.tracing.TracingUtil.executeInSpan(TracingUtil.java:159)
              at org.apache.hadoop.hdds.tracing.TracingUtil.executeInNewSpan(TracingUtil.java:149)
              at org.apache.hadoop.hdds.scm.XceiverClientRatis.sendRequestAsync(XceiverClientRatis.java:222)
              at org.apache.hadoop.hdds.scm.XceiverClientRatis.sendCommandAsync(XceiverClientRatis.java:325)
              at org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.writeChunkAsync(ContainerProtocolCalls.java:441)
              at org.apache.hadoop.hdds.scm.storage.BlockOutputStream.writeChunkToContainer(BlockOutputStream.java:800)
              at org.apache.hadoop.hdds.scm.storage.BlockOutputStream.writeChunk(BlockOutputStream.java:573)
              at org.apache.hadoop.hdds.scm.storage.BlockOutputStream.handleFlushInternal(BlockOutputStream.java:620)
              at org.apache.hadoop.hdds.scm.storage.BlockOutputStream.handleFlush(BlockOutputStream.java:587)
              at org.apache.hadoop.hdds.scm.storage.RatisBlockOutputStream.hsync(RatisBlockOutputStream.java:139)
              at org.apache.hadoop.ozone.client.io.BlockOutputStreamEntry.hsync(BlockOutputStreamEntry.java:163)
              at org.apache.hadoop.ozone.client.io.KeyOutputStream.handleStreamAction(KeyOutputStream.java:524)
              at org.apache.hadoop.ozone.client.io.KeyOutputStream.handleFlushOrClose(KeyOutputStream.java:487)
              at org.apache.hadoop.ozone.client.io.KeyOutputStream.hsync(KeyOutputStream.java:457)
              at org.apache.hadoop.ozone.client.io.OzoneOutputStream.hsync(OzoneOutputStream.java:118)
              at org.apache.hadoop.fs.ozone.OzoneFSOutputStream.hsync(OzoneFSOutputStream.java:70)
              at org.apache.hadoop.fs.ozone.OzoneFSOutputStream.hflush(OzoneFSOutputStream.java:65)
              at org.apache.hadoop.fs.FSDataOutputStream.hflush(FSDataOutputStream.java:136)
              at org.apache.hadoop.hbase.io.asyncfs.WrapperAsyncFSOutput.flush0(WrapperAsyncFSOutput.java:92)
              at org.apache.hadoop.hbase.io.asyncfs.WrapperAsyncFSOutput.lambda$flush$0(WrapperAsyncFSOutput.java:113)
              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: org.apache.ratis.protocol.exceptions.AlreadyClosedException: client-546F8A3A61A4->0efdc82f-f19e-4bba-bc6b-ee61f973a24a is closed. 

      And just after this error HMaster aborts, exactly same pattern with the second HMaster too.

      2024-04-15 07:08:45,184 ERROR org.apache.hadoop.hbase.master.HMaster: ***** ABORTING master vc0122.xyz,22001,1713187869079: Log rolling failed *****
      java.lang.RuntimeException
              at org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.writeWALMetadata(AsyncProtobufLogWriter.java:217)
              at org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.writeMagicAndWALHeader(AsyncProtobufLogWriter.java:223)
              at org.apache.hadoop.hbase.regionserver.wal.AbstractProtobufLogWriter.init(AbstractProtobufLogWriter.java:164)
              at org.apache.hadoop.hbase.wal.AsyncFSWALProvider.createAsyncWriter(AsyncFSWALProvider.java:116)
              at org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(AsyncFSWAL.java:726)
              at org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(AsyncFSWAL.java:129)
              at org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.rollWriter(AbstractFSWAL.java:886)
              at org.apache.hadoop.hbase.wal.AbstractWALRoller$RollController.rollWal(AbstractWALRoller.java:304)
              at org.apache.hadoop.hbase.wal.AbstractWALRoller.run(AbstractWALRoller.java:211)
      2024-04-15 07:08:45,188 INFO org.apache.ranger.plugin.util.PolicyRefresher: PolicyRefresher(serviceName=cm_hbase).run(): interrupted! Exiting thread
      java.lang.InterruptedException
              at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.reportInterruptAfterWait(AbstractQueuedSynchronizer.java:2014)
              at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2048)
              at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
              at org.apache.ranger.plugin.util.PolicyRefresher.run(PolicyRefresher.java:208) 

      Attachments

        Issue Links

          Activity

            People

              ashishk Ashish Kumar
              pratyush.bhatt Pratyush Bhatt
              Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: