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.filter.FilterList.Operator;
37  import org.apache.hadoop.hbase.util.Bytes;
38  
39  /**
40   * Tests filter sets
41   *
42   */
43  public class TestFilterList extends TestCase {
44    static final int MAX_PAGES = 2;
45    static final char FIRST_CHAR = 'a';
46    static final char LAST_CHAR = 'e';
47    static byte[] GOOD_BYTES = Bytes.toBytes("abc");
48    static byte[] BAD_BYTES = Bytes.toBytes("def");
49  
50    /**
51     * Test "must pass one"
52     * @throws Exception
53     */
54    public void testMPONE() throws Exception {
55      List<Filter> filters = new ArrayList<Filter>();
56      filters.add(new PageFilter(MAX_PAGES));
57      filters.add(new WhileMatchFilter(new PrefixFilter(Bytes.toBytes("yyy"))));
58      Filter filterMPONE =
59          new FilterList(FilterList.Operator.MUST_PASS_ONE, filters);
60      /* Filter must do all below steps:
61       * <ul>
62       * <li>{@link #reset()}</li>
63       * <li>{@link #filterAllRemaining()} -> true indicates scan is over, false, keep going on.</li>
64       * <li>{@link #filterRowKey(byte[],int,int)} -> true to drop this row,
65       * if false, we will also call</li>
66       * <li>{@link #filterKeyValue(org.apache.hadoop.hbase.KeyValue)} -> true to drop this key/value</li>
67       * <li>{@link #filterRow()} -> last chance to drop entire row based on the sequence of
68       * filterValue() calls. Eg: filter a row if it doesn't contain a specified column.
69       * </li>
70       * </ul>
71      */
72      filterMPONE.reset();
73      assertFalse(filterMPONE.filterAllRemaining());
74  
75      /* Will pass both */
76      byte [] rowkey = Bytes.toBytes("yyyyyyyyy");
77      for (int i = 0; i < MAX_PAGES - 1; i++) {
78        assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
79        assertFalse(filterMPONE.filterRow());
80        KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
81          Bytes.toBytes(i));
82        assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
83      }
84  
85      /* Only pass PageFilter */
86      rowkey = Bytes.toBytes("z");
87      assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
88      assertFalse(filterMPONE.filterRow());
89      KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(0),
90          Bytes.toBytes(0));
91      assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
92  
93      /* PageFilter will fail now, but should pass because we match yyy */
94      rowkey = Bytes.toBytes("yyy");
95      assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
96      assertFalse(filterMPONE.filterRow());
97      kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(0),
98          Bytes.toBytes(0));
99      assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
100 
101     /* We should filter any row */
102     rowkey = Bytes.toBytes("z");
103     assertTrue(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
104     assertTrue(filterMPONE.filterRow());
105     assertTrue(filterMPONE.filterAllRemaining());
106 
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     // Both filters in Set should be satisfied by now
148     assertTrue(filterMPALL.filterRow());
149   }
150 
151   /**
152    * Test list ordering
153    * @throws Exception
154    */
155   public void testOrdering() throws Exception {
156     List<Filter> filters = new ArrayList<Filter>();
157     filters.add(new PrefixFilter(Bytes.toBytes("yyy")));
158     filters.add(new PageFilter(MAX_PAGES));
159     Filter filterMPONE =
160         new FilterList(FilterList.Operator.MUST_PASS_ONE, filters);
161     /* Filter must do all below steps:
162      * <ul>
163      * <li>{@link #reset()}</li>
164      * <li>{@link #filterAllRemaining()} -> true indicates scan is over, false, keep going on.</li>
165      * <li>{@link #filterRowKey(byte[],int,int)} -> true to drop this row,
166      * if false, we will also call</li>
167      * <li>{@link #filterKeyValue(org.apache.hadoop.hbase.KeyValue)} -> true to drop this key/value</li>
168      * <li>{@link #filterRow()} -> last chance to drop entire row based on the sequence of
169      * filterValue() calls. Eg: filter a row if it doesn't contain a specified column.
170      * </li>
171      * </ul>
172     */
173     filterMPONE.reset();
174     assertFalse(filterMPONE.filterAllRemaining());
175 
176     /* We should be able to fill MAX_PAGES without incrementing page counter */
177     byte [] rowkey = Bytes.toBytes("yyyyyyyy");
178     for (int i = 0; i < MAX_PAGES; i++) {
179       assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
180       KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
181           Bytes.toBytes(i));
182         assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
183       assertFalse(filterMPONE.filterRow());
184     }
185 
186     /* Now let's fill the page filter */
187     rowkey = Bytes.toBytes("xxxxxxx");
188     for (int i = 0; i < MAX_PAGES; i++) {
189       assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
190       KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
191           Bytes.toBytes(i));
192         assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
193       assertFalse(filterMPONE.filterRow());
194     }
195 
196     /* We should still be able to include even though page filter is at max */
197     rowkey = Bytes.toBytes("yyy");
198     for (int i = 0; i < MAX_PAGES; i++) {
199       assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
200       KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
201           Bytes.toBytes(i));
202         assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
203       assertFalse(filterMPONE.filterRow());
204     }
205   }
206 
207   /**
208    * Test serialization
209    * @throws Exception
210    */
211   public void testSerialization() throws Exception {
212     List<Filter> filters = new ArrayList<Filter>();
213     filters.add(new PageFilter(MAX_PAGES));
214     filters.add(new WhileMatchFilter(new PrefixFilter(Bytes.toBytes("yyy"))));
215     Filter filterMPALL =
216       new FilterList(FilterList.Operator.MUST_PASS_ALL, filters);
217 
218     // Decompose filterMPALL to bytes.
219     ByteArrayOutputStream stream = new ByteArrayOutputStream();
220     DataOutputStream out = new DataOutputStream(stream);
221     filterMPALL.write(out);
222     out.close();
223     byte[] buffer = stream.toByteArray();
224 
225     // Recompose filterMPALL.
226     DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
227     FilterList newFilter = new FilterList();
228     newFilter.readFields(in);
229 
230     // TODO: Run TESTS!!!
231   }
232 
233   /**
234    * Test pass-thru of hints.
235    */
236   public void testHintPassThru() throws Exception {
237 
238     final KeyValue minKeyValue = new KeyValue(Bytes.toBytes(0L), null, null);
239     final KeyValue maxKeyValue = new KeyValue(Bytes.toBytes(Long.MAX_VALUE),
240         null, null);
241 
242     Filter filterNoHint = new FilterBase() {
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 filterMinHint = new FilterBase() {
251       @Override
252       public KeyValue getNextKeyHint(KeyValue currentKV) {
253         return minKeyValue;
254       }
255 
256       @Override
257       public void readFields(DataInput arg0) throws IOException {}
258 
259       @Override
260       public void write(DataOutput arg0) throws IOException {}
261     };
262 
263     Filter filterMaxHint = new FilterBase() {
264       @Override
265       public KeyValue getNextKeyHint(KeyValue currentKV) {
266         return new KeyValue(Bytes.toBytes(Long.MAX_VALUE), null, null);
267       }
268 
269       @Override
270       public void readFields(DataInput arg0) throws IOException {}
271 
272       @Override
273       public void write(DataOutput arg0) throws IOException {}
274     };
275 
276     // MUST PASS ONE
277 
278     // Should take the min if given two hints
279     FilterList filterList = new FilterList(Operator.MUST_PASS_ONE,
280         Arrays.asList(new Filter [] { filterMinHint, filterMaxHint } ));
281     assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
282         minKeyValue));
283 
284     // Should have no hint if any filter has no hint
285     filterList = new FilterList(Operator.MUST_PASS_ONE,
286         Arrays.asList(
287             new Filter [] { filterMinHint, filterMaxHint, filterNoHint } ));
288     assertNull(filterList.getNextKeyHint(null));
289     filterList = new FilterList(Operator.MUST_PASS_ONE,
290         Arrays.asList(new Filter [] { filterNoHint, filterMaxHint } ));
291     assertNull(filterList.getNextKeyHint(null));
292 
293     // Should give max hint if its the only one
294     filterList = new FilterList(Operator.MUST_PASS_ONE,
295         Arrays.asList(new Filter [] { filterMaxHint, filterMaxHint } ));
296     assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
297         maxKeyValue));
298 
299     // MUST PASS ALL
300 
301     // Should take the max if given two hints
302     filterList = new FilterList(Operator.MUST_PASS_ALL,
303         Arrays.asList(new Filter [] { filterMinHint, filterMaxHint } ));
304     assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
305         maxKeyValue));
306 
307     // Should have max hint even if a filter has no hint
308     filterList = new FilterList(Operator.MUST_PASS_ALL,
309         Arrays.asList(
310             new Filter [] { filterMinHint, filterMaxHint, filterNoHint } ));
311     assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
312         maxKeyValue));
313     filterList = new FilterList(Operator.MUST_PASS_ALL,
314         Arrays.asList(new Filter [] { filterNoHint, filterMaxHint } ));
315     assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
316         maxKeyValue));
317     filterList = new FilterList(Operator.MUST_PASS_ALL,
318         Arrays.asList(new Filter [] { filterNoHint, filterMinHint } ));
319     assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
320         minKeyValue));
321 
322     // Should give min hint if its the only one
323     filterList = new FilterList(Operator.MUST_PASS_ALL,
324         Arrays.asList(new Filter [] { filterNoHint, filterMinHint } ));
325     assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
326         minKeyValue));
327   }
328 }