You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by kt...@apache.org on 2016/01/20 20:43:05 UTC

accumulo git commit: ACCUMULO-4113 Fix incorrect usage of ByteBuffer

Repository: accumulo
Updated Branches:
  refs/heads/1.6 a2c2d38aa -> 27300d81e


ACCUMULO-4113 Fix incorrect usage of ByteBuffer


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/27300d81
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/27300d81
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/27300d81

Branch: refs/heads/1.6
Commit: 27300d81eaee14516e705d0032bc0bdc46880d4c
Parents: a2c2d38
Author: Keith Turner <kt...@apache.org>
Authored: Wed Jan 20 12:36:20 2016 -0500
Committer: Keith Turner <kt...@apache.org>
Committed: Wed Jan 20 12:36:20 2016 -0500

----------------------------------------------------------------------
 .../accumulo/core/data/ArrayByteSequence.java   | 12 ++-
 .../accumulo/core/util/ByteBufferUtil.java      | 34 +++++--
 .../core/util/UnsynchronizedBuffer.java         |  4 +-
 .../accumulo/core/util/ByteBufferUtilTest.java  | 98 ++++++++++++++++++++
 4 files changed, 132 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/27300d81/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java b/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java
index d9ddc67..d040139 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java
@@ -21,6 +21,8 @@ import static com.google.common.base.Charsets.UTF_8;
 import java.io.Serializable;
 import java.nio.ByteBuffer;
 
+import org.apache.accumulo.core.util.ByteBufferUtil;
+
 public class ArrayByteSequence extends ByteSequence implements Serializable {
 
   private static final long serialVersionUID = 1L;
@@ -52,15 +54,14 @@ public class ArrayByteSequence extends ByteSequence implements Serializable {
   }
 
   public ArrayByteSequence(ByteBuffer buffer) {
-    this.length = buffer.remaining();
-
     if (buffer.hasArray()) {
       this.data = buffer.array();
-      this.offset = buffer.position();
+      this.offset = buffer.position() + buffer.arrayOffset();
+      this.length = buffer.remaining();
     } else {
-      this.data = new byte[length];
       this.offset = 0;
-      buffer.get(data);
+      this.data = ByteBufferUtil.toBytes(buffer);
+      this.length = data.length;
     }
   }
 
@@ -118,6 +119,7 @@ public class ArrayByteSequence extends ByteSequence implements Serializable {
     return copy;
   }
 
+  @Override
   public String toString() {
     return new String(data, offset, length, UTF_8);
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/27300d81/core/src/main/java/org/apache/accumulo/core/util/ByteBufferUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/ByteBufferUtil.java b/core/src/main/java/org/apache/accumulo/core/util/ByteBufferUtil.java
index be5cddf..cdde05c 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/ByteBufferUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/ByteBufferUtil.java
@@ -31,7 +31,15 @@ public class ByteBufferUtil {
   public static byte[] toBytes(ByteBuffer buffer) {
     if (buffer == null)
       return null;
-    return Arrays.copyOfRange(buffer.array(), buffer.position(), buffer.limit());
+    if (buffer.hasArray()) {
+      // did not use buffer.get() because it changes the position
+      return Arrays.copyOfRange(buffer.array(), buffer.position() + buffer.arrayOffset(), buffer.limit() + buffer.arrayOffset());
+    } else {
+      byte[] data = new byte[buffer.remaining()];
+      // duplicate inorder to avoid changing position
+      buffer.duplicate().get(data);
+      return data;
+    }
   }
 
   public static List<ByteBuffer> toByteBuffers(Collection<byte[]> bytesList) {
@@ -47,23 +55,32 @@ public class ByteBufferUtil {
   public static List<byte[]> toBytesList(Collection<ByteBuffer> bytesList) {
     if (bytesList == null)
       return null;
-    ArrayList<byte[]> result = new ArrayList<byte[]>();
+    ArrayList<byte[]> result = new ArrayList<byte[]>(bytesList.size());
     for (ByteBuffer bytes : bytesList) {
       result.add(toBytes(bytes));
     }
     return result;
   }
 
-  public static Text toText(ByteBuffer bytes) {
-    if (bytes == null)
+  public static Text toText(ByteBuffer byteBuffer) {
+    if (byteBuffer == null)
       return null;
-    Text result = new Text();
-    result.set(bytes.array(), bytes.position(), bytes.remaining());
-    return result;
+
+    if (byteBuffer.hasArray()) {
+      Text result = new Text();
+      result.set(byteBuffer.array(), byteBuffer.arrayOffset() + byteBuffer.position(), byteBuffer.remaining());
+      return result;
+    } else {
+      return new Text(toBytes(byteBuffer));
+    }
   }
 
   public static String toString(ByteBuffer bytes) {
-    return new String(bytes.array(), bytes.position(), bytes.remaining(), UTF_8);
+    if (bytes.hasArray()) {
+      return new String(bytes.array(), bytes.arrayOffset() + bytes.position(), bytes.remaining(), UTF_8);
+    } else {
+      return new String(toBytes(bytes), UTF_8);
+    }
   }
 
   public static ByteBuffer toByteBuffers(ByteSequence bs) {
@@ -73,7 +90,6 @@ public class ByteBufferUtil {
     if (bs.isBackedByArray()) {
       return ByteBuffer.wrap(bs.getBackingArray(), bs.offset(), bs.length());
     } else {
-      // TODO create more efficient impl
       return ByteBuffer.wrap(bs.toArray());
     }
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/27300d81/core/src/main/java/org/apache/accumulo/core/util/UnsynchronizedBuffer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/UnsynchronizedBuffer.java b/core/src/main/java/org/apache/accumulo/core/util/UnsynchronizedBuffer.java
index f353613..b776553 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/UnsynchronizedBuffer.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/UnsynchronizedBuffer.java
@@ -122,8 +122,8 @@ public class UnsynchronizedBuffer {
         offset = buffer.arrayOffset() + buffer.position();
         data = buffer.array();
       } else {
-        data = new byte[buffer.remaining()];
-        buffer.get(data);
+        offset = 0;
+        data = ByteBufferUtil.toBytes(buffer);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/27300d81/core/src/test/java/org/apache/accumulo/core/util/ByteBufferUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/util/ByteBufferUtilTest.java b/core/src/test/java/org/apache/accumulo/core/util/ByteBufferUtilTest.java
new file mode 100644
index 0000000..f4ca0e9
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/util/ByteBufferUtilTest.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.accumulo.core.util;
+
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.accumulo.core.data.ArrayByteSequence;
+import org.apache.hadoop.io.Text;
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.google.common.base.Charsets;
+
+public class ByteBufferUtilTest {
+
+  private static void assertEquals(String expected, ByteBuffer bb) {
+    Assert.assertEquals(new Text(expected), ByteBufferUtil.toText(bb));
+    Assert.assertEquals(expected, new String(ByteBufferUtil.toBytes(bb), Charsets.UTF_8));
+    Assert.assertEquals(expected, ByteBufferUtil.toString(bb));
+
+    List<byte[]> bal = ByteBufferUtil.toBytesList(Collections.singletonList(bb));
+    Assert.assertEquals(1, bal.size());
+    Assert.assertEquals(expected, new String(bal.get(0), Charsets.UTF_8));
+
+    Assert.assertEquals(new ArrayByteSequence(expected), new ArrayByteSequence(bb));
+  }
+
+  @Test
+  public void testNonZeroArrayOffset() {
+    byte[] data = "0123456789".getBytes(Charsets.UTF_8);
+
+    ByteBuffer bb1 = ByteBuffer.wrap(data, 3, 4);
+
+    // create a ByteBuffer with a non-zero array offset
+    ByteBuffer bb2 = bb1.slice();
+
+    // The purpose of this test is to ensure ByteBufferUtil code works when arrayOffset is non-zero. The following asserts are not to test ByteBuffer, but
+    // ensure the behavior of slice() is as expected.
+
+    Assert.assertEquals(3, bb2.arrayOffset());
+    Assert.assertEquals(0, bb2.position());
+    Assert.assertEquals(4, bb2.limit());
+
+    // start test with non zero arrayOffset
+    assertEquals("3456", bb2);
+
+    // read one byte from byte buffer... this should cause position to be non-zero in addition to array offset
+    bb2.get();
+    assertEquals("456", bb2);
+
+  }
+
+  @Test
+  public void testZeroArrayOffsetAndNonZeroPosition() {
+    byte[] data = "0123456789".getBytes(Charsets.UTF_8);
+    ByteBuffer bb1 = ByteBuffer.wrap(data, 3, 4);
+
+    assertEquals("3456", bb1);
+  }
+
+  @Test
+  public void testZeroArrayOffsetAndPosition() {
+    byte[] data = "0123456789".getBytes(Charsets.UTF_8);
+    ByteBuffer bb1 = ByteBuffer.wrap(data, 0, 4);
+    assertEquals("0123", bb1);
+  }
+
+  @Test
+  public void testDirectByteBuffer() {
+    // allocate direct so it does not have a backing array
+    ByteBuffer bb = ByteBuffer.allocateDirect(10);
+    bb.put("0123456789".getBytes(Charsets.UTF_8));
+    bb.rewind();
+
+    assertEquals("0123456789", bb);
+
+    // advance byte buffer position
+    bb.get();
+    assertEquals("123456789", bb);
+  }
+}