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);
- }
- }
}