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.regionserver.wal;
019
020import static org.junit.Assert.assertEquals;
021
022import java.io.ByteArrayInputStream;
023import java.io.ByteArrayOutputStream;
024import java.io.InputStream;
025import java.nio.ByteBuffer;
026import java.util.ArrayList;
027import java.util.List;
028import org.apache.hadoop.conf.Configuration;
029import org.apache.hadoop.hbase.ArrayBackedTag;
030import org.apache.hadoop.hbase.ByteBufferKeyValue;
031import org.apache.hadoop.hbase.HBaseClassTestRule;
032import org.apache.hadoop.hbase.HConstants;
033import org.apache.hadoop.hbase.KeyValue;
034import org.apache.hadoop.hbase.PrivateCellUtil;
035import org.apache.hadoop.hbase.Tag;
036import org.apache.hadoop.hbase.codec.Codec.Decoder;
037import org.apache.hadoop.hbase.codec.Codec.Encoder;
038import org.apache.hadoop.hbase.io.util.LRUDictionary;
039import org.apache.hadoop.hbase.testclassification.RegionServerTests;
040import org.apache.hadoop.hbase.testclassification.SmallTests;
041import org.apache.hadoop.hbase.util.Bytes;
042import org.junit.ClassRule;
043import org.junit.Test;
044import org.junit.experimental.categories.Category;
045
046@Category({RegionServerTests.class, SmallTests.class})
047public class TestWALCellCodecWithCompression {
048
049  @ClassRule
050  public static final HBaseClassTestRule CLASS_RULE =
051      HBaseClassTestRule.forClass(TestWALCellCodecWithCompression.class);
052
053  @Test
054  public void testEncodeDecodeKVsWithTags() throws Exception {
055    doTest(false, false);
056  }
057
058  @Test
059  public void testEncodeDecodeKVsWithTagsWithTagsCompression() throws Exception {
060    doTest(true, false);
061  }
062
063  @Test
064  public void testEncodeDecodeOffKVsWithTagsWithTagsCompression() throws Exception {
065    doTest(true, true);
066  }
067
068  private void doTest(boolean compressTags, boolean offheapKV) throws Exception {
069    Configuration conf = new Configuration(false);
070    conf.setBoolean(CompressionContext.ENABLE_WAL_TAGS_COMPRESSION, compressTags);
071    WALCellCodec codec = new WALCellCodec(conf, new CompressionContext(LRUDictionary.class, false,
072        compressTags));
073    ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
074    Encoder encoder = codec.getEncoder(bos);
075    if (offheapKV) {
076      encoder.write(createOffheapKV(1));
077      encoder.write(createOffheapKV(0));
078      encoder.write(createOffheapKV(2));
079    } else {
080      encoder.write(createKV(1));
081      encoder.write(createKV(0));
082      encoder.write(createKV(2));
083    }
084
085    InputStream is = new ByteArrayInputStream(bos.toByteArray());
086    Decoder decoder = codec.getDecoder(is);
087    decoder.advance();
088    KeyValue kv = (KeyValue) decoder.current();
089    List<Tag> tags = PrivateCellUtil.getTags(kv);
090    assertEquals(1, tags.size());
091    assertEquals("tagValue1", Bytes.toString(Tag.cloneValue(tags.get(0))));
092    decoder.advance();
093    kv = (KeyValue) decoder.current();
094    tags = PrivateCellUtil.getTags(kv);
095    assertEquals(0, tags.size());
096    decoder.advance();
097    kv = (KeyValue) decoder.current();
098    tags = PrivateCellUtil.getTags(kv);
099    assertEquals(2, tags.size());
100    assertEquals("tagValue1", Bytes.toString(Tag.cloneValue(tags.get(0))));
101    assertEquals("tagValue2", Bytes.toString(Tag.cloneValue(tags.get(1))));
102  }
103
104  private KeyValue createKV(int noOfTags) {
105    byte[] row = Bytes.toBytes("myRow");
106    byte[] cf = Bytes.toBytes("myCF");
107    byte[] q = Bytes.toBytes("myQualifier");
108    byte[] value = Bytes.toBytes("myValue");
109    List<Tag> tags = new ArrayList<>(noOfTags);
110    for (int i = 1; i <= noOfTags; i++) {
111      tags.add(new ArrayBackedTag((byte) i, Bytes.toBytes("tagValue" + i)));
112    }
113    return new KeyValue(row, cf, q, HConstants.LATEST_TIMESTAMP, value, tags);
114  }
115
116  private ByteBufferKeyValue createOffheapKV(int noOfTags) {
117    byte[] row = Bytes.toBytes("myRow");
118    byte[] cf = Bytes.toBytes("myCF");
119    byte[] q = Bytes.toBytes("myQualifier");
120    byte[] value = Bytes.toBytes("myValue");
121    List<Tag> tags = new ArrayList<>(noOfTags);
122    for (int i = 1; i <= noOfTags; i++) {
123      tags.add(new ArrayBackedTag((byte) i, Bytes.toBytes("tagValue" + i)));
124    }
125    KeyValue kv = new KeyValue(row, cf, q, HConstants.LATEST_TIMESTAMP, value, tags);
126    ByteBuffer dbb = ByteBuffer.allocateDirect(kv.getBuffer().length);
127    dbb.put(kv.getBuffer());
128    return new ByteBufferKeyValue(dbb, 0, kv.getBuffer().length);
129  }
130}