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 java.io.IOException;
22 import java.util.ArrayList;
23 import java.util.Arrays;
24 import java.util.List;
25
26 import static org.junit.Assert.assertEquals;
27 import static org.junit.Assert.assertFalse;
28 import static org.junit.Assert.assertNotNull;
29 import static org.junit.Assert.assertTrue;
30 import static org.junit.Assert.assertNull;
31
32 import org.apache.hadoop.hbase.Cell;
33 import org.apache.hadoop.hbase.KeyValue;
34 import org.apache.hadoop.hbase.KeyValueUtil;
35 import org.apache.hadoop.hbase.testclassification.SmallTests;
36 import org.apache.hadoop.hbase.exceptions.DeserializationException;
37 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
38 import org.apache.hadoop.hbase.filter.Filter.ReturnCode;
39 import org.apache.hadoop.hbase.filter.FilterList.Operator;
40 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
41 import org.apache.hadoop.hbase.util.Bytes;
42 import org.junit.Test;
43 import org.junit.experimental.categories.Category;
44
45 import com.google.common.collect.Lists;
46
47
48
49
50
51 @Category(SmallTests.class)
52 public class TestFilterList {
53 static final int MAX_PAGES = 2;
54 static final char FIRST_CHAR = 'a';
55 static final char LAST_CHAR = 'e';
56 static byte[] GOOD_BYTES = Bytes.toBytes("abc");
57 static byte[] BAD_BYTES = Bytes.toBytes("def");
58
59
60 @Test
61 public void testAddFilter() throws Exception {
62 Filter filter1 = new FirstKeyOnlyFilter();
63 Filter filter2 = new FirstKeyOnlyFilter();
64
65 FilterList filterList = new FilterList(filter1, filter2);
66 filterList.addFilter(new FirstKeyOnlyFilter());
67
68 filterList = new FilterList(Arrays.asList(filter1, filter2));
69 filterList.addFilter(new FirstKeyOnlyFilter());
70
71 filterList = new FilterList(Operator.MUST_PASS_ALL, filter1, filter2);
72 filterList.addFilter(new FirstKeyOnlyFilter());
73
74 filterList = new FilterList(Operator.MUST_PASS_ALL, Arrays.asList(filter1, filter2));
75 filterList.addFilter(new FirstKeyOnlyFilter());
76
77 }
78
79
80
81
82
83
84 @Test
85 public void testMPONE() throws Exception {
86 mpOneTest(getFilterMPONE());
87 }
88
89 private Filter getFilterMPONE() {
90 List<Filter> filters = new ArrayList<Filter>();
91 filters.add(new PageFilter(MAX_PAGES));
92 filters.add(new WhileMatchFilter(new PrefixFilter(Bytes.toBytes("yyy"))));
93 Filter filterMPONE =
94 new FilterList(FilterList.Operator.MUST_PASS_ONE, filters);
95 return filterMPONE;
96 }
97
98 private void mpOneTest(Filter filterMPONE) throws Exception {
99
100
101
102
103
104
105
106
107
108
109
110
111 filterMPONE.reset();
112 assertFalse(filterMPONE.filterAllRemaining());
113
114
115 byte [] rowkey = Bytes.toBytes("yyyyyyyyy");
116 for (int i = 0; i < MAX_PAGES - 1; i++) {
117 assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
118 KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
119 Bytes.toBytes(i));
120 assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
121 assertFalse(filterMPONE.filterRow());
122 }
123
124
125 rowkey = Bytes.toBytes("z");
126 assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
127 KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(0),
128 Bytes.toBytes(0));
129 assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
130 assertFalse(filterMPONE.filterRow());
131
132
133 rowkey = Bytes.toBytes("yyy");
134 assertTrue(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
135 kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(0),
136 Bytes.toBytes(0));
137 assertFalse(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
138 assertFalse(filterMPONE.filterRow());
139
140
141 rowkey = Bytes.toBytes("z");
142 assertTrue(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
143 assertTrue(filterMPONE.filterAllRemaining());
144 }
145
146
147
148
149
150 @Test
151 public void testMPALL() throws Exception {
152 mpAllTest(getMPALLFilter());
153 }
154
155 private Filter getMPALLFilter() {
156 List<Filter> filters = new ArrayList<Filter>();
157 filters.add(new PageFilter(MAX_PAGES));
158 filters.add(new WhileMatchFilter(new PrefixFilter(Bytes.toBytes("yyy"))));
159 Filter filterMPALL =
160 new FilterList(FilterList.Operator.MUST_PASS_ALL, filters);
161 return filterMPALL;
162 }
163
164 private void mpAllTest(Filter filterMPALL) throws Exception {
165
166
167
168
169
170
171
172
173
174
175
176
177 filterMPALL.reset();
178 assertFalse(filterMPALL.filterAllRemaining());
179 byte [] rowkey = Bytes.toBytes("yyyyyyyyy");
180 for (int i = 0; i < MAX_PAGES - 1; i++) {
181 assertFalse(filterMPALL.filterRowKey(rowkey, 0, rowkey.length));
182 KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
183 Bytes.toBytes(i));
184 assertTrue(Filter.ReturnCode.INCLUDE == filterMPALL.filterKeyValue(kv));
185 }
186 filterMPALL.reset();
187 rowkey = Bytes.toBytes("z");
188 assertTrue(filterMPALL.filterRowKey(rowkey, 0, rowkey.length));
189
190 KeyValue kv = new KeyValue(rowkey, rowkey, rowkey, rowkey);
191 assertTrue(Filter.ReturnCode.NEXT_ROW == filterMPALL.filterKeyValue(kv));
192 }
193
194
195
196
197
198 @Test
199 public void testOrdering() throws Exception {
200 orderingTest(getOrderingFilter());
201 }
202
203 public Filter getOrderingFilter() {
204 List<Filter> filters = new ArrayList<Filter>();
205 filters.add(new PrefixFilter(Bytes.toBytes("yyy")));
206 filters.add(new PageFilter(MAX_PAGES));
207 Filter filterMPONE =
208 new FilterList(FilterList.Operator.MUST_PASS_ONE, filters);
209 return filterMPONE;
210 }
211
212 public void orderingTest(Filter filterMPONE) throws Exception {
213
214
215
216
217
218
219
220
221
222
223
224
225 filterMPONE.reset();
226 assertFalse(filterMPONE.filterAllRemaining());
227
228
229 byte [] rowkey = Bytes.toBytes("yyyyyyyy");
230 for (int i = 0; i < MAX_PAGES; i++) {
231 assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
232 KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
233 Bytes.toBytes(i));
234 assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
235 assertFalse(filterMPONE.filterRow());
236 }
237
238
239 rowkey = Bytes.toBytes("xxxxxxx");
240 for (int i = 0; i < MAX_PAGES; i++) {
241 assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
242 KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
243 Bytes.toBytes(i));
244 assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
245 assertFalse(filterMPONE.filterRow());
246 }
247
248
249 rowkey = Bytes.toBytes("yyy");
250 for (int i = 0; i < MAX_PAGES; i++) {
251 assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
252 KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
253 Bytes.toBytes(i));
254 assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
255 assertFalse(filterMPONE.filterRow());
256 }
257 }
258
259
260
261
262
263
264 public void testFilterListTwoFiltersMustPassOne() throws Exception {
265 byte[] r1 = Bytes.toBytes("Row1");
266 byte[] r11 = Bytes.toBytes("Row11");
267 byte[] r2 = Bytes.toBytes("Row2");
268
269 FilterList flist = new FilterList(FilterList.Operator.MUST_PASS_ONE);
270 flist.addFilter(new PrefixFilter(r1));
271 flist.filterRowKey(r1, 0, r1.length);
272 assertEquals(flist.filterKeyValue(new KeyValue(r1,r1,r1)), ReturnCode.INCLUDE);
273 assertEquals(flist.filterKeyValue(new KeyValue(r11,r11,r11)), ReturnCode.INCLUDE);
274
275 flist.reset();
276 flist.filterRowKey(r2, 0, r2.length);
277 assertEquals(flist.filterKeyValue(new KeyValue(r2,r2,r2)), ReturnCode.SKIP);
278
279 flist = new FilterList(FilterList.Operator.MUST_PASS_ONE);
280 flist.addFilter(new AlwaysNextColFilter());
281 flist.addFilter(new PrefixFilter(r1));
282 flist.filterRowKey(r1, 0, r1.length);
283 assertEquals(flist.filterKeyValue(new KeyValue(r1,r1,r1)), ReturnCode.INCLUDE);
284 assertEquals(flist.filterKeyValue(new KeyValue(r11,r11,r11)), ReturnCode.INCLUDE);
285
286 flist.reset();
287 flist.filterRowKey(r2, 0, r2.length);
288 assertEquals(flist.filterKeyValue(new KeyValue(r2,r2,r2)), ReturnCode.SKIP);
289 }
290
291
292
293
294
295
296 public void testFilterListWithInclusiveStopFilteMustPassOne() throws Exception {
297 byte[] r1 = Bytes.toBytes("Row1");
298 byte[] r11 = Bytes.toBytes("Row11");
299 byte[] r2 = Bytes.toBytes("Row2");
300
301 FilterList flist = new FilterList(FilterList.Operator.MUST_PASS_ONE);
302 flist.addFilter(new AlwaysNextColFilter());
303 flist.addFilter(new InclusiveStopFilter(r1));
304 flist.filterRowKey(r1, 0, r1.length);
305 assertEquals(flist.filterKeyValue(new KeyValue(r1,r1,r1)), ReturnCode.INCLUDE);
306 assertEquals(flist.filterKeyValue(new KeyValue(r11,r11,r11)), ReturnCode.INCLUDE);
307
308 flist.reset();
309 flist.filterRowKey(r2, 0, r2.length);
310 assertEquals(flist.filterKeyValue(new KeyValue(r2,r2,r2)), ReturnCode.SKIP);
311 }
312
313 public static class AlwaysNextColFilter extends FilterBase {
314 public AlwaysNextColFilter() {
315 super();
316 }
317 @Override
318 public ReturnCode filterKeyValue(Cell v) {
319 return ReturnCode.NEXT_COL;
320 }
321 public static AlwaysNextColFilter parseFrom(final byte [] pbBytes)
322 throws DeserializationException {
323 return new AlwaysNextColFilter();
324 }
325 }
326
327
328
329
330
331 @Test
332 public void testSerialization() throws Exception {
333 List<Filter> filters = new ArrayList<Filter>();
334 filters.add(new PageFilter(MAX_PAGES));
335 filters.add(new WhileMatchFilter(new PrefixFilter(Bytes.toBytes("yyy"))));
336 Filter filterMPALL =
337 new FilterList(FilterList.Operator.MUST_PASS_ALL, filters);
338
339
340 byte[] buffer = filterMPALL.toByteArray();
341
342
343 FilterList newFilter = FilterList.parseFrom(buffer);
344
345
346 mpOneTest(ProtobufUtil.toFilter(ProtobufUtil.toFilter(getFilterMPONE())));
347 mpAllTest(ProtobufUtil.toFilter(ProtobufUtil.toFilter(getMPALLFilter())));
348 orderingTest(ProtobufUtil.toFilter(ProtobufUtil.toFilter(getOrderingFilter())));
349 }
350
351
352
353
354
355 public void testFilterKeyValue() throws Exception {
356 Filter includeFilter = new FilterBase() {
357 @Override
358 public Filter.ReturnCode filterKeyValue(Cell v) {
359 return Filter.ReturnCode.INCLUDE;
360 }
361 };
362
363 Filter alternateFilter = new FilterBase() {
364 boolean returnInclude = true;
365
366 @Override
367 public Filter.ReturnCode filterKeyValue(Cell v) {
368 Filter.ReturnCode returnCode = returnInclude ? Filter.ReturnCode.INCLUDE :
369 Filter.ReturnCode.SKIP;
370 returnInclude = !returnInclude;
371 return returnCode;
372 }
373 };
374
375 Filter alternateIncludeFilter = new FilterBase() {
376 boolean returnIncludeOnly = false;
377
378 @Override
379 public Filter.ReturnCode filterKeyValue(Cell v) {
380 Filter.ReturnCode returnCode = returnIncludeOnly ? Filter.ReturnCode.INCLUDE :
381 Filter.ReturnCode.INCLUDE_AND_NEXT_COL;
382 returnIncludeOnly = !returnIncludeOnly;
383 return returnCode;
384 }
385 };
386
387
388 FilterList mpOnefilterList = new FilterList(Operator.MUST_PASS_ONE,
389 Arrays.asList(new Filter[] { includeFilter, alternateIncludeFilter, alternateFilter }));
390
391 assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL, mpOnefilterList.filterKeyValue(null));
392
393 assertEquals(Filter.ReturnCode.INCLUDE, mpOnefilterList.filterKeyValue(null));
394
395
396 FilterList mpAllfilterList = new FilterList(Operator.MUST_PASS_ALL,
397 Arrays.asList(new Filter[] { includeFilter, alternateIncludeFilter, alternateFilter }));
398
399 assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL, mpAllfilterList.filterKeyValue(null));
400
401 assertEquals(Filter.ReturnCode.SKIP, mpAllfilterList.filterKeyValue(null));
402 }
403
404
405
406
407 @Test
408 public void testHintPassThru() throws Exception {
409
410 final KeyValue minKeyValue = new KeyValue(Bytes.toBytes(0L), null, null);
411 final KeyValue maxKeyValue = new KeyValue(Bytes.toBytes(Long.MAX_VALUE),
412 null, null);
413
414 Filter filterNoHint = new FilterBase() {
415 @Override
416 public byte [] toByteArray() {
417 return null;
418 }
419
420 @Override
421 public ReturnCode filterKeyValue(Cell ignored) throws IOException {
422 return ReturnCode.INCLUDE;
423 }
424 };
425
426 Filter filterMinHint = new FilterBase() {
427 @Override
428 public ReturnCode filterKeyValue(Cell ignored) {
429 return ReturnCode.SEEK_NEXT_USING_HINT;
430 }
431
432 @Override
433 public Cell getNextCellHint(Cell currentKV) {
434 return minKeyValue;
435 }
436
437 @Override
438 public byte [] toByteArray() {return null;}
439 };
440
441 Filter filterMaxHint = new FilterBase() {
442 @Override
443 public ReturnCode filterKeyValue(Cell ignored) {
444 return ReturnCode.SEEK_NEXT_USING_HINT;
445 }
446
447 @Override
448 public Cell getNextCellHint(Cell currentKV) {
449 return new KeyValue(Bytes.toBytes(Long.MAX_VALUE), null, null);
450 }
451
452 @Override
453 public byte [] toByteArray() {return null;}
454 };
455
456
457
458
459 FilterList filterList = new FilterList(Operator.MUST_PASS_ONE,
460 Arrays.asList(new Filter [] { filterMinHint, filterMaxHint } ));
461 assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
462 minKeyValue));
463
464
465 filterList = new FilterList(Operator.MUST_PASS_ONE,
466 Arrays.asList(
467 new Filter [] { filterMinHint, filterMaxHint, filterNoHint } ));
468 assertNull(filterList.getNextKeyHint(null));
469 filterList = new FilterList(Operator.MUST_PASS_ONE,
470 Arrays.asList(new Filter [] { filterNoHint, filterMaxHint } ));
471 assertNull(filterList.getNextKeyHint(null));
472
473
474 filterList = new FilterList(Operator.MUST_PASS_ONE,
475 Arrays.asList(new Filter [] { filterMaxHint, filterMaxHint } ));
476 assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
477 maxKeyValue));
478
479
480
481
482 filterList = new FilterList(Operator.MUST_PASS_ALL,
483 Arrays.asList(new Filter [] { filterMinHint, filterMaxHint } ));
484 filterList.filterKeyValue(null);
485 assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
486 minKeyValue));
487
488 filterList = new FilterList(Operator.MUST_PASS_ALL,
489 Arrays.asList(new Filter [] { filterMaxHint, filterMinHint } ));
490 filterList.filterKeyValue(null);
491 assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
492 maxKeyValue));
493
494
495 filterList = new FilterList(Operator.MUST_PASS_ALL,
496 Arrays.asList(
497 new Filter [] { filterNoHint, filterMinHint, filterMaxHint } ));
498 filterList.filterKeyValue(null);
499 assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
500 minKeyValue));
501 filterList = new FilterList(Operator.MUST_PASS_ALL,
502 Arrays.asList(new Filter [] { filterNoHint, filterMaxHint } ));
503 filterList.filterKeyValue(null);
504 assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
505 maxKeyValue));
506 filterList = new FilterList(Operator.MUST_PASS_ALL,
507 Arrays.asList(new Filter [] { filterNoHint, filterMinHint } ));
508 filterList.filterKeyValue(null);
509 assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
510 minKeyValue));
511 }
512
513
514
515
516
517
518
519 @Test
520 public void testTransformMPO() throws Exception {
521
522
523
524 final FilterList flist = new FilterList(Operator.MUST_PASS_ONE, Lists.<Filter>newArrayList(
525 new FilterList(Operator.MUST_PASS_ALL, Lists.<Filter>newArrayList(
526 new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(Bytes.toBytes("fam"))),
527 new QualifierFilter(CompareOp.EQUAL, new BinaryComparator(Bytes.toBytes("qual1"))),
528 new KeyOnlyFilter())),
529 new FilterList(Operator.MUST_PASS_ALL, Lists.<Filter>newArrayList(
530 new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(Bytes.toBytes("fam"))),
531 new QualifierFilter(CompareOp.EQUAL, new BinaryComparator(Bytes.toBytes("qual2")))))));
532
533 final KeyValue kvQual1 = new KeyValue(
534 Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("qual1"), Bytes.toBytes("value"));
535 final KeyValue kvQual2 = new KeyValue(
536 Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("qual2"), Bytes.toBytes("value"));
537 final KeyValue kvQual3 = new KeyValue(
538 Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("qual3"), Bytes.toBytes("value"));
539
540
541 assertEquals(Filter.ReturnCode.INCLUDE, flist.filterKeyValue(kvQual1));
542 final KeyValue transformedQual1 = KeyValueUtil.ensureKeyValue(flist.transform(kvQual1));
543 assertEquals(0, transformedQual1.getValue().length);
544
545
546 assertEquals(Filter.ReturnCode.INCLUDE, flist.filterKeyValue(kvQual2));
547 final KeyValue transformedQual2 = KeyValueUtil.ensureKeyValue(flist.transform(kvQual2));
548 assertEquals("value", Bytes.toString(transformedQual2.getValue()));
549
550
551 assertEquals(Filter.ReturnCode.SKIP, flist.filterKeyValue(kvQual3));
552 }
553
554 }
555