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