001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.io.encoding;
019
020import static org.junit.Assert.assertEquals;
021import static org.junit.Assert.assertFalse;
022import static org.junit.Assert.assertTrue;
023
024import java.io.ByteArrayInputStream;
025import java.io.ByteArrayOutputStream;
026import java.nio.ByteBuffer;
027import org.apache.hadoop.hbase.CellUtil;
028import org.apache.hadoop.hbase.HBaseClassTestRule;
029import org.apache.hadoop.hbase.HConstants;
030import org.apache.hadoop.hbase.KeyValue;
031import org.apache.hadoop.hbase.KeyValue.Type;
032import org.apache.hadoop.hbase.codec.Codec.Decoder;
033import org.apache.hadoop.hbase.codec.Codec.Encoder;
034import org.apache.hadoop.hbase.codec.KeyValueCodecWithTags;
035import org.apache.hadoop.hbase.io.encoding.BufferedDataBlockEncoder.OnheapDecodedCell;
036import org.apache.hadoop.hbase.testclassification.IOTests;
037import org.apache.hadoop.hbase.testclassification.MediumTests;
038import org.apache.hadoop.hbase.util.Bytes;
039import org.apache.hadoop.hbase.util.ObjectIntPair;
040import org.junit.ClassRule;
041import org.junit.Test;
042import org.junit.experimental.categories.Category;
043
044@Category({ IOTests.class, MediumTests.class })
045public class TestBufferedDataBlockEncoder {
046
047  @ClassRule
048  public static final HBaseClassTestRule CLASS_RULE =
049    HBaseClassTestRule.forClass(TestBufferedDataBlockEncoder.class);
050
051  byte[] row1 = Bytes.toBytes("row1");
052  byte[] row2 = Bytes.toBytes("row2");
053  byte[] row_1_0 = Bytes.toBytes("row10");
054
055  byte[] fam1 = Bytes.toBytes("fam1");
056  byte[] fam2 = Bytes.toBytes("fam2");
057  byte[] fam_1_2 = Bytes.toBytes("fam12");
058
059  byte[] qual1 = Bytes.toBytes("qual1");
060  byte[] qual2 = Bytes.toBytes("qual2");
061
062  byte[] val = Bytes.toBytes("val");
063
064  @Test
065  public void testEnsureSpaceForKey() {
066    BufferedDataBlockEncoder.SeekerState state =
067      new BufferedDataBlockEncoder.SeekerState(new ObjectIntPair<>(), false);
068    for (int i = 1; i <= 65536; ++i) {
069      state.keyLength = i;
070      state.ensureSpaceForKey();
071      state.keyBuffer[state.keyLength - 1] = (byte) ((i - 1) % 0xff);
072      for (int j = 0; j < i - 1; ++j) {
073        // Check that earlier bytes were preserved as the buffer grew.
074        assertEquals((byte) (j % 0xff), state.keyBuffer[j]);
075      }
076    }
077  }
078
079  @Test
080  public void testCommonPrefixComparators() {
081    KeyValue kv1 = new KeyValue(row1, fam1, qual1, 1L, Type.Put);
082    KeyValue kv2 = new KeyValue(row1, fam_1_2, qual1, 1L, Type.Maximum);
083    assertTrue((BufferedDataBlockEncoder.compareCommonFamilyPrefix(kv1, kv2, 4) < 0));
084
085    kv1 = new KeyValue(row1, fam1, qual1, 1L, Type.Put);
086    kv2 = new KeyValue(row_1_0, fam_1_2, qual1, 1L, Type.Maximum);
087    assertTrue((BufferedDataBlockEncoder.compareCommonRowPrefix(kv1, kv2, 4) < 0));
088
089    kv1 = new KeyValue(row1, fam1, qual2, 1L, Type.Put);
090    kv2 = new KeyValue(row1, fam1, qual1, 1L, Type.Maximum);
091    assertTrue((BufferedDataBlockEncoder.compareCommonQualifierPrefix(kv1, kv2, 4) > 0));
092  }
093
094  @Test
095  public void testKVCodecWithTagsForDecodedCellsWithNoTags() throws Exception {
096    KeyValue kv1 = new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("1"),
097      HConstants.LATEST_TIMESTAMP, Bytes.toBytes("1"));
098    // kv1.getKey() return a copy of the Key bytes which starts from RK_length. Means from offsets,
099    // we need to reduce the KL and VL parts.
100    OnheapDecodedCell c1 = new OnheapDecodedCell(kv1.getKey(), kv1.getRowLength(),
101      kv1.getFamilyOffset() - KeyValue.ROW_OFFSET, kv1.getFamilyLength(),
102      kv1.getQualifierOffset() - KeyValue.ROW_OFFSET, kv1.getQualifierLength(), kv1.getTimestamp(),
103      kv1.getTypeByte(), kv1.getValueArray(), kv1.getValueOffset(), kv1.getValueLength(),
104      kv1.getSequenceId(), kv1.getTagsArray(), kv1.getTagsOffset(), kv1.getTagsLength());
105    KeyValue kv2 = new KeyValue(Bytes.toBytes("r2"), Bytes.toBytes("f"), Bytes.toBytes("2"),
106      HConstants.LATEST_TIMESTAMP, Bytes.toBytes("2"));
107    OnheapDecodedCell c2 = new OnheapDecodedCell(kv2.getKey(), kv2.getRowLength(),
108      kv2.getFamilyOffset() - KeyValue.ROW_OFFSET, kv2.getFamilyLength(),
109      kv2.getQualifierOffset() - KeyValue.ROW_OFFSET, kv2.getQualifierLength(), kv2.getTimestamp(),
110      kv2.getTypeByte(), kv2.getValueArray(), kv2.getValueOffset(), kv2.getValueLength(),
111      kv2.getSequenceId(), kv2.getTagsArray(), kv2.getTagsOffset(), kv2.getTagsLength());
112    KeyValue kv3 = new KeyValue(Bytes.toBytes("r3"), Bytes.toBytes("cf"), Bytes.toBytes("qual"),
113      HConstants.LATEST_TIMESTAMP, Bytes.toBytes("3"));
114    BufferedDataBlockEncoder.OffheapDecodedExtendedCell c3 =
115      new BufferedDataBlockEncoder.OffheapDecodedExtendedCell(ByteBuffer.wrap(kv2.getKey()),
116        kv2.getRowLength(), kv2.getFamilyOffset() - KeyValue.ROW_OFFSET, kv2.getFamilyLength(),
117        kv2.getQualifierOffset() - KeyValue.ROW_OFFSET, kv2.getQualifierLength(),
118        kv2.getTimestamp(), kv2.getTypeByte(), ByteBuffer.wrap(kv2.getValueArray()),
119        kv2.getValueOffset(), kv2.getValueLength(), kv2.getSequenceId(),
120        ByteBuffer.wrap(kv2.getTagsArray()), kv2.getTagsOffset(), kv2.getTagsLength());
121    ByteArrayOutputStream os = new ByteArrayOutputStream();
122    KeyValueCodecWithTags codec = new KeyValueCodecWithTags();
123    Encoder encoder = codec.getEncoder(os);
124    encoder.write(c1);
125    encoder.write(c2);
126    encoder.write(c3);
127    ByteArrayInputStream is = new ByteArrayInputStream(os.toByteArray());
128    Decoder decoder = codec.getDecoder(is);
129    assertTrue(decoder.advance());
130    assertTrue(CellUtil.equals(c1, decoder.current()));
131    assertTrue(decoder.advance());
132    assertTrue(CellUtil.equals(c2, decoder.current()));
133    assertTrue(decoder.advance());
134    assertTrue(CellUtil.equals(c3, decoder.current()));
135    assertFalse(decoder.advance());
136  }
137}