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; 021 022import java.io.IOException; 023import java.nio.ByteBuffer; 024import java.util.ArrayList; 025import java.util.Collection; 026import java.util.List; 027import org.apache.hadoop.hbase.Cell; 028import org.apache.hadoop.hbase.CellComparatorImpl; 029import org.apache.hadoop.hbase.HBaseClassTestRule; 030import org.apache.hadoop.hbase.HBaseCommonTestingUtility; 031import org.apache.hadoop.hbase.HConstants; 032import org.apache.hadoop.hbase.KeyValue; 033import org.apache.hadoop.hbase.PrivateCellUtil; 034import org.apache.hadoop.hbase.io.compress.Compression; 035import org.apache.hadoop.hbase.io.hfile.HFileContext; 036import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; 037import org.apache.hadoop.hbase.nio.SingleByteBuff; 038import org.apache.hadoop.hbase.testclassification.IOTests; 039import org.apache.hadoop.hbase.testclassification.SmallTests; 040import org.apache.hadoop.hbase.util.Bytes; 041import org.junit.ClassRule; 042import org.junit.Test; 043import org.junit.experimental.categories.Category; 044import org.junit.runner.RunWith; 045import org.junit.runners.Parameterized; 046import org.junit.runners.Parameterized.Parameters; 047 048@Category({IOTests.class, SmallTests.class}) 049@RunWith(Parameterized.class) 050public class TestSeekToBlockWithEncoders { 051 052 @ClassRule 053 public static final HBaseClassTestRule CLASS_RULE = 054 HBaseClassTestRule.forClass(TestSeekToBlockWithEncoders.class); 055 056 static final byte[] HFILEBLOCK_DUMMY_HEADER = new byte[HConstants.HFILEBLOCK_HEADER_SIZE]; 057 private final boolean useOffheapData; 058 059 @Parameters 060 public static Collection<Object[]> parameters() { 061 return HBaseCommonTestingUtility.BOOLEAN_PARAMETERIZED; 062 } 063 064 public TestSeekToBlockWithEncoders(boolean useOffheapData) { 065 this.useOffheapData = useOffheapData; 066 } 067 068 /** 069 * Test seeking while file is encoded. 070 */ 071 @Test 072 public void testSeekToBlockWithNonMatchingSeekKey() throws IOException { 073 List<KeyValue> sampleKv = new ArrayList<>(); 074 KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), 075 Bytes.toBytes("val")); 076 sampleKv.add(kv1); 077 KeyValue kv2 = new KeyValue(Bytes.toBytes("aab"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), 078 Bytes.toBytes("val")); 079 sampleKv.add(kv2); 080 KeyValue kv3 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), 081 Bytes.toBytes("val")); 082 sampleKv.add(kv3); 083 KeyValue kv4 = new KeyValue(Bytes.toBytes("aad"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), 084 Bytes.toBytes("val")); 085 sampleKv.add(kv4); 086 KeyValue kv5 = new KeyValue(Bytes.toBytes("bba"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), 087 Bytes.toBytes("val")); 088 sampleKv.add(kv5); 089 KeyValue toSeek = new KeyValue(Bytes.toBytes("aae"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), 090 Bytes.toBytes("val")); 091 seekToTheKey(kv4, sampleKv, toSeek); 092 } 093 094 /** 095 * Test seeking while file is encoded. 096 */ 097 @Test 098 public void testSeekingToBlockWithBiggerNonLength1() throws IOException { 099 List<KeyValue> sampleKv = new ArrayList<>(); 100 KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), 101 Bytes.toBytes("val")); 102 sampleKv.add(kv1); 103 KeyValue kv2 = new KeyValue(Bytes.toBytes("aab"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), 104 Bytes.toBytes("val")); 105 sampleKv.add(kv2); 106 KeyValue kv3 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), 107 Bytes.toBytes("val")); 108 sampleKv.add(kv3); 109 KeyValue kv4 = new KeyValue(Bytes.toBytes("aad"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), 110 Bytes.toBytes("val")); 111 sampleKv.add(kv4); 112 KeyValue kv5 = new KeyValue(Bytes.toBytes("aaddd"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), 113 Bytes.toBytes("val")); 114 sampleKv.add(kv5); 115 KeyValue toSeek = new KeyValue(Bytes.toBytes("aaaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), 116 Bytes.toBytes("val")); 117 seekToTheKey(kv1, sampleKv, toSeek); 118 } 119 120 /** 121 * Test seeking while file is encoded. 122 */ 123 @Test 124 public void testSeekingToBlockToANotAvailableKey() throws IOException { 125 List<KeyValue> sampleKv = new ArrayList<>(); 126 KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), 127 Bytes.toBytes("val")); 128 sampleKv.add(kv1); 129 KeyValue kv2 = new KeyValue(Bytes.toBytes("aab"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), 130 Bytes.toBytes("val")); 131 sampleKv.add(kv2); 132 KeyValue kv3 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), 133 Bytes.toBytes("val")); 134 sampleKv.add(kv3); 135 KeyValue kv4 = new KeyValue(Bytes.toBytes("aade"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), 136 Bytes.toBytes("val")); 137 sampleKv.add(kv4); 138 KeyValue kv5 = new KeyValue(Bytes.toBytes("bbbcd"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), 139 Bytes.toBytes("val")); 140 sampleKv.add(kv5); 141 KeyValue toSeek = new KeyValue(Bytes.toBytes("bbbce"), Bytes.toBytes("f1"), 142 Bytes.toBytes("q1"), Bytes.toBytes("val")); 143 seekToTheKey(kv5, sampleKv, toSeek); 144 } 145 146 /** 147 * Test seeking while file is encoded. 148 */ 149 @Test 150 public void testSeekToBlockWithDecreasingCommonPrefix() throws IOException { 151 List<KeyValue> sampleKv = new ArrayList<>(); 152 KeyValue kv1 = new KeyValue(Bytes.toBytes("row10aaa"), Bytes.toBytes("f1"), 153 Bytes.toBytes("q1"), Bytes.toBytes("val")); 154 sampleKv.add(kv1); 155 KeyValue kv2 = new KeyValue(Bytes.toBytes("row10aaa"), Bytes.toBytes("f1"), 156 Bytes.toBytes("q2"), Bytes.toBytes("val")); 157 sampleKv.add(kv2); 158 KeyValue kv3 = new KeyValue(Bytes.toBytes("row10aaa"), Bytes.toBytes("f1"), 159 Bytes.toBytes("q3"), Bytes.toBytes("val")); 160 sampleKv.add(kv3); 161 KeyValue kv4 = new KeyValue(Bytes.toBytes("row11baa"), Bytes.toBytes("f1"), 162 Bytes.toBytes("q1"), Bytes.toBytes("val")); 163 sampleKv.add(kv4); 164 Cell toSeek = PrivateCellUtil.createLastOnRow(kv3); 165 seekToTheKey(kv3, sampleKv, toSeek); 166 } 167 168 @Test 169 public void testSeekToBlockWithDiffQualifer() throws IOException { 170 List<KeyValue> sampleKv = new ArrayList<>(); 171 KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), 172 Bytes.toBytes("val")); 173 sampleKv.add(kv1); 174 KeyValue kv2 = new KeyValue(Bytes.toBytes("aab"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), 175 Bytes.toBytes("val")); 176 sampleKv.add(kv2); 177 KeyValue kv4 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), 178 Bytes.toBytes("val")); 179 sampleKv.add(kv4); 180 KeyValue kv5 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q2"), 181 Bytes.toBytes("val")); 182 sampleKv.add(kv5); 183 KeyValue toSeek = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q2"), 184 Bytes.toBytes("val")); 185 seekToTheKey(kv5, sampleKv, toSeek); 186 } 187 188 @Test 189 public void testSeekToBlockWithDiffQualiferOnSameRow() throws IOException { 190 List<KeyValue> sampleKv = new ArrayList<>(); 191 KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), 192 Bytes.toBytes("val")); 193 sampleKv.add(kv1); 194 KeyValue kv2 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q2"), 195 Bytes.toBytes("val")); 196 sampleKv.add(kv2); 197 KeyValue kv4 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q3"), 198 Bytes.toBytes("val")); 199 sampleKv.add(kv4); 200 KeyValue kv5 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q4"), 201 Bytes.toBytes("val")); 202 sampleKv.add(kv5); 203 KeyValue kv6 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q5"), 204 Bytes.toBytes("val")); 205 sampleKv.add(kv6); 206 KeyValue toSeek = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q5"), 207 Bytes.toBytes("val")); 208 seekToTheKey(kv6, sampleKv, toSeek); 209 } 210 211 @Test 212 public void testSeekToBlockWithDiffQualiferOnSameRow1() throws IOException { 213 List<KeyValue> sampleKv = new ArrayList<>(); 214 KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), 215 Bytes.toBytes("val")); 216 sampleKv.add(kv1); 217 KeyValue kv2 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q2"), 218 Bytes.toBytes("val")); 219 sampleKv.add(kv2); 220 KeyValue kv4 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q3"), 221 Bytes.toBytes("val")); 222 sampleKv.add(kv4); 223 KeyValue kv5 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q4"), 224 Bytes.toBytes("val")); 225 sampleKv.add(kv5); 226 KeyValue kv6 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("z5"), 227 Bytes.toBytes("val")); 228 sampleKv.add(kv6); 229 KeyValue toSeek = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q5"), 230 Bytes.toBytes("val")); 231 seekToTheKey(kv5, sampleKv, toSeek); 232 } 233 234 @Test 235 public void testSeekToBlockWithDiffQualiferOnSameRowButDescendingInSize() throws IOException { 236 List<KeyValue> sampleKv = new ArrayList<>(); 237 KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qual1"), 238 Bytes.toBytes("val")); 239 sampleKv.add(kv1); 240 KeyValue kv2 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qual2"), 241 Bytes.toBytes("val")); 242 sampleKv.add(kv2); 243 KeyValue kv4 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qual3"), 244 Bytes.toBytes("val")); 245 sampleKv.add(kv4); 246 KeyValue kv5 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qual4"), 247 Bytes.toBytes("val")); 248 sampleKv.add(kv5); 249 KeyValue kv6 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qz"), 250 Bytes.toBytes("val")); 251 sampleKv.add(kv6); 252 KeyValue toSeek = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qz"), 253 Bytes.toBytes("val")); 254 seekToTheKey(kv6, sampleKv, toSeek); 255 } 256 257 @Test 258 public void testSeekToBlockWithDiffFamilyAndQualifer() throws IOException { 259 List<KeyValue> sampleKv = new ArrayList<>(); 260 KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("fam1"), Bytes.toBytes("q1"), 261 Bytes.toBytes("val")); 262 sampleKv.add(kv1); 263 KeyValue kv2 = new KeyValue(Bytes.toBytes("aab"), Bytes.toBytes("fam1"), Bytes.toBytes("q1"), 264 Bytes.toBytes("val")); 265 sampleKv.add(kv2); 266 KeyValue kv4 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("fam1"), Bytes.toBytes("q1"), 267 Bytes.toBytes("val")); 268 sampleKv.add(kv4); 269 KeyValue kv5 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("fam1"), Bytes.toBytes("q2"), 270 Bytes.toBytes("val")); 271 sampleKv.add(kv5); 272 KeyValue toSeek = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("fam2"), 273 Bytes.toBytes("q2"), Bytes.toBytes("val")); 274 seekToTheKey(kv5, sampleKv, toSeek); 275 } 276 277 private void seekToTheKey(KeyValue expected, List<KeyValue> kvs, Cell toSeek) 278 throws IOException { 279 // create all seekers 280 List<DataBlockEncoder.EncodedSeeker> encodedSeekers = new ArrayList<>(); 281 for (DataBlockEncoding encoding : DataBlockEncoding.values()) { 282 if (encoding.getEncoder() == null) { 283 continue; 284 } 285 DataBlockEncoder encoder = encoding.getEncoder(); 286 HFileContext meta = new HFileContextBuilder().withHBaseCheckSum(false) 287 .withIncludesMvcc(false).withIncludesTags(false) 288 .withCompression(Compression.Algorithm.NONE).build(); 289 HFileBlockEncodingContext encodingContext = encoder.newDataBlockEncodingContext(encoding, 290 HFILEBLOCK_DUMMY_HEADER, meta); 291 ByteBuffer encodedBuffer = TestDataBlockEncoders.encodeKeyValues(encoding, kvs, 292 encodingContext, this.useOffheapData); 293 DataBlockEncoder.EncodedSeeker seeker = encoder.createSeeker(CellComparatorImpl.COMPARATOR, 294 encoder.newDataBlockDecodingContext(meta)); 295 seeker.setCurrentBuffer(new SingleByteBuff(encodedBuffer)); 296 encodedSeekers.add(seeker); 297 } 298 // test it! 299 // try a few random seeks 300 checkSeekingConsistency(encodedSeekers, toSeek, expected); 301 } 302 303 private void checkSeekingConsistency(List<DataBlockEncoder.EncodedSeeker> encodedSeekers, 304 Cell keyValue, KeyValue expected) { 305 for (DataBlockEncoder.EncodedSeeker seeker : encodedSeekers) { 306 seeker.seekToKeyInBlock(keyValue, false); 307 Cell keyValue2 = seeker.getCell(); 308 assertEquals(expected, keyValue2); 309 seeker.rewind(); 310 } 311 } 312}