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