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.filter; 019 020import static org.junit.Assert.assertEquals; 021import static org.junit.Assert.assertFalse; 022import static org.junit.Assert.assertTrue; 023 024import java.io.IOException; 025import java.util.ArrayList; 026import java.util.Arrays; 027import java.util.List; 028import org.apache.hadoop.hbase.Cell; 029import org.apache.hadoop.hbase.CellComparator; 030import org.apache.hadoop.hbase.CellUtil; 031import org.apache.hadoop.hbase.CompareOperator; 032import org.apache.hadoop.hbase.HBaseClassTestRule; 033import org.apache.hadoop.hbase.HBaseTestingUtil; 034import org.apache.hadoop.hbase.HConstants; 035import org.apache.hadoop.hbase.KeyValue; 036import org.apache.hadoop.hbase.TableName; 037import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; 038import org.apache.hadoop.hbase.client.Delete; 039import org.apache.hadoop.hbase.client.Durability; 040import org.apache.hadoop.hbase.client.Put; 041import org.apache.hadoop.hbase.client.RegionInfo; 042import org.apache.hadoop.hbase.client.RegionInfoBuilder; 043import org.apache.hadoop.hbase.client.Scan; 044import org.apache.hadoop.hbase.client.TableDescriptor; 045import org.apache.hadoop.hbase.client.TableDescriptorBuilder; 046import org.apache.hadoop.hbase.filter.FilterList.Operator; 047import org.apache.hadoop.hbase.regionserver.HRegion; 048import org.apache.hadoop.hbase.regionserver.InternalScanner; 049import org.apache.hadoop.hbase.regionserver.RegionScanner; 050import org.apache.hadoop.hbase.testclassification.FilterTests; 051import org.apache.hadoop.hbase.testclassification.MediumTests; 052import org.apache.hadoop.hbase.util.Bytes; 053import org.apache.hadoop.hbase.wal.WAL; 054import org.junit.After; 055import org.junit.Assert; 056import org.junit.Before; 057import org.junit.ClassRule; 058import org.junit.Ignore; 059import org.junit.Rule; 060import org.junit.Test; 061import org.junit.experimental.categories.Category; 062import org.junit.rules.TestName; 063import org.slf4j.Logger; 064import org.slf4j.LoggerFactory; 065 066import org.apache.hbase.thirdparty.com.google.common.base.Throwables; 067 068/** 069 * Test filters at the HRegion doorstep. 070 */ 071@Category({ FilterTests.class, MediumTests.class }) 072public class TestFilter { 073 074 @ClassRule 075 public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass(TestFilter.class); 076 077 private final static Logger LOG = LoggerFactory.getLogger(TestFilter.class); 078 private HRegion region; 079 private final static HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); 080 081 @Rule 082 public TestName name = new TestName(); 083 084 // 085 // Rows, Qualifiers, and Values are in two groups, One and Two. 086 // 087 088 private static final byte[][] ROWS_ONE = { Bytes.toBytes("testRowOne-0"), 089 Bytes.toBytes("testRowOne-1"), Bytes.toBytes("testRowOne-2"), Bytes.toBytes("testRowOne-3") }; 090 091 private static final byte[][] ROWS_TWO = { Bytes.toBytes("testRowTwo-0"), 092 Bytes.toBytes("testRowTwo-1"), Bytes.toBytes("testRowTwo-2"), Bytes.toBytes("testRowTwo-3") }; 093 094 private static final byte[][] ROWS_THREE = 095 { Bytes.toBytes("testRowThree-0"), Bytes.toBytes("testRowThree-1"), 096 Bytes.toBytes("testRowThree-2"), Bytes.toBytes("testRowThree-3") }; 097 098 private static final byte[][] ROWS_FOUR = 099 { Bytes.toBytes("testRowFour-0"), Bytes.toBytes("testRowFour-1"), 100 Bytes.toBytes("testRowFour-2"), Bytes.toBytes("testRowFour-3") }; 101 102 private static final byte[][] FAMILIES = 103 { Bytes.toBytes("testFamilyOne"), Bytes.toBytes("testFamilyTwo") }; 104 105 private static final byte[][] FAMILIES_1 = 106 { Bytes.toBytes("testFamilyThree"), Bytes.toBytes("testFamilyFour") }; 107 108 private static final byte[][] QUALIFIERS_ONE = 109 { Bytes.toBytes("testQualifierOne-0"), Bytes.toBytes("testQualifierOne-1"), 110 Bytes.toBytes("testQualifierOne-2"), Bytes.toBytes("testQualifierOne-3") }; 111 112 private static final byte[][] QUALIFIERS_TWO = 113 { Bytes.toBytes("testQualifierTwo-0"), Bytes.toBytes("testQualifierTwo-1"), 114 Bytes.toBytes("testQualifierTwo-2"), Bytes.toBytes("testQualifierTwo-3") }; 115 116 private static final byte[][] QUALIFIERS_THREE = 117 { Bytes.toBytes("testQualifierThree-0"), Bytes.toBytes("testQualifierThree-1"), 118 Bytes.toBytes("testQualifierThree-2"), Bytes.toBytes("testQualifierThree-3") }; 119 120 private static final byte[][] QUALIFIERS_FOUR = 121 { Bytes.toBytes("testQualifierFour-0"), Bytes.toBytes("testQualifierFour-1"), 122 Bytes.toBytes("testQualifierFour-2"), Bytes.toBytes("testQualifierFour-3") }; 123 124 private static final byte[][] QUALIFIERS_FIVE = 125 { Bytes.toBytes("testQualifierFive-0"), Bytes.toBytes("testQualifierFive-1") }; 126 127 private static final byte[][] VALUES = 128 { Bytes.toBytes("testValueOne"), Bytes.toBytes("testValueTwo") }; 129 130 byte[][] NEW_FAMILIES = { Bytes.toBytes("f1"), Bytes.toBytes("f2") }; 131 132 private long numRows = (long) ROWS_ONE.length + ROWS_TWO.length; 133 private long colsPerRow = (long) FAMILIES.length * QUALIFIERS_ONE.length; 134 135 @Before 136 public void setUp() throws Exception { 137 TableDescriptor tableDescriptor = TableDescriptorBuilder 138 .newBuilder(TableName.valueOf("TestFilter")) 139 .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILIES[0]).setMinVersions(100) 140 .setMaxVersions(100).build()) 141 .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILIES[1])) 142 .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILIES_1[0])) 143 .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILIES_1[1])) 144 .setColumnFamily(ColumnFamilyDescriptorBuilder.of(NEW_FAMILIES[0])) 145 .setColumnFamily(ColumnFamilyDescriptorBuilder.of(NEW_FAMILIES[1])).build(); 146 RegionInfo info = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build(); 147 this.region = HBaseTestingUtil.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), 148 TEST_UTIL.getConfiguration(), tableDescriptor); 149 150 // Insert first half 151 for (byte[] ROW : ROWS_ONE) { 152 Put p = new Put(ROW); 153 p.setDurability(Durability.SKIP_WAL); 154 for (byte[] QUALIFIER : QUALIFIERS_ONE) { 155 p.addColumn(FAMILIES[0], QUALIFIER, VALUES[0]); 156 } 157 this.region.put(p); 158 } 159 for (byte[] ROW : ROWS_TWO) { 160 Put p = new Put(ROW); 161 p.setDurability(Durability.SKIP_WAL); 162 for (byte[] QUALIFIER : QUALIFIERS_TWO) { 163 p.addColumn(FAMILIES[1], QUALIFIER, VALUES[1]); 164 } 165 this.region.put(p); 166 } 167 168 // Flush 169 this.region.flush(true); 170 171 // Insert second half (reverse families) 172 for (byte[] ROW : ROWS_ONE) { 173 Put p = new Put(ROW); 174 p.setDurability(Durability.SKIP_WAL); 175 for (byte[] QUALIFIER : QUALIFIERS_ONE) { 176 p.addColumn(FAMILIES[1], QUALIFIER, VALUES[0]); 177 } 178 this.region.put(p); 179 } 180 for (byte[] ROW : ROWS_TWO) { 181 Put p = new Put(ROW); 182 p.setDurability(Durability.SKIP_WAL); 183 for (byte[] QUALIFIER : QUALIFIERS_TWO) { 184 p.addColumn(FAMILIES[0], QUALIFIER, VALUES[1]); 185 } 186 this.region.put(p); 187 } 188 189 // Delete the second qualifier from all rows and families 190 for (byte[] ROW : ROWS_ONE) { 191 Delete d = new Delete(ROW); 192 d.addColumns(FAMILIES[0], QUALIFIERS_ONE[1]); 193 d.addColumns(FAMILIES[1], QUALIFIERS_ONE[1]); 194 this.region.delete(d); 195 } 196 for (byte[] ROW : ROWS_TWO) { 197 Delete d = new Delete(ROW); 198 d.addColumns(FAMILIES[0], QUALIFIERS_TWO[1]); 199 d.addColumns(FAMILIES[1], QUALIFIERS_TWO[1]); 200 this.region.delete(d); 201 } 202 colsPerRow -= 2; 203 204 // Delete the second rows from both groups, one column at a time 205 for (byte[] QUALIFIER : QUALIFIERS_ONE) { 206 Delete d = new Delete(ROWS_ONE[1]); 207 d.addColumns(FAMILIES[0], QUALIFIER); 208 d.addColumns(FAMILIES[1], QUALIFIER); 209 this.region.delete(d); 210 } 211 for (byte[] QUALIFIER : QUALIFIERS_TWO) { 212 Delete d = new Delete(ROWS_TWO[1]); 213 d.addColumns(FAMILIES[0], QUALIFIER); 214 d.addColumns(FAMILIES[1], QUALIFIER); 215 this.region.delete(d); 216 } 217 numRows -= 2; 218 } 219 220 @After 221 public void tearDown() throws Exception { 222 HBaseTestingUtil.closeRegionAndWAL(region); 223 } 224 225 @Test 226 public void testRegionScannerReseek() throws Exception { 227 // create new rows and column family to show how reseek works.. 228 for (byte[] ROW : ROWS_THREE) { 229 Put p = new Put(ROW); 230 p.setDurability(Durability.SKIP_WAL); 231 for (byte[] QUALIFIER : QUALIFIERS_THREE) { 232 p.addColumn(FAMILIES[0], QUALIFIER, VALUES[0]); 233 234 } 235 this.region.put(p); 236 } 237 for (byte[] ROW : ROWS_FOUR) { 238 Put p = new Put(ROW); 239 p.setDurability(Durability.SKIP_WAL); 240 for (byte[] QUALIFIER : QUALIFIERS_FOUR) { 241 p.addColumn(FAMILIES[1], QUALIFIER, VALUES[1]); 242 } 243 this.region.put(p); 244 } 245 // Flush 246 this.region.flush(true); 247 248 // Insert second half (reverse families) 249 for (byte[] ROW : ROWS_THREE) { 250 Put p = new Put(ROW); 251 p.setDurability(Durability.SKIP_WAL); 252 for (byte[] QUALIFIER : QUALIFIERS_THREE) { 253 p.addColumn(FAMILIES[1], QUALIFIER, VALUES[0]); 254 } 255 this.region.put(p); 256 } 257 for (byte[] ROW : ROWS_FOUR) { 258 Put p = new Put(ROW); 259 p.setDurability(Durability.SKIP_WAL); 260 for (byte[] QUALIFIER : QUALIFIERS_FOUR) { 261 p.addColumn(FAMILIES[0], QUALIFIER, VALUES[1]); 262 } 263 this.region.put(p); 264 } 265 266 Scan s = new Scan(); 267 // set a start row 268 s.withStartRow(ROWS_FOUR[1]); 269 RegionScanner scanner = region.getScanner(s); 270 271 // reseek to row three. 272 scanner.reseek(ROWS_THREE[1]); 273 List<Cell> results = new ArrayList<>(); 274 275 // the results should belong to ROWS_THREE[1] 276 scanner.next(results); 277 for (Cell keyValue : results) { 278 assertTrue("The rows with ROWS_TWO as row key should be appearing.", 279 CellUtil.matchingRows(keyValue, ROWS_THREE[1])); 280 } 281 // again try to reseek to a value before ROWS_THREE[1] 282 scanner.reseek(ROWS_ONE[1]); 283 results = new ArrayList<>(); 284 // This time no seek would have been done to ROWS_ONE[1] 285 scanner.next(results); 286 for (Cell keyValue : results) { 287 assertFalse("Cannot rewind back to a value less than previous reseek.", 288 Bytes.toString(CellUtil.cloneRow(keyValue)).contains("testRowOne")); 289 } 290 } 291 292 @Test 293 public void testNoFilter() throws Exception { 294 // No filter 295 long expectedRows = this.numRows; 296 long expectedKeys = this.colsPerRow; 297 298 // Both families 299 Scan s = new Scan(); 300 verifyScan(s, expectedRows, expectedKeys); 301 302 // One family 303 s = new Scan(); 304 s.addFamily(FAMILIES[0]); 305 verifyScan(s, expectedRows, expectedKeys / 2); 306 } 307 308 @Test 309 public void testPrefixFilter() throws Exception { 310 // Grab rows from group one (half of total) 311 long expectedRows = this.numRows / 2; 312 long expectedKeys = this.colsPerRow; 313 Scan s = new Scan(); 314 s.setFilter(new PrefixFilter(Bytes.toBytes("testRowOne"))); 315 verifyScan(s, expectedRows, expectedKeys); 316 } 317 318 @Test 319 public void testPrefixFilterWithReverseScan() throws Exception { 320 // Grab rows from group one (half of total) 321 long expectedRows = this.numRows / 2; 322 long expectedKeys = this.colsPerRow; 323 Scan s = new Scan(); 324 s.setReversed(true); 325 s.setFilter(new PrefixFilter(Bytes.toBytes("testRowOne"))); 326 verifyScan(s, expectedRows, expectedKeys); 327 } 328 329 @Test 330 public void testPageFilter() throws Exception { 331 332 // KVs in first 6 rows 333 KeyValue[] expectedKVs = { 334 // testRowOne-0 335 new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), 336 new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]), 337 new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]), 338 new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]), 339 new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]), 340 new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]), 341 // testRowOne-2 342 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), 343 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]), 344 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]), 345 new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]), 346 new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]), 347 new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]), 348 // testRowOne-3 349 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), 350 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]), 351 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]), 352 new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]), 353 new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]), 354 new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]), 355 // testRowTwo-0 356 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 357 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 358 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 359 new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]), 360 new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]), 361 new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]), 362 // testRowTwo-2 363 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 364 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 365 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 366 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]), 367 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]), 368 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]), 369 // testRowTwo-3 370 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 371 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 372 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 373 new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]), 374 new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]), 375 new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]) }; 376 377 // Grab all 6 rows 378 long expectedRows = 6; 379 long expectedKeys = this.colsPerRow; 380 Scan s = new Scan(); 381 s.setFilter(new PageFilter(expectedRows)); 382 verifyScan(s, expectedRows, expectedKeys); 383 s.setFilter(new PageFilter(expectedRows)); 384 verifyScanFull(s, expectedKVs); 385 386 // Grab first 4 rows (6 cols per row) 387 expectedRows = 4; 388 expectedKeys = this.colsPerRow; 389 s = new Scan(); 390 s.setFilter(new PageFilter(expectedRows)); 391 verifyScan(s, expectedRows, expectedKeys); 392 s.setFilter(new PageFilter(expectedRows)); 393 verifyScanFull(s, Arrays.copyOf(expectedKVs, 24)); 394 395 // Grab first 2 rows 396 expectedRows = 2; 397 expectedKeys = this.colsPerRow; 398 s = new Scan(); 399 s.setFilter(new PageFilter(expectedRows)); 400 verifyScan(s, expectedRows, expectedKeys); 401 s.setFilter(new PageFilter(expectedRows)); 402 verifyScanFull(s, Arrays.copyOf(expectedKVs, 12)); 403 404 // Grab first row 405 expectedRows = 1; 406 expectedKeys = this.colsPerRow; 407 s = new Scan(); 408 s.setFilter(new PageFilter(expectedRows)); 409 verifyScan(s, expectedRows, expectedKeys); 410 s.setFilter(new PageFilter(expectedRows)); 411 verifyScanFull(s, Arrays.copyOf(expectedKVs, 6)); 412 413 } 414 415 @Test 416 public void testPageFilterWithReverseScan() throws Exception { 417 // KVs in first 6 rows 418 KeyValue[] expectedKVs = { 419 // testRowOne-0 420 new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), 421 new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]), 422 new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]), 423 new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]), 424 new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]), 425 new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]), 426 // testRowOne-2 427 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), 428 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]), 429 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]), 430 new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]), 431 new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]), 432 new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]), 433 // testRowOne-3 434 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), 435 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]), 436 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]), 437 new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]), 438 new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]), 439 new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]), 440 // testRowTwo-0 441 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 442 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 443 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 444 new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]), 445 new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]), 446 new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]), 447 // testRowTwo-2 448 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 449 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 450 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 451 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]), 452 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]), 453 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]), 454 // testRowTwo-3 455 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 456 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 457 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 458 new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]), 459 new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]), 460 new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]) }; 461 462 // Grab all 6 rows 463 long expectedRows = 6; 464 long expectedKeys = this.colsPerRow; 465 Scan s = new Scan(); 466 s.setReversed(true); 467 s.setFilter(new PageFilter(expectedRows)); 468 verifyScan(s, expectedRows, expectedKeys); 469 470 // Grab first 4 rows (6 cols per row) 471 expectedRows = 4; 472 expectedKeys = this.colsPerRow; 473 s = new Scan(); 474 s.setReversed(true); 475 s.setFilter(new PageFilter(expectedRows)); 476 verifyScan(s, expectedRows, expectedKeys); 477 478 // Grab first 2 rows 479 expectedRows = 2; 480 expectedKeys = this.colsPerRow; 481 s = new Scan(); 482 s.setReversed(true); 483 s.setFilter(new PageFilter(expectedRows)); 484 verifyScan(s, expectedRows, expectedKeys); 485 486 // Grab first row 487 expectedRows = 1; 488 expectedKeys = this.colsPerRow; 489 s = new Scan(); 490 s.setReversed(true); 491 s.setFilter(new PageFilter(expectedRows)); 492 verifyScan(s, expectedRows, expectedKeys); 493 } 494 495 @Test 496 public void testWhileMatchFilterWithFilterRowWithReverseScan() throws Exception { 497 final int pageSize = 4; 498 499 Scan s = new Scan(); 500 s.setReversed(true); 501 WhileMatchFilter filter = new WhileMatchFilter(new PageFilter(pageSize)); 502 s.setFilter(filter); 503 504 InternalScanner scanner = this.region.getScanner(s); 505 int scannerCounter = 0; 506 while (true) { 507 boolean isMoreResults = scanner.next(new ArrayList<>()); 508 scannerCounter++; 509 510 if (scannerCounter >= pageSize) { 511 Assert.assertTrue("The WhileMatchFilter should now filter all remaining", 512 filter.filterAllRemaining()); 513 } 514 if (!isMoreResults) { 515 break; 516 } 517 } 518 scanner.close(); 519 Assert.assertEquals("The page filter returned more rows than expected", pageSize, 520 scannerCounter); 521 } 522 523 @Test 524 public void testWhileMatchFilterWithFilterRowKeyWithReverseScan() throws Exception { 525 Scan s = new Scan(); 526 String prefix = "testRowOne"; 527 WhileMatchFilter filter = new WhileMatchFilter(new PrefixFilter(Bytes.toBytes(prefix))); 528 s.setFilter(filter); 529 s.setReversed(true); 530 531 InternalScanner scanner = this.region.getScanner(s); 532 while (true) { 533 ArrayList<Cell> values = new ArrayList<>(); 534 boolean isMoreResults = scanner.next(values); 535 if (!isMoreResults || !Bytes.toString(CellUtil.cloneRow(values.get(0))).startsWith(prefix)) { 536 Assert.assertTrue("The WhileMatchFilter should now filter all remaining", 537 filter.filterAllRemaining()); 538 } 539 if (!isMoreResults) { 540 break; 541 } 542 } 543 scanner.close(); 544 } 545 546 /** 547 * Tests the the {@link WhileMatchFilter} works in combination with a {@link Filter} that uses the 548 * {@link Filter#filterRow()} method. See HBASE-2258. 549 */ 550 @Test 551 public void testWhileMatchFilterWithFilterRow() throws Exception { 552 final int pageSize = 4; 553 554 Scan s = new Scan(); 555 WhileMatchFilter filter = new WhileMatchFilter(new PageFilter(pageSize)); 556 s.setFilter(filter); 557 558 InternalScanner scanner = this.region.getScanner(s); 559 int scannerCounter = 0; 560 while (true) { 561 boolean isMoreResults = scanner.next(new ArrayList<>()); 562 scannerCounter++; 563 564 if (scannerCounter >= pageSize) { 565 assertTrue("The WhileMatchFilter should now filter all remaining", 566 filter.filterAllRemaining()); 567 } 568 if (!isMoreResults) { 569 break; 570 } 571 } 572 assertEquals("The page filter returned more rows than expected", pageSize, scannerCounter); 573 } 574 575 /** 576 * The following filter simulates a pre-0.96 filter where filterRow() is defined while 577 * hasFilterRow() returns false 578 */ 579 static class OldTestFilter extends FilterBase { 580 @Override 581 public byte[] toByteArray() { 582 return null; 583 } 584 585 @Override 586 public boolean hasFilterRow() { 587 return false; 588 } 589 590 @Override 591 public boolean filterRow() { 592 // always filter out rows 593 return true; 594 } 595 596 @Override 597 public ReturnCode filterCell(final Cell ignored) throws IOException { 598 return ReturnCode.INCLUDE; 599 } 600 } 601 602 /** 603 * The following test is to ensure old(such as hbase0.94) filterRow() can be correctly fired in 604 * 0.96+ code base. See HBASE-10366 605 */ 606 @Test 607 public void test94FilterRowCompatibility() throws Exception { 608 Scan s = new Scan(); 609 OldTestFilter filter = new OldTestFilter(); 610 s.setFilter(filter); 611 612 InternalScanner scanner = this.region.getScanner(s); 613 ArrayList<Cell> values = new ArrayList<>(); 614 scanner.next(values); 615 assertTrue("All rows should be filtered out", values.isEmpty()); 616 } 617 618 /** 619 * Tests the the {@link WhileMatchFilter} works in combination with a {@link Filter} that uses the 620 * {@link Filter#filterRowKey(Cell)} method. See HBASE-2258. 621 */ 622 @Test 623 public void testWhileMatchFilterWithFilterRowKey() throws Exception { 624 Scan s = new Scan(); 625 String prefix = "testRowOne"; 626 WhileMatchFilter filter = new WhileMatchFilter(new PrefixFilter(Bytes.toBytes(prefix))); 627 s.setFilter(filter); 628 629 InternalScanner scanner = this.region.getScanner(s); 630 while (true) { 631 ArrayList<Cell> values = new ArrayList<>(); 632 boolean isMoreResults = scanner.next(values); 633 if (!isMoreResults || !Bytes.toString(CellUtil.cloneRow(values.get(0))).startsWith(prefix)) { 634 assertTrue("The WhileMatchFilter should now filter all remaining", 635 filter.filterAllRemaining()); 636 } 637 if (!isMoreResults) { 638 break; 639 } 640 } 641 } 642 643 /** 644 * Tests the the {@link WhileMatchFilter} works in combination with a {@link Filter} that uses the 645 * {@link Filter#filterCell(Cell)} method. See HBASE-2258. 646 */ 647 @Test 648 public void testWhileMatchFilterWithFilterCell() throws Exception { 649 Scan s = new Scan(); 650 WhileMatchFilter filter = new WhileMatchFilter(new SingleColumnValueFilter(FAMILIES[0], 651 QUALIFIERS_ONE[0], CompareOperator.EQUAL, Bytes.toBytes("foo"))); 652 s.setFilter(filter); 653 654 InternalScanner scanner = this.region.getScanner(s); 655 while (true) { 656 ArrayList<Cell> values = new ArrayList<>(); 657 boolean isMoreResults = scanner.next(values); 658 assertTrue("The WhileMatchFilter should now filter all remaining", 659 filter.filterAllRemaining()); 660 if (!isMoreResults) { 661 break; 662 } 663 } 664 } 665 666 @Test 667 public void testInclusiveStopFilter() throws IOException { 668 669 // Grab rows from group one 670 671 // If we just use start/stop row, we get total/2 - 1 rows 672 long expectedRows = (this.numRows / 2) - 1; 673 long expectedKeys = this.colsPerRow; 674 Scan s = new Scan().withStartRow(Bytes.toBytes("testRowOne-0")) 675 .withStopRow(Bytes.toBytes("testRowOne-3")); 676 verifyScan(s, expectedRows, expectedKeys); 677 678 // Now use start row with inclusive stop filter 679 expectedRows = this.numRows / 2; 680 s = new Scan().withStartRow(Bytes.toBytes("testRowOne-0")); 681 s.setFilter(new InclusiveStopFilter(Bytes.toBytes("testRowOne-3"))); 682 verifyScan(s, expectedRows, expectedKeys); 683 684 // Grab rows from group two 685 686 // If we just use start/stop row, we get total/2 - 1 rows 687 expectedRows = (this.numRows / 2) - 1; 688 expectedKeys = this.colsPerRow; 689 s = new Scan().withStartRow(Bytes.toBytes("testRowTwo-0")) 690 .withStopRow(Bytes.toBytes("testRowTwo-3")); 691 verifyScan(s, expectedRows, expectedKeys); 692 693 // Now use start row with inclusive stop filter 694 expectedRows = this.numRows / 2; 695 s = new Scan().withStartRow(Bytes.toBytes("testRowTwo-0")); 696 s.setFilter(new InclusiveStopFilter(Bytes.toBytes("testRowTwo-3"))); 697 verifyScan(s, expectedRows, expectedKeys); 698 699 } 700 701 @Test 702 public void testInclusiveStopFilterWithReverseScan() throws IOException { 703 704 // Grab rows from group one 705 706 // If we just use start/stop row, we get total/2 - 1 rows 707 long expectedRows = (this.numRows / 2) - 1; 708 long expectedKeys = this.colsPerRow; 709 Scan s = new Scan().withStartRow(Bytes.toBytes("testRowOne-3")) 710 .withStopRow(Bytes.toBytes("testRowOne-0")); 711 s.setReversed(true); 712 verifyScan(s, expectedRows, expectedKeys); 713 714 // Now use start row with inclusive stop filter 715 expectedRows = this.numRows / 2; 716 s = new Scan().withStartRow(Bytes.toBytes("testRowOne-3")); 717 s.setReversed(true); 718 s.setFilter(new InclusiveStopFilter(Bytes.toBytes("testRowOne-0"))); 719 verifyScan(s, expectedRows, expectedKeys); 720 721 // Grab rows from group two 722 723 // If we just use start/stop row, we get total/2 - 1 rows 724 expectedRows = (this.numRows / 2) - 1; 725 expectedKeys = this.colsPerRow; 726 s = new Scan().withStartRow(Bytes.toBytes("testRowTwo-3")) 727 .withStopRow(Bytes.toBytes("testRowTwo-0")); 728 s.setReversed(true); 729 verifyScan(s, expectedRows, expectedKeys); 730 731 // Now use start row with inclusive stop filter 732 expectedRows = this.numRows / 2; 733 s = new Scan().withStartRow(Bytes.toBytes("testRowTwo-3")); 734 s.setReversed(true); 735 s.setFilter(new InclusiveStopFilter(Bytes.toBytes("testRowTwo-0"))); 736 verifyScan(s, expectedRows, expectedKeys); 737 738 } 739 740 @Test 741 public void testQualifierFilter() throws IOException { 742 743 // Match two keys (one from each family) in half the rows 744 long expectedRows = this.numRows / 2; 745 long expectedKeys = 2; 746 Filter f = new QualifierFilter(CompareOperator.EQUAL, 747 new BinaryComparator(Bytes.toBytes("testQualifierOne-2"))); 748 Scan s = new Scan(); 749 s.setFilter(f); 750 verifyScanNoEarlyOut(s, expectedRows, expectedKeys); 751 752 // Match keys less than same qualifier 753 // Expect only two keys (one from each family) in half the rows 754 expectedRows = this.numRows / 2; 755 expectedKeys = 2; 756 f = new QualifierFilter(CompareOperator.LESS, 757 new BinaryComparator(Bytes.toBytes("testQualifierOne-2"))); 758 s = new Scan(); 759 s.setFilter(f); 760 verifyScanNoEarlyOut(s, expectedRows, expectedKeys); 761 762 // Match keys less than or equal 763 // Expect four keys (two from each family) in half the rows 764 expectedRows = this.numRows / 2; 765 expectedKeys = 4; 766 f = new QualifierFilter(CompareOperator.LESS_OR_EQUAL, 767 new BinaryComparator(Bytes.toBytes("testQualifierOne-2"))); 768 s = new Scan(); 769 s.setFilter(f); 770 verifyScanNoEarlyOut(s, expectedRows, expectedKeys); 771 772 // Match keys not equal 773 // Expect four keys (two from each family) 774 // Only look in first group of rows 775 expectedRows = this.numRows / 2; 776 expectedKeys = 4; 777 f = new QualifierFilter(CompareOperator.NOT_EQUAL, 778 new BinaryComparator(Bytes.toBytes("testQualifierOne-2"))); 779 s = 780 new Scan().withStartRow(HConstants.EMPTY_START_ROW).withStopRow(Bytes.toBytes("testRowTwo")); 781 s.setFilter(f); 782 verifyScanNoEarlyOut(s, expectedRows, expectedKeys); 783 784 // Match keys greater or equal 785 // Expect four keys (two from each family) 786 // Only look in first group of rows 787 expectedRows = this.numRows / 2; 788 expectedKeys = 4; 789 f = new QualifierFilter(CompareOperator.GREATER_OR_EQUAL, 790 new BinaryComparator(Bytes.toBytes("testQualifierOne-2"))); 791 s = 792 new Scan().withStartRow(HConstants.EMPTY_START_ROW).withStopRow(Bytes.toBytes("testRowTwo")); 793 s.setFilter(f); 794 verifyScanNoEarlyOut(s, expectedRows, expectedKeys); 795 796 // Match keys greater 797 // Expect two keys (one from each family) 798 // Only look in first group of rows 799 expectedRows = this.numRows / 2; 800 expectedKeys = 2; 801 f = new QualifierFilter(CompareOperator.GREATER, 802 new BinaryComparator(Bytes.toBytes("testQualifierOne-2"))); 803 s = 804 new Scan().withStartRow(HConstants.EMPTY_START_ROW).withStopRow(Bytes.toBytes("testRowTwo")); 805 s.setFilter(f); 806 verifyScanNoEarlyOut(s, expectedRows, expectedKeys); 807 808 // Match keys not equal to 809 // Look across rows and fully validate the keys and ordering 810 // Expect varied numbers of keys, 4 per row in group one, 6 per row in group two 811 f = new QualifierFilter(CompareOperator.NOT_EQUAL, new BinaryComparator(QUALIFIERS_ONE[2])); 812 s = new Scan(); 813 s.setFilter(f); 814 815 KeyValue[] kvs = { 816 // testRowOne-0 817 new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), 818 new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]), 819 new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]), 820 new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]), 821 // testRowOne-2 822 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), 823 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]), 824 new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]), 825 new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]), 826 // testRowOne-3 827 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), 828 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]), 829 new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]), 830 new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]), 831 // testRowTwo-0 832 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 833 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 834 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 835 new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]), 836 new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]), 837 new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]), 838 // testRowTwo-2 839 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 840 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 841 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 842 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]), 843 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]), 844 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]), 845 // testRowTwo-3 846 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 847 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 848 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 849 new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]), 850 new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]), 851 new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]), }; 852 verifyScanFull(s, kvs); 853 854 // Test across rows and groups with a regex 855 // Filter out "test*-2" 856 // Expect 4 keys per row across both groups 857 f = new QualifierFilter(CompareOperator.NOT_EQUAL, new RegexStringComparator("test.+-2")); 858 s = new Scan(); 859 s.setFilter(f); 860 861 kvs = new KeyValue[] { 862 // testRowOne-0 863 new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), 864 new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]), 865 new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]), 866 new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]), 867 // testRowOne-2 868 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), 869 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]), 870 new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]), 871 new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]), 872 // testRowOne-3 873 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), 874 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]), 875 new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]), 876 new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]), 877 // testRowTwo-0 878 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 879 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 880 new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]), 881 new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]), 882 // testRowTwo-2 883 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 884 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 885 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]), 886 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]), 887 // testRowTwo-3 888 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 889 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 890 new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]), 891 new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]), }; 892 verifyScanFull(s, kvs); 893 894 } 895 896 @Test 897 public void testFamilyFilter() throws IOException { 898 // Match family, only half of columns returned. 899 long expectedRows = this.numRows; 900 long expectedKeys = this.colsPerRow / 2; 901 Filter f = 902 new FamilyFilter(CompareOperator.EQUAL, new BinaryComparator(Bytes.toBytes("testFamilyOne"))); 903 Scan s = new Scan(); 904 s.setFilter(f); 905 verifyScanNoEarlyOut(s, expectedRows, expectedKeys); 906 907 // Match keys less than given family, should return nothing 908 expectedRows = 0; 909 expectedKeys = 0; 910 f = new FamilyFilter(CompareOperator.LESS, new BinaryComparator(Bytes.toBytes("testFamily"))); 911 s = new Scan(); 912 s.setFilter(f); 913 verifyScanNoEarlyOut(s, expectedRows, expectedKeys); 914 915 // Match keys less than or equal, should return half of columns 916 expectedRows = this.numRows; 917 expectedKeys = this.colsPerRow / 2; 918 f = new FamilyFilter(CompareOperator.LESS_OR_EQUAL, 919 new BinaryComparator(Bytes.toBytes("testFamilyOne"))); 920 s = new Scan(); 921 s.setFilter(f); 922 verifyScanNoEarlyOut(s, expectedRows, expectedKeys); 923 924 // Match keys from second family 925 // look only in second group of rows 926 expectedRows = this.numRows / 2; 927 expectedKeys = this.colsPerRow / 2; 928 f = new FamilyFilter(CompareOperator.NOT_EQUAL, 929 new BinaryComparator(Bytes.toBytes("testFamilyOne"))); 930 s = 931 new Scan().withStartRow(HConstants.EMPTY_START_ROW).withStopRow(Bytes.toBytes("testRowTwo")); 932 s.setFilter(f); 933 verifyScanNoEarlyOut(s, expectedRows, expectedKeys); 934 935 // Match all columns 936 // look only in second group of rows 937 expectedRows = this.numRows / 2; 938 expectedKeys = this.colsPerRow; 939 f = new FamilyFilter(CompareOperator.GREATER_OR_EQUAL, 940 new BinaryComparator(Bytes.toBytes("testFamilyOne"))); 941 s = 942 new Scan().withStartRow(HConstants.EMPTY_START_ROW).withStopRow(Bytes.toBytes("testRowTwo")); 943 s.setFilter(f); 944 verifyScanNoEarlyOut(s, expectedRows, expectedKeys); 945 946 // Match all columns in second family 947 // look only in second group of rows 948 expectedRows = this.numRows / 2; 949 expectedKeys = this.colsPerRow / 2; 950 f = new FamilyFilter(CompareOperator.GREATER, 951 new BinaryComparator(Bytes.toBytes("testFamilyOne"))); 952 s = 953 new Scan().withStartRow(HConstants.EMPTY_START_ROW).withStopRow(Bytes.toBytes("testRowTwo")); 954 s.setFilter(f); 955 verifyScanNoEarlyOut(s, expectedRows, expectedKeys); 956 957 // Match keys not equal to given family 958 // Look across rows and fully validate the keys and ordering 959 f = new FamilyFilter(CompareOperator.NOT_EQUAL, new BinaryComparator(FAMILIES[1])); 960 s = new Scan(); 961 s.setFilter(f); 962 963 KeyValue[] kvs = { 964 // testRowOne-0 965 new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), 966 new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]), 967 new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]), 968 // testRowOne-2 969 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), 970 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]), 971 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]), 972 // testRowOne-3 973 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), 974 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]), 975 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]), 976 // testRowTwo-0 977 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 978 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 979 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 980 // testRowTwo-2 981 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 982 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 983 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 984 // testRowTwo-3 985 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 986 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 987 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), }; 988 verifyScanFull(s, kvs); 989 990 // Test across rows and groups with a regex 991 // Filter out "test*-2" 992 // Expect 4 keys per row across both groups 993 f = new FamilyFilter(CompareOperator.NOT_EQUAL, new RegexStringComparator("test.*One")); 994 s = new Scan(); 995 s.setFilter(f); 996 997 kvs = new KeyValue[] { 998 // testRowOne-0 999 new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]), 1000 new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]), 1001 new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]), 1002 // testRowOne-2 1003 new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]), 1004 new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]), 1005 new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]), 1006 // testRowOne-3 1007 new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]), 1008 new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]), 1009 new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]), 1010 // testRowTwo-0 1011 new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]), 1012 new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]), 1013 new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]), 1014 // testRowTwo-2 1015 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]), 1016 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]), 1017 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]), 1018 // testRowTwo-3 1019 new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]), 1020 new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]), 1021 new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]), }; 1022 verifyScanFull(s, kvs); 1023 } 1024 1025 @Test 1026 public void testRowFilter() throws IOException { 1027 1028 // Match a single row, all keys 1029 long expectedRows = 1; 1030 long expectedKeys = this.colsPerRow; 1031 Filter f = 1032 new RowFilter(CompareOperator.EQUAL, new BinaryComparator(Bytes.toBytes("testRowOne-2"))); 1033 Scan s = new Scan(); 1034 s.setFilter(f); 1035 verifyScanNoEarlyOut(s, expectedRows, expectedKeys); 1036 1037 // Match a two rows, one from each group, using regex 1038 expectedRows = 2; 1039 expectedKeys = this.colsPerRow; 1040 f = new RowFilter(CompareOperator.EQUAL, new RegexStringComparator("testRow.+-2")); 1041 s = new Scan(); 1042 s.setFilter(f); 1043 verifyScanNoEarlyOut(s, expectedRows, expectedKeys); 1044 1045 // Match rows less than 1046 // Expect all keys in one row 1047 expectedRows = 1; 1048 expectedKeys = this.colsPerRow; 1049 f = new RowFilter(CompareOperator.LESS, new BinaryComparator(Bytes.toBytes("testRowOne-2"))); 1050 s = new Scan(); 1051 s.setFilter(f); 1052 verifyScanNoEarlyOut(s, expectedRows, expectedKeys); 1053 1054 // Match rows less than or equal 1055 // Expect all keys in two rows 1056 expectedRows = 2; 1057 expectedKeys = this.colsPerRow; 1058 f = new RowFilter(CompareOperator.LESS_OR_EQUAL, 1059 new BinaryComparator(Bytes.toBytes("testRowOne-2"))); 1060 s = new Scan(); 1061 s.setFilter(f); 1062 verifyScanNoEarlyOut(s, expectedRows, expectedKeys); 1063 1064 // Match rows not equal 1065 // Expect all keys in all but one row 1066 expectedRows = this.numRows - 1; 1067 expectedKeys = this.colsPerRow; 1068 f = 1069 new RowFilter(CompareOperator.NOT_EQUAL, new BinaryComparator(Bytes.toBytes("testRowOne-2"))); 1070 s = new Scan(); 1071 s.setFilter(f); 1072 verifyScanNoEarlyOut(s, expectedRows, expectedKeys); 1073 1074 // Match keys greater or equal 1075 // Expect all keys in all but one row 1076 expectedRows = this.numRows - 1; 1077 expectedKeys = this.colsPerRow; 1078 f = new RowFilter(CompareOperator.GREATER_OR_EQUAL, 1079 new BinaryComparator(Bytes.toBytes("testRowOne-2"))); 1080 s = new Scan(); 1081 s.setFilter(f); 1082 verifyScanNoEarlyOut(s, expectedRows, expectedKeys); 1083 1084 // Match keys greater 1085 // Expect all keys in all but two rows 1086 expectedRows = this.numRows - 2; 1087 expectedKeys = this.colsPerRow; 1088 f = new RowFilter(CompareOperator.GREATER, new BinaryComparator(Bytes.toBytes("testRowOne-2"))); 1089 s = new Scan(); 1090 s.setFilter(f); 1091 verifyScanNoEarlyOut(s, expectedRows, expectedKeys); 1092 1093 // Match rows not equal to testRowTwo-2 1094 // Look across rows and fully validate the keys and ordering 1095 // Should see all keys in all rows but testRowTwo-2 1096 f = 1097 new RowFilter(CompareOperator.NOT_EQUAL, new BinaryComparator(Bytes.toBytes("testRowOne-2"))); 1098 s = new Scan(); 1099 s.setFilter(f); 1100 1101 KeyValue[] kvs = { 1102 // testRowOne-0 1103 new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), 1104 new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]), 1105 new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]), 1106 new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]), 1107 new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]), 1108 new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]), 1109 // testRowOne-3 1110 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), 1111 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]), 1112 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]), 1113 new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]), 1114 new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]), 1115 new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]), 1116 // testRowTwo-0 1117 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 1118 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 1119 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 1120 new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]), 1121 new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]), 1122 new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]), 1123 // testRowTwo-2 1124 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 1125 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 1126 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 1127 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]), 1128 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]), 1129 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]), 1130 // testRowTwo-3 1131 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 1132 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 1133 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 1134 new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]), 1135 new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]), 1136 new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]), }; 1137 verifyScanFull(s, kvs); 1138 1139 // Test across rows and groups with a regex 1140 // Filter out everything that doesn't match "*-2" 1141 // Expect all keys in two rows 1142 f = new RowFilter(CompareOperator.EQUAL, new RegexStringComparator(".+-2")); 1143 s = new Scan(); 1144 s.setFilter(f); 1145 1146 kvs = new KeyValue[] { 1147 // testRowOne-2 1148 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), 1149 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]), 1150 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]), 1151 new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]), 1152 new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]), 1153 new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]), 1154 // testRowTwo-2 1155 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 1156 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 1157 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 1158 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]), 1159 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]), 1160 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]) }; 1161 verifyScanFull(s, kvs); 1162 1163 } 1164 1165 @Test 1166 public void testValueFilter() throws IOException { 1167 1168 // Match group one rows 1169 long expectedRows = this.numRows / 2; 1170 long expectedKeys = this.colsPerRow; 1171 Filter f = 1172 new ValueFilter(CompareOperator.EQUAL, new BinaryComparator(Bytes.toBytes("testValueOne"))); 1173 Scan s = new Scan(); 1174 s.setFilter(f); 1175 verifyScanNoEarlyOut(s, expectedRows, expectedKeys); 1176 1177 // Match group two rows 1178 expectedRows = this.numRows / 2; 1179 expectedKeys = this.colsPerRow; 1180 f = new ValueFilter(CompareOperator.EQUAL, new BinaryComparator(Bytes.toBytes("testValueTwo"))); 1181 s = new Scan(); 1182 s.setFilter(f); 1183 verifyScanNoEarlyOut(s, expectedRows, expectedKeys); 1184 1185 // Match all values using regex 1186 expectedRows = this.numRows; 1187 expectedKeys = this.colsPerRow; 1188 f = new ValueFilter(CompareOperator.EQUAL, new RegexStringComparator("testValue((One)|(Two))")); 1189 s = new Scan(); 1190 s.setFilter(f); 1191 verifyScanNoEarlyOut(s, expectedRows, expectedKeys); 1192 1193 // Match values less than 1194 // Expect group one rows 1195 expectedRows = this.numRows / 2; 1196 expectedKeys = this.colsPerRow; 1197 f = new ValueFilter(CompareOperator.LESS, new BinaryComparator(Bytes.toBytes("testValueTwo"))); 1198 s = new Scan(); 1199 s.setFilter(f); 1200 verifyScanNoEarlyOut(s, expectedRows, expectedKeys); 1201 1202 // Match values less than or equal 1203 // Expect all rows 1204 expectedRows = this.numRows; 1205 expectedKeys = this.colsPerRow; 1206 f = new ValueFilter(CompareOperator.LESS_OR_EQUAL, 1207 new BinaryComparator(Bytes.toBytes("testValueTwo"))); 1208 s = new Scan(); 1209 s.setFilter(f); 1210 verifyScanNoEarlyOut(s, expectedRows, expectedKeys); 1211 1212 // Match values less than or equal 1213 // Expect group one rows 1214 expectedRows = this.numRows / 2; 1215 expectedKeys = this.colsPerRow; 1216 f = new ValueFilter(CompareOperator.LESS_OR_EQUAL, 1217 new BinaryComparator(Bytes.toBytes("testValueOne"))); 1218 s = new Scan(); 1219 s.setFilter(f); 1220 verifyScanNoEarlyOut(s, expectedRows, expectedKeys); 1221 1222 // Match values not equal 1223 // Expect half the rows 1224 expectedRows = this.numRows / 2; 1225 expectedKeys = this.colsPerRow; 1226 f = new ValueFilter(CompareOperator.NOT_EQUAL, 1227 new BinaryComparator(Bytes.toBytes("testValueOne"))); 1228 s = new Scan(); 1229 s.setFilter(f); 1230 verifyScanNoEarlyOut(s, expectedRows, expectedKeys); 1231 1232 // Match values greater or equal 1233 // Expect all rows 1234 expectedRows = this.numRows; 1235 expectedKeys = this.colsPerRow; 1236 f = new ValueFilter(CompareOperator.GREATER_OR_EQUAL, 1237 new BinaryComparator(Bytes.toBytes("testValueOne"))); 1238 s = new Scan(); 1239 s.setFilter(f); 1240 verifyScanNoEarlyOut(s, expectedRows, expectedKeys); 1241 1242 // Match values greater 1243 // Expect half rows 1244 expectedRows = this.numRows / 2; 1245 expectedKeys = this.colsPerRow; 1246 f = 1247 new ValueFilter(CompareOperator.GREATER, new BinaryComparator(Bytes.toBytes("testValueOne"))); 1248 s = new Scan(); 1249 s.setFilter(f); 1250 verifyScanNoEarlyOut(s, expectedRows, expectedKeys); 1251 1252 // Match values not equal to testValueOne 1253 // Look across rows and fully validate the keys and ordering 1254 // Should see all keys in all group two rows 1255 f = new ValueFilter(CompareOperator.NOT_EQUAL, 1256 new BinaryComparator(Bytes.toBytes("testValueOne"))); 1257 s = new Scan(); 1258 s.setFilter(f); 1259 1260 KeyValue[] kvs = { 1261 // testRowTwo-0 1262 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 1263 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 1264 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 1265 new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]), 1266 new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]), 1267 new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]), 1268 // testRowTwo-2 1269 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 1270 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 1271 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 1272 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]), 1273 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]), 1274 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]), 1275 // testRowTwo-3 1276 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 1277 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 1278 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 1279 new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]), 1280 new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]), 1281 new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]), }; 1282 verifyScanFull(s, kvs); 1283 } 1284 1285 @Test 1286 public void testSkipFilter() throws IOException { 1287 1288 // Test for qualifier regex: "testQualifierOne-2" 1289 // Should only get rows from second group, and all keys 1290 Filter f = new SkipFilter(new QualifierFilter(CompareOperator.NOT_EQUAL, 1291 new BinaryComparator(Bytes.toBytes("testQualifierOne-2")))); 1292 Scan s = new Scan(); 1293 s.setFilter(f); 1294 1295 KeyValue[] kvs = { 1296 // testRowTwo-0 1297 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 1298 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 1299 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 1300 new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]), 1301 new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]), 1302 new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]), 1303 // testRowTwo-2 1304 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 1305 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 1306 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 1307 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]), 1308 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]), 1309 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]), 1310 // testRowTwo-3 1311 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 1312 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 1313 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 1314 new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]), 1315 new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]), 1316 new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]), }; 1317 verifyScanFull(s, kvs); 1318 } 1319 1320 // TODO: This is important... need many more tests for ordering, etc 1321 // There are limited tests elsewhere but we need HRegion level ones here 1322 @Test 1323 public void testFilterList() throws IOException { 1324 1325 // Test getting a single row, single key using Row, Qualifier, and Value 1326 // regular expression and substring filters 1327 // Use must pass all 1328 List<Filter> filters = new ArrayList<>(); 1329 filters.add(new RowFilter(CompareOperator.EQUAL, new RegexStringComparator(".+-2"))); 1330 filters.add(new QualifierFilter(CompareOperator.EQUAL, new RegexStringComparator(".+-2"))); 1331 filters.add(new ValueFilter(CompareOperator.EQUAL, new SubstringComparator("One"))); 1332 Filter f = new FilterList(Operator.MUST_PASS_ALL, filters); 1333 Scan s = new Scan(); 1334 s.addFamily(FAMILIES[0]); 1335 s.setFilter(f); 1336 KeyValue[] kvs = { new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]) }; 1337 verifyScanFull(s, kvs); 1338 1339 // Test getting everything with a MUST_PASS_ONE filter including row, qf, val 1340 // regular expression and substring filters 1341 filters.clear(); 1342 filters.add(new RowFilter(CompareOperator.EQUAL, new RegexStringComparator(".+Two.+"))); 1343 filters.add(new QualifierFilter(CompareOperator.EQUAL, new RegexStringComparator(".+-2"))); 1344 filters.add(new ValueFilter(CompareOperator.EQUAL, new SubstringComparator("One"))); 1345 f = new FilterList(Operator.MUST_PASS_ONE, filters); 1346 s = new Scan(); 1347 s.setFilter(f); 1348 verifyScanNoEarlyOut(s, this.numRows, this.colsPerRow); 1349 1350 } 1351 1352 @Test 1353 public void testFirstKeyOnlyFilter() throws IOException { 1354 Scan s = new Scan(); 1355 s.setFilter(new FirstKeyOnlyFilter()); 1356 // Expected KVs, the first KV from each of the remaining 6 rows 1357 KeyValue[] kvs = { new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), 1358 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), 1359 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), 1360 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 1361 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 1362 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]) }; 1363 verifyScanFull(s, kvs); 1364 } 1365 1366 @Test 1367 public void testFilterListWithSingleColumnValueFilter() throws IOException { 1368 // Test for HBASE-3191 1369 1370 // Scan using SingleColumnValueFilter 1371 SingleColumnValueFilter f1 = 1372 new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[0], CompareOperator.EQUAL, VALUES[0]); 1373 f1.setFilterIfMissing(true); 1374 Scan s1 = new Scan(); 1375 s1.addFamily(FAMILIES[0]); 1376 s1.setFilter(f1); 1377 KeyValue[] kvs1 = { new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), 1378 new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]), 1379 new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]), 1380 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), 1381 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]), 1382 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]), 1383 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), 1384 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]), 1385 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]), }; 1386 verifyScanNoEarlyOut(s1, 3, 3); 1387 verifyScanFull(s1, kvs1); 1388 1389 // Scan using another SingleColumnValueFilter, expect disjoint result 1390 SingleColumnValueFilter f2 = 1391 new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_TWO[0], CompareOperator.EQUAL, VALUES[1]); 1392 f2.setFilterIfMissing(true); 1393 Scan s2 = new Scan(); 1394 s2.addFamily(FAMILIES[0]); 1395 s2.setFilter(f2); 1396 KeyValue[] kvs2 = { new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 1397 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 1398 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 1399 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 1400 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 1401 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 1402 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 1403 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 1404 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), }; 1405 verifyScanNoEarlyOut(s2, 3, 3); 1406 verifyScanFull(s2, kvs2); 1407 1408 // Scan, ORing the two previous filters, expect unified result 1409 FilterList f = new FilterList(Operator.MUST_PASS_ONE); 1410 f.addFilter(f1); 1411 f.addFilter(f2); 1412 Scan s = new Scan(); 1413 s.addFamily(FAMILIES[0]); 1414 s.setFilter(f); 1415 KeyValue[] kvs = { new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), 1416 new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]), 1417 new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]), 1418 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), 1419 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]), 1420 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]), 1421 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), 1422 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]), 1423 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]), 1424 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 1425 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 1426 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 1427 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 1428 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 1429 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 1430 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 1431 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 1432 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), }; 1433 verifyScanNoEarlyOut(s, 6, 3); 1434 verifyScanFull(s, kvs); 1435 } 1436 1437 // HBASE-9747 1438 @Test 1439 public void testFilterListWithPrefixFilter() throws IOException { 1440 byte[] family = Bytes.toBytes("f1"); 1441 byte[] qualifier = Bytes.toBytes("q1"); 1442 TableDescriptor tableDescriptor = 1443 TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName())) 1444 .setColumnFamily(ColumnFamilyDescriptorBuilder.of(family)).build(); 1445 RegionInfo info = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build(); 1446 HRegion testRegion = HBaseTestingUtil.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), 1447 TEST_UTIL.getConfiguration(), tableDescriptor); 1448 1449 for (int i = 0; i < 5; i++) { 1450 Put p = new Put(Bytes.toBytes((char) ('a' + i) + "row")); 1451 p.setDurability(Durability.SKIP_WAL); 1452 p.addColumn(family, qualifier, Bytes.toBytes(String.valueOf(111 + i))); 1453 testRegion.put(p); 1454 } 1455 testRegion.flush(true); 1456 1457 // rows starting with "b" 1458 PrefixFilter pf = new PrefixFilter(new byte[] { 'b' }); 1459 // rows with value of column 'q1' set to '113' 1460 SingleColumnValueFilter scvf = 1461 new SingleColumnValueFilter(family, qualifier, CompareOperator.EQUAL, Bytes.toBytes("113")); 1462 // combine these two with OR in a FilterList 1463 FilterList filterList = new FilterList(Operator.MUST_PASS_ONE, pf, scvf); 1464 1465 Scan s1 = new Scan(); 1466 s1.setFilter(filterList); 1467 InternalScanner scanner = testRegion.getScanner(s1); 1468 List<Cell> results = new ArrayList<>(); 1469 int resultCount = 0; 1470 while (scanner.next(results)) { 1471 resultCount++; 1472 byte[] row = CellUtil.cloneRow(results.get(0)); 1473 LOG.debug("Found row: " + Bytes.toStringBinary(row)); 1474 assertTrue( 1475 Bytes.equals(row, Bytes.toBytes("brow")) || Bytes.equals(row, Bytes.toBytes("crow"))); 1476 results.clear(); 1477 } 1478 assertEquals(2, resultCount); 1479 scanner.close(); 1480 1481 WAL wal = ((HRegion) testRegion).getWAL(); 1482 ((HRegion) testRegion).close(); 1483 wal.close(); 1484 } 1485 1486 @Test 1487 public void testSingleColumnValueFilter() throws IOException { 1488 1489 // From HBASE-1821 1490 // Desired action is to combine two SCVF in a FilterList 1491 // Want to return only rows that match both conditions 1492 1493 // Need to change one of the group one columns to use group two value 1494 Put p = new Put(ROWS_ONE[2]); 1495 p.addColumn(FAMILIES[0], QUALIFIERS_ONE[2], VALUES[1]); 1496 this.region.put(p); 1497 1498 // Now let's grab rows that have Q_ONE[0](VALUES[0]) and Q_ONE[2](VALUES[1]) 1499 // Since group two rows don't have these qualifiers, they will pass 1500 // so limiting scan to group one 1501 List<Filter> filters = new ArrayList<>(); 1502 filters.add(new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[0], CompareOperator.EQUAL, 1503 VALUES[0])); 1504 filters.add(new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[2], CompareOperator.EQUAL, 1505 VALUES[1])); 1506 Filter f = new FilterList(Operator.MUST_PASS_ALL, filters); 1507 Scan s = new Scan().withStartRow(ROWS_ONE[0]).withStopRow(ROWS_TWO[0]); 1508 s.addFamily(FAMILIES[0]); 1509 s.setFilter(f); 1510 // Expect only one row, all qualifiers 1511 KeyValue[] kvs = { new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), 1512 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[1]), 1513 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]) }; 1514 verifyScanNoEarlyOut(s, 1, 3); 1515 verifyScanFull(s, kvs); 1516 1517 // In order to get expected behavior without limiting to group one 1518 // need to wrap SCVFs in SkipFilters 1519 filters = new ArrayList<>(); 1520 filters.add(new SkipFilter(new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[0], 1521 CompareOperator.EQUAL, VALUES[0]))); 1522 filters.add(new SkipFilter(new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[2], 1523 CompareOperator.EQUAL, VALUES[1]))); 1524 f = new FilterList(Operator.MUST_PASS_ALL, filters); 1525 s = new Scan().withStartRow(ROWS_ONE[0]).withStopRow(ROWS_TWO[0]); 1526 s.addFamily(FAMILIES[0]); 1527 s.setFilter(f); 1528 // Expect same KVs 1529 verifyScanNoEarlyOut(s, 1, 3); 1530 verifyScanFull(s, kvs); 1531 1532 // More tests from HBASE-1821 for Clint and filterIfMissing flag 1533 1534 byte[][] ROWS_THREE = { Bytes.toBytes("rowThree-0"), Bytes.toBytes("rowThree-1"), 1535 Bytes.toBytes("rowThree-2"), Bytes.toBytes("rowThree-3") }; 1536 1537 // Give row 0 and 2 QUALIFIERS_ONE[0] (VALUE[0] VALUE[1]) 1538 // Give row 1 and 3 QUALIFIERS_ONE[1] (VALUE[0] VALUE[1]) 1539 1540 KeyValue[] srcKVs = 1541 new KeyValue[] { new KeyValue(ROWS_THREE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), 1542 new KeyValue(ROWS_THREE[1], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[1]), 1543 new KeyValue(ROWS_THREE[2], FAMILIES[0], QUALIFIERS_ONE[1], VALUES[0]), 1544 new KeyValue(ROWS_THREE[3], FAMILIES[0], QUALIFIERS_ONE[1], VALUES[1]) }; 1545 1546 for (KeyValue kv : srcKVs) { 1547 Put put = new Put(CellUtil.cloneRow(kv)).add(kv); 1548 put.setDurability(Durability.SKIP_WAL); 1549 this.region.put(put); 1550 } 1551 1552 // Match VALUES[0] against QUALIFIERS_ONE[0] with filterIfMissing = false 1553 // Expect 3 rows (0, 2, 3) 1554 SingleColumnValueFilter scvf = 1555 new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[0], CompareOperator.EQUAL, VALUES[0]); 1556 s = new Scan().withStartRow(ROWS_THREE[0]).withStopRow(Bytes.toBytes("rowThree-4")); 1557 s.addFamily(FAMILIES[0]); 1558 s.setFilter(scvf); 1559 kvs = new KeyValue[] { srcKVs[0], srcKVs[2], srcKVs[3] }; 1560 verifyScanFull(s, kvs); 1561 1562 // Match VALUES[0] against QUALIFIERS_ONE[0] with filterIfMissing = true 1563 // Expect 1 row (0) 1564 scvf = 1565 new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[0], CompareOperator.EQUAL, VALUES[0]); 1566 scvf.setFilterIfMissing(true); 1567 s = new Scan().withStartRow(ROWS_THREE[0]).withStopRow(Bytes.toBytes("rowThree-4")); 1568 s.addFamily(FAMILIES[0]); 1569 s.setFilter(scvf); 1570 kvs = new KeyValue[] { srcKVs[0] }; 1571 verifyScanFull(s, kvs); 1572 1573 // Match VALUES[1] against QUALIFIERS_ONE[1] with filterIfMissing = true 1574 // Expect 1 row (3) 1575 scvf = 1576 new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[1], CompareOperator.EQUAL, VALUES[1]); 1577 scvf.setFilterIfMissing(true); 1578 s = new Scan().withStartRow(ROWS_THREE[0]).withStopRow(Bytes.toBytes("rowThree-4")); 1579 s.addFamily(FAMILIES[0]); 1580 s.setFilter(scvf); 1581 kvs = new KeyValue[] { srcKVs[3] }; 1582 verifyScanFull(s, kvs); 1583 1584 // Add QUALIFIERS_ONE[1] to ROWS_THREE[0] with VALUES[0] 1585 KeyValue kvA = new KeyValue(ROWS_THREE[0], FAMILIES[0], QUALIFIERS_ONE[1], VALUES[0]); 1586 this.region.put(new Put(CellUtil.cloneRow(kvA)).add(kvA)); 1587 1588 // Match VALUES[1] against QUALIFIERS_ONE[1] with filterIfMissing = true 1589 // Expect 1 row (3) 1590 scvf = 1591 new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[1], CompareOperator.EQUAL, VALUES[1]); 1592 scvf.setFilterIfMissing(true); 1593 s = new Scan().withStartRow(ROWS_THREE[0]).withStopRow(Bytes.toBytes("rowThree-4")); 1594 s.addFamily(FAMILIES[0]); 1595 s.setFilter(scvf); 1596 kvs = new KeyValue[] { srcKVs[3] }; 1597 verifyScanFull(s, kvs); 1598 1599 } 1600 1601 @Test 1602 public void testColumnValueFilter() throws Exception { 1603 // Prepare test rows: 1604 for (int i = 0; i < 2; i++) { 1605 for (int j = 0; j < ROWS_ONE.length; j++) { 1606 Put p1 = new Put(ROWS_ONE[j]).setDurability(Durability.SKIP_WAL); 1607 Put p2 = new Put(ROWS_TWO[j]).setDurability(Durability.SKIP_WAL); 1608 for (byte[] q5 : QUALIFIERS_FIVE) { 1609 p1.addColumn(FAMILIES[0], q5, VALUES[0 + i]).addColumn(FAMILIES[1], q5, VALUES[0 + i]); 1610 p2.addColumn(FAMILIES[0], q5, VALUES[1 - i]).addColumn(FAMILIES[1], q5, VALUES[1 - i]); 1611 } 1612 this.region.put(p1); 1613 this.region.put(p2); 1614 } 1615 this.region.flush(true); 1616 } 1617 // 1. Test = f[0]:q5[0]:v[1] 1618 Scan scan = new Scan().setFilter( 1619 new ColumnValueFilter(FAMILIES[0], QUALIFIERS_FIVE[0], CompareOperator.EQUAL, VALUES[1])); 1620 KeyValue[] expectedEquals = 1621 { new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]), 1622 new KeyValue(ROWS_ONE[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]), 1623 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]), 1624 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]), 1625 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]), 1626 new KeyValue(ROWS_TWO[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]), 1627 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]), 1628 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]) }; 1629 verifyScanFull(scan, expectedEquals); 1630 // 2. Test > f[0]:q5[0]:v[0] 1631 scan.setFilter( 1632 new ColumnValueFilter(FAMILIES[0], QUALIFIERS_FIVE[0], CompareOperator.GREATER, VALUES[0])); 1633 KeyValue[] expectedGreater = 1634 { new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]), 1635 new KeyValue(ROWS_ONE[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]), 1636 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]), 1637 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]), 1638 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]), 1639 new KeyValue(ROWS_TWO[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]), 1640 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]), 1641 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]) }; 1642 verifyScanFull(scan, expectedGreater); 1643 // 3. Test >= f[0]:q5[0]:v[0] 1644 // also test readAllVersions(), since FAMILIES[0] allow multiple versions. 1645 scan.readAllVersions().setFilter(new ColumnValueFilter(FAMILIES[0], QUALIFIERS_FIVE[0], 1646 CompareOperator.GREATER_OR_EQUAL, VALUES[0])); 1647 KeyValue[] expectedGreaterOrEqual = 1648 { new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]), 1649 new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]), 1650 new KeyValue(ROWS_ONE[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]), 1651 new KeyValue(ROWS_ONE[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]), 1652 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]), 1653 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]), 1654 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]), 1655 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]), 1656 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]), 1657 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]), 1658 new KeyValue(ROWS_TWO[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]), 1659 new KeyValue(ROWS_TWO[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]), 1660 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]), 1661 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]), 1662 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]), 1663 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]) }; 1664 verifyScanFull(scan, expectedGreaterOrEqual); 1665 // 4. Test < f[1]:q5[1]:v[1], FAMILIES[1] doesn't support multiple versions 1666 scan.readVersions(1).setFilter( 1667 new ColumnValueFilter(FAMILIES[1], QUALIFIERS_FIVE[1], CompareOperator.LESS, VALUES[1])); 1668 KeyValue[] expectedLess = 1669 { new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]), 1670 new KeyValue(ROWS_TWO[1], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]), 1671 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]), 1672 new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]) }; 1673 verifyScanFull(scan, expectedLess); 1674 // 5. Test <= f[1]:q5[0]:v[1] 1675 scan.setFilter(new ColumnValueFilter(FAMILIES[1], QUALIFIERS_FIVE[1], 1676 CompareOperator.LESS_OR_EQUAL, VALUES[1])); 1677 KeyValue[] expectedLessOrEqual = 1678 { new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[1]), 1679 new KeyValue(ROWS_ONE[1], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[1]), 1680 new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[1]), 1681 new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[1]), 1682 new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]), 1683 new KeyValue(ROWS_TWO[1], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]), 1684 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]), 1685 new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]) }; 1686 verifyScanFull(scan, expectedLessOrEqual); 1687 // 6. Test != f[1]:q5[1]:v[1] 1688 scan.setFilter( 1689 new ColumnValueFilter(FAMILIES[1], QUALIFIERS_FIVE[1], CompareOperator.NOT_EQUAL, VALUES[1])); 1690 KeyValue[] expectedNotEqual = 1691 { new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]), 1692 new KeyValue(ROWS_TWO[1], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]), 1693 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]), 1694 new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]) }; 1695 verifyScanFull(scan, expectedNotEqual); 1696 // 7. Test FilterList(MUST_PASS_ONE) combining ColumnValueFilter and QualifierFilter 1697 // (ColumnValueFilter, != f[1]:q5[1]:v[1]) || (QualifierFilter, = q5[0]) 1698 List<Filter> orFilters = new ArrayList<>(2); 1699 orFilters.add( 1700 new ColumnValueFilter(FAMILIES[1], QUALIFIERS_FIVE[1], CompareOperator.NOT_EQUAL, VALUES[1])); 1701 orFilters 1702 .add(new QualifierFilter(CompareOperator.EQUAL, new BinaryComparator(QUALIFIERS_FIVE[0]))); 1703 scan.setFilter(new FilterList(Operator.MUST_PASS_ONE, orFilters)); 1704 KeyValue[] expectedMustPassOne = 1705 { new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]), 1706 new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_FIVE[0], VALUES[1]), 1707 new KeyValue(ROWS_ONE[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]), 1708 new KeyValue(ROWS_ONE[1], FAMILIES[1], QUALIFIERS_FIVE[0], VALUES[1]), 1709 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]), 1710 new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_FIVE[0], VALUES[1]), 1711 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]), 1712 new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_FIVE[0], VALUES[1]), 1713 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]), 1714 new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_FIVE[0], VALUES[0]), 1715 new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]), // this pass scvf 1716 new KeyValue(ROWS_TWO[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]), 1717 new KeyValue(ROWS_TWO[1], FAMILIES[1], QUALIFIERS_FIVE[0], VALUES[0]), 1718 new KeyValue(ROWS_TWO[1], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]), // this pass scvf 1719 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]), 1720 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_FIVE[0], VALUES[0]), 1721 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]), // this pass scvf 1722 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]), 1723 new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_FIVE[0], VALUES[0]), 1724 new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]) }; // this pass scvf 1725 verifyScanFull(scan, expectedMustPassOne); 1726 // 8. Test FilterList(MUST_PASS_ALL) combining ColumnValueFilter and RowFilter 1727 // (ColumnValueFilter, != f[1]:q5[1]:v[1]) && (RowFilter, = prefix:"testRow") 1728 List<Filter> andFilters = new ArrayList<>(2); 1729 andFilters.add( 1730 new ColumnValueFilter(FAMILIES[1], QUALIFIERS_FIVE[1], CompareOperator.NOT_EQUAL, VALUES[1])); 1731 andFilters.add( 1732 new RowFilter(CompareOperator.EQUAL, new BinaryPrefixComparator(Bytes.toBytes("testRow")))); 1733 scan.setFilter(new FilterList(Operator.MUST_PASS_ALL, andFilters)); 1734 KeyValue[] expectedMustPassAll = 1735 { new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]), 1736 new KeyValue(ROWS_TWO[1], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]), 1737 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]), 1738 new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]) }; 1739 verifyScanFull(scan, expectedMustPassAll); 1740 // 9. Test specified columns with FilterList(MUST_PASS_ONE) which sused in case 7. 1741 // Result is different from case 7, because column is strongly constrained by specified columns 1742 Scan anotherScan = new Scan().addColumn(FAMILIES[1], QUALIFIERS_FIVE[1]) 1743 .setFilter(new FilterList(Operator.MUST_PASS_ONE, orFilters)); 1744 KeyValue[] expectedValues = 1745 { new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]), 1746 new KeyValue(ROWS_TWO[1], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]), 1747 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]), 1748 new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]) }; 1749 verifyScanFull(anotherScan, expectedValues); 1750 } 1751 1752 private void verifyScan(Scan s, long expectedRows, long expectedKeys) throws IOException { 1753 InternalScanner scanner = this.region.getScanner(s); 1754 List<Cell> results = new ArrayList<>(); 1755 int i = 0; 1756 for (boolean done = true; done; i++) { 1757 done = scanner.next(results); 1758 Arrays.sort(results.toArray(new Cell[results.size()]), CellComparator.getInstance()); 1759 LOG.info("counter=" + i + ", " + results); 1760 if (results.isEmpty()) break; 1761 assertTrue("Scanned too many rows! Only expected " + expectedRows 1762 + " total but already scanned " + (i + 1), expectedRows > i); 1763 assertEquals("Expected " + expectedKeys + " keys per row but " + "returned " + results.size(), 1764 expectedKeys, results.size()); 1765 results.clear(); 1766 } 1767 assertEquals("Expected " + expectedRows + " rows but scanned " + i + " rows", expectedRows, i); 1768 } 1769 1770 private void verifyScanNoEarlyOut(Scan s, long expectedRows, long expectedKeys) 1771 throws IOException { 1772 InternalScanner scanner = this.region.getScanner(s); 1773 List<Cell> results = new ArrayList<>(); 1774 int i = 0; 1775 for (boolean done = true; done; i++) { 1776 done = scanner.next(results); 1777 Arrays.sort(results.toArray(new Cell[results.size()]), CellComparator.getInstance()); 1778 LOG.info("counter=" + i + ", " + results); 1779 if (results.isEmpty()) break; 1780 assertTrue("Scanned too many rows! Only expected " + expectedRows 1781 + " total but already scanned " + (i + 1), expectedRows > i); 1782 assertEquals("Expected " + expectedKeys + " keys per row but " + "returned " + results.size(), 1783 expectedKeys, results.size()); 1784 results.clear(); 1785 } 1786 assertEquals("Expected " + expectedRows + " rows but scanned " + i + " rows", expectedRows, i); 1787 } 1788 1789 private void verifyScanFull(Scan s, KeyValue[] kvs) throws IOException { 1790 InternalScanner scanner = this.region.getScanner(s); 1791 List<Cell> results = new ArrayList<>(); 1792 int row = 0; 1793 int idx = 0; 1794 for (boolean done = true; done; row++) { 1795 done = scanner.next(results); 1796 Arrays.sort(results.toArray(new Cell[results.size()]), CellComparator.getInstance()); 1797 if (results.isEmpty()) break; 1798 assertTrue( 1799 "Scanned too many keys! Only expected " + kvs.length + " total but already scanned " 1800 + (results.size() + idx) 1801 + (results.isEmpty() ? "" : "(" + results.get(0).toString() + ")"), 1802 kvs.length >= idx + results.size()); 1803 for (Cell kv : results) { 1804 LOG.info("row=" + row + ", result=" + kv.toString() + ", match=" + kvs[idx].toString()); 1805 assertTrue("Row mismatch", CellUtil.matchingRows(kv, kvs[idx])); 1806 assertTrue("Family mismatch", CellUtil.matchingFamily(kv, kvs[idx])); 1807 assertTrue("Qualifier mismatch", CellUtil.matchingQualifier(kv, kvs[idx])); 1808 assertTrue("Value mismatch", CellUtil.matchingValue(kv, kvs[idx])); 1809 idx++; 1810 } 1811 results.clear(); 1812 } 1813 LOG.info("Looked at " + row + " rows with " + idx + " keys"); 1814 assertEquals("Expected " + kvs.length + " total keys but scanned " + idx, kvs.length, idx); 1815 } 1816 1817 private void verifyScanFullNoValues(Scan s, KeyValue[] kvs, boolean useLen) throws IOException { 1818 InternalScanner scanner = this.region.getScanner(s); 1819 List<Cell> results = new ArrayList<>(); 1820 int row = 0; 1821 int idx = 0; 1822 for (boolean more = true; more; row++) { 1823 more = scanner.next(results); 1824 Arrays.sort(results.toArray(new Cell[results.size()]), CellComparator.getInstance()); 1825 if (results.isEmpty()) break; 1826 assertTrue( 1827 "Scanned too many keys! Only expected " + kvs.length + " total but already scanned " 1828 + (results.size() + idx) 1829 + (results.isEmpty() ? "" : "(" + results.get(0).toString() + ")"), 1830 kvs.length >= idx + results.size()); 1831 for (Cell kv : results) { 1832 LOG.info("row=" + row + ", result=" + kv.toString() + ", match=" + kvs[idx].toString()); 1833 1834 assertTrue("Row mismatch", CellUtil.matchingRows(kv, kvs[idx])); 1835 assertTrue("Family mismatch", CellUtil.matchingFamily(kv, kvs[idx])); 1836 assertTrue("Qualifier mismatch", CellUtil.matchingQualifier(kv, kvs[idx])); 1837 assertFalse("Should not have returned whole value", CellUtil.matchingValue(kv, kvs[idx])); 1838 if (useLen) { 1839 assertEquals("Value in result is not SIZEOF_INT", Bytes.SIZEOF_INT, kv.getValueLength()); 1840 LOG.info("idx = " + idx + ", len=" + kvs[idx].getValueLength() + ", actual=" 1841 + Bytes.toInt(CellUtil.cloneValue(kv))); 1842 assertEquals("Scan value should be the length of the actual value. ", 1843 kvs[idx].getValueLength(), Bytes.toInt(CellUtil.cloneValue(kv))); 1844 LOG.info("good"); 1845 } else { 1846 assertEquals("Value in result is not empty", 0, kv.getValueLength()); 1847 } 1848 idx++; 1849 } 1850 results.clear(); 1851 } 1852 LOG.info("Looked at " + row + " rows with " + idx + " keys"); 1853 assertEquals("Expected " + kvs.length + " total keys but scanned " + idx, kvs.length, idx); 1854 } 1855 1856 @Test 1857 public void testColumnPaginationFilterColumnOffset() throws Exception { 1858 KeyValue[] expectedKVs = { 1859 // testRowOne-0 1860 new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]), 1861 // testRowOne-2 1862 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]), 1863 // testRowOne-3 1864 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]), 1865 // testRowTwo-0 1866 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 1867 // testRowTwo-2 1868 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 1869 // testRowTwo-3 1870 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), }; 1871 KeyValue[] expectedKVs1 = { 1872 // testRowTwo-0 1873 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 1874 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 1875 // testRowTwo-2 1876 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 1877 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 1878 // testRowTwo-3 1879 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 1880 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]) }; 1881 KeyValue[] expectedKVs2 = { 1882 // testRowTwo-0 1883 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 1884 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 1885 new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]), 1886 // testRowTwo-2 1887 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 1888 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 1889 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]), 1890 // testRowTwo-3 1891 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 1892 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 1893 new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]) }; 1894 KeyValue[] expectedKVs3 = { 1895 // testRowTwo-0 1896 new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]), 1897 new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]), 1898 // testRowTwo-2 1899 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]), 1900 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]), 1901 // testRowTwo-3 1902 new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]), 1903 new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]), }; 1904 Scan s = new Scan(); 1905 1906 // Page size 1. 1907 long expectedRows = 6; 1908 long expectedKeys = 1; 1909 s.setFilter(new ColumnPaginationFilter(1, QUALIFIERS_ONE[1])); 1910 verifyScan(s, expectedRows, expectedKeys); 1911 this.verifyScanFull(s, expectedKVs); 1912 1913 // Page size 2. 1914 expectedRows = 3; 1915 expectedKeys = 2; 1916 s = new Scan(); 1917 s.setFilter(new ColumnPaginationFilter(2, QUALIFIERS_TWO[2])); 1918 verifyScan(s, expectedRows, expectedKeys); 1919 this.verifyScanFull(s, expectedKVs1); 1920 1921 // Page size 3 across multiple column families. 1922 expectedRows = 3; 1923 expectedKeys = 3; 1924 s = new Scan(); 1925 s.setFilter(new ColumnPaginationFilter(3, QUALIFIERS_TWO[2])); 1926 verifyScan(s, expectedRows, expectedKeys); 1927 this.verifyScanFull(s, expectedKVs2); 1928 1929 // Page size 2 restricted to one column family. 1930 expectedRows = 3; 1931 expectedKeys = 2; 1932 s = new Scan(); 1933 s.addFamily(FAMILIES[1]); 1934 s.setFilter(new ColumnPaginationFilter(2, QUALIFIERS_TWO[2])); 1935 this.verifyScanFull(s, expectedKVs3); 1936 } 1937 1938 @Test 1939 public void testLatestVersionFilterWithExplicitColumn() throws Exception { 1940 // Add multiple versions 1941 Put p = new Put(ROWS_ONE[0]); 1942 p.setDurability(Durability.SKIP_WAL); 1943 p.addColumn(FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]); 1944 this.region.put(p); 1945 p = new Put(ROWS_ONE[0]); 1946 p.setDurability(Durability.SKIP_WAL); 1947 p.addColumn(FAMILIES[0], QUALIFIERS_ONE[0], VALUES[1]); 1948 this.region.put(p); 1949 this.region.flush(true); 1950 Scan s = new Scan(); 1951 s.setFilter(new FilterBase() { 1952 @Override 1953 public ReturnCode filterCell(Cell c) throws IOException { 1954 return ReturnCode.INCLUDE_AND_NEXT_COL; 1955 } 1956 }); 1957 s.readVersions(100); 1958 s.addColumn(FAMILIES[0], QUALIFIERS_ONE[0]); 1959 s.addColumn(FAMILIES[0], QUALIFIERS_ONE[1]); 1960 s.addColumn(FAMILIES[0], QUALIFIERS_ONE[2]); 1961 s.addColumn(FAMILIES[0], QUALIFIERS_ONE[3]); 1962 s.addColumn(FAMILIES[0], QUALIFIERS_TWO[0]); 1963 s.addColumn(FAMILIES[0], QUALIFIERS_TWO[1]); 1964 s.addColumn(FAMILIES[0], QUALIFIERS_TWO[2]); 1965 s.addColumn(FAMILIES[0], QUALIFIERS_TWO[3]); 1966 KeyValue[] kvs = { 1967 // testRowOne-0 1968 new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[1]), 1969 new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]), 1970 new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]), 1971 1972 // testRowOne-2 1973 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), 1974 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]), 1975 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]), 1976 1977 // testRowOne-3 1978 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), 1979 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]), 1980 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]), 1981 // testRowTwo-0 1982 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 1983 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 1984 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 1985 // testRowTwo-2 1986 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 1987 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 1988 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 1989 // testRowTwo-3 1990 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 1991 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 1992 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), }; 1993 verifyScanFull(s, kvs); 1994 1995 } 1996 1997 @Test 1998 public void testColumnPaginationFilter() throws Exception { 1999 // Test that the filter skips multiple column versions. 2000 Put p = new Put(ROWS_ONE[0]); 2001 p.setDurability(Durability.SKIP_WAL); 2002 p.addColumn(FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]); 2003 this.region.put(p); 2004 this.region.flush(true); 2005 2006 // Set of KVs (page: 1; pageSize: 1) - the first set of 1 column per row 2007 KeyValue[] expectedKVs = { 2008 // testRowOne-0 2009 new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), 2010 // testRowOne-2 2011 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), 2012 // testRowOne-3 2013 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), 2014 // testRowTwo-0 2015 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 2016 // testRowTwo-2 2017 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 2018 // testRowTwo-3 2019 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]) }; 2020 2021 // Set of KVs (page: 3; pageSize: 1) - the third set of 1 column per row 2022 KeyValue[] expectedKVs2 = { 2023 // testRowOne-0 2024 new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]), 2025 // testRowOne-2 2026 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]), 2027 // testRowOne-3 2028 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]), 2029 // testRowTwo-0 2030 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 2031 // testRowTwo-2 2032 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 2033 // testRowTwo-3 2034 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), }; 2035 2036 // Set of KVs (page: 2; pageSize 2) - the 2nd set of 2 columns per row 2037 KeyValue[] expectedKVs3 = { 2038 // testRowOne-0 2039 new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]), 2040 new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]), 2041 // testRowOne-2 2042 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]), 2043 new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]), 2044 // testRowOne-3 2045 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]), 2046 new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]), 2047 // testRowTwo-0 2048 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 2049 new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]), 2050 // testRowTwo-2 2051 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 2052 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]), 2053 // testRowTwo-3 2054 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 2055 new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]), }; 2056 2057 // Set of KVs (page: 2; pageSize 2) - the 2nd set of 2 columns per row 2058 KeyValue[] expectedKVs4 = { 2059 2060 }; 2061 2062 long expectedRows = this.numRows; 2063 long expectedKeys = 1; 2064 Scan s = new Scan(); 2065 2066 // Page 1; 1 Column per page (Limit 1, Offset 0) 2067 s.setFilter(new ColumnPaginationFilter(1, 0)); 2068 verifyScan(s, expectedRows, expectedKeys); 2069 this.verifyScanFull(s, expectedKVs); 2070 2071 // Page 3; 1 Result per page (Limit 1, Offset 2) 2072 s.setFilter(new ColumnPaginationFilter(1, 2)); 2073 verifyScan(s, expectedRows, expectedKeys); 2074 this.verifyScanFull(s, expectedKVs2); 2075 2076 // Page 2; 2 Results per page (Limit 2, Offset 2) 2077 s.setFilter(new ColumnPaginationFilter(2, 2)); 2078 expectedKeys = 2; 2079 verifyScan(s, expectedRows, expectedKeys); 2080 this.verifyScanFull(s, expectedKVs3); 2081 2082 // Page 8; 20 Results per page (no results) (Limit 20, Offset 140) 2083 s.setFilter(new ColumnPaginationFilter(20, 140)); 2084 expectedKeys = 0; 2085 expectedRows = 0; 2086 verifyScan(s, expectedRows, 0); 2087 this.verifyScanFull(s, expectedKVs4); 2088 } 2089 2090 @Test 2091 public void testKeyOnlyFilter() throws Exception { 2092 2093 // KVs in first 6 rows 2094 KeyValue[] expectedKVs = { 2095 // testRowOne-0 2096 new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), 2097 new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]), 2098 new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]), 2099 new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]), 2100 new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]), 2101 new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]), 2102 // testRowOne-2 2103 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), 2104 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]), 2105 new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]), 2106 new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]), 2107 new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]), 2108 new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]), 2109 // testRowOne-3 2110 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), 2111 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]), 2112 new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]), 2113 new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]), 2114 new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]), 2115 new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]), 2116 // testRowTwo-0 2117 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 2118 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 2119 new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 2120 new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]), 2121 new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]), 2122 new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]), 2123 // testRowTwo-2 2124 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 2125 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 2126 new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 2127 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]), 2128 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]), 2129 new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]), 2130 // testRowTwo-3 2131 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), 2132 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]), 2133 new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), 2134 new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]), 2135 new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]), 2136 new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]) }; 2137 2138 // Grab all 6 rows 2139 long expectedRows = 6; 2140 long expectedKeys = this.colsPerRow; 2141 for (boolean useLen : new boolean[] { false, true }) { 2142 Scan s = new Scan(); 2143 s.setFilter(new KeyOnlyFilter(useLen)); 2144 verifyScan(s, expectedRows, expectedKeys); 2145 verifyScanFullNoValues(s, expectedKVs, useLen); 2146 } 2147 } 2148 2149 /** 2150 * Filter which makes sleeps for a second between each row of a scan. This can be useful for 2151 * manual testing of bugs like HBASE-5973. For example: <code> 2152 * create 't1', 'f1' 2153 * 1.upto(100) { |x| put 't1', 'r' + x.to_s, 'f1:q1', 'hi' } 2154 * import org.apache.hadoop.hbase.filter.TestFilter 2155 * scan 't1', { FILTER => TestFilter::SlowScanFilter.new(), CACHE => 50 } 2156 * </code> 2157 */ 2158 public static class SlowScanFilter extends FilterBase { 2159 private static Thread ipcHandlerThread = null; 2160 2161 @Override 2162 public byte[] toByteArray() { 2163 return null; 2164 } 2165 2166 @Override 2167 public ReturnCode filterCell(final Cell ignored) throws IOException { 2168 return ReturnCode.INCLUDE; 2169 } 2170 2171 @Override 2172 public boolean filterRow() throws IOException { 2173 ipcHandlerThread = Thread.currentThread(); 2174 try { 2175 LOG.info("Handler thread " + ipcHandlerThread + " sleeping in filter..."); 2176 Thread.sleep(1000); 2177 } catch (InterruptedException e) { 2178 Throwables.propagate(e); 2179 } 2180 return super.filterRow(); 2181 } 2182 } 2183 2184 @Test 2185 @Ignore("TODO: intentionally disabled?") 2186 public void testNestedFilterListWithSCVF() throws IOException { 2187 byte[] columnStatus = Bytes.toBytes("S"); 2188 TableDescriptor tableDescriptor = 2189 TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName())) 2190 .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILIES[0])).build(); 2191 RegionInfo info = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build(); 2192 HRegion testRegion = HBaseTestingUtil.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), 2193 TEST_UTIL.getConfiguration(), tableDescriptor); 2194 for (int i = 0; i < 10; i++) { 2195 Put p = new Put(Bytes.toBytes("row" + i)); 2196 p.setDurability(Durability.SKIP_WAL); 2197 p.addColumn(FAMILIES[0], columnStatus, Bytes.toBytes(i % 2)); 2198 testRegion.put(p); 2199 } 2200 testRegion.flush(true); 2201 // 1. got rows > "row4" 2202 Filter rowFilter = 2203 new RowFilter(CompareOperator.GREATER, new BinaryComparator(Bytes.toBytes("row4"))); 2204 Scan s1 = new Scan(); 2205 s1.setFilter(rowFilter); 2206 InternalScanner scanner = testRegion.getScanner(s1); 2207 List<Cell> results = new ArrayList<>(); 2208 int i = 5; 2209 for (boolean done = true; done; i++) { 2210 done = scanner.next(results); 2211 assertTrue(CellUtil.matchingRows(results.get(0), Bytes.toBytes("row" + i))); 2212 assertEquals(Bytes.toInt(CellUtil.cloneValue(results.get(0))), i % 2); 2213 results.clear(); 2214 } 2215 // 2. got rows <= "row4" and S= 2216 FilterList subFilterList = new FilterList(FilterList.Operator.MUST_PASS_ALL); 2217 Filter subFilter1 = 2218 new RowFilter(CompareOperator.LESS_OR_EQUAL, new BinaryComparator(Bytes.toBytes("row4"))); 2219 subFilterList.addFilter(subFilter1); 2220 Filter subFilter2 = new SingleColumnValueFilter(FAMILIES[0], columnStatus, 2221 CompareOperator.EQUAL, Bytes.toBytes(0)); 2222 subFilterList.addFilter(subFilter2); 2223 s1 = new Scan(); 2224 s1.setFilter(subFilterList); 2225 scanner = testRegion.getScanner(s1); 2226 results = new ArrayList<>(); 2227 for (i = 0; i <= 4; i += 2) { 2228 scanner.next(results); 2229 assertTrue(CellUtil.matchingRows(results.get(0), Bytes.toBytes("row" + i))); 2230 assertEquals(Bytes.toInt(CellUtil.cloneValue(results.get(0))), i % 2); 2231 results.clear(); 2232 } 2233 assertFalse(scanner.next(results)); 2234 // 3. let's begin to verify nested filter list 2235 // 3.1 add rowFilter, then add subFilterList 2236 FilterList filterList = new FilterList(FilterList.Operator.MUST_PASS_ONE); 2237 filterList.addFilter(rowFilter); 2238 filterList.addFilter(subFilterList); 2239 s1 = new Scan(); 2240 s1.setFilter(filterList); 2241 scanner = testRegion.getScanner(s1); 2242 results = new ArrayList<>(); 2243 for (i = 0; i <= 4; i += 2) { 2244 scanner.next(results); 2245 assertTrue(CellUtil.matchingRows(results.get(0), Bytes.toBytes("row" + i))); 2246 assertEquals(Bytes.toInt(CellUtil.cloneValue(results.get(0))), i % 2); 2247 results.clear(); 2248 } 2249 for (i = 5; i <= 9; i++) { 2250 scanner.next(results); 2251 assertTrue(CellUtil.matchingRows(results.get(0), Bytes.toBytes("row" + i))); 2252 assertEquals(Bytes.toInt(CellUtil.cloneValue(results.get(0))), i % 2); 2253 results.clear(); 2254 } 2255 assertFalse(scanner.next(results)); 2256 // 3.2 MAGIC here! add subFilterList first, then add rowFilter 2257 filterList = new FilterList(FilterList.Operator.MUST_PASS_ONE); 2258 filterList.addFilter(subFilterList); 2259 filterList.addFilter(rowFilter); 2260 s1 = new Scan(); 2261 s1.setFilter(filterList); 2262 scanner = testRegion.getScanner(s1); 2263 results = new ArrayList<>(); 2264 for (i = 0; i <= 4; i += 2) { 2265 scanner.next(results); 2266 assertTrue(CellUtil.matchingRows(results.get(0), Bytes.toBytes("row" + i))); 2267 assertEquals(Bytes.toInt(CellUtil.cloneValue(results.get(0))), i % 2); 2268 results.clear(); 2269 } 2270 for (i = 5; i <= 9; i++) { 2271 scanner.next(results); 2272 assertTrue(CellUtil.matchingRows(results.get(0), Bytes.toBytes("row" + i))); 2273 assertEquals(Bytes.toInt(CellUtil.cloneValue(results.get(0))), i % 2); 2274 results.clear(); 2275 } 2276 assertFalse(scanner.next(results)); 2277 WAL wal = ((HRegion) testRegion).getWAL(); 2278 ((HRegion) testRegion).close(); 2279 wal.close(); 2280 } 2281}