You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by mb...@apache.org on 2012/10/23 23:58:44 UTC

svn commit: r1401499 [2/2] - in /hbase/branches/0.89-fb/src: main/java/org/apache/hadoop/hbase/ main/java/org/apache/hadoop/hbase/io/encoding/ main/java/org/apache/hadoop/hbase/io/hfile/ main/java/org/apache/hadoop/hbase/util/ test/java/org/apache/hado...

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java?rev=1401499&r1=1401498&r2=1401499&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java Tue Oct 23 21:58:43 2012
@@ -120,16 +120,24 @@ public final class ByteBufferUtils {
   }
 
    /**
-    * Put in output stream 32 bit integer (Big Endian byte order).
+    * Write a big-endian integer at a specific position in the output array
     * @param out Where to put integer.
+    * @param offset Offset in array.
+    * @param length Length of the part of the array we are allowed to write into, starting with the
+    *        specified offset.
     * @param value Value of integer.
-    * @throws IOException On stream error.
+    * @throws IOException When insufficient size to write.
     */
-   public static void putInt(OutputStream out, final int value)
-       throws IOException {
-     for (int i = Bytes.SIZEOF_INT - 1; i >= 0; --i) {
-       out.write((byte) (value >>> (i * 8)));
+   public static void putInt(byte[] out, final int offset, final int length,
+       final int v) throws IOException {
+     if (length < Bytes.SIZEOF_INT) {
+      throw new IOException("Not enough space to write an int: offset=" + offset + ", length="
+          + length);
      }
+     out[offset] = (byte) ((v >>> 24) & 0xFF);
+     out[offset + 1] = (byte) ((v >>> 16) & 0xFF);
+     out[offset + 2] = (byte) ((v >>> 8) & 0xFF);
+     out[offset + 3] = (byte) (v & 0xFF);
    }
 
   /**
@@ -156,8 +164,13 @@ public final class ByteBufferUtils {
   public static void copyBufferToStream(OutputStream out, ByteBuffer in,
       int offset, int length) throws IOException {
     if (in.hasArray()) {
-      out.write(in.array(), in.arrayOffset() + offset,
-          length);
+      try {
+        out.write(in.array(), in.arrayOffset() + offset,
+            length);
+      } catch (IndexOutOfBoundsException ex) {
+        throw new IOException("Array out of bounds: arrayOffset=" + in.arrayOffset() + ", " +
+            "offset=" + offset + ", length=" + in.array().length, ex); 
+      }
     } else {
       for (int i = 0; i < length; ++i) {
         out.write(in.get(offset + i));

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java?rev=1401499&r1=1401498&r2=1401499&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java Tue Oct 23 21:58:43 2012
@@ -72,8 +72,7 @@ public class TestDataBlockEncoders {
     // decode
     ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
     DataInputStream dis = new DataInputStream(bais);
-    ByteBuffer actualDataset;
-    actualDataset = encoder.decodeKeyValues(dis, includesMemstoreTS);
+    ByteBuffer actualDataset = encoder.decodeKeyValues(dis, 0, includesMemstoreTS, dis.available());
 
     dataset.rewind();
     actualDataset.rewind();

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java?rev=1401499&r1=1401498&r2=1401499&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java Tue Oct 23 21:58:43 2012
@@ -252,9 +252,18 @@ public class TestCacheOnWrite {
     String countByType = blockCountByType.toString();
     BlockType cachedDataBlockType =
         encoderType.encodeInCache ? BlockType.ENCODED_DATA : BlockType.DATA;
-    assertEquals("{" + cachedDataBlockType
-        + "=1379, LEAF_INDEX=173, BLOOM_CHUNK=9, INTERMEDIATE_INDEX=24}",
-        countByType);
+
+    // Block size depends on whether encoding on disk has been enabled
+    // so number of blocks depends on this parameter as well.
+    if (encoder.getEncodingOnDisk() == DataBlockEncoding.PREFIX) {
+      assertEquals("{" + cachedDataBlockType
+          + "=965, LEAF_INDEX=121, BLOOM_CHUNK=9, INTERMEDIATE_INDEX=17}",
+          countByType);
+    } else {
+      assertEquals("{" + cachedDataBlockType
+          + "=1379, LEAF_INDEX=173, BLOOM_CHUNK=9, INTERMEDIATE_INDEX=24}",
+          countByType);
+    }
 
     reader.close();
   }

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java?rev=1401499&r1=1401498&r2=1401499&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java Tue Oct 23 21:58:43 2012
@@ -114,14 +114,8 @@ public class TestHFile extends HBaseTest
       ByteBuffer val = scanner.getValue();
       String keyStr = String.format(localFormatter, Integer.valueOf(i));
       String valStr = value + keyStr;
-      byte [] keyBytes = Bytes.toBytes(key);
-      assertTrue("bytes for keys do not match " + keyStr + " " +
-        Bytes.toString(Bytes.toBytes(key)),
-          Arrays.equals(Bytes.toBytes(keyStr), keyBytes));
-      byte [] valBytes = Bytes.toBytes(val);
-      assertTrue("bytes for vals do not match " + valStr + " " +
-        Bytes.toString(valBytes),
-        Arrays.equals(Bytes.toBytes(valStr), valBytes));
+      assertEquals("bytes for keys do not match", keyStr, Bytes.toStringBinaryRemaining(key));
+      assertEquals("bytes for vals do not match", valStr, Bytes.toStringBinaryRemaining(val));
       if (!scanner.next()) {
         break;
       }

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java?rev=1401499&r1=1401498&r2=1401499&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java Tue Oct 23 21:58:43 2012
@@ -51,8 +51,8 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.io.DoubleOutputStream;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
+import org.apache.hadoop.hbase.io.hfile.HFileBlock.Writer;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.hadoop.io.WritableUtils;
@@ -60,8 +60,8 @@ import org.apache.hadoop.io.compress.Com
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized.Parameters;
 import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
 
 @RunWith(Parameterized.class)
 public class TestHFileBlock {
@@ -105,11 +105,12 @@ public class TestHFileBlock {
 
   public void writeTestBlockContents(DataOutputStream dos) throws IOException {
     // This compresses really well.
-    for (int i = 0; i < 1000; ++i)
+    for (int i = 0; i < 1000; ++i) {
       dos.writeInt(i / 100);
+    }
   }
 
-  private int writeTestKeyValues(OutputStream dos, int seed)
+  private void writeTestKeyValues(OutputStream dos, Writer hbw, int seed)
       throws IOException {
     List<KeyValue> keyValues = new ArrayList<KeyValue>();
     Random randomizer = new Random(42l + seed); // just any fixed number
@@ -159,20 +160,19 @@ public class TestHFileBlock {
     }
 
     // sort it and write to stream
-    int totalSize = 0;
     Collections.sort(keyValues, KeyValue.COMPARATOR);
     DataOutputStream dataOutputStream = new DataOutputStream(dos);
     for (KeyValue kv : keyValues) {
-      totalSize += kv.getLength();
+      long memstoreTS = randomizer.nextLong();
+      hbw.appendEncodedKV(memstoreTS, kv.getBuffer(), kv.getKeyOffset(), kv.getKeyLength(),
+          kv.getBuffer(), kv.getValueOffset(), kv.getValueLength());
+
+      // Write raw key/value pair for validation.
       dataOutputStream.write(kv.getBuffer(), kv.getOffset(), kv.getLength());
       if (includesMemstoreTS) {
-        long memstoreTS = randomizer.nextLong();
         WritableUtils.writeVLong(dataOutputStream, memstoreTS);
-        totalSize += WritableUtils.getVIntSize(memstoreTS);
       }
     }
-
-    return totalSize;
   }
 
   public byte[] createTestV1Block(Compression.Algorithm algo)
@@ -194,7 +194,8 @@ public class TestHFileBlock {
     final BlockType blockType = BlockType.DATA;
     HFileBlock.Writer hbw = new HFileBlock.Writer(algo, null,
         includesMemstoreTS);
-    DataOutputStream dos = hbw.startWriting(blockType);
+    hbw.startWriting(blockType);
+    DataOutputStream dos = hbw.getUserDataStreamUnsafe();
     writeTestBlockContents(dos);
     byte[] headerAndData = hbw.getHeaderAndData();
     assertEquals(1000 * 4, hbw.getUncompressedSizeWithoutHeader());
@@ -290,9 +291,11 @@ public class TestHFileBlock {
             includesMemstoreTS);
         long totalSize = 0;
         for (int blockId = 0; blockId < 2; ++blockId) {
-          DataOutputStream dos = hbw.startWriting(BlockType.DATA);
-          for (int i = 0; i < 1234; ++i)
+          hbw.startWriting(BlockType.DATA);
+          DataOutputStream dos = hbw.getUserDataStreamUnsafe();
+          for (int i = 0; i < 1234; ++i) {
             dos.writeInt(i);
+          }
           hbw.writeHeaderAndData(os);
           totalSize += hbw.getOnDiskSizeWithHeader();
         }
@@ -343,6 +346,10 @@ public class TestHFileBlock {
     for (Compression.Algorithm algo : COMPRESSION_ALGORITHMS) {
       for (boolean pread : new boolean[] { false, true }) {
         for (DataBlockEncoding encoding : DataBlockEncoding.values()) {
+          LOG.info("\n\nUsing includesMemstoreTS: " + includesMemstoreTS +
+              ", compression: " + algo +
+              ", pread: " + pread +
+              ", encoding: " + encoding + "\n");
           Path path = new Path(TEST_UTIL.getTestDir(), "blocks_v2_"
               + algo + "_" + encoding.toString());
           FSDataOutputStream os = fs.create(path);
@@ -359,6 +366,9 @@ public class TestHFileBlock {
 
             hbw.writeHeaderAndData(os);
             totalSize += hbw.getOnDiskSizeWithHeader();
+            LOG.info("Wrote block #" + blockId + ": " +
+                "onDiskSizeWithHeader=" + hbw.getOnDiskSizeWithHeader() + ", " +
+                "uncompressedSizeWithHeader=" + hbw.getUncompressedSizeWithHeader());
           }
           os.close();
 
@@ -370,19 +380,19 @@ public class TestHFileBlock {
 
           HFileBlock b;
           int pos = 0;
+          LOG.info("\n\nStarting to read blocks\n");
           for (int blockId = 0; blockId < numBlocks; ++blockId) {
             b = hbr.readBlockData(pos, -1, -1, pread);
             b.sanityCheck();
             pos += b.getOnDiskSizeWithHeader();
 
-            assertEquals((int) encodedSizes.get(blockId),
+            LOG.info("Read block #" + blockId + ": " + b);
+            assertEquals("Invalid encoded size:", (int) encodedSizes.get(blockId),
                 b.getUncompressedSizeWithoutHeader());
             ByteBuffer actualBuffer = b.getBufferWithoutHeader();
             if (encoding != DataBlockEncoding.NONE) {
               // We expect a two-byte big-endian encoding id.
-              assertEquals(0, actualBuffer.get(0));
-              assertEquals(encoding.getId(), actualBuffer.get(1));
-              actualBuffer.position(2);
+              assertEquals(encoding.getId(), actualBuffer.getShort());
               actualBuffer = actualBuffer.slice();
             }
 
@@ -390,8 +400,7 @@ public class TestHFileBlock {
             expectedBuffer.rewind();
 
             // test if content matches, produce nice message
-            assertBuffersEqual(expectedBuffer, actualBuffer, algo, encoding,
-                pread);
+            assertBuffersEqual(expectedBuffer, actualBuffer, algo, encoding, pread);
           }
           is.close();
         }
@@ -402,34 +411,24 @@ public class TestHFileBlock {
   private void writeEncodedBlock(DataBlockEncoding encoding,
       HFileBlock.Writer hbw, final List<Integer> encodedSizes,
       final List<ByteBuffer> encodedBlocks, int blockId) throws IOException {
-    DataOutputStream dos = hbw.startWriting(BlockType.DATA);
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    DoubleOutputStream doubleOutputStream =
-        new DoubleOutputStream(dos, baos);
+    hbw.startWriting(BlockType.DATA);
+    ByteArrayOutputStream rawKVBytes = new ByteArrayOutputStream();
 
-    final int rawBlockSize = writeTestKeyValues(doubleOutputStream,
-        blockId);
+    writeTestKeyValues(rawKVBytes, hbw, blockId);
 
-    ByteBuffer rawBuf = ByteBuffer.wrap(baos.toByteArray());
-    rawBuf.rewind();
+    byte[] rawBuf = rawKVBytes.toByteArray();
+    ByteArrayOutputStream encodedOut = new ByteArrayOutputStream();
+    encoding.getEncoder().encodeKeyValues(
+        new DataOutputStream(encodedOut),
+        ByteBuffer.wrap(rawBuf), includesMemstoreTS);
+
+    // We need to account for the two-byte encoding algorithm ID that
+    // comes after the 24-byte block header but before encoded KVs.
+    int encodedSize = encoding.encodingIdSize() + encodedOut.size();
 
-    final int encodedSize;
-    final ByteBuffer encodedBuf;
-    if (encoding == DataBlockEncoding.NONE) {
-      encodedSize = rawBlockSize;
-      encodedBuf = rawBuf;
-    } else {
-      ByteArrayOutputStream encodedOut = new ByteArrayOutputStream();
-      encoding.getEncoder().encodeKeyValues(
-          new DataOutputStream(encodedOut),
-          rawBuf.duplicate(), includesMemstoreTS);
-      // We need to account for the two-byte encoding algorithm ID that
-      // comes after the 24-byte block header but before encoded KVs.
-      encodedSize = encodedOut.size() + DataBlockEncoding.ID_SIZE;
-      encodedBuf = ByteBuffer.wrap(encodedOut.toByteArray());
-    }
+    LOG.info("Raw size: " + rawBuf.length + ", encoded size: " + encodedSize);
     encodedSizes.add(encodedSize);
-    encodedBlocks.add(encodedBuf);
+    encodedBlocks.add(ByteBuffer.wrap(encodedOut.toByteArray()));
   }
 
   private void assertBuffersEqual(ByteBuffer expectedBuffer,
@@ -442,27 +441,16 @@ public class TestHFileBlock {
           expectedBuffer.get(prefix) == actualBuffer.get(prefix)) {
         prefix++;
       }
-
-      fail(String.format(
-          "Content mismath for compression %s, encoding %s, " +
-          "pread %s, commonPrefix %d, expected %s, got %s",
+      assertEquals(String.format(
+          "Content mismatch for compression %s, encoding %s, " +
+          "pread %s, commonPrefix %d, expected length %d, actual length %d",
           compression, encoding, pread, prefix,
-          nextBytesToStr(expectedBuffer, prefix),
-          nextBytesToStr(actualBuffer, prefix)));
+          expectedBuffer.limit(), actualBuffer.limit()),
+          Bytes.toStringBinary(expectedBuffer),
+          Bytes.toStringBinary(actualBuffer));
     }
   }
 
-  /**
-   * Convert a few next bytes in the given buffer at the given position to
-   * string. Used for error messages.
-   */
-  private static String nextBytesToStr(ByteBuffer buf, int pos) {
-    int maxBytes = buf.limit() - pos;
-    int numBytes = Math.min(16, maxBytes);
-    return Bytes.toStringBinary(buf.array(), buf.arrayOffset() + pos,
-        numBytes) + (numBytes < maxBytes ? "..." : "");
-  }
-
   @Test
   public void testPreviousOffset() throws IOException {
     for (Compression.Algorithm algo : COMPRESSION_ALGORITHMS) {
@@ -673,7 +661,8 @@ public class TestHFileBlock {
         blockTypeOrdinal = BlockType.DATA.ordinal();
       }
       BlockType bt = BlockType.values()[blockTypeOrdinal];
-      DataOutputStream dos = hbw.startWriting(bt);
+      hbw.startWriting(bt);
+      DataOutputStream dos = hbw.getUserDataStreamUnsafe(); 
       for (int j = 0; j < rand.nextInt(500); ++j) {
         // This might compress well.
         dos.writeShort(i + 1);

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java?rev=1401499&r1=1401498&r2=1401499&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java Tue Oct 23 21:58:43 2012
@@ -220,7 +220,8 @@ public class TestHFileBlockIndex {
         new HFileBlockIndex.BlockIndexWriter(hbw, null, null);
 
     for (int i = 0; i < NUM_DATA_BLOCKS; ++i) {
-      hbw.startWriting(BlockType.DATA).write(
+      hbw.startWriting(BlockType.DATA);
+      hbw.getUserDataStreamUnsafe().write(
           String.valueOf(rand.nextInt(1000)).getBytes());
       long blockOffset = outputStream.getPos();
       hbw.writeHeaderAndData(outputStream);

Added: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/io/hfile/TestIncrementalEncoding.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/io/hfile/TestIncrementalEncoding.java?rev=1401499&view=auto
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/io/hfile/TestIncrementalEncoding.java (added)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/io/hfile/TestIncrementalEncoding.java Tue Oct 23 21:58:43 2012
@@ -0,0 +1,127 @@
+/*
+ * 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.io.hfile;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Test;
+
+public class TestIncrementalEncoding {
+  
+  private static final Log LOG = LogFactory.getLog(TestIncrementalEncoding.class);
+
+  private static final int BLOCK_SIZE = 1024;
+  private static final int KVTYPES = 4;
+  private static final byte[] FAMILY = Bytes.toBytes("family");
+
+  public void testEncoding(DataBlockEncoding dataEncoding, boolean includeMemstoreTS,
+      int kvType) throws IOException {
+    LOG.info("encoding=" + dataEncoding + ", includeMemstoreTS=" + includeMemstoreTS + ", " +
+      "kvType=" + kvType);
+    HFileDataBlockEncoder blockEncoder = new HFileDataBlockEncoderImpl(dataEncoding);
+    HFileBlock.Writer writerEncoded = new HFileBlock.Writer(null, blockEncoder,
+        includeMemstoreTS);
+    HFileBlock.Writer writerUnencoded = new HFileBlock.Writer(null,
+        NoOpDataBlockEncoder.INSTANCE, includeMemstoreTS);
+    writerEncoded.startWriting(BlockType.DATA);
+    writerUnencoded.startWriting(BlockType.DATA);
+
+    // Fill block with data
+    long time = 1 << 10;
+    while (writerEncoded.blockSizeWritten() < BLOCK_SIZE) {
+      KeyValue kv;
+      switch (kvType) {
+        case 3:
+          kv = new KeyValue(Bytes.toBytes(time), FAMILY,
+              Bytes.toBytes(time), time, Bytes.toBytes(time));
+          break;
+        case 2:
+          kv = new KeyValue(Bytes.toBytes("row"), FAMILY,
+              Bytes.toBytes("qf" + time), 0, Bytes.toBytes("V"));
+          break;
+        case 1:
+          kv = new KeyValue(Bytes.toBytes("row"), FAMILY,
+              Bytes.toBytes("qf" + time), time, Bytes.toBytes("V" + time));
+          break;
+        default:
+          kv = new KeyValue(Bytes.toBytes("row" + time), FAMILY,
+              Bytes.toBytes("qf"), 0, Bytes.toBytes("Value"));
+      }
+      time++;
+      appendEncoded(kv, writerEncoded);
+      appendEncoded(kv, writerUnencoded);
+    }
+
+    ByteArrayOutputStream encoded = new ByteArrayOutputStream();
+    writerEncoded.writeHeaderAndData(new DataOutputStream(encoded));
+
+    ByteArrayOutputStream unencoded = new ByteArrayOutputStream();
+    writerUnencoded.writeHeaderAndData(new DataOutputStream(unencoded));
+
+    ByteArrayOutputStream encodedAgain = new ByteArrayOutputStream();
+    DataOutputStream dataOut = new DataOutputStream(encodedAgain);
+    int bytesToSkip = HFileBlock.HEADER_SIZE;
+    ByteBuffer unencodedWithoutHeader = ByteBuffer.wrap(unencoded.toByteArray(), bytesToSkip,
+        unencoded.size() - bytesToSkip).slice();
+    dataEncoding.getEncoder().encodeKeyValues(dataOut,
+        unencodedWithoutHeader, includeMemstoreTS);
+
+    assertEquals(encodedAgain.size() + HFileBlock.HEADER_SIZE +
+        dataEncoding.encodingIdSize(), encoded.size());
+
+    byte[] en = encoded.toByteArray();
+    byte[] en2 = encodedAgain.toByteArray();
+    int shift = HFileBlock.HEADER_SIZE + dataEncoding.encodingIdSize();
+    for (int i = 0; i < encodedAgain.size(); i++) {
+      assertEquals("Byte" + i, en2[i], en[i + shift]);
+    }
+  }
+
+  private void testOneEncodingWithAllKVTypes(DataBlockEncoding blockEncoding,
+      boolean includeMemstoreTS) throws IOException {
+    for (int i = 0; i < KVTYPES; i++) {
+      testEncoding(blockEncoding, includeMemstoreTS, i);
+    }
+  }
+
+  @Test
+  public void testAllEncodings() throws IOException {
+    for (DataBlockEncoding encoding : DataBlockEncoding.values()) {
+      for (boolean includeMemstoreTS : HConstants.BOOLEAN_VALUES) {
+        testOneEncodingWithAllKVTypes(encoding, includeMemstoreTS);
+      }
+    }
+  }
+
+  public void appendEncoded(final KeyValue kv, HFileBlock.Writer writer)
+      throws IOException {
+    writer.appendEncodedKV(kv.getMemstoreTS(), kv.getBuffer(),
+        kv.getKeyOffset(), kv.getKeyLength(), kv.getBuffer(),
+        kv.getValueOffset(), kv.getValueLength());
+  }
+}

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java?rev=1401499&r1=1401498&r2=1401499&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java Tue Oct 23 21:58:43 2012
@@ -259,23 +259,6 @@ public class TestByteBufferUtils {
     assertFalse(ByteBufferUtils.arePartsEqual(buffer, 0, 3, 6, 3));
   }
 
-  /**
-   * Test serializing int to bytes
-   */
-  @Test
-  public void testPutInt() {
-    testPutInt(0);
-    testPutInt(Integer.MAX_VALUE);
-
-    for (int i = 0; i < 3; i++) {
-      testPutInt((128 << i) - 1);
-    }
-
-    for (int i = 0; i < 3; i++) {
-      testPutInt((128 << i));
-    }
-  }
-
   // Utility methods invoked from test methods
 
   private void testCompressedInt(int value) throws IOException {
@@ -291,20 +274,4 @@ public class TestByteBufferUtils {
     assertEquals(value, parsedValue);
   }
 
-  private void testPutInt(int value) {
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    try {
-      ByteBufferUtils.putInt(baos, value);
-    } catch (IOException e) {
-      throw new RuntimeException("Bug in putIn()", e);
-    }
-
-    ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
-    DataInputStream dis = new DataInputStream(bais);
-    try {
-      assertEquals(dis.readInt(), value);
-    } catch (IOException e) {
-      throw new RuntimeException("Bug in test!", e);
-    }
-  }
 }