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:44:16 UTC

[1/5] accumulo git commit: ACCUMULO-4113 Fix incorrect usage of ByteBuffer

Repository: accumulo
Updated Branches:
  refs/heads/master a12b3741a -> 739f906cf


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/master
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);
+  }
+}


[4/5] accumulo git commit: Merge branch '1.7'

Posted by kt...@apache.org.
Merge branch '1.7'


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

Branch: refs/heads/master
Commit: 90a4c6159f7b8f1d855ad630a6744f4eacdb3ffb
Parents: a12b374 0720515
Author: Keith Turner <kt...@apache.org>
Authored: Wed Jan 20 13:27:50 2016 -0500
Committer: Keith Turner <kt...@apache.org>
Committed: Wed Jan 20 13:27:50 2016 -0500

----------------------------------------------------------------------
 .../impl/AuthenticationTokenIdentifier.java     |   3 +-
 .../accumulo/core/data/ArrayByteSequence.java   |  11 +-
 .../accumulo/core/util/ByteBufferUtil.java      |  45 ++++++--
 .../accumulo/core/util/ThriftMessageUtil.java   |   8 --
 .../core/util/UnsynchronizedBuffer.java         |   4 +-
 .../accumulo/core/util/ByteBufferUtilTest.java  | 112 +++++++++++++++++++
 .../core/util/ThriftMessageUtilTest.java        |   8 --
 .../org/apache/accumulo/proxy/ProxyServer.java  |   2 +-
 .../security/delegation/AuthenticationKey.java  |   3 +-
 9 files changed, 161 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/90a4c615/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
----------------------------------------------------------------------


[3/5] accumulo git commit: ACCUMULO-4113 Fix ByteBuffer misuse thats occurs in 1.7 and not in 1.6

Posted by kt...@apache.org.
ACCUMULO-4113 Fix ByteBuffer misuse thats occurs in 1.7 and not in 1.6


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

Branch: refs/heads/master
Commit: 07205156b6f87780d7ca04c475ccafdc0ba5fc36
Parents: 1b905ce
Author: Keith Turner <kt...@apache.org>
Authored: Wed Jan 20 13:24:41 2016 -0500
Committer: Keith Turner <kt...@apache.org>
Committed: Wed Jan 20 13:24:41 2016 -0500

----------------------------------------------------------------------
 .../client/impl/AuthenticationTokenIdentifier.java    |  3 ++-
 .../org/apache/accumulo/core/util/ByteBufferUtil.java | 11 +++++++++++
 .../apache/accumulo/core/util/ThriftMessageUtil.java  |  8 --------
 .../apache/accumulo/core/util/ByteBufferUtilTest.java | 14 ++++++++++++++
 .../accumulo/core/util/ThriftMessageUtilTest.java     |  8 --------
 .../java/org/apache/accumulo/proxy/ProxyServer.java   |  2 +-
 .../server/security/delegation/AuthenticationKey.java |  3 ++-
 7 files changed, 30 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/07205156/core/src/main/java/org/apache/accumulo/core/client/impl/AuthenticationTokenIdentifier.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/AuthenticationTokenIdentifier.java b/core/src/main/java/org/apache/accumulo/core/client/impl/AuthenticationTokenIdentifier.java
index 1f548bc..eaf0ce7 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/AuthenticationTokenIdentifier.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/AuthenticationTokenIdentifier.java
@@ -25,6 +25,7 @@ import java.nio.ByteBuffer;
 
 import org.apache.accumulo.core.client.admin.DelegationTokenConfig;
 import org.apache.accumulo.core.security.thrift.TAuthenticationTokenIdentifier;
+import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.core.util.ThriftMessageUtil;
 import org.apache.commons.lang.builder.HashCodeBuilder;
 import org.apache.hadoop.io.Text;
@@ -131,7 +132,7 @@ public class AuthenticationTokenIdentifier extends TokenIdentifier {
       ThriftMessageUtil msgUtil = new ThriftMessageUtil();
       ByteBuffer serialized = msgUtil.serialize(impl);
       out.writeInt(serialized.limit());
-      out.write(serialized.array(), serialized.arrayOffset(), serialized.limit());
+      ByteBufferUtil.write(out, serialized);
     } else {
       out.writeInt(0);
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/07205156/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 ad41540..85c3e12 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
@@ -18,6 +18,8 @@ package org.apache.accumulo.core.util;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 
+import java.io.DataOutput;
+import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -93,4 +95,13 @@ public class ByteBufferUtil {
       return ByteBuffer.wrap(bs.toArray());
     }
   }
+
+  public static void write(DataOutput out, ByteBuffer buffer) throws IOException {
+    if (buffer.hasArray()) {
+      out.write(buffer.array(), buffer.arrayOffset() + buffer.position(), buffer.remaining());
+    } else {
+      out.write(toBytes(buffer));
+    }
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/07205156/core/src/main/java/org/apache/accumulo/core/util/ThriftMessageUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/ThriftMessageUtil.java b/core/src/main/java/org/apache/accumulo/core/util/ThriftMessageUtil.java
index 611d21c..7405f27 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/ThriftMessageUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/ThriftMessageUtil.java
@@ -69,14 +69,6 @@ public class ThriftMessageUtil {
   }
 
   /**
-   * @see #deserialize(byte[], int, int, TBase)
-   */
-  public <T extends TBase<?,?>> T deserialize(ByteBuffer serialized, T instance) throws IOException {
-    checkNotNull(serialized);
-    return deserialize(serialized.array(), serialized.arrayOffset(), serialized.limit(), instance);
-  }
-
-  /**
    * Assumes the entire contents of the byte array compose the serialized {@code instance}
    *
    * @see #deserialize(byte[], int, int, TBase)

http://git-wip-us.apache.org/repos/asf/accumulo/blob/07205156/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
index f4ca0e9..f27a1ba 100644
--- a/core/src/test/java/org/apache/accumulo/core/util/ByteBufferUtilTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/util/ByteBufferUtilTest.java
@@ -17,6 +17,9 @@
 
 package org.apache.accumulo.core.util;
 
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Collections;
 import java.util.List;
@@ -40,6 +43,17 @@ public class ByteBufferUtilTest {
     Assert.assertEquals(expected, new String(bal.get(0), Charsets.UTF_8));
 
     Assert.assertEquals(new ArrayByteSequence(expected), new ArrayByteSequence(bb));
+
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    DataOutputStream dos = new DataOutputStream(baos);
+    try {
+      ByteBufferUtil.write(dos, bb);
+      dos.close();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+
+    Assert.assertEquals(expected, new String(baos.toByteArray(), Charsets.UTF_8));
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/accumulo/blob/07205156/core/src/test/java/org/apache/accumulo/core/util/ThriftMessageUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/util/ThriftMessageUtilTest.java b/core/src/test/java/org/apache/accumulo/core/util/ThriftMessageUtilTest.java
index 765d9ca..a84af88 100644
--- a/core/src/test/java/org/apache/accumulo/core/util/ThriftMessageUtilTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/util/ThriftMessageUtilTest.java
@@ -37,14 +37,6 @@ public class ThriftMessageUtilTest {
   }
 
   @Test
-  public void testSerialization() throws IOException {
-    ByteBuffer buff = util.serialize(msg);
-    TAuthenticationTokenIdentifier bbMsg = new TAuthenticationTokenIdentifier();
-    util.deserialize(buff, bbMsg);
-    assertEquals(msg, bbMsg);
-  }
-
-  @Test
   public void testSerializationAsByteArray() throws IOException {
     ByteBuffer buff = util.serialize(msg);
     TAuthenticationTokenIdentifier copy = new TAuthenticationTokenIdentifier();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/07205156/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
----------------------------------------------------------------------
diff --git a/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java b/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
index 88dad8d..d8b678a 100644
--- a/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
+++ b/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
@@ -226,7 +226,7 @@ public class ProxyServer implements AccumuloProxy.Iface {
   }
 
   protected Connector getConnector(ByteBuffer login) throws Exception {
-    String[] pair = new String(login.array(), login.position(), login.remaining(), UTF_8).split(",", 2);
+    String[] pair = ByteBufferUtil.toString(login).split(",", 2);
     if (instance.getInstanceID().equals(pair[0])) {
       Credentials creds = Credentials.deserialize(pair[1]);
       return instance.getConnector(creds.getPrincipal(), creds.getToken());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/07205156/server/base/src/main/java/org/apache/accumulo/server/security/delegation/AuthenticationKey.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/delegation/AuthenticationKey.java b/server/base/src/main/java/org/apache/accumulo/server/security/delegation/AuthenticationKey.java
index 35fd115..2907099 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/delegation/AuthenticationKey.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/delegation/AuthenticationKey.java
@@ -29,6 +29,7 @@ import java.util.Objects;
 import javax.crypto.SecretKey;
 
 import org.apache.accumulo.core.security.thrift.TAuthenticationKey;
+import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.core.util.ThriftMessageUtil;
 import org.apache.commons.lang.builder.HashCodeBuilder;
 import org.apache.hadoop.io.Writable;
@@ -124,7 +125,7 @@ public class AuthenticationKey implements Writable {
     ThriftMessageUtil util = new ThriftMessageUtil();
     ByteBuffer serialized = util.serialize(authKey);
     WritableUtils.writeVInt(out, serialized.limit() - serialized.arrayOffset());
-    out.write(serialized.array(), serialized.arrayOffset(), serialized.limit());
+    ByteBufferUtil.write(out, serialized);
   }
 
   @Override


[2/5] accumulo git commit: Merge branch '1.6' into 1.7

Posted by kt...@apache.org.
Merge branch '1.6' into 1.7

Conflicts:
	core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java


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

Branch: refs/heads/master
Commit: 1b905ce112565b9450fe66c762ea1cfa30212295
Parents: c9c60d3 27300d8
Author: Keith Turner <kt...@apache.org>
Authored: Wed Jan 20 12:51:51 2016 -0500
Committer: Keith Turner <kt...@apache.org>
Committed: Wed Jan 20 12:51:51 2016 -0500

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


http://git-wip-us.apache.org/repos/asf/accumulo/blob/1b905ce1/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java
index f894dbf,d040139..5d16541
--- 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,9 -21,8 +21,11 @@@ import static java.nio.charset.Standard
  import java.io.Serializable;
  import java.nio.ByteBuffer;
  
+ import org.apache.accumulo.core.util.ByteBufferUtil;
+ 
 +/**
 + * An implementation of {@link ByteSequence} that uses a backing byte array.
 + */
  public class ArrayByteSequence extends ByteSequence implements Serializable {
  
    private static final long serialVersionUID = 1L;
@@@ -79,24 -53,15 +81,23 @@@
      this(s.getBytes(UTF_8));
    }
  
 +  /**
 +   * Creates a new sequence based on a byte buffer. If the byte buffer has an array, that array (and the buffer's offset and limit) are used; otherwise, a new
 +   * backing array is created and a relative bulk get is performed to transfer the buffer's contents (starting at its current position and not beyond its
 +   * limit).
 +   *
 +   * @param buffer
 +   *          byte buffer
 +   */
    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;
      }
    }
  

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1b905ce1/core/src/main/java/org/apache/accumulo/core/util/ByteBufferUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1b905ce1/core/src/main/java/org/apache/accumulo/core/util/UnsynchronizedBuffer.java
----------------------------------------------------------------------


[5/5] accumulo git commit: ACCUMULO-4113 Fix ByteBuffer misuse thats occurs in 1.8 and not in 1.7

Posted by kt...@apache.org.
ACCUMULO-4113 Fix ByteBuffer misuse thats occurs in 1.8 and not in 1.7


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

Branch: refs/heads/master
Commit: 739f906cfba9ca300f615f820f588de3f195cff9
Parents: 90a4c61
Author: Keith Turner <kt...@apache.org>
Authored: Wed Jan 20 14:40:43 2016 -0500
Committer: Keith Turner <kt...@apache.org>
Committed: Wed Jan 20 14:40:43 2016 -0500

----------------------------------------------------------------------
 .../org/apache/accumulo/core/util/ByteBufferUtil.java     |  8 ++++++++
 .../org/apache/accumulo/core/util/ByteBufferUtilTest.java | 10 ++++++++++
 .../replication/BatchWriterReplicationReplayer.java       |  4 ++--
 .../org/apache/accumulo/test/proxy/SimpleProxyBase.java   |  8 ++++----
 4 files changed, 24 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/739f906c/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 85c3e12..006755f 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
@@ -18,6 +18,7 @@ package org.apache.accumulo.core.util;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 
+import java.io.ByteArrayInputStream;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -102,6 +103,13 @@ public class ByteBufferUtil {
     } else {
       out.write(toBytes(buffer));
     }
+  }
 
+  public static ByteArrayInputStream toByteArrayInputStream(ByteBuffer buffer){
+    if (buffer.hasArray()) {
+      return new ByteArrayInputStream(buffer.array(), buffer.arrayOffset() + buffer.position(), buffer.remaining());
+    } else {
+      return new ByteArrayInputStream(toBytes(buffer));
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/739f906c/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
index f27a1ba..d6e1b35 100644
--- a/core/src/test/java/org/apache/accumulo/core/util/ByteBufferUtilTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/util/ByteBufferUtilTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.accumulo.core.util;
 
+import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
@@ -54,6 +55,15 @@ public class ByteBufferUtilTest {
     }
 
     Assert.assertEquals(expected, new String(baos.toByteArray(), Charsets.UTF_8));
+
+    ByteArrayInputStream bais = ByteBufferUtil.toByteArrayInputStream(bb);
+    byte[] buffer = new byte[expected.length()];
+    try {
+      bais.read(buffer);
+      Assert.assertEquals(expected, new String(buffer, Charsets.UTF_8));
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/accumulo/blob/739f906c/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/BatchWriterReplicationReplayer.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/BatchWriterReplicationReplayer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/BatchWriterReplicationReplayer.java
index 8a80ea3..e5e9e80 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/BatchWriterReplicationReplayer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/BatchWriterReplicationReplayer.java
@@ -16,7 +16,6 @@
  */
 package org.apache.accumulo.tserver.replication;
 
-import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -40,6 +39,7 @@ import org.apache.accumulo.core.replication.thrift.RemoteReplicationErrorCode;
 import org.apache.accumulo.core.replication.thrift.RemoteReplicationException;
 import org.apache.accumulo.core.replication.thrift.WalEdits;
 import org.apache.accumulo.core.security.ColumnVisibility;
+import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.server.data.ServerMutation;
 import org.apache.accumulo.tserver.logger.LogFileKey;
 import org.apache.accumulo.tserver.logger.LogFileValue;
@@ -64,7 +64,7 @@ public class BatchWriterReplicationReplayer implements AccumuloReplicationReplay
     long mutationsApplied = 0l;
     try {
       for (ByteBuffer edit : data.getEdits()) {
-        DataInputStream dis = new DataInputStream(new ByteArrayInputStream(edit.array()));
+        DataInputStream dis = new DataInputStream(ByteBufferUtil.toByteArrayInputStream(edit));
         try {
           key.readFields(dis);
           // TODO this is brittle because AccumuloReplicaSystem isn't actually calling LogFileValue.write, but we're expecting

http://git-wip-us.apache.org/repos/asf/accumulo/blob/739f906c/test/src/main/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java b/test/src/main/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
index 8c6811f..5c892ef 100644
--- a/test/src/main/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
+++ b/test/src/main/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.test.proxy;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -126,7 +127,6 @@ import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.Iterators;
 import com.google.common.net.HostAndPort;
-import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 
 /**
  * Call every method on the proxy and try to verify that it works.
@@ -1188,7 +1188,7 @@ public abstract class SimpleProxyBase extends SharedMiniClusterBase {
     if (!isKerberosEnabled()) {
       password = s2bb("");
       client.changeLocalUserPassword(creds, user, password);
-      assertTrue(client.authenticateUser(creds, user, s2pp(new String(password.array(), password.position(), password.limit(), UTF_8))));
+      assertTrue(client.authenticateUser(creds, user, s2pp(ByteBufferUtil.toString(password))));
     }
 
     if (isKerberosEnabled()) {
@@ -1207,7 +1207,7 @@ public abstract class SimpleProxyBase extends SharedMiniClusterBase {
       }
     } else {
       // check login with new password
-      client.login(user, s2pp(new String(password.array(), password.position(), password.limit(), UTF_8)));
+      client.login(user, s2pp(ByteBufferUtil.toString(password)));
     }
   }
 
@@ -1242,7 +1242,7 @@ public abstract class SimpleProxyBase extends SharedMiniClusterBase {
       userName = getUniqueNames(1)[0];
       // create a user
       client.createLocalUser(creds, userName, password);
-      user = client.login(userName, s2pp(new String(password.array(), password.position(), password.limit(), UTF_8)));
+      user = client.login(userName, s2pp(ByteBufferUtil.toString(password)));
     }
 
     // check permission failure