1   /**
2    * Copyright 2009 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
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   * Tests filter sets
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     * Test "must pass one"
55     * @throws Exception
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      /* Filter must do all below steps:
64       * <ul>
65       * <li>{@link #reset()}</li>
66       * <li>{@link #filterAllRemaining()} -> true indicates scan is over, false, keep going on.</li>
67       * <li>{@link #filterRowKey(byte[],int,int)} -> true to drop this row,
68       * if false, we will also call</li>
69       * <li>{@link #filterKeyValue(org.apache.hadoop.hbase.KeyValue)} -> true to drop this key/value</li>
70       * <li>{@link #filterRow()} -> last chance to drop entire row based on the sequence of
71       * filterValue() calls. Eg: filter a row if it doesn't contain a specified column.
72       * </li>
73       * </ul>
74      */
75      filterMPONE.reset();
76      assertFalse(filterMPONE.filterAllRemaining());
77  
78      /* Will pass both */
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        KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
83          Bytes.toBytes(i));
84        assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
85        assertFalse(filterMPONE.filterRow());
86      }
87  
88      /* Only pass PageFilter */
89      rowkey = Bytes.toBytes("z");
90      assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
91      KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(0),
92          Bytes.toBytes(0));
93      assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
94      assertFalse(filterMPONE.filterRow());
95  
96      /* reach MAX_PAGES already, should filter any rows */
97      rowkey = Bytes.toBytes("yyy");
98      assertTrue(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
99      kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(0),
100         Bytes.toBytes(0));
101     assertFalse(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
102 
103     /* We should filter any row */
104     rowkey = Bytes.toBytes("z");
105     assertTrue(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
106     assertTrue(filterMPONE.filterAllRemaining());
107   }
108 
109   /**
110    * Test "must pass all"
111    * @throws Exception
112    */
113   public void testMPALL() throws Exception {
114     List<Filter> filters = new ArrayList<Filter>();
115     filters.add(new PageFilter(MAX_PAGES));
116     filters.add(new WhileMatchFilter(new PrefixFilter(Bytes.toBytes("yyy"))));
117     Filter filterMPALL =
118       new FilterList(FilterList.Operator.MUST_PASS_ALL, filters);
119     /* Filter must do all below steps:
120      * <ul>
121      * <li>{@link #reset()}</li>
122      * <li>{@link #filterAllRemaining()} -> true indicates scan is over, false, keep going on.</li>
123      * <li>{@link #filterRowKey(byte[],int,int)} -> true to drop this row,
124      * if false, we will also call</li>
125      * <li>{@link #filterKeyValue(org.apache.hadoop.hbase.KeyValue)} -> true to drop this key/value</li>
126      * <li>{@link #filterRow()} -> last chance to drop entire row based on the sequence of
127      * filterValue() calls. Eg: filter a row if it doesn't contain a specified column.
128      * </li>
129      * </ul>
130     */
131     filterMPALL.reset();
132     assertFalse(filterMPALL.filterAllRemaining());
133     byte [] rowkey = Bytes.toBytes("yyyyyyyyy");
134     for (int i = 0; i < MAX_PAGES - 1; i++) {
135       assertFalse(filterMPALL.filterRowKey(rowkey, 0, rowkey.length));
136       KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
137         Bytes.toBytes(i));
138       assertTrue(Filter.ReturnCode.INCLUDE == filterMPALL.filterKeyValue(kv));
139     }
140     filterMPALL.reset();
141     rowkey = Bytes.toBytes("z");
142     assertTrue(filterMPALL.filterRowKey(rowkey, 0, rowkey.length));
143     // Should fail here; row should be filtered out.
144     KeyValue kv = new KeyValue(rowkey, rowkey, rowkey, rowkey);
145     assertTrue(Filter.ReturnCode.NEXT_ROW == filterMPALL.filterKeyValue(kv));
146   }
147 
148   /**
149    * Test list ordering
150    * @throws Exception
151    */
152   public void testOrdering() throws Exception {
153     List<Filter> filters = new ArrayList<Filter>();
154     filters.add(new PrefixFilter(Bytes.toBytes("yyy")));
155     filters.add(new PageFilter(MAX_PAGES));
156     Filter filterMPONE =
157         new FilterList(FilterList.Operator.MUST_PASS_ONE, filters);
158     /* Filter must do all below steps:
159      * <ul>
160      * <li>{@link #reset()}</li>
161      * <li>{@link #filterAllRemaining()} -> true indicates scan is over, false, keep going on.</li>
162      * <li>{@link #filterRowKey(byte[],int,int)} -> true to drop this row,
163      * if false, we will also call</li>
164      * <li>{@link #filterKeyValue(org.apache.hadoop.hbase.KeyValue)} -> true to drop this key/value</li>
165      * <li>{@link #filterRow()} -> last chance to drop entire row based on the sequence of
166      * filterValue() calls. Eg: filter a row if it doesn't contain a specified column.
167      * </li>
168      * </ul>
169     */
170     filterMPONE.reset();
171     assertFalse(filterMPONE.filterAllRemaining());
172 
173     /* We should be able to fill MAX_PAGES without incrementing page counter */
174     byte [] rowkey = Bytes.toBytes("yyyyyyyy");
175     for (int i = 0; i < MAX_PAGES; i++) {
176       assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
177       KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
178           Bytes.toBytes(i));
179         assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
180       assertFalse(filterMPONE.filterRow());
181     }
182 
183     /* Now let's fill the page filter */
184     rowkey = Bytes.toBytes("xxxxxxx");
185     for (int i = 0; i < MAX_PAGES; i++) {
186       assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
187       KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
188           Bytes.toBytes(i));
189         assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
190       assertFalse(filterMPONE.filterRow());
191     }
192 
193     /* We should still be able to include even though page filter is at max */
194     rowkey = Bytes.toBytes("yyy");
195     for (int i = 0; i < MAX_PAGES; i++) {
196       assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
197       KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
198           Bytes.toBytes(i));
199         assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
200       assertFalse(filterMPONE.filterRow());
201     }
202   }
203 
204   /**
205    * Test serialization
206    * @throws Exception
207    */
208   public void testSerialization() throws Exception {
209     List<Filter> filters = new ArrayList<Filter>();
210     filters.add(new PageFilter(MAX_PAGES));
211     filters.add(new WhileMatchFilter(new PrefixFilter(Bytes.toBytes("yyy"))));
212     Filter filterMPALL =
213       new FilterList(FilterList.Operator.MUST_PASS_ALL, filters);
214 
215     // Decompose filterMPALL to bytes.
216     ByteArrayOutputStream stream = new ByteArrayOutputStream();
217     DataOutputStream out = new DataOutputStream(stream);
218     filterMPALL.write(out);
219     out.close();
220     byte[] buffer = stream.toByteArray();
221 
222     // Recompose filterMPALL.
223     DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
224     FilterList newFilter = new FilterList();
225     newFilter.readFields(in);
226 
227     // TODO: Run TESTS!!!
228   }
229 
230   /**
231    * Test filterKeyValue logic.
232    * @throws Exception
233    */
234   public void testFilterKeyValue() throws Exception {
235     Filter includeFilter = new FilterBase() {
236       @Override
237       public Filter.ReturnCode filterKeyValue(KeyValue v) {
238         return Filter.ReturnCode.INCLUDE;
239       }
240 
241       @Override
242       public void readFields(DataInput arg0) throws IOException {}
243 
244       @Override
245       public void write(DataOutput arg0) throws IOException {}
246     };
247 
248     Filter alternateFilter = new FilterBase() {
249       boolean returnInclude = true;
250 
251       @Override
252       public Filter.ReturnCode filterKeyValue(KeyValue v) {
253         Filter.ReturnCode returnCode = returnInclude ? Filter.ReturnCode.INCLUDE :
254                                                        Filter.ReturnCode.SKIP;
255         returnInclude = !returnInclude;
256         return returnCode;
257       }
258 
259       @Override
260       public void readFields(DataInput arg0) throws IOException {}
261 
262       @Override
263       public void write(DataOutput arg0) throws IOException {}
264     };
265 
266     Filter alternateIncludeFilter = new FilterBase() {
267       boolean returnIncludeOnly = false;
268 
269       @Override
270       public Filter.ReturnCode filterKeyValue(KeyValue v) {
271         Filter.ReturnCode returnCode = returnIncludeOnly ? Filter.ReturnCode.INCLUDE :
272                                                            Filter.ReturnCode.INCLUDE_AND_NEXT_COL;
273         returnIncludeOnly = !returnIncludeOnly;
274         return returnCode;
275       }
276 
277       @Override
278       public void readFields(DataInput arg0) throws IOException {}
279 
280       @Override
281       public void write(DataOutput arg0) throws IOException {}
282     };
283 
284     // Check must pass one filter.
285     FilterList mpOnefilterList = new FilterList(Operator.MUST_PASS_ONE,
286         Arrays.asList(new Filter[] { includeFilter, alternateIncludeFilter, alternateFilter }));
287     // INCLUDE, INCLUDE, INCLUDE_AND_NEXT_COL.
288     assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL, mpOnefilterList.filterKeyValue(null));
289     // INCLUDE, SKIP, INCLUDE. 
290     assertEquals(Filter.ReturnCode.INCLUDE, mpOnefilterList.filterKeyValue(null));
291 
292     // Check must pass all filter.
293     FilterList mpAllfilterList = new FilterList(Operator.MUST_PASS_ALL,
294         Arrays.asList(new Filter[] { includeFilter, alternateIncludeFilter, alternateFilter }));
295     // INCLUDE, INCLUDE, INCLUDE_AND_NEXT_COL.
296     assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL, mpAllfilterList.filterKeyValue(null));
297     // INCLUDE, SKIP, INCLUDE. 
298     assertEquals(Filter.ReturnCode.SKIP, mpAllfilterList.filterKeyValue(null));
299   }
300 
301   /**
302    * Test pass-thru of hints.
303    */
304   public void testHintPassThru() throws Exception {
305 
306     final KeyValue minKeyValue = new KeyValue(Bytes.toBytes(0L), null, null);
307     final KeyValue maxKeyValue = new KeyValue(Bytes.toBytes(Long.MAX_VALUE),
308         null, null);
309 
310     Filter filterNoHint = new FilterBase() {
311       @Override
312       public void readFields(DataInput arg0) throws IOException {}
313 
314       @Override
315       public void write(DataOutput arg0) throws IOException {}
316     };
317 
318     Filter filterMinHint = new FilterBase() {
319       @Override
320       public KeyValue getNextKeyHint(KeyValue currentKV) {
321         return minKeyValue;
322       }
323 
324       @Override
325       public void readFields(DataInput arg0) throws IOException {}
326 
327       @Override
328       public void write(DataOutput arg0) throws IOException {}
329     };
330 
331     Filter filterMaxHint = new FilterBase() {
332       @Override
333       public KeyValue getNextKeyHint(KeyValue currentKV) {
334         return new KeyValue(Bytes.toBytes(Long.MAX_VALUE), null, null);
335       }
336 
337       @Override
338       public void readFields(DataInput arg0) throws IOException {}
339 
340       @Override
341       public void write(DataOutput arg0) throws IOException {}
342     };
343 
344     // MUST PASS ONE
345 
346     // Should take the min if given two hints
347     FilterList filterList = new FilterList(Operator.MUST_PASS_ONE,
348         Arrays.asList(new Filter [] { filterMinHint, filterMaxHint } ));
349     assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
350         minKeyValue));
351 
352     // Should have no hint if any filter has no hint
353     filterList = new FilterList(Operator.MUST_PASS_ONE,
354         Arrays.asList(
355             new Filter [] { filterMinHint, filterMaxHint, filterNoHint } ));
356     assertNull(filterList.getNextKeyHint(null));
357     filterList = new FilterList(Operator.MUST_PASS_ONE,
358         Arrays.asList(new Filter [] { filterNoHint, filterMaxHint } ));
359     assertNull(filterList.getNextKeyHint(null));
360 
361     // Should give max hint if its the only one
362     filterList = new FilterList(Operator.MUST_PASS_ONE,
363         Arrays.asList(new Filter [] { filterMaxHint, filterMaxHint } ));
364     assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
365         maxKeyValue));
366 
367     // MUST PASS ALL
368 
369     // Should take the max if given two hints
370     filterList = new FilterList(Operator.MUST_PASS_ALL,
371         Arrays.asList(new Filter [] { filterMinHint, filterMaxHint } ));
372     assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
373         maxKeyValue));
374 
375     // Should have max hint even if a filter has no hint
376     filterList = new FilterList(Operator.MUST_PASS_ALL,
377         Arrays.asList(
378             new Filter [] { filterMinHint, filterMaxHint, filterNoHint } ));
379     assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
380         maxKeyValue));
381     filterList = new FilterList(Operator.MUST_PASS_ALL,
382         Arrays.asList(new Filter [] { filterNoHint, filterMaxHint } ));
383     assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
384         maxKeyValue));
385     filterList = new FilterList(Operator.MUST_PASS_ALL,
386         Arrays.asList(new Filter [] { filterNoHint, filterMinHint } ));
387     assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
388         minKeyValue));
389 
390     // Should give min hint if its the only one
391     filterList = new FilterList(Operator.MUST_PASS_ALL,
392         Arrays.asList(new Filter [] { filterNoHint, filterMinHint } ));
393     assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
394         minKeyValue));
395   }
396 
397   @org.junit.Rule
398   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
399     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
400 }
401