Repository: hbase
Updated Branches:
  refs/heads/master a249989b9 -> 834f87b23


http://git-wip-us.apache.org/repos/asf/hbase/blob/834f87b2/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV2.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV2.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV2.java
index 253dff8..429f630 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV2.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV2.java
@@ -27,7 +27,6 @@ import static org.junit.Assert.assertTrue;
 import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Random;
@@ -45,6 +44,7 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
 import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
+import org.apache.hadoop.hbase.nio.ByteBuff;
 import org.apache.hadoop.hbase.testclassification.IOTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -194,7 +194,7 @@ public class TestHFileWriterV2 {
         assertFalse(block.isUnpacked());
         block = block.unpack(meta, blockReader);
       }
-      ByteBuffer buf = block.getBufferWithoutHeader();
+      ByteBuff buf = block.getBufferWithoutHeader();
       while (buf.hasRemaining()) {
         int keyLen = buf.getInt();
         int valueLen = buf.getInt();
@@ -241,7 +241,7 @@ public class TestHFileWriterV2 {
         .unpack(meta, blockReader);
       assertEquals(BlockType.META, block.getBlockType());
       Text t = new Text();
-      ByteBuffer buf = block.getBufferWithoutHeader();
+      ByteBuff buf = block.getBufferWithoutHeader();
       if (Writables.getWritable(buf.array(), buf.arrayOffset(), buf.limit(), 
t) == null) {
         throw new IOException("Failed to deserialize block " + this + " into a 
" + t.getClass().getSimpleName());
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/834f87b2/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java
index 9adeaca..979c9f6 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java
@@ -26,7 +26,6 @@ import static org.junit.Assert.assertTrue;
 import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
@@ -47,6 +46,7 @@ import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
 import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
+import org.apache.hadoop.hbase.nio.ByteBuff;
 import org.apache.hadoop.hbase.testclassification.IOTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -220,7 +220,7 @@ public class TestHFileWriterV3 {
       HFileBlock block = blockReader.readBlockData(curBlockPos, -1, -1, false)
         .unpack(context, blockReader);
       assertEquals(BlockType.DATA, block.getBlockType());
-      ByteBuffer buf = block.getBufferWithoutHeader();
+      ByteBuff buf = block.getBufferWithoutHeader();
       int keyLen = -1;
       while (buf.hasRemaining()) {
 
@@ -282,7 +282,7 @@ public class TestHFileWriterV3 {
         .unpack(context, blockReader);
       assertEquals(BlockType.META, block.getBlockType());
       Text t = new Text();
-      ByteBuffer buf = block.getBufferWithoutHeader();
+      ByteBuff buf = block.getBufferWithoutHeader();
       if (Writables.getWritable(buf.array(), buf.arrayOffset(), buf.limit(), 
t) == null) {
         throw new IOException("Failed to deserialize block " + this + " into a 
" + t.getClass().getSimpleName());
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/834f87b2/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestByteBufferIOEngine.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestByteBufferIOEngine.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestByteBufferIOEngine.java
index 511f942..f68271e 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestByteBufferIOEngine.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestByteBufferIOEngine.java
@@ -22,6 +22,8 @@ import static org.junit.Assert.assertTrue;
 
 import java.nio.ByteBuffer;
 
+import org.apache.hadoop.hbase.nio.ByteBuff;
+import org.apache.hadoop.hbase.nio.MultiByteBuff;
 import org.apache.hadoop.hbase.testclassification.IOTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.junit.Test;
@@ -73,4 +75,47 @@ public class TestByteBufferIOEngine {
     assert testOffsetAtStartNum == 0;
     assert testOffsetAtEndNum == 0;
   }
+
+  @Test
+  public void testByteBufferIOEngineWithMBB() throws Exception {
+    int capacity = 32 * 1024 * 1024; // 32 MB
+    int testNum = 100;
+    int maxBlockSize = 64 * 1024;
+    ByteBufferIOEngine ioEngine = new ByteBufferIOEngine(capacity, false);
+    int testOffsetAtStartNum = testNum / 10;
+    int testOffsetAtEndNum = testNum / 10;
+    for (int i = 0; i < testNum; i++) {
+      byte val = (byte) (Math.random() * 255);
+      int blockSize = (int) (Math.random() * maxBlockSize);
+      if (blockSize == 0) {
+        blockSize = 1;
+      }
+      byte[] byteArray = new byte[blockSize];
+      for (int j = 0; j < byteArray.length; ++j) {
+        byteArray[j] = val;
+      }
+      ByteBuffer srcBuffer = ByteBuffer.wrap(byteArray);
+      int offset = 0;
+      if (testOffsetAtStartNum > 0) {
+        testOffsetAtStartNum--;
+        offset = 0;
+      } else if (testOffsetAtEndNum > 0) {
+        testOffsetAtEndNum--;
+        offset = capacity - blockSize;
+      } else {
+        offset = (int) (Math.random() * (capacity - maxBlockSize));
+      }
+      ioEngine.write(srcBuffer, offset);
+      //ByteBuffer dstBuffer = ByteBuffer.allocate(blockSize);
+      //ioEngine.read(dstBuffer, offset);
+      //MultiByteBuffer read = new MultiByteBuffer(dstBuffer);
+      // TODO : this will get changed after HBASE-12295 goes in
+      ByteBuff read = ioEngine.read(offset, blockSize);
+      for (int j = 0; j < byteArray.length; ++j) {
+        assertTrue(srcBuffer.get(j) == read.get(j));
+      }
+    }
+    assert testOffsetAtStartNum == 0;
+    assert testOffsetAtEndNum == 0;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/834f87b2/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestBloomFilterChunk.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestBloomFilterChunk.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestBloomFilterChunk.java
index 4d8ad4b..dd46119 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestBloomFilterChunk.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestBloomFilterChunk.java
@@ -24,6 +24,8 @@ import java.io.DataOutputStream;
 import java.nio.ByteBuffer;
 
 import junit.framework.TestCase;
+
+import org.apache.hadoop.hbase.nio.MultiByteBuff;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.junit.experimental.categories.Category;
@@ -44,14 +46,14 @@ public class TestBloomFilterChunk extends TestCase {
     bf1.add(key1);
     bf2.add(key2);
 
-    assertTrue(BloomFilterUtil.contains(key1, 0, key1.length, bf1.bloom, 0, 
(int) bf1.byteSize,
-        bf1.hash, bf1.hashCount));
-    assertFalse(BloomFilterUtil.contains(key2, 0, key2.length, bf1.bloom, 0, 
(int) bf1.byteSize,
-        bf1.hash, bf1.hashCount));
-    assertFalse(BloomFilterUtil.contains(key1, 0, key1.length, bf2.bloom, 0, 
(int) bf2.byteSize,
-        bf2.hash, bf2.hashCount));
-    assertTrue(BloomFilterUtil.contains(key2, 0, key2.length, bf2.bloom, 0, 
(int) bf2.byteSize,
-        bf2.hash, bf2.hashCount));
+    assertTrue(BloomFilterUtil.contains(key1, 0, key1.length, new 
MultiByteBuff(bf1.bloom), 0,
+        (int) bf1.byteSize, bf1.hash, bf1.hashCount));
+    assertFalse(BloomFilterUtil.contains(key2, 0, key2.length, new 
MultiByteBuff(bf1.bloom), 0,
+        (int) bf1.byteSize, bf1.hash, bf1.hashCount));
+    assertFalse(BloomFilterUtil.contains(key1, 0, key1.length, new 
MultiByteBuff(bf2.bloom), 0,
+        (int) bf2.byteSize, bf2.hash, bf2.hashCount));
+    assertTrue(BloomFilterUtil.contains(key2, 0, key2.length, new 
MultiByteBuff(bf2.bloom), 0,
+        (int) bf2.byteSize, bf2.hash, bf2.hashCount));
 
     byte [] bkey = {1,2,3,4};
     byte [] bval = "this is a much larger byte array".getBytes();
@@ -59,12 +61,12 @@ public class TestBloomFilterChunk extends TestCase {
     bf1.add(bkey);
     bf1.add(bval, 1, bval.length-1);
 
-    assertTrue(BloomFilterUtil.contains(bkey, 0, bkey.length, bf1.bloom, 0, 
(int) bf1.byteSize,
-        bf1.hash, bf1.hashCount));
-    assertTrue(BloomFilterUtil.contains(bval, 1, bval.length - 1, bf1.bloom, 
0, (int) bf1.byteSize,
-        bf1.hash, bf1.hashCount));
-    assertFalse(BloomFilterUtil.contains(bval, 0, bval.length, bf1.bloom, 0, 
(int) bf1.byteSize,
-        bf1.hash, bf1.hashCount));
+    assertTrue(BloomFilterUtil.contains(bkey, 0, bkey.length, new 
MultiByteBuff(bf1.bloom), 0,
+        (int) bf1.byteSize, bf1.hash, bf1.hashCount));
+    assertTrue(BloomFilterUtil.contains(bval, 1, bval.length - 1, new 
MultiByteBuff(bf1.bloom),
+        0, (int) bf1.byteSize, bf1.hash, bf1.hashCount));
+    assertFalse(BloomFilterUtil.contains(bval, 0, bval.length, new 
MultiByteBuff(bf1.bloom), 0,
+        (int) bf1.byteSize, bf1.hash, bf1.hashCount));
 
     // test 2: serialization & deserialization.
     // (convert bloom to byte array & read byte array back in as input)
@@ -73,18 +75,18 @@ public class TestBloomFilterChunk extends TestCase {
     ByteBuffer bb = ByteBuffer.wrap(bOut.toByteArray());
     BloomFilterChunk newBf1 = new BloomFilterChunk(1000, (float)0.01,
         Hash.MURMUR_HASH, 0);
-    assertTrue(BloomFilterUtil.contains(key1, 0, key1.length, bb, 0, (int) 
newBf1.byteSize,
-        newBf1.hash, newBf1.hashCount));
-    assertFalse(BloomFilterUtil.contains(key2, 0, key2.length, bb, 0, (int) 
newBf1.byteSize,
-        newBf1.hash, newBf1.hashCount));
-    assertTrue(BloomFilterUtil.contains(bkey, 0, bkey.length, bb, 0, (int) 
newBf1.byteSize,
-        newBf1.hash, newBf1.hashCount));
-    assertTrue(BloomFilterUtil.contains(bval, 1, bval.length - 1, bb, 0, (int) 
newBf1.byteSize,
-        newBf1.hash, newBf1.hashCount));
-    assertFalse(BloomFilterUtil.contains(bval, 0, bval.length, bb, 0, (int) 
newBf1.byteSize,
-        newBf1.hash, newBf1.hashCount));
-    assertFalse(BloomFilterUtil.contains(bval, 0, bval.length, bb, 0, (int) 
newBf1.byteSize,
-        newBf1.hash, newBf1.hashCount));
+    assertTrue(BloomFilterUtil.contains(key1, 0, key1.length, new 
MultiByteBuff(bb), 0,
+        (int) newBf1.byteSize, newBf1.hash, newBf1.hashCount));
+    assertFalse(BloomFilterUtil.contains(key2, 0, key2.length, new 
MultiByteBuff(bb), 0,
+        (int) newBf1.byteSize, newBf1.hash, newBf1.hashCount));
+    assertTrue(BloomFilterUtil.contains(bkey, 0, bkey.length, new 
MultiByteBuff(bb), 0,
+        (int) newBf1.byteSize, newBf1.hash, newBf1.hashCount));
+    assertTrue(BloomFilterUtil.contains(bval, 1, bval.length - 1, new 
MultiByteBuff(bb), 0,
+        (int) newBf1.byteSize, newBf1.hash, newBf1.hashCount));
+    assertFalse(BloomFilterUtil.contains(bval, 0, bval.length, new 
MultiByteBuff(bb), 0,
+        (int) newBf1.byteSize, newBf1.hash, newBf1.hashCount));
+    assertFalse(BloomFilterUtil.contains(bval, 0, bval.length, new 
MultiByteBuff(bb), 0,
+        (int) newBf1.byteSize, newBf1.hash, newBf1.hashCount));
 
     System.out.println("Serialized as " + bOut.size() + " bytes");
     assertTrue(bOut.size() - bf1.byteSize < 10); //... allow small padding
@@ -105,9 +107,10 @@ public class TestBloomFilterChunk extends TestCase {
     int falsePositives = 0;
     for (int i = 0; i < 25; ++i) {
       byte[] bytes = Bytes.toBytes(i);
-      if (BloomFilterUtil.contains(bytes, 0, bytes.length, b.bloom, 0, (int) 
b.byteSize, b.hash,
-          b.hashCount)) {
-        if(i >= 12) falsePositives++;
+      if (BloomFilterUtil.contains(bytes, 0, bytes.length, new 
MultiByteBuff(b.bloom), 0,
+          (int) b.byteSize, b.hash, b.hashCount)) {
+        if (i >= 12)
+          falsePositives++;
       } else {
         assertFalse(i < 12);
       }
@@ -143,9 +146,10 @@ public class TestBloomFilterChunk extends TestCase {
     for (int i = 0; i < 2*1000*1000; ++i) {
 
       byte[] bytes = Bytes.toBytes(i);
-      if (BloomFilterUtil.contains(bytes, 0, bytes.length, b.bloom, 0, (int) 
b.byteSize, b.hash,
-          b.hashCount)) {
-        if(i >= 1*1000*1000) falsePositives++;
+      if (BloomFilterUtil.contains(bytes, 0, bytes.length, new 
MultiByteBuff(b.bloom), 0,
+          (int) b.byteSize, b.hash, b.hashCount)) {
+        if (i >= 1 * 1000 * 1000)
+          falsePositives++;
       } else {
         assertFalse(i < 1*1000*1000);
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/834f87b2/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestByteBuffUtils.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestByteBuffUtils.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestByteBuffUtils.java
new file mode 100644
index 0000000..4c6990e
--- /dev/null
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestByteBuffUtils.java
@@ -0,0 +1,78 @@
+/*
+ * 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.hadoop.hbase.util;
+
+import static org.junit.Assert.assertTrue;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.hbase.nio.ByteBuff;
+import org.apache.hadoop.hbase.nio.MultiByteBuff;
+import org.apache.hadoop.hbase.nio.SingleByteBuff;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ MiscTests.class, SmallTests.class })
+public class TestByteBuffUtils {
+  @Test
+  public void testCopyAndCompare() throws Exception {
+    ByteBuffer bb1 = ByteBuffer.allocate(50);
+    ByteBuffer bb2 = ByteBuffer.allocate(50);
+    MultiByteBuff src = new MultiByteBuff(bb1, bb2);
+    for (int i = 0; i < 7; i++) {
+      src.putLong(8l);
+    }
+    src.put((byte) 1);
+    src.put((byte) 1);
+    ByteBuffer bb3 = ByteBuffer.allocate(50);
+    ByteBuffer bb4 = ByteBuffer.allocate(50);
+    MultiByteBuff mbbDst = new MultiByteBuff(bb3, bb4);
+    // copy from MBB to MBB
+    mbbDst.put(0, src, 0, 100);
+    int compareTo = ByteBuff.compareTo(src, 0, 100, mbbDst, 0, 100);
+    assertTrue(compareTo == 0);
+    // Copy from MBB to SBB
+    bb3 = ByteBuffer.allocate(100);
+    SingleByteBuff sbbDst = new SingleByteBuff(bb3);
+    src.rewind();
+    sbbDst.put(0, src, 0, 100);
+    compareTo = ByteBuff.compareTo(src, 0, 100, sbbDst, 0, 100);
+    assertTrue(compareTo == 0);
+    // Copy from SBB to SBB
+    bb3 = ByteBuffer.allocate(100);
+    SingleByteBuff sbb = new SingleByteBuff(bb3);
+    for (int i = 0; i < 7; i++) {
+      sbb.putLong(8l);
+    }
+    sbb.put((byte) 1);
+    sbb.put((byte) 1);
+    bb4 = ByteBuffer.allocate(100);
+    sbbDst = new SingleByteBuff(bb4);
+    sbbDst.put(0, sbb, 0, 100);
+    compareTo = ByteBuff.compareTo(sbb, 0, 100, sbbDst, 0, 100);
+    assertTrue(compareTo == 0);
+    // copy from SBB to MBB
+    sbb.rewind();
+    mbbDst = new MultiByteBuff(bb3, bb4);
+    mbbDst.rewind();
+    mbbDst.put(0, sbb, 0, 100);
+    compareTo = ByteBuff.compareTo(sbb, 0, 100, mbbDst, 0, 100);
+    assertTrue(compareTo == 0);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/834f87b2/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java
index 547c046..2403c82 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java
@@ -194,7 +194,7 @@ public class TestByteBufferUtils {
     ByteBuffer dstBuffer = ByteBuffer.allocate(array.length);
     srcBuffer.put(array);
 
-    ByteBufferUtils.copyFromBufferToBuffer(dstBuffer, srcBuffer,
+    ByteBufferUtils.copyFromBufferToBuffer(srcBuffer, dstBuffer,
         array.length / 2, array.length / 4);
     for (int i = 0; i < array.length / 4; ++i) {
       assertEquals(srcBuffer.get(i + array.length / 2),

Reply via email to