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;
019
020import static org.junit.Assert.assertTrue;
021
022import java.io.ByteArrayInputStream;
023import java.io.DataOutputStream;
024import java.nio.ByteBuffer;
025import java.util.ArrayList;
026import java.util.List;
027import org.apache.hadoop.hbase.ArrayBackedTag;
028import org.apache.hadoop.hbase.ByteBufferExtendedCell;
029import org.apache.hadoop.hbase.ByteBufferKeyValue;
030import org.apache.hadoop.hbase.Cell;
031import org.apache.hadoop.hbase.HBaseClassTestRule;
032import org.apache.hadoop.hbase.KeyValue;
033import org.apache.hadoop.hbase.Tag;
034import org.apache.hadoop.hbase.io.util.LRUDictionary;
035import org.apache.hadoop.hbase.nio.SingleByteBuff;
036import org.apache.hadoop.hbase.testclassification.MiscTests;
037import org.apache.hadoop.hbase.testclassification.SmallTests;
038import org.apache.hadoop.hbase.util.ByteBufferUtils;
039import org.apache.hadoop.hbase.util.Bytes;
040import org.junit.ClassRule;
041import org.junit.Test;
042import org.junit.experimental.categories.Category;
043
044@Category({ MiscTests.class, SmallTests.class })
045public class TestTagCompressionContext {
046
047  @ClassRule
048  public static final HBaseClassTestRule CLASS_RULE =
049    HBaseClassTestRule.forClass(TestTagCompressionContext.class);
050
051  private static final byte[] ROW = Bytes.toBytes("r1");
052  private static final byte[] CF = Bytes.toBytes("f");
053  private static final byte[] Q = Bytes.toBytes("q");
054  private static final byte[] V = Bytes.toBytes("v");
055
056  @Test
057  public void testCompressUncompressTags1() throws Exception {
058    ByteArrayOutputStream baos = new ByteArrayOutputStream();
059    TagCompressionContext context = new TagCompressionContext(LRUDictionary.class, Byte.MAX_VALUE);
060    KeyValue kv1 = createKVWithTags(2);
061    int tagsLength1 = kv1.getTagsLength();
062    ByteBuffer ib = ByteBuffer.wrap(kv1.getTagsArray());
063    context.compressTags(baos, ib, kv1.getTagsOffset(), tagsLength1);
064    KeyValue kv2 = createKVWithTags(3);
065    int tagsLength2 = kv2.getTagsLength();
066    ib = ByteBuffer.wrap(kv2.getTagsArray());
067    context.compressTags(baos, ib, kv2.getTagsOffset(), tagsLength2);
068
069    context.clear();
070
071    byte[] dest = new byte[tagsLength1];
072    ByteBuffer ob = ByteBuffer.wrap(baos.toByteArray());
073    context.uncompressTags(new SingleByteBuff(ob), dest, 0, tagsLength1);
074    assertTrue(
075      Bytes.equals(kv1.getTagsArray(), kv1.getTagsOffset(), tagsLength1, dest, 0, tagsLength1));
076    dest = new byte[tagsLength2];
077    context.uncompressTags(new SingleByteBuff(ob), dest, 0, tagsLength2);
078    assertTrue(
079      Bytes.equals(kv2.getTagsArray(), kv2.getTagsOffset(), tagsLength2, dest, 0, tagsLength2));
080  }
081
082  @Test
083  public void testCompressUncompressTagsWithOffheapKeyValue1() throws Exception {
084    ByteArrayOutputStream baos = new ByteArrayOutputStream();
085    DataOutputStream daos = new ByteBufferWriterDataOutputStream(baos);
086    TagCompressionContext context = new TagCompressionContext(LRUDictionary.class, Byte.MAX_VALUE);
087    ByteBufferExtendedCell kv1 = (ByteBufferExtendedCell) createOffheapKVWithTags(2);
088    int tagsLength1 = kv1.getTagsLength();
089    context.compressTags(daos, kv1.getTagsByteBuffer(), kv1.getTagsPosition(), tagsLength1);
090    ByteBufferExtendedCell kv2 = (ByteBufferExtendedCell) createOffheapKVWithTags(3);
091    int tagsLength2 = kv2.getTagsLength();
092    context.compressTags(daos, kv2.getTagsByteBuffer(), kv2.getTagsPosition(), tagsLength2);
093
094    context.clear();
095
096    byte[] dest = new byte[tagsLength1];
097    ByteBuffer ob = ByteBuffer.wrap(baos.getBuffer());
098    context.uncompressTags(new SingleByteBuff(ob), dest, 0, tagsLength1);
099    assertTrue(
100      Bytes.equals(kv1.getTagsArray(), kv1.getTagsOffset(), tagsLength1, dest, 0, tagsLength1));
101    dest = new byte[tagsLength2];
102    context.uncompressTags(new SingleByteBuff(ob), dest, 0, tagsLength2);
103    assertTrue(
104      Bytes.equals(kv2.getTagsArray(), kv2.getTagsOffset(), tagsLength2, dest, 0, tagsLength2));
105  }
106
107  @Test
108  public void testCompressUncompressTags2() throws Exception {
109    ByteArrayOutputStream baos = new ByteArrayOutputStream();
110    TagCompressionContext context = new TagCompressionContext(LRUDictionary.class, Byte.MAX_VALUE);
111    KeyValue kv1 = createKVWithTags(1);
112    int tagsLength1 = kv1.getTagsLength();
113    context.compressTags(baos, kv1.getTagsArray(), kv1.getTagsOffset(), tagsLength1);
114    KeyValue kv2 = createKVWithTags(3);
115    int tagsLength2 = kv2.getTagsLength();
116    context.compressTags(baos, kv2.getTagsArray(), kv2.getTagsOffset(), tagsLength2);
117
118    context.clear();
119
120    ByteArrayInputStream bais = new ByteArrayInputStream(baos.getBuffer());
121    byte[] dest = new byte[tagsLength1];
122    context.uncompressTags(bais, dest, 0, tagsLength1);
123    assertTrue(
124      Bytes.equals(kv1.getTagsArray(), kv1.getTagsOffset(), tagsLength1, dest, 0, tagsLength1));
125    dest = new byte[tagsLength2];
126    context.uncompressTags(bais, dest, 0, tagsLength2);
127    assertTrue(
128      Bytes.equals(kv2.getTagsArray(), kv2.getTagsOffset(), tagsLength2, dest, 0, tagsLength2));
129  }
130
131  @Test
132  public void testCompressUncompressTagsWithOffheapKeyValue2() throws Exception {
133    ByteArrayOutputStream baos = new ByteArrayOutputStream();
134    DataOutputStream daos = new ByteBufferWriterDataOutputStream(baos);
135    TagCompressionContext context = new TagCompressionContext(LRUDictionary.class, Byte.MAX_VALUE);
136    ByteBufferExtendedCell kv1 = (ByteBufferExtendedCell) createOffheapKVWithTags(1);
137    int tagsLength1 = kv1.getTagsLength();
138    context.compressTags(daos, kv1.getTagsByteBuffer(), kv1.getTagsPosition(), tagsLength1);
139    ByteBufferExtendedCell kv2 = (ByteBufferExtendedCell) createOffheapKVWithTags(3);
140    int tagsLength2 = kv2.getTagsLength();
141    context.compressTags(daos, kv2.getTagsByteBuffer(), kv2.getTagsPosition(), tagsLength2);
142
143    context.clear();
144
145    ByteArrayInputStream bais = new ByteArrayInputStream(baos.getBuffer());
146    byte[] dest = new byte[tagsLength1];
147    context.uncompressTags(bais, dest, 0, tagsLength1);
148    assertTrue(
149      Bytes.equals(kv1.getTagsArray(), kv1.getTagsOffset(), tagsLength1, dest, 0, tagsLength1));
150    dest = new byte[tagsLength2];
151    context.uncompressTags(bais, dest, 0, tagsLength2);
152    assertTrue(
153      Bytes.equals(kv2.getTagsArray(), kv2.getTagsOffset(), tagsLength2, dest, 0, tagsLength2));
154  }
155
156  private KeyValue createKVWithTags(int noOfTags) {
157    List<Tag> tags = new ArrayList<>();
158    for (int i = 0; i < noOfTags; i++) {
159      tags.add(new ArrayBackedTag((byte) i, "tagValue" + i));
160    }
161    KeyValue kv = new KeyValue(ROW, CF, Q, 1234L, V, tags);
162    return kv;
163  }
164
165  private Cell createOffheapKVWithTags(int noOfTags) {
166    List<Tag> tags = new ArrayList<>();
167    for (int i = 0; i < noOfTags; i++) {
168      tags.add(new ArrayBackedTag((byte) i, "tagValue" + i));
169    }
170    KeyValue kv = new KeyValue(ROW, CF, Q, 1234L, V, tags);
171    ByteBuffer dbb = ByteBuffer.allocateDirect(kv.getBuffer().length);
172    ByteBufferUtils.copyFromArrayToBuffer(dbb, kv.getBuffer(), 0, kv.getBuffer().length);
173    ByteBufferKeyValue offheapKV = new ByteBufferKeyValue(dbb, 0, kv.getBuffer().length, 0);
174    return offheapKV;
175  }
176}