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