Uploaded image for project: 'HBase'
  1. HBase
  2. HBASE-27926

DBB release too early for replication

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Open
    • Major
    • Resolution: Unresolved
    • 3.0.0-alpha-4, 2.4.17, 2.5.5
    • None
    • Replication
    • None

    Description

      When the RS of destination cluster acts as client to forward the replicated entries and encounters exception, the DBB will be released too early by calling RpcResponse#done() in NettyRpcServerResponseEncoder.

      The coredump and log details are as follows,

      Stack: [0x00007f92d9e6d000,0x00007f92d9f6e000],  sp=0x00007f92d9f6be18,  free space=1019kNative frames: (J=compiled Java code, j=interpreted, Vv=VM code, C=native code)C  [libc.so.6+0x89db4]  _wordcopy_fwd_dest_aligned+0xd4
      Java frames: (J=compiled Java code, j=interpreted, Vv=VM code)J 3297  sun.misc.Unsafe.copyMemory(Ljava/lang/Object;JLjava/lang/Object;JJ)V (0 bytes) @ 0x00007fad7d9aa267 [0x00007fad7d9aa200+0x67]j  org.apache.hadoop.hbase.util.UnsafeAccess.unsafeCopy(Ljava/lang/Object;JLjava/lang/Object;JJ)V+36j  org.apache.hadoop.hbase.util.UnsafeAccess.copy(Ljava/nio/ByteBuffer;I[BII)V+69j  org.apache.hadoop.hbase.util.ByteBufferUtils.copyFromBufferToArray([BLjava/nio/ByteBuffer;III)V+39j  org.apache.hadoop.hbase.CellUtil.copyQualifierTo(Lorg/apache/hadoop/hbase/Cell;[BI)I+31J 15658 C1 org.apache.hadoop.hbase.CellUtil.cloneQualifier(Lorg/apache/hadoop/hbase/Cell;)[B (18 bytes) @ 0x00007fad7e9a6c2c [0x00007fad7e9a6aa0+0x18c]j  org.apache.hadoop.hbase.ByteBufferKeyValue.getQualifierArray()[B+1j  org.apache.hadoop.hbase.client.Mutation.cellToStringMap(Lorg/apache/hadoop/hbase/Cell;)Ljava/util/Map;+12j  org.apache.hadoop.hbase.client.Mutation.toMap(I)Ljava/util/Map;+189j  org.apache.hadoop.hbase.client.Operation.toJSON(I)Ljava/lang/String;+2j  org.apache.hadoop.hbase.client.Operation.toString(I)Ljava/lang/String;+2j  org.apache.hadoop.hbase.client.Operation.toString()Ljava/lang/String;+2J 8353 C2 java.lang.StringBuilder.append(Ljava/lang/Object;)Ljava/lang/StringBuilder; (9 bytes) @ 0x00007fad7ea0a1bc [0x00007fad7ea0a180+0x3c]j  org.apache.hadoop.hbase.client.AsyncRequestFutureImpl.manageLocationError(Lorg/apache/hadoop/hbase/client/Action;Ljava/lang/Exception;)V+28j  org.apache.hadoop.hbase.client.AsyncRequestFutureImpl.groupAndSendMultiAction(Ljava/util/List;I)V+163J 23463 C2 org.apache.hadoop.hbase.client.AsyncRequestFutureImpl.resubmit(Lorg/apache/hadoop/hbase/ServerName;Ljava/util/List;IILjava/lang/Throwable;)V (214 bytes) @ 0x00007fad80effb54 [0x00007fad80eff7a0+0x3b4]J 19097 C2 org.apache.hadoop.hbase.client.AsyncRequestFutureImpl.receiveGlobalFailure(Lorg/apache/hadoop/hbase/client/MultiAction;Lorg/apache/hadoop/hbase/ServerName;ILjava/lang/Throwable;Z)V (312 bytes) @ 0x00007fad7ff53370 [0x00007fad7ff52fa0+0x3d0]J 20201 C1 org.apache.hadoop.hbase.client.AsyncRequestFutureImpl.access$1600(Lorg/apache/hadoop/hbase/client/AsyncRequestFutureImpl;Lorg/apache/hadoop/hbase/client/MultiAction;Lorg/apache/hadoop/hbase/ServerName;ILjava/lang/Throwable;Z)V (12 bytes) @ 0x00007fad803f31dc [0x00007fad803f3180+0x5c]J 18619 C2 org.apache.hadoop.hbase.client.AsyncRequestFutureImpl$SingleServerRequestRunnable.run()V (677 bytes) @ 0x00007fad7f40a8b4 [0x00007fad7f409160+0x1754]J 13220 C2 java.util.concurrent.ThreadPoolExecutor.runWorker(Ljava/util/concurrent/ThreadPoolExecutor$Worker;)V (225 bytes) @ 0x00007fad7f3b3a28 [0x00007fad7f3b38a0+0x188]J 10884 C1 java.util.concurrent.ThreadPoolExecutor$Worker.run()V (9 bytes) @ 0x00007fad7db53c44 [0x00007fad7db53b40+0x104]J 7961 C1 java.lang.Thread.run()V (17 bytes) @ 0x00007fad7d61bbfc [0x00007fad7d61bac0+0x13c]v  ~StubRoutines::call_stubStack: [0x00007f92d9e6d000,0x00007f92d9f6e000],  sp=0x00007f92d9f6be18,  free space=1019kNative frames: (J=compiled Java code, j=interpreted, Vv=VM code, C=native code)C  [libc.so.6+0x89db4]  _wordcopy_fwd_dest_aligned+0xd4
      Java frames: (J=compiled Java code, j=interpreted, Vv=VM code)J 3297  sun.misc.Unsafe.copyMemory(Ljava/lang/Object;JLjava/lang/Object;JJ)V (0 bytes) @ 0x00007fad7d9aa267 [0x00007fad7d9aa200+0x67]j  org.apache.hadoop.hbase.util.UnsafeAccess.unsafeCopy(Ljava/lang/Object;JLjava/lang/Object;JJ)V+36j  org.apache.hadoop.hbase.util.UnsafeAccess.copy(Ljava/nio/ByteBuffer;I[BII)V+69j  org.apache.hadoop.hbase.util.ByteBufferUtils.copyFromBufferToArray([BLjava/nio/ByteBuffer;III)V+39j  org.apache.hadoop.hbase.CellUtil.copyQualifierTo(Lorg/apache/hadoop/hbase/Cell;[BI)I+31J 15658 C1 org.apache.hadoop.hbase.CellUtil.cloneQualifier(Lorg/apache/hadoop/hbase/Cell;)[B (18 bytes) @ 0x00007fad7e9a6c2c [0x00007fad7e9a6aa0+0x18c]j  org.apache.hadoop.hbase.ByteBufferKeyValue.getQualifierArray()[B+1j  org.apache.hadoop.hbase.client.Mutation.cellToStringMap(Lorg/apache/hadoop/hbase/Cell;)Ljava/util/Map;+12j  org.apache.hadoop.hbase.client.Mutation.toMap(I)Ljava/util/Map;+189j  org.apache.hadoop.hbase.client.Operation.toJSON(I)Ljava/lang/String;+2j  org.apache.hadoop.hbase.client.Operation.toString(I)Ljava/lang/String;+2j  org.apache.hadoop.hbase.client.Operation.toString()Ljava/lang/String;+2J 8353 C2 java.lang.StringBuilder.append(Ljava/lang/Object;)Ljava/lang/StringBuilder; (9 bytes) @ 0x00007fad7ea0a1bc [0x00007fad7ea0a180+0x3c]j  org.apache.hadoop.hbase.client.AsyncRequestFutureImpl.manageLocationError(Lorg/apache/hadoop/hbase/client/Action;Ljava/lang/Exception;)V+28j  org.apache.hadoop.hbase.client.AsyncRequestFutureImpl.groupAndSendMultiAction(Ljava/util/List;I)V+163J 23463 C2 org.apache.hadoop.hbase.client.AsyncRequestFutureImpl.resubmit(Lorg/apache/hadoop/hbase/ServerName;Ljava/util/List;IILjava/lang/Throwable;)V (214 bytes) @ 0x00007fad80effb54 [0x00007fad80eff7a0+0x3b4]J 19097 C2 org.apache.hadoop.hbase.client.AsyncRequestFutureImpl.receiveGlobalFailure(Lorg/apache/hadoop/hbase/client/MultiAction;Lorg/apache/hadoop/hbase/ServerName;ILjava/lang/Throwable;Z)V (312 bytes) @ 0x00007fad7ff53370 [0x00007fad7ff52fa0+0x3d0]J 20201 C1 org.apache.hadoop.hbase.client.AsyncRequestFutureImpl.access$1600(Lorg/apache/hadoop/hbase/client/AsyncRequestFutureImpl;Lorg/apache/hadoop/hbase/client/MultiAction;Lorg/apache/hadoop/hbase/ServerName;ILjava/lang/Throwable;Z)V (12 bytes) @ 0x00007fad803f31dc [0x00007fad803f3180+0x5c]J 18619 C2 org.apache.hadoop.hbase.client.AsyncRequestFutureImpl$SingleServerRequestRunnable.run()V (677 bytes) @ 0x00007fad7f40a8b4 [0x00007fad7f409160+0x1754]J 13220 C2 java.util.concurrent.ThreadPoolExecutor.runWorker(Ljava/util/concurrent/ThreadPoolExecutor$Worker;)V (225 bytes) @ 0x00007fad7f3b3a28 [0x00007fad7f3b38a0+0x188]J 10884 C1 java.util.concurrent.ThreadPoolExecutor$Worker.run()V (9 bytes) @ 0x00007fad7db53c44 [0x00007fad7db53b40+0x104]J 7961 C1 java.lang.Thread.run()V (17 bytes) @ 0x00007fad7d61bbfc [0x00007fad7d61bac0+0x13c]v  ~StubRoutines::call_stub 
      2023-06-13 09:04:08,106 ERROR [RpcServer.replication.FPBQ.Fifo.handler=9,queue=0,port=16020] regionserver.ReplicationSink: Unable to accept edit because:
      org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException: Failed 816 actions: NotServingRegionException: 816 times, servers with issues: XXXXXX,XXXX,1686558427722,
              at org.apache.hadoop.hbase.client.BatchErrors.makeException(BatchErrors.java:53)
              at org.apache.hadoop.hbase.client.AsyncRequestFutureImpl.getErrors(AsyncRequestFutureImpl.java:1309)
              at org.apache.hadoop.hbase.client.HTable.batch(HTable.java:936)
              at org.apache.hadoop.hbase.client.HTable.batch(HTable.java:927)
              at org.apache.hadoop.hbase.replication.regionserver.ReplicationSink.batch(ReplicationSink.java:434)
              at org.apache.hadoop.hbase.replication.regionserver.ReplicationSink.replicateEntries(ReplicationSink.java:261)
              at org.apache.hadoop.hbase.replication.regionserver.Replication.replicateLogEntries(Replication.java:187)
              at org.apache.hadoop.hbase.regionserver.RSRpcServices.replicateWALEntry(RSRpcServices.java:2094)
              at org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos$AdminService$2.callBlockingMethod(AdminProtos.java:32335)
              at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:396)
              at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:135)
              at org.apache.hadoop.hbase.ipc.RpcHandler.run(RpcHandler.java:103)
              at org.apache.hadoop.hbase.ipc.RpcHandler.run(RpcHandler.java:83) 

      Attachments

        Issue Links

          Activity

            People

              Xiaolin Ha Xiaolin Ha
              Xiaolin Ha Xiaolin Ha
              Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

                Created:
                Updated: