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