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