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; 023import java.io.IOException; 024import java.nio.charset.StandardCharsets; 025import java.util.ArrayList; 026import java.util.List; 027import org.apache.hadoop.hbase.CompareOperator; 028import org.apache.hadoop.hbase.HBaseClassTestRule; 029import org.apache.hadoop.hbase.testclassification.MediumTests; 030import org.apache.hadoop.hbase.testclassification.RegionServerTests; 031import org.apache.hadoop.hbase.util.Bytes; 032import org.junit.After; 033import org.junit.Before; 034import org.junit.ClassRule; 035import org.junit.Test; 036import org.junit.experimental.categories.Category; 037 038/** 039 * This class tests ParseFilter.java 040 * It tests the entire work flow from when a string is given by the user 041 * and how it is parsed to construct the corresponding Filter object 042 */ 043@Category({RegionServerTests.class, MediumTests.class}) 044public class TestParseFilter { 045 046 @ClassRule 047 public static final HBaseClassTestRule CLASS_RULE = 048 HBaseClassTestRule.forClass(TestParseFilter.class); 049 050 ParseFilter f; 051 Filter filter; 052 053 @Before 054 public void setUp() throws Exception { 055 f = new ParseFilter(); 056 } 057 058 @After 059 public void tearDown() throws Exception { 060 // Nothing to do. 061 } 062 063 @Test 064 public void testKeyOnlyFilter() throws IOException { 065 String filterString = "KeyOnlyFilter()"; 066 doTestFilter(filterString, KeyOnlyFilter.class); 067 068 String filterString2 = "KeyOnlyFilter ('') "; 069 byte [] filterStringAsByteArray2 = Bytes.toBytes(filterString2); 070 try { 071 filter = f.parseFilterString(filterStringAsByteArray2); 072 assertTrue(false); 073 } catch (IllegalArgumentException e) { 074 System.out.println(e.getMessage()); 075 } 076 } 077 078 @Test 079 public void testFirstKeyOnlyFilter() throws IOException { 080 String filterString = " FirstKeyOnlyFilter( ) "; 081 doTestFilter(filterString, FirstKeyOnlyFilter.class); 082 083 String filterString2 = " FirstKeyOnlyFilter ('') "; 084 byte [] filterStringAsByteArray2 = Bytes.toBytes(filterString2); 085 try { 086 filter = f.parseFilterString(filterStringAsByteArray2); 087 assertTrue(false); 088 } catch (IllegalArgumentException e) { 089 System.out.println(e.getMessage()); 090 } 091 } 092 093 @Test 094 public void testPrefixFilter() throws IOException { 095 String filterString = " PrefixFilter('row' ) "; 096 PrefixFilter prefixFilter = doTestFilter(filterString, PrefixFilter.class); 097 byte [] prefix = prefixFilter.getPrefix(); 098 assertEquals("row", new String(prefix, StandardCharsets.UTF_8)); 099 100 101 filterString = " PrefixFilter(row)"; 102 try { 103 doTestFilter(filterString, PrefixFilter.class); 104 assertTrue(false); 105 } catch (IllegalArgumentException e) { 106 System.out.println(e.getMessage()); 107 } 108 } 109 110 @Test 111 public void testColumnPrefixFilter() throws IOException { 112 String filterString = " ColumnPrefixFilter('qualifier' ) "; 113 ColumnPrefixFilter columnPrefixFilter = 114 doTestFilter(filterString, ColumnPrefixFilter.class); 115 byte [] columnPrefix = columnPrefixFilter.getPrefix(); 116 assertEquals("qualifier", new String(columnPrefix, StandardCharsets.UTF_8)); 117 } 118 119 @Test 120 public void testMultipleColumnPrefixFilter() throws IOException { 121 String filterString = " MultipleColumnPrefixFilter('qualifier1', 'qualifier2' ) "; 122 MultipleColumnPrefixFilter multipleColumnPrefixFilter = 123 doTestFilter(filterString, MultipleColumnPrefixFilter.class); 124 byte [][] prefixes = multipleColumnPrefixFilter.getPrefix(); 125 assertEquals("qualifier1", new String(prefixes[0], StandardCharsets.UTF_8)); 126 assertEquals("qualifier2", new String(prefixes[1], StandardCharsets.UTF_8)); 127 } 128 129 @Test 130 public void testColumnCountGetFilter() throws IOException { 131 String filterString = " ColumnCountGetFilter(4)"; 132 ColumnCountGetFilter columnCountGetFilter = 133 doTestFilter(filterString, ColumnCountGetFilter.class); 134 int limit = columnCountGetFilter.getLimit(); 135 assertEquals(4, limit); 136 137 filterString = " ColumnCountGetFilter('abc')"; 138 try { 139 doTestFilter(filterString, ColumnCountGetFilter.class); 140 assertTrue(false); 141 } catch (IllegalArgumentException e) { 142 System.out.println(e.getMessage()); 143 } 144 145 filterString = " ColumnCountGetFilter(2147483648)"; 146 try { 147 doTestFilter(filterString, ColumnCountGetFilter.class); 148 assertTrue(false); 149 } catch (IllegalArgumentException e) { 150 System.out.println(e.getMessage()); 151 } 152 } 153 154 @Test 155 public void testPageFilter() throws IOException { 156 String filterString = " PageFilter(4)"; 157 PageFilter pageFilter = 158 doTestFilter(filterString, PageFilter.class); 159 long pageSize = pageFilter.getPageSize(); 160 assertEquals(4, pageSize); 161 162 filterString = " PageFilter('123')"; 163 try { 164 doTestFilter(filterString, PageFilter.class); 165 assertTrue(false); 166 } catch (IllegalArgumentException e) { 167 System.out.println("PageFilter needs an int as an argument"); 168 } 169 } 170 171 @Test 172 public void testColumnPaginationFilter() throws IOException { 173 String filterString = "ColumnPaginationFilter(4, 6)"; 174 ColumnPaginationFilter columnPaginationFilter = 175 doTestFilter(filterString, ColumnPaginationFilter.class); 176 int limit = columnPaginationFilter.getLimit(); 177 assertEquals(4, limit); 178 int offset = columnPaginationFilter.getOffset(); 179 assertEquals(6, offset); 180 181 filterString = " ColumnPaginationFilter('124')"; 182 try { 183 doTestFilter(filterString, ColumnPaginationFilter.class); 184 assertTrue(false); 185 } catch (IllegalArgumentException e) { 186 System.out.println("ColumnPaginationFilter needs two arguments"); 187 } 188 189 filterString = " ColumnPaginationFilter('4' , '123a')"; 190 try { 191 doTestFilter(filterString, ColumnPaginationFilter.class); 192 assertTrue(false); 193 } catch (IllegalArgumentException e) { 194 System.out.println("ColumnPaginationFilter needs two ints as arguments"); 195 } 196 197 filterString = " ColumnPaginationFilter('4' , '-123')"; 198 try { 199 doTestFilter(filterString, ColumnPaginationFilter.class); 200 assertTrue(false); 201 } catch (IllegalArgumentException e) { 202 System.out.println("ColumnPaginationFilter arguments should not be negative"); 203 } 204 } 205 206 @Test 207 public void testInclusiveStopFilter() throws IOException { 208 String filterString = "InclusiveStopFilter ('row 3')"; 209 InclusiveStopFilter inclusiveStopFilter = 210 doTestFilter(filterString, InclusiveStopFilter.class); 211 byte [] stopRowKey = inclusiveStopFilter.getStopRowKey(); 212 assertEquals("row 3", new String(stopRowKey, StandardCharsets.UTF_8)); 213 } 214 215 216 @Test 217 public void testTimestampsFilter() throws IOException { 218 String filterString = "TimestampsFilter(9223372036854775806, 6)"; 219 TimestampsFilter timestampsFilter = 220 doTestFilter(filterString, TimestampsFilter.class); 221 List<Long> timestamps = timestampsFilter.getTimestamps(); 222 assertEquals(2, timestamps.size()); 223 assertEquals(Long.valueOf(6), timestamps.get(0)); 224 225 filterString = "TimestampsFilter()"; 226 timestampsFilter = doTestFilter(filterString, TimestampsFilter.class); 227 timestamps = timestampsFilter.getTimestamps(); 228 assertEquals(0, timestamps.size()); 229 230 filterString = "TimestampsFilter(9223372036854775808, 6)"; 231 try { 232 doTestFilter(filterString, ColumnPaginationFilter.class); 233 assertTrue(false); 234 } catch (IllegalArgumentException e) { 235 System.out.println("Long Argument was too large"); 236 } 237 238 filterString = "TimestampsFilter(-45, 6)"; 239 try { 240 doTestFilter(filterString, ColumnPaginationFilter.class); 241 assertTrue(false); 242 } catch (IllegalArgumentException e) { 243 System.out.println("Timestamp Arguments should not be negative"); 244 } 245 } 246 247 @Test 248 public void testRowFilter() throws IOException { 249 String filterString = "RowFilter ( =, 'binary:regionse')"; 250 RowFilter rowFilter = 251 doTestFilter(filterString, RowFilter.class); 252 assertEquals(CompareOperator.EQUAL, rowFilter.getCompareOperator()); 253 assertTrue(rowFilter.getComparator() instanceof BinaryComparator); 254 BinaryComparator binaryComparator = (BinaryComparator) rowFilter.getComparator(); 255 assertEquals("regionse", new String(binaryComparator.getValue(), StandardCharsets.UTF_8)); 256 } 257 258 @Test 259 public void testFamilyFilter() throws IOException { 260 String filterString = "FamilyFilter(>=, 'binaryprefix:pre')"; 261 FamilyFilter familyFilter = 262 doTestFilter(filterString, FamilyFilter.class); 263 assertEquals(CompareOperator.GREATER_OR_EQUAL, familyFilter.getCompareOperator()); 264 assertTrue(familyFilter.getComparator() instanceof BinaryPrefixComparator); 265 BinaryPrefixComparator binaryPrefixComparator = 266 (BinaryPrefixComparator) familyFilter.getComparator(); 267 assertEquals("pre", new String(binaryPrefixComparator.getValue(), StandardCharsets.UTF_8)); 268 } 269 270 @Test 271 public void testQualifierFilter() throws IOException { 272 String filterString = "QualifierFilter(=, 'regexstring:pre*')"; 273 QualifierFilter qualifierFilter = 274 doTestFilter(filterString, QualifierFilter.class); 275 assertEquals(CompareOperator.EQUAL, qualifierFilter.getCompareOperator()); 276 assertTrue(qualifierFilter.getComparator() instanceof RegexStringComparator); 277 RegexStringComparator regexStringComparator = 278 (RegexStringComparator) qualifierFilter.getComparator(); 279 assertEquals("pre*", new String(regexStringComparator.getValue(), StandardCharsets.UTF_8)); 280 } 281 282 @Test 283 public void testValueFilter() throws IOException { 284 String filterString = "ValueFilter(!=, 'substring:pre')"; 285 ValueFilter valueFilter = 286 doTestFilter(filterString, ValueFilter.class); 287 assertEquals(CompareOperator.NOT_EQUAL, valueFilter.getCompareOperator()); 288 assertTrue(valueFilter.getComparator() instanceof SubstringComparator); 289 SubstringComparator substringComparator = 290 (SubstringComparator) valueFilter.getComparator(); 291 assertEquals("pre", new String(substringComparator.getValue(), StandardCharsets.UTF_8)); 292 } 293 294 @Test 295 public void testColumnRangeFilter() throws IOException { 296 String filterString = "ColumnRangeFilter('abc', true, 'xyz', false)"; 297 ColumnRangeFilter columnRangeFilter = 298 doTestFilter(filterString, ColumnRangeFilter.class); 299 assertEquals("abc", new String(columnRangeFilter.getMinColumn(), StandardCharsets.UTF_8)); 300 assertEquals("xyz", new String(columnRangeFilter.getMaxColumn(), StandardCharsets.UTF_8)); 301 assertTrue(columnRangeFilter.isMinColumnInclusive()); 302 assertFalse(columnRangeFilter.isMaxColumnInclusive()); 303 } 304 305 @Test 306 public void testDependentColumnFilter() throws IOException { 307 String filterString = "DependentColumnFilter('family', 'qualifier', true, =, 'binary:abc')"; 308 DependentColumnFilter dependentColumnFilter = 309 doTestFilter(filterString, DependentColumnFilter.class); 310 assertEquals("family", new String(dependentColumnFilter.getFamily(), StandardCharsets.UTF_8)); 311 assertEquals("qualifier", 312 new String(dependentColumnFilter.getQualifier(), StandardCharsets.UTF_8)); 313 assertTrue(dependentColumnFilter.getDropDependentColumn()); 314 assertEquals(CompareOperator.EQUAL, dependentColumnFilter.getCompareOperator()); 315 assertTrue(dependentColumnFilter.getComparator() instanceof BinaryComparator); 316 BinaryComparator binaryComparator = (BinaryComparator)dependentColumnFilter.getComparator(); 317 assertEquals("abc", new String(binaryComparator.getValue(), StandardCharsets.UTF_8)); 318 } 319 320 @Test 321 public void testSingleColumnValueFilter() throws IOException { 322 String filterString = "SingleColumnValueFilter " + 323 "('family', 'qualifier', >=, 'binary:a', true, false)"; 324 SingleColumnValueFilter singleColumnValueFilter = 325 doTestFilter(filterString, SingleColumnValueFilter.class); 326 assertEquals("family", new String(singleColumnValueFilter.getFamily(), StandardCharsets.UTF_8)); 327 assertEquals("qualifier", 328 new String(singleColumnValueFilter.getQualifier(), StandardCharsets.UTF_8)); 329 assertEquals(CompareOperator.GREATER_OR_EQUAL, singleColumnValueFilter.getCompareOperator()); 330 assertTrue(singleColumnValueFilter.getComparator() instanceof BinaryComparator); 331 BinaryComparator binaryComparator = (BinaryComparator) singleColumnValueFilter.getComparator(); 332 assertEquals("a", new String(binaryComparator.getValue(), StandardCharsets.UTF_8)); 333 assertTrue(singleColumnValueFilter.getFilterIfMissing()); 334 assertFalse(singleColumnValueFilter.getLatestVersionOnly()); 335 336 337 filterString = "SingleColumnValueFilter ('family', 'qualifier', >, 'binaryprefix:a')"; 338 singleColumnValueFilter = doTestFilter(filterString, SingleColumnValueFilter.class); 339 assertEquals("family", new String(singleColumnValueFilter.getFamily(), StandardCharsets.UTF_8)); 340 assertEquals("qualifier", 341 new String(singleColumnValueFilter.getQualifier(), StandardCharsets.UTF_8)); 342 assertEquals(CompareOperator.GREATER, singleColumnValueFilter.getCompareOperator()); 343 assertTrue(singleColumnValueFilter.getComparator() instanceof BinaryPrefixComparator); 344 BinaryPrefixComparator binaryPrefixComparator = 345 (BinaryPrefixComparator) singleColumnValueFilter.getComparator(); 346 assertEquals("a", new String(binaryPrefixComparator.getValue(), StandardCharsets.UTF_8)); 347 assertFalse(singleColumnValueFilter.getFilterIfMissing()); 348 assertTrue(singleColumnValueFilter.getLatestVersionOnly()); 349 } 350 351 @Test 352 public void testSingleColumnValueExcludeFilter() throws IOException { 353 String filterString = 354 "SingleColumnValueExcludeFilter ('family', 'qualifier', <, 'binaryprefix:a')"; 355 SingleColumnValueExcludeFilter singleColumnValueExcludeFilter = 356 doTestFilter(filterString, SingleColumnValueExcludeFilter.class); 357 assertEquals(CompareOperator.LESS, singleColumnValueExcludeFilter.getCompareOperator()); 358 assertEquals("family", 359 new String(singleColumnValueExcludeFilter.getFamily(), StandardCharsets.UTF_8)); 360 assertEquals("qualifier", 361 new String(singleColumnValueExcludeFilter.getQualifier(), StandardCharsets.UTF_8)); 362 assertEquals("a", new String(singleColumnValueExcludeFilter.getComparator().getValue(), 363 StandardCharsets.UTF_8)); 364 assertFalse(singleColumnValueExcludeFilter.getFilterIfMissing()); 365 assertTrue(singleColumnValueExcludeFilter.getLatestVersionOnly()); 366 367 filterString = "SingleColumnValueExcludeFilter " + 368 "('family', 'qualifier', <=, 'binaryprefix:a', true, false)"; 369 singleColumnValueExcludeFilter = 370 doTestFilter(filterString, SingleColumnValueExcludeFilter.class); 371 assertEquals("family", 372 new String(singleColumnValueExcludeFilter.getFamily(), StandardCharsets.UTF_8)); 373 assertEquals("qualifier", 374 new String(singleColumnValueExcludeFilter.getQualifier(), StandardCharsets.UTF_8)); 375 assertEquals(CompareOperator.LESS_OR_EQUAL, 376 singleColumnValueExcludeFilter.getCompareOperator()); 377 assertTrue(singleColumnValueExcludeFilter.getComparator() instanceof BinaryPrefixComparator); 378 BinaryPrefixComparator binaryPrefixComparator = 379 (BinaryPrefixComparator) singleColumnValueExcludeFilter.getComparator(); 380 assertEquals("a", new String(binaryPrefixComparator.getValue(), StandardCharsets.UTF_8)); 381 assertTrue(singleColumnValueExcludeFilter.getFilterIfMissing()); 382 assertFalse(singleColumnValueExcludeFilter.getLatestVersionOnly()); 383 } 384 385 @Test 386 public void testSkipFilter() throws IOException { 387 String filterString = "SKIP ValueFilter( =, 'binary:0')"; 388 SkipFilter skipFilter = 389 doTestFilter(filterString, SkipFilter.class); 390 assertTrue(skipFilter.getFilter() instanceof ValueFilter); 391 ValueFilter valueFilter = (ValueFilter) skipFilter.getFilter(); 392 393 assertEquals(CompareOperator.EQUAL, valueFilter.getCompareOperator()); 394 assertTrue(valueFilter.getComparator() instanceof BinaryComparator); 395 BinaryComparator binaryComparator = (BinaryComparator) valueFilter.getComparator(); 396 assertEquals("0", new String(binaryComparator.getValue(), StandardCharsets.UTF_8)); 397 } 398 399 @Test 400 public void testWhileFilter() throws IOException { 401 String filterString = " WHILE RowFilter ( !=, 'binary:row1')"; 402 WhileMatchFilter whileMatchFilter = 403 doTestFilter(filterString, WhileMatchFilter.class); 404 assertTrue(whileMatchFilter.getFilter() instanceof RowFilter); 405 RowFilter rowFilter = (RowFilter) whileMatchFilter.getFilter(); 406 407 assertEquals(CompareOperator.NOT_EQUAL, rowFilter.getCompareOperator()); 408 assertTrue(rowFilter.getComparator() instanceof BinaryComparator); 409 BinaryComparator binaryComparator = (BinaryComparator) rowFilter.getComparator(); 410 assertEquals("row1", new String(binaryComparator.getValue(), StandardCharsets.UTF_8)); 411 } 412 413 @Test 414 public void testCompoundFilter1() throws IOException { 415 String filterString = " (PrefixFilter ('realtime')AND FirstKeyOnlyFilter())"; 416 FilterList filterList = 417 doTestFilter(filterString, FilterList.class); 418 ArrayList<Filter> filters = (ArrayList<Filter>) filterList.getFilters(); 419 420 assertTrue(filters.get(0) instanceof PrefixFilter); 421 assertTrue(filters.get(1) instanceof FirstKeyOnlyFilter); 422 PrefixFilter PrefixFilter = (PrefixFilter) filters.get(0); 423 byte [] prefix = PrefixFilter.getPrefix(); 424 assertEquals("realtime", new String(prefix, StandardCharsets.UTF_8)); 425 FirstKeyOnlyFilter firstKeyOnlyFilter = (FirstKeyOnlyFilter) filters.get(1); 426 } 427 428 @Test 429 public void testCompoundFilter2() throws IOException { 430 String filterString = "(PrefixFilter('realtime') AND QualifierFilter (>=, 'binary:e'))" + 431 "OR FamilyFilter (=, 'binary:qualifier') "; 432 FilterList filterList = 433 doTestFilter(filterString, FilterList.class); 434 ArrayList<Filter> filterListFilters = (ArrayList<Filter>) filterList.getFilters(); 435 assertTrue(filterListFilters.get(0) instanceof FilterList); 436 assertTrue(filterListFilters.get(1) instanceof FamilyFilter); 437 assertEquals(FilterList.Operator.MUST_PASS_ONE, filterList.getOperator()); 438 439 filterList = (FilterList) filterListFilters.get(0); 440 FamilyFilter familyFilter = (FamilyFilter) filterListFilters.get(1); 441 442 filterListFilters = (ArrayList<Filter>)filterList.getFilters(); 443 assertTrue(filterListFilters.get(0) instanceof PrefixFilter); 444 assertTrue(filterListFilters.get(1) instanceof QualifierFilter); 445 assertEquals(FilterList.Operator.MUST_PASS_ALL, filterList.getOperator()); 446 447 assertEquals(CompareOperator.EQUAL, familyFilter.getCompareOperator()); 448 assertTrue(familyFilter.getComparator() instanceof BinaryComparator); 449 BinaryComparator binaryComparator = (BinaryComparator) familyFilter.getComparator(); 450 assertEquals("qualifier", new String(binaryComparator.getValue(), StandardCharsets.UTF_8)); 451 452 PrefixFilter prefixFilter = (PrefixFilter) filterListFilters.get(0); 453 byte [] prefix = prefixFilter.getPrefix(); 454 assertEquals("realtime", new String(prefix, StandardCharsets.UTF_8)); 455 456 QualifierFilter qualifierFilter = (QualifierFilter) filterListFilters.get(1); 457 assertEquals(CompareOperator.GREATER_OR_EQUAL, qualifierFilter.getCompareOperator()); 458 assertTrue(qualifierFilter.getComparator() instanceof BinaryComparator); 459 binaryComparator = (BinaryComparator) qualifierFilter.getComparator(); 460 assertEquals("e", new String(binaryComparator.getValue(), StandardCharsets.UTF_8)); 461 } 462 463 @Test 464 public void testCompoundFilter3() throws IOException { 465 String filterString = " ColumnPrefixFilter ('realtime')AND " + 466 "FirstKeyOnlyFilter() OR SKIP FamilyFilter(=, 'substring:hihi')"; 467 FilterList filterList = 468 doTestFilter(filterString, FilterList.class); 469 ArrayList<Filter> filters = (ArrayList<Filter>) filterList.getFilters(); 470 471 assertTrue(filters.get(0) instanceof FilterList); 472 assertTrue(filters.get(1) instanceof SkipFilter); 473 474 filterList = (FilterList) filters.get(0); 475 SkipFilter skipFilter = (SkipFilter) filters.get(1); 476 477 filters = (ArrayList<Filter>) filterList.getFilters(); 478 assertTrue(filters.get(0) instanceof ColumnPrefixFilter); 479 assertTrue(filters.get(1) instanceof FirstKeyOnlyFilter); 480 481 ColumnPrefixFilter columnPrefixFilter = (ColumnPrefixFilter) filters.get(0); 482 byte [] columnPrefix = columnPrefixFilter.getPrefix(); 483 assertEquals("realtime", new String(columnPrefix, StandardCharsets.UTF_8)); 484 485 FirstKeyOnlyFilter firstKeyOnlyFilter = (FirstKeyOnlyFilter) filters.get(1); 486 487 assertTrue(skipFilter.getFilter() instanceof FamilyFilter); 488 FamilyFilter familyFilter = (FamilyFilter) skipFilter.getFilter(); 489 490 assertEquals(CompareOperator.EQUAL, familyFilter.getCompareOperator()); 491 assertTrue(familyFilter.getComparator() instanceof SubstringComparator); 492 SubstringComparator substringComparator = 493 (SubstringComparator) familyFilter.getComparator(); 494 assertEquals("hihi", new String(substringComparator.getValue(), StandardCharsets.UTF_8)); 495 } 496 497 @Test 498 public void testCompoundFilter4() throws IOException { 499 String filterString = " ColumnPrefixFilter ('realtime') OR " + 500 "FirstKeyOnlyFilter() OR SKIP FamilyFilter(=, 'substring:hihi')"; 501 FilterList filterList = 502 doTestFilter(filterString, FilterList.class); 503 ArrayList<Filter> filters = (ArrayList<Filter>) filterList.getFilters(); 504 505 assertTrue(filters.get(0) instanceof ColumnPrefixFilter); 506 assertTrue(filters.get(1) instanceof FirstKeyOnlyFilter); 507 assertTrue(filters.get(2) instanceof SkipFilter); 508 509 ColumnPrefixFilter columnPrefixFilter = (ColumnPrefixFilter) filters.get(0); 510 FirstKeyOnlyFilter firstKeyOnlyFilter = (FirstKeyOnlyFilter) filters.get(1); 511 SkipFilter skipFilter = (SkipFilter) filters.get(2); 512 513 byte [] columnPrefix = columnPrefixFilter.getPrefix(); 514 assertEquals("realtime", new String(columnPrefix, StandardCharsets.UTF_8)); 515 516 assertTrue(skipFilter.getFilter() instanceof FamilyFilter); 517 FamilyFilter familyFilter = (FamilyFilter) skipFilter.getFilter(); 518 519 assertEquals(CompareOperator.EQUAL, familyFilter.getCompareOperator()); 520 assertTrue(familyFilter.getComparator() instanceof SubstringComparator); 521 SubstringComparator substringComparator = 522 (SubstringComparator) familyFilter.getComparator(); 523 assertEquals("hihi", new String(substringComparator.getValue(), StandardCharsets.UTF_8)); 524 } 525 526 @Test 527 public void testCompoundFilter5() throws IOException { 528 String filterStr = "(ValueFilter(!=, 'substring:pre'))"; 529 ValueFilter valueFilter = doTestFilter(filterStr, ValueFilter.class); 530 assertTrue(valueFilter.getComparator() instanceof SubstringComparator); 531 532 filterStr = "(ValueFilter(>=,'binary:x') AND (ValueFilter(<=,'binary:y')))" 533 + " OR ValueFilter(=,'binary:ab')"; 534 filter = f.parseFilterString(filterStr); 535 assertTrue(filter instanceof FilterList); 536 List<Filter> list = ((FilterList) filter).getFilters(); 537 assertEquals(2, list.size()); 538 assertTrue(list.get(0) instanceof FilterList); 539 assertTrue(list.get(1) instanceof ValueFilter); 540 } 541 542 @Test 543 public void testIncorrectCompareOperator() throws IOException { 544 String filterString = "RowFilter ('>>' , 'binary:region')"; 545 try { 546 doTestFilter(filterString, RowFilter.class); 547 assertTrue(false); 548 } catch (IllegalArgumentException e) { 549 System.out.println("Incorrect compare operator >>"); 550 } 551 } 552 553 @Test 554 public void testIncorrectComparatorType() throws IOException { 555 String filterString = "RowFilter ('>=' , 'binaryoperator:region')"; 556 try { 557 doTestFilter(filterString, RowFilter.class); 558 assertTrue(false); 559 } catch (IllegalArgumentException e) { 560 System.out.println("Incorrect comparator type: binaryoperator"); 561 } 562 563 filterString = "RowFilter ('>=' 'regexstring:pre*')"; 564 try { 565 doTestFilter(filterString, RowFilter.class); 566 assertTrue(false); 567 } catch (IllegalArgumentException e) { 568 System.out.println("RegexStringComparator can only be used with EQUAL or NOT_EQUAL"); 569 } 570 571 filterString = "SingleColumnValueFilter" + 572 " ('family', 'qualifier', '>=', 'substring:a', 'true', 'false')')"; 573 try { 574 doTestFilter(filterString, RowFilter.class); 575 assertTrue(false); 576 } catch (IllegalArgumentException e) { 577 System.out.println("SubtringComparator can only be used with EQUAL or NOT_EQUAL"); 578 } 579 } 580 581 @Test 582 public void testPrecedence1() throws IOException { 583 String filterString = " (PrefixFilter ('realtime')AND FirstKeyOnlyFilter()" + 584 " OR KeyOnlyFilter())"; 585 FilterList filterList = 586 doTestFilter(filterString, FilterList.class); 587 588 ArrayList<Filter> filters = (ArrayList<Filter>) filterList.getFilters(); 589 590 assertTrue(filters.get(0) instanceof FilterList); 591 assertTrue(filters.get(1) instanceof KeyOnlyFilter); 592 593 filterList = (FilterList) filters.get(0); 594 filters = (ArrayList<Filter>) filterList.getFilters(); 595 596 assertTrue(filters.get(0) instanceof PrefixFilter); 597 assertTrue(filters.get(1) instanceof FirstKeyOnlyFilter); 598 599 PrefixFilter prefixFilter = (PrefixFilter)filters.get(0); 600 byte [] prefix = prefixFilter.getPrefix(); 601 assertEquals("realtime", new String(prefix, StandardCharsets.UTF_8)); 602 } 603 604 @Test 605 public void testPrecedence2() throws IOException { 606 String filterString = " PrefixFilter ('realtime')AND SKIP FirstKeyOnlyFilter()" + 607 "OR KeyOnlyFilter()"; 608 FilterList filterList = 609 doTestFilter(filterString, FilterList.class); 610 ArrayList<Filter> filters = (ArrayList<Filter>) filterList.getFilters(); 611 612 assertTrue(filters.get(0) instanceof FilterList); 613 assertTrue(filters.get(1) instanceof KeyOnlyFilter); 614 615 filterList = (FilterList) filters.get(0); 616 filters = (ArrayList<Filter>) filterList.getFilters(); 617 618 assertTrue(filters.get(0) instanceof PrefixFilter); 619 assertTrue(filters.get(1) instanceof SkipFilter); 620 621 PrefixFilter prefixFilter = (PrefixFilter)filters.get(0); 622 byte [] prefix = prefixFilter.getPrefix(); 623 assertEquals("realtime", new String(prefix, StandardCharsets.UTF_8)); 624 625 SkipFilter skipFilter = (SkipFilter)filters.get(1); 626 assertTrue(skipFilter.getFilter() instanceof FirstKeyOnlyFilter); 627 } 628 629 @Test 630 public void testUnescapedQuote1() throws IOException { 631 String filterString = "InclusiveStopFilter ('row''3')"; 632 InclusiveStopFilter inclusiveStopFilter = 633 doTestFilter(filterString, InclusiveStopFilter.class); 634 byte [] stopRowKey = inclusiveStopFilter.getStopRowKey(); 635 assertEquals("row'3", new String(stopRowKey, StandardCharsets.UTF_8)); 636 } 637 638 @Test 639 public void testUnescapedQuote2() throws IOException { 640 String filterString = "InclusiveStopFilter ('row''3''')"; 641 InclusiveStopFilter inclusiveStopFilter = 642 doTestFilter(filterString, InclusiveStopFilter.class); 643 byte [] stopRowKey = inclusiveStopFilter.getStopRowKey(); 644 assertEquals("row'3'", new String(stopRowKey, StandardCharsets.UTF_8)); 645 } 646 647 @Test 648 public void testUnescapedQuote3() throws IOException { 649 String filterString = " InclusiveStopFilter ('''')"; 650 InclusiveStopFilter inclusiveStopFilter = doTestFilter(filterString, InclusiveStopFilter.class); 651 byte [] stopRowKey = inclusiveStopFilter.getStopRowKey(); 652 assertEquals("'", new String(stopRowKey, StandardCharsets.UTF_8)); 653 } 654 655 @Test 656 public void testIncorrectFilterString() throws IOException { 657 String filterString = "()"; 658 byte [] filterStringAsByteArray = Bytes.toBytes(filterString); 659 try { 660 filter = f.parseFilterString(filterStringAsByteArray); 661 assertTrue(false); 662 } catch (IllegalArgumentException e) { 663 System.out.println(e.getMessage()); 664 } 665 } 666 667 @Test 668 public void testCorrectFilterString() throws IOException { 669 String filterString = "(FirstKeyOnlyFilter())"; 670 FirstKeyOnlyFilter firstKeyOnlyFilter = doTestFilter(filterString, FirstKeyOnlyFilter.class); 671 } 672 673 @Test 674 public void testRegisterFilter() { 675 ParseFilter.registerFilter("MyFilter", "some.class"); 676 677 assertTrue(f.getSupportedFilters().contains("MyFilter")); 678 } 679 680 private <T extends Filter> T doTestFilter(String filterString, Class<T> clazz) 681 throws IOException { 682 byte [] filterStringAsByteArray = Bytes.toBytes(filterString); 683 filter = f.parseFilterString(filterStringAsByteArray); 684 assertEquals(clazz, filter.getClass()); 685 return clazz.cast(filter); 686 } 687 688 @Test 689 public void testColumnValueFilter() throws IOException { 690 String filterString = "ColumnValueFilter ('family', 'qualifier', <, 'binaryprefix:value')"; 691 ColumnValueFilter cvf = doTestFilter(filterString, ColumnValueFilter.class); 692 assertEquals("family", new String(cvf.getFamily(), StandardCharsets.UTF_8)); 693 assertEquals("qualifier", new String(cvf.getQualifier(), StandardCharsets.UTF_8)); 694 assertEquals(CompareOperator.LESS, cvf.getCompareOperator()); 695 assertTrue(cvf.getComparator() instanceof BinaryPrefixComparator); 696 assertEquals("value", new String(cvf.getComparator().getValue(), StandardCharsets.UTF_8)); 697 } 698}