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