You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zg...@apache.org on 2020/06/04 01:06:38 UTC

[hbase] branch branch-1 updated: HBASE-24501 Backport to branch-1 HBASE-15785 Unnecessary lock in ByteBufferArray (#1842)

This is an automated email from the ASF dual-hosted git repository.

zghao pushed a commit to branch branch-1
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-1 by this push:
     new 4118383  HBASE-24501 Backport to branch-1 HBASE-15785 Unnecessary lock in ByteBufferArray (#1842)
4118383 is described below

commit 4118383fb186592c758696339d745c4862363c1d
Author: thangTang <ta...@gmail.com>
AuthorDate: Thu Jun 4 09:06:22 2020 +0800

    HBASE-24501 Backport to branch-1 HBASE-15785 Unnecessary lock in ByteBufferArray (#1842)
    
    Signed-off-by: Guanghao Zhang <zg...@apache.org>
---
 .../apache/hadoop/hbase/util/ByteBufferArray.java  | 49 +++++++++-------------
 1 file changed, 20 insertions(+), 29 deletions(-)

diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferArray.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferArray.java
index d9f8dcf..4404926 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferArray.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferArray.java
@@ -19,8 +19,6 @@
 package org.apache.hadoop.hbase.util;
 
 import java.nio.ByteBuffer;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -38,7 +36,6 @@ public final class ByteBufferArray {
 
   static final int DEFAULT_BUFFER_SIZE = 4 * 1024 * 1024;
   private ByteBuffer buffers[];
-  private Lock locks[];
   private int bufferSize;
   private int bufferCount;
 
@@ -58,9 +55,7 @@ public final class ByteBufferArray {
         + ", sizePerBuffer=" + StringUtils.byteDesc(bufferSize) + ", count="
         + bufferCount + ", direct=" + directByteBuffer);
     buffers = new ByteBuffer[bufferCount + 1];
-    locks = new Lock[bufferCount + 1];
     for (int i = 0; i <= bufferCount; i++) {
-      locks[i] = new ReentrantLock();
       if (i < bufferCount) {
         buffers[i] = directByteBuffer ? ByteBuffer.allocateDirect(bufferSize)
             : ByteBuffer.allocate(bufferSize);
@@ -102,8 +97,8 @@ public final class ByteBufferArray {
 
   private final static Visitor GET_MULTIPLE_VISTOR = new Visitor() {
     @Override
-    public void visit(ByteBuffer bb, byte[] array, int arrayIdx, int len) {
-      bb.get(array, arrayIdx, len);
+    public void visit(ByteBuffer bb, int pos, byte[] array, int arrayIdx, int len) {
+      ByteBufferUtils.copyFromBufferToArray(array, bb, pos, arrayIdx, len);
     }
   };
 
@@ -131,8 +126,8 @@ public final class ByteBufferArray {
 
   private final static Visitor PUT_MULTIPLE_VISITOR = new Visitor() {
     @Override
-    public void visit(ByteBuffer bb, byte[] array, int arrayIdx, int len) {
-      bb.put(array, arrayIdx, len);
+    public void visit(ByteBuffer bb, int pos, byte[] array, int arrayIdx, int len) {
+      ByteBufferUtils.copyFromArrayToBuffer(bb, pos, array, arrayIdx, len);
     }
   };
 
@@ -142,11 +137,12 @@ public final class ByteBufferArray {
      * bytes from the buffer to the destination array, else if it is a write
      * action, we will transfer the bytes from the source array to the buffer
      * @param bb byte buffer
+     * @param pos Start position in ByteBuffer
      * @param array a source or destination byte array
      * @param arrayOffset offset of the byte array
      * @param len read/write length
      */
-    void visit(ByteBuffer bb, byte[] array, int arrayOffset, int len);
+    void visit(ByteBuffer bb, int pos, byte[] array, int arrayOffset, int len);
   }
 
   /**
@@ -169,7 +165,7 @@ public final class ByteBufferArray {
     assert startBuffer >= 0 && startBuffer < bufferCount;
     assert (endBuffer >= 0 && endBuffer < bufferCount)
         || (endBuffer == bufferCount && endOffset == 0);
-    if (startBuffer >= locks.length || startBuffer < 0) {
+    if (startBuffer >= buffers.length || startBuffer < 0) {
       String msg = "Failed multiple, start=" + start + ",startBuffer="
           + startBuffer + ",bufferSize=" + bufferSize;
       LOG.error(msg);
@@ -177,26 +173,21 @@ public final class ByteBufferArray {
     }
     int srcIndex = 0, cnt = -1;
     for (int i = startBuffer; i <= endBuffer; ++i) {
-      Lock lock = locks[i];
-      lock.lock();
-      try {
-        ByteBuffer bb = buffers[i];
-        if (i == startBuffer) {
-          cnt = bufferSize - startOffset;
-          if (cnt > len) cnt = len;
-          bb.limit(startOffset + cnt).position(startOffset);
-        } else if (i == endBuffer) {
-          cnt = endOffset;
-          bb.limit(cnt).position(0);
-        } else {
-          cnt = bufferSize;
-          bb.limit(cnt).position(0);
+      ByteBuffer bb = buffers[i].duplicate();
+      int pos = 0;
+      if (i == startBuffer) {
+        cnt = bufferSize - startOffset;
+        if (cnt > len) {
+          cnt = len;
         }
-        visitor.visit(bb, array, srcIndex + arrayOffset, cnt);
-        srcIndex += cnt;
-      } finally {
-        lock.unlock();
+        pos = startOffset;
+      } else if (i == endBuffer) {
+        cnt = endOffset;
+      } else {
+        cnt = bufferSize;
       }
+      visitor.visit(bb, pos, array, srcIndex + arrayOffset, cnt);
+      srcIndex += cnt;
     }
     assert srcIndex == len;
   }