1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.filter;
20
21 import static org.junit.Assert.*;
22
23 import java.io.IOException;
24 import java.nio.ByteBuffer;
25 import java.util.ArrayList;
26 import java.util.HashMap;
27 import java.util.HashSet;
28 import java.util.List;
29 import java.util.Map;
30 import java.util.Set;
31 import java.util.TreeSet;
32
33 import org.apache.hadoop.hbase.*;
34 import org.apache.hadoop.hbase.client.Put;
35 import org.apache.hadoop.hbase.client.Scan;
36 import org.apache.hadoop.hbase.regionserver.HRegion;
37 import org.apache.hadoop.hbase.regionserver.InternalScanner;
38 import org.apache.hadoop.hbase.util.Bytes;
39 import org.junit.After;
40 import org.junit.Before;
41 import org.junit.Test;
42 import org.junit.experimental.categories.Category;
43
44
45
46
47
48
49 @Category(SmallTests.class)
50 public class TestParseFilter {
51
52 ParseFilter f;
53 Filter filter;
54
55 @Before
56 public void setUp() throws Exception {
57 f = new ParseFilter();
58 }
59
60 @After
61 public void tearDown() throws Exception {
62
63 }
64
65 @Test
66 public void testKeyOnlyFilter() throws IOException {
67 String filterString = "KeyOnlyFilter()";
68 doTestFilter(filterString, KeyOnlyFilter.class);
69
70 String filterString2 = "KeyOnlyFilter ('') ";
71 byte [] filterStringAsByteArray2 = Bytes.toBytes(filterString2);
72 try {
73 filter = f.parseFilterString(filterStringAsByteArray2);
74 assertTrue(false);
75 } catch (IllegalArgumentException e) {
76 System.out.println(e.getMessage());
77 }
78 }
79
80 @Test
81 public void testFirstKeyOnlyFilter() throws IOException {
82 String filterString = " FirstKeyOnlyFilter( ) ";
83 doTestFilter(filterString, FirstKeyOnlyFilter.class);
84
85 String filterString2 = " FirstKeyOnlyFilter ('') ";
86 byte [] filterStringAsByteArray2 = Bytes.toBytes(filterString2);
87 try {
88 filter = f.parseFilterString(filterStringAsByteArray2);
89 assertTrue(false);
90 } catch (IllegalArgumentException e) {
91 System.out.println(e.getMessage());
92 }
93 }
94
95 @Test
96 public void testPrefixFilter() throws IOException {
97 String filterString = " PrefixFilter('row' ) ";
98 PrefixFilter prefixFilter = doTestFilter(filterString, PrefixFilter.class);
99 byte [] prefix = prefixFilter.getPrefix();
100 assertEquals(new String(prefix), "row");
101
102
103 filterString = " PrefixFilter(row)";
104 try {
105 doTestFilter(filterString, PrefixFilter.class);
106 assertTrue(false);
107 } catch (IllegalArgumentException e) {
108 System.out.println(e.getMessage());
109 }
110 }
111
112 @Test
113 public void testColumnPrefixFilter() throws IOException {
114 String filterString = " ColumnPrefixFilter('qualifier' ) ";
115 ColumnPrefixFilter columnPrefixFilter =
116 doTestFilter(filterString, ColumnPrefixFilter.class);
117 byte [] columnPrefix = columnPrefixFilter.getPrefix();
118 assertEquals(new String(columnPrefix), "qualifier");
119 }
120
121 @Test
122 public void testMultipleColumnPrefixFilter() throws IOException {
123 String filterString = " MultipleColumnPrefixFilter('qualifier1', 'qualifier2' ) ";
124 MultipleColumnPrefixFilter multipleColumnPrefixFilter =
125 doTestFilter(filterString, MultipleColumnPrefixFilter.class);
126 byte [][] prefixes = multipleColumnPrefixFilter.getPrefix();
127 assertEquals(new String(prefixes[0]), "qualifier1");
128 assertEquals(new String(prefixes[1]), "qualifier2");
129 }
130
131 @Test
132 public void testColumnCountGetFilter() throws IOException {
133 String filterString = " ColumnCountGetFilter(4)";
134 ColumnCountGetFilter columnCountGetFilter =
135 doTestFilter(filterString, ColumnCountGetFilter.class);
136 int limit = columnCountGetFilter.getLimit();
137 assertEquals(limit, 4);
138
139 filterString = " ColumnCountGetFilter('abc')";
140 try {
141 doTestFilter(filterString, ColumnCountGetFilter.class);
142 assertTrue(false);
143 } catch (IllegalArgumentException e) {
144 System.out.println(e.getMessage());
145 }
146
147 filterString = " ColumnCountGetFilter(2147483648)";
148 try {
149 doTestFilter(filterString, ColumnCountGetFilter.class);
150 assertTrue(false);
151 } catch (IllegalArgumentException e) {
152 System.out.println(e.getMessage());
153 }
154 }
155
156 @Test
157 public void testPageFilter() throws IOException {
158 String filterString = " PageFilter(4)";
159 PageFilter pageFilter =
160 doTestFilter(filterString, PageFilter.class);
161 long pageSize = pageFilter.getPageSize();
162 assertEquals(pageSize, 4);
163
164 filterString = " PageFilter('123')";
165 try {
166 doTestFilter(filterString, PageFilter.class);
167 assertTrue(false);
168 } catch (IllegalArgumentException e) {
169 System.out.println("PageFilter needs an int as an argument");
170 }
171 }
172
173 @Test
174 public void testColumnPaginationFilter() throws IOException {
175 String filterString = "ColumnPaginationFilter(4, 6)";
176 ColumnPaginationFilter columnPaginationFilter =
177 doTestFilter(filterString, ColumnPaginationFilter.class);
178 int limit = columnPaginationFilter.getLimit();
179 assertEquals(limit, 4);
180 int offset = columnPaginationFilter.getOffset();
181 assertEquals(offset, 6);
182
183 filterString = " ColumnPaginationFilter('124')";
184 try {
185 doTestFilter(filterString, ColumnPaginationFilter.class);
186 assertTrue(false);
187 } catch (IllegalArgumentException e) {
188 System.out.println("ColumnPaginationFilter needs two arguments");
189 }
190
191 filterString = " ColumnPaginationFilter('4' , '123a')";
192 try {
193 doTestFilter(filterString, ColumnPaginationFilter.class);
194 assertTrue(false);
195 } catch (IllegalArgumentException e) {
196 System.out.println("ColumnPaginationFilter needs two ints as arguments");
197 }
198
199 filterString = " ColumnPaginationFilter('4' , '-123')";
200 try {
201 doTestFilter(filterString, ColumnPaginationFilter.class);
202 assertTrue(false);
203 } catch (IllegalArgumentException e) {
204 System.out.println("ColumnPaginationFilter arguments should not be negative");
205 }
206 }
207
208 @Test
209 public void testInclusiveStopFilter() throws IOException {
210 String filterString = "InclusiveStopFilter ('row 3')";
211 InclusiveStopFilter inclusiveStopFilter =
212 doTestFilter(filterString, InclusiveStopFilter.class);
213 byte [] stopRowKey = inclusiveStopFilter.getStopRowKey();
214 assertEquals(new String(stopRowKey), "row 3");
215 }
216
217
218 @Test
219 public void testTimestampsFilter() throws IOException {
220 String filterString = "TimestampsFilter(9223372036854775806, 6)";
221 TimestampsFilter timestampsFilter =
222 doTestFilter(filterString, TimestampsFilter.class);
223 List<Long> timestamps = timestampsFilter.getTimestamps();
224 assertEquals(timestamps.size(), 2);
225 assertEquals(timestamps.get(0), new Long(6));
226
227 filterString = "TimestampsFilter()";
228 timestampsFilter = doTestFilter(filterString, TimestampsFilter.class);
229 timestamps = timestampsFilter.getTimestamps();
230 assertEquals(timestamps.size(), 0);
231
232 filterString = "TimestampsFilter(9223372036854775808, 6)";
233 try {
234 doTestFilter(filterString, ColumnPaginationFilter.class);
235 assertTrue(false);
236 } catch (IllegalArgumentException e) {
237 System.out.println("Long Argument was too large");
238 }
239
240 filterString = "TimestampsFilter(-45, 6)";
241 try {
242 doTestFilter(filterString, ColumnPaginationFilter.class);
243 assertTrue(false);
244 } catch (IllegalArgumentException e) {
245 System.out.println("Timestamp Arguments should not be negative");
246 }
247 }
248
249 @Test
250 public void testRowFilter() throws IOException {
251 String filterString = "RowFilter ( =, 'binary:regionse')";
252 RowFilter rowFilter =
253 doTestFilter(filterString, RowFilter.class);
254 assertEquals(CompareFilter.CompareOp.EQUAL, rowFilter.getOperator());
255 assertTrue(rowFilter.getComparator() instanceof BinaryComparator);
256 BinaryComparator binaryComparator = (BinaryComparator) rowFilter.getComparator();
257 assertEquals("regionse", new String(binaryComparator.getValue()));
258 }
259
260 @Test
261 public void testFamilyFilter() throws IOException {
262 String filterString = "FamilyFilter(>=, 'binaryprefix:pre')";
263 FamilyFilter familyFilter =
264 doTestFilter(filterString, FamilyFilter.class);
265 assertEquals(CompareFilter.CompareOp.GREATER_OR_EQUAL, familyFilter.getOperator());
266 assertTrue(familyFilter.getComparator() instanceof BinaryPrefixComparator);
267 BinaryPrefixComparator binaryPrefixComparator =
268 (BinaryPrefixComparator) familyFilter.getComparator();
269 assertEquals("pre", new String(binaryPrefixComparator.getValue()));
270 }
271
272 @Test
273 public void testQualifierFilter() throws IOException {
274 String filterString = "QualifierFilter(=, 'regexstring:pre*')";
275 QualifierFilter qualifierFilter =
276 doTestFilter(filterString, QualifierFilter.class);
277 assertEquals(CompareFilter.CompareOp.EQUAL, qualifierFilter.getOperator());
278 assertTrue(qualifierFilter.getComparator() instanceof RegexStringComparator);
279 RegexStringComparator regexStringComparator =
280 (RegexStringComparator) qualifierFilter.getComparator();
281 assertEquals("pre*", new String(regexStringComparator.getValue()));
282 }
283
284 @Test
285 public void testValueFilter() throws IOException {
286 String filterString = "ValueFilter(!=, 'substring:pre')";
287 ValueFilter valueFilter =
288 doTestFilter(filterString, ValueFilter.class);
289 assertEquals(CompareFilter.CompareOp.NOT_EQUAL, valueFilter.getOperator());
290 assertTrue(valueFilter.getComparator() instanceof SubstringComparator);
291 SubstringComparator substringComparator =
292 (SubstringComparator) valueFilter.getComparator();
293 assertEquals("pre", new String(substringComparator.getValue()));
294 }
295
296 @Test
297 public void testColumnRangeFilter() throws IOException {
298 String filterString = "ColumnRangeFilter('abc', true, 'xyz', false)";
299 ColumnRangeFilter columnRangeFilter =
300 doTestFilter(filterString, ColumnRangeFilter.class);
301 assertEquals("abc", new String(columnRangeFilter.getMinColumn()));
302 assertEquals("xyz", new String(columnRangeFilter.getMaxColumn()));
303 assertTrue(columnRangeFilter.isMinColumnInclusive());
304 assertFalse(columnRangeFilter.isMaxColumnInclusive());
305 }
306
307 @Test
308 public void testDependentColumnFilter() throws IOException {
309 String filterString = "DependentColumnFilter('family', 'qualifier', true, =, 'binary:abc')";
310 DependentColumnFilter dependentColumnFilter =
311 doTestFilter(filterString, DependentColumnFilter.class);
312 assertEquals("family", new String(dependentColumnFilter.getFamily()));
313 assertEquals("qualifier", new String(dependentColumnFilter.getQualifier()));
314 assertTrue(dependentColumnFilter.getDropDependentColumn());
315 assertEquals(CompareFilter.CompareOp.EQUAL, dependentColumnFilter.getOperator());
316 assertTrue(dependentColumnFilter.getComparator() instanceof BinaryComparator);
317 BinaryComparator binaryComparator = (BinaryComparator)dependentColumnFilter.getComparator();
318 assertEquals("abc", new String(binaryComparator.getValue()));
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()));
328 assertEquals("qualifier", new String(singleColumnValueFilter.getQualifier()));
329 assertEquals(singleColumnValueFilter.getOperator(), CompareFilter.CompareOp.GREATER_OR_EQUAL);
330 assertTrue(singleColumnValueFilter.getComparator() instanceof BinaryComparator);
331 BinaryComparator binaryComparator = (BinaryComparator) singleColumnValueFilter.getComparator();
332 assertEquals(new String(binaryComparator.getValue()), "a");
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()));
340 assertEquals("qualifier", new String(singleColumnValueFilter.getQualifier()));
341 assertEquals(singleColumnValueFilter.getOperator(), CompareFilter.CompareOp.GREATER);
342 assertTrue(singleColumnValueFilter.getComparator() instanceof BinaryPrefixComparator);
343 BinaryPrefixComparator binaryPrefixComparator =
344 (BinaryPrefixComparator) singleColumnValueFilter.getComparator();
345 assertEquals(new String(binaryPrefixComparator.getValue()), "a");
346 assertFalse(singleColumnValueFilter.getFilterIfMissing());
347 assertTrue(singleColumnValueFilter.getLatestVersionOnly());
348 }
349
350 @Test
351 public void testSingleColumnValueExcludeFilter() throws IOException {
352 String filterString =
353 "SingleColumnValueExcludeFilter ('family', 'qualifier', <, 'binaryprefix:a')";
354 SingleColumnValueExcludeFilter singleColumnValueExcludeFilter =
355 doTestFilter(filterString, SingleColumnValueExcludeFilter.class);
356 assertEquals(singleColumnValueExcludeFilter.getOperator(), CompareFilter.CompareOp.LESS);
357 assertEquals("family", new String(singleColumnValueExcludeFilter.getFamily()));
358 assertEquals("qualifier", new String(singleColumnValueExcludeFilter.getQualifier()));
359 assertEquals(new String(singleColumnValueExcludeFilter.getComparator().getValue()), "a");
360 assertFalse(singleColumnValueExcludeFilter.getFilterIfMissing());
361 assertTrue(singleColumnValueExcludeFilter.getLatestVersionOnly());
362
363 filterString = "SingleColumnValueExcludeFilter " +
364 "('family', 'qualifier', <=, 'binaryprefix:a', true, false)";
365 singleColumnValueExcludeFilter =
366 doTestFilter(filterString, SingleColumnValueExcludeFilter.class);
367 assertEquals("family", new String(singleColumnValueExcludeFilter.getFamily()));
368 assertEquals("qualifier", new String(singleColumnValueExcludeFilter.getQualifier()));
369 assertEquals(singleColumnValueExcludeFilter.getOperator(),
370 CompareFilter.CompareOp.LESS_OR_EQUAL);
371 assertTrue(singleColumnValueExcludeFilter.getComparator() instanceof BinaryPrefixComparator);
372 BinaryPrefixComparator binaryPrefixComparator =
373 (BinaryPrefixComparator) singleColumnValueExcludeFilter.getComparator();
374 assertEquals(new String(binaryPrefixComparator.getValue()), "a");
375 assertTrue(singleColumnValueExcludeFilter.getFilterIfMissing());
376 assertFalse(singleColumnValueExcludeFilter.getLatestVersionOnly());
377 }
378
379 @Test
380 public void testSkipFilter() throws IOException {
381 String filterString = "SKIP ValueFilter( =, 'binary:0')";
382 SkipFilter skipFilter =
383 doTestFilter(filterString, SkipFilter.class);
384 assertTrue(skipFilter.getFilter() instanceof ValueFilter);
385 ValueFilter valueFilter = (ValueFilter) skipFilter.getFilter();
386
387 assertEquals(CompareFilter.CompareOp.EQUAL, valueFilter.getOperator());
388 assertTrue(valueFilter.getComparator() instanceof BinaryComparator);
389 BinaryComparator binaryComparator = (BinaryComparator) valueFilter.getComparator();
390 assertEquals("0", new String(binaryComparator.getValue()));
391 }
392
393 @Test
394 public void testWhileFilter() throws IOException {
395 String filterString = " WHILE RowFilter ( !=, 'binary:row1')";
396 WhileMatchFilter whileMatchFilter =
397 doTestFilter(filterString, WhileMatchFilter.class);
398 assertTrue(whileMatchFilter.getFilter() instanceof RowFilter);
399 RowFilter rowFilter = (RowFilter) whileMatchFilter.getFilter();
400
401 assertEquals(CompareFilter.CompareOp.NOT_EQUAL, rowFilter.getOperator());
402 assertTrue(rowFilter.getComparator() instanceof BinaryComparator);
403 BinaryComparator binaryComparator = (BinaryComparator) rowFilter.getComparator();
404 assertEquals("row1", new String(binaryComparator.getValue()));
405 }
406
407 @Test
408 public void testCompoundFilter1() throws IOException {
409 String filterString = " (PrefixFilter ('realtime')AND FirstKeyOnlyFilter())";
410 FilterList filterList =
411 doTestFilter(filterString, FilterList.class);
412 ArrayList<Filter> filters = (ArrayList<Filter>) filterList.getFilters();
413
414 assertTrue(filters.get(0) instanceof PrefixFilter);
415 assertTrue(filters.get(1) instanceof FirstKeyOnlyFilter);
416 PrefixFilter PrefixFilter = (PrefixFilter) filters.get(0);
417 byte [] prefix = PrefixFilter.getPrefix();
418 assertEquals(new String(prefix), "realtime");
419 FirstKeyOnlyFilter firstKeyOnlyFilter = (FirstKeyOnlyFilter) filters.get(1);
420 }
421
422 @Test
423 public void testCompoundFilter2() throws IOException {
424 String filterString = "(PrefixFilter('realtime') AND QualifierFilter (>=, 'binary:e'))" +
425 "OR FamilyFilter (=, 'binary:qualifier') ";
426 FilterList filterList =
427 doTestFilter(filterString, FilterList.class);
428 ArrayList<Filter> filterListFilters = (ArrayList<Filter>) filterList.getFilters();
429 assertTrue(filterListFilters.get(0) instanceof FilterList);
430 assertTrue(filterListFilters.get(1) instanceof FamilyFilter);
431 assertEquals(filterList.getOperator(), FilterList.Operator.MUST_PASS_ONE);
432
433 filterList = (FilterList) filterListFilters.get(0);
434 FamilyFilter familyFilter = (FamilyFilter) filterListFilters.get(1);
435
436 filterListFilters = (ArrayList<Filter>)filterList.getFilters();
437 assertTrue(filterListFilters.get(0) instanceof PrefixFilter);
438 assertTrue(filterListFilters.get(1) instanceof QualifierFilter);
439 assertEquals(filterList.getOperator(), FilterList.Operator.MUST_PASS_ALL);
440
441 assertEquals(CompareFilter.CompareOp.EQUAL, familyFilter.getOperator());
442 assertTrue(familyFilter.getComparator() instanceof BinaryComparator);
443 BinaryComparator binaryComparator = (BinaryComparator) familyFilter.getComparator();
444 assertEquals("qualifier", new String(binaryComparator.getValue()));
445
446 PrefixFilter prefixFilter = (PrefixFilter) filterListFilters.get(0);
447 byte [] prefix = prefixFilter.getPrefix();
448 assertEquals(new String(prefix), "realtime");
449
450 QualifierFilter qualifierFilter = (QualifierFilter) filterListFilters.get(1);
451 assertEquals(CompareFilter.CompareOp.GREATER_OR_EQUAL, qualifierFilter.getOperator());
452 assertTrue(qualifierFilter.getComparator() instanceof BinaryComparator);
453 binaryComparator = (BinaryComparator) qualifierFilter.getComparator();
454 assertEquals("e", new String(binaryComparator.getValue()));
455 }
456
457 @Test
458 public void testCompoundFilter3() throws IOException {
459 String filterString = " ColumnPrefixFilter ('realtime')AND " +
460 "FirstKeyOnlyFilter() OR SKIP FamilyFilter(=, 'substring:hihi')";
461 FilterList filterList =
462 doTestFilter(filterString, FilterList.class);
463 ArrayList<Filter> filters = (ArrayList<Filter>) filterList.getFilters();
464
465 assertTrue(filters.get(0) instanceof FilterList);
466 assertTrue(filters.get(1) instanceof SkipFilter);
467
468 filterList = (FilterList) filters.get(0);
469 SkipFilter skipFilter = (SkipFilter) filters.get(1);
470
471 filters = (ArrayList<Filter>) filterList.getFilters();
472 assertTrue(filters.get(0) instanceof ColumnPrefixFilter);
473 assertTrue(filters.get(1) instanceof FirstKeyOnlyFilter);
474
475 ColumnPrefixFilter columnPrefixFilter = (ColumnPrefixFilter) filters.get(0);
476 byte [] columnPrefix = columnPrefixFilter.getPrefix();
477 assertEquals(new String(columnPrefix), "realtime");
478
479 FirstKeyOnlyFilter firstKeyOnlyFilter = (FirstKeyOnlyFilter) filters.get(1);
480
481 assertTrue(skipFilter.getFilter() instanceof FamilyFilter);
482 FamilyFilter familyFilter = (FamilyFilter) skipFilter.getFilter();
483
484 assertEquals(CompareFilter.CompareOp.EQUAL, familyFilter.getOperator());
485 assertTrue(familyFilter.getComparator() instanceof SubstringComparator);
486 SubstringComparator substringComparator =
487 (SubstringComparator) familyFilter.getComparator();
488 assertEquals("hihi", new String(substringComparator.getValue()));
489 }
490
491 @Test
492 public void testCompoundFilter4() throws IOException {
493 String filterString = " ColumnPrefixFilter ('realtime') OR " +
494 "FirstKeyOnlyFilter() OR SKIP FamilyFilter(=, 'substring:hihi')";
495 FilterList filterList =
496 doTestFilter(filterString, FilterList.class);
497 ArrayList<Filter> filters = (ArrayList<Filter>) filterList.getFilters();
498
499 assertTrue(filters.get(0) instanceof ColumnPrefixFilter);
500 assertTrue(filters.get(1) instanceof FirstKeyOnlyFilter);
501 assertTrue(filters.get(2) instanceof SkipFilter);
502
503 ColumnPrefixFilter columnPrefixFilter = (ColumnPrefixFilter) filters.get(0);
504 FirstKeyOnlyFilter firstKeyOnlyFilter = (FirstKeyOnlyFilter) filters.get(1);
505 SkipFilter skipFilter = (SkipFilter) filters.get(2);
506
507 byte [] columnPrefix = columnPrefixFilter.getPrefix();
508 assertEquals(new String(columnPrefix), "realtime");
509
510 assertTrue(skipFilter.getFilter() instanceof FamilyFilter);
511 FamilyFilter familyFilter = (FamilyFilter) skipFilter.getFilter();
512
513 assertEquals(CompareFilter.CompareOp.EQUAL, familyFilter.getOperator());
514 assertTrue(familyFilter.getComparator() instanceof SubstringComparator);
515 SubstringComparator substringComparator =
516 (SubstringComparator) familyFilter.getComparator();
517 assertEquals("hihi", new String(substringComparator.getValue()));
518 }
519
520 @Test
521 public void testIncorrectCompareOperator() throws IOException {
522 String filterString = "RowFilter ('>>' , 'binary:region')";
523 try {
524 doTestFilter(filterString, RowFilter.class);
525 assertTrue(false);
526 } catch (IllegalArgumentException e) {
527 System.out.println("Incorrect compare operator >>");
528 }
529 }
530
531 @Test
532 public void testIncorrectComparatorType () throws IOException {
533 String filterString = "RowFilter ('>=' , 'binaryoperator:region')";
534 try {
535 doTestFilter(filterString, RowFilter.class);
536 assertTrue(false);
537 } catch (IllegalArgumentException e) {
538 System.out.println("Incorrect comparator type: binaryoperator");
539 }
540
541 filterString = "RowFilter ('>=' 'regexstring:pre*')";
542 try {
543 doTestFilter(filterString, RowFilter.class);
544 assertTrue(false);
545 } catch (IllegalArgumentException e) {
546 System.out.println("RegexStringComparator can only be used with EQUAL or NOT_EQUAL");
547 }
548
549 filterString = "SingleColumnValueFilter" +
550 " ('family', 'qualifier', '>=', 'substring:a', 'true', 'false')')";
551 try {
552 doTestFilter(filterString, RowFilter.class);
553 assertTrue(false);
554 } catch (IllegalArgumentException e) {
555 System.out.println("SubtringComparator can only be used with EQUAL or NOT_EQUAL");
556 }
557 }
558
559 @Test
560 public void testPrecedence1() throws IOException {
561 String filterString = " (PrefixFilter ('realtime')AND FirstKeyOnlyFilter()" +
562 " OR KeyOnlyFilter())";
563 FilterList filterList =
564 doTestFilter(filterString, FilterList.class);
565
566 ArrayList<Filter> filters = (ArrayList<Filter>) filterList.getFilters();
567
568 assertTrue(filters.get(0) instanceof FilterList);
569 assertTrue(filters.get(1) instanceof KeyOnlyFilter);
570
571 filterList = (FilterList) filters.get(0);
572 filters = (ArrayList<Filter>) filterList.getFilters();
573
574 assertTrue(filters.get(0) instanceof PrefixFilter);
575 assertTrue(filters.get(1) instanceof FirstKeyOnlyFilter);
576
577 PrefixFilter prefixFilter = (PrefixFilter)filters.get(0);
578 byte [] prefix = prefixFilter.getPrefix();
579 assertEquals(new String(prefix), "realtime");
580 }
581
582 @Test
583 public void testPrecedence2() throws IOException {
584 String filterString = " PrefixFilter ('realtime')AND SKIP FirstKeyOnlyFilter()" +
585 "OR KeyOnlyFilter()";
586 FilterList filterList =
587 doTestFilter(filterString, FilterList.class);
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 SkipFilter);
598
599 PrefixFilter prefixFilter = (PrefixFilter)filters.get(0);
600 byte [] prefix = prefixFilter.getPrefix();
601 assertEquals(new String(prefix), "realtime");
602
603 SkipFilter skipFilter = (SkipFilter)filters.get(1);
604 assertTrue(skipFilter.getFilter() instanceof FirstKeyOnlyFilter);
605 }
606
607 @Test
608 public void testUnescapedQuote1 () throws IOException {
609 String filterString = "InclusiveStopFilter ('row''3')";
610 InclusiveStopFilter inclusiveStopFilter =
611 doTestFilter(filterString, InclusiveStopFilter.class);
612 byte [] stopRowKey = inclusiveStopFilter.getStopRowKey();
613 assertEquals(new String(stopRowKey), "row'3");
614 }
615
616 @Test
617 public void testUnescapedQuote2 () throws IOException {
618 String filterString = "InclusiveStopFilter ('row''3''')";
619 InclusiveStopFilter inclusiveStopFilter =
620 doTestFilter(filterString, InclusiveStopFilter.class);
621 byte [] stopRowKey = inclusiveStopFilter.getStopRowKey();
622 assertEquals(new String(stopRowKey), "row'3'");
623 }
624
625 @Test
626 public void testUnescapedQuote3 () throws IOException {
627 String filterString = " InclusiveStopFilter ('''')";
628 InclusiveStopFilter inclusiveStopFilter =
629 doTestFilter(filterString, InclusiveStopFilter.class);
630 byte [] stopRowKey = inclusiveStopFilter.getStopRowKey();
631 assertEquals(new String(stopRowKey), "'");
632 }
633
634 @Test
635 public void testIncorrectFilterString () throws IOException {
636 String filterString = "()";
637 byte [] filterStringAsByteArray = Bytes.toBytes(filterString);
638 try {
639 filter = f.parseFilterString(filterStringAsByteArray);
640 assertTrue(false);
641 } catch (IllegalArgumentException e) {
642 System.out.println(e.getMessage());
643 }
644 }
645
646 @Test
647 public void testCorrectFilterString () throws IOException {
648 String filterString = "(FirstKeyOnlyFilter())";
649 FirstKeyOnlyFilter firstKeyOnlyFilter =
650 doTestFilter(filterString, FirstKeyOnlyFilter.class);
651 }
652
653 @Test
654 public void testRegisterFilter() {
655 ParseFilter.registerFilter("MyFilter", "some.class");
656
657 assertTrue(f.getSupportedFilters().contains("MyFilter"));
658 }
659
660 private <T extends Filter> T doTestFilter(String filterString, Class<T> clazz) throws IOException {
661 byte [] filterStringAsByteArray = Bytes.toBytes(filterString);
662 filter = f.parseFilterString(filterStringAsByteArray);
663 assertEquals(clazz, filter.getClass());
664 return clazz.cast(filter);
665 }
666
667 }
668