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