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