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