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.DataInputStream;
25  import java.io.DataOutputStream;
26  import java.util.ArrayList;
27  import java.util.List;
28  
29  import org.apache.hadoop.hbase.util.Bytes;
30  import org.apache.hadoop.hbase.KeyValue;
31  
32  
33  import junit.framework.TestCase;
34  
35  /**
36   * Tests filter sets
37   *
38   */
39  public class TestFilterList extends TestCase {
40    static final int MAX_PAGES = 2;
41    static final char FIRST_CHAR = 'a';
42    static final char LAST_CHAR = 'e';
43    static byte[] GOOD_BYTES = Bytes.toBytes("abc");
44    static byte[] BAD_BYTES = Bytes.toBytes("def");
45  
46    /**
47     * Test "must pass one"
48     * @throws Exception
49     */
50    public void testMPONE() throws Exception {
51      List<Filter> filters = new ArrayList<Filter>();
52      filters.add(new PageFilter(MAX_PAGES));
53      filters.add(new WhileMatchFilter(new PrefixFilter(Bytes.toBytes("yyy"))));
54      Filter filterMPONE =
55          new FilterList(FilterList.Operator.MUST_PASS_ONE, filters);
56      /* Filter must do all below steps:
57       * <ul>
58       * <li>{@link #reset()}</li>
59       * <li>{@link #filterAllRemaining()} -> true indicates scan is over, false, keep going on.</li>
60       * <li>{@link #filterRowKey(byte[],int,int)} -> true to drop this row,
61       * if false, we will also call</li>
62       * <li>{@link #filterKeyValue(org.apache.hadoop.hbase.KeyValue)} -> true to drop this key/value</li>
63       * <li>{@link #filterRow()} -> last chance to drop entire row based on the sequence of
64       * filterValue() calls. Eg: filter a row if it doesn't contain a specified column.
65       * </li>
66       * </ul>
67      */
68      filterMPONE.reset();
69      assertFalse(filterMPONE.filterAllRemaining());
70  
71      /* Will pass both */
72      byte [] rowkey = Bytes.toBytes("yyyyyyyyy");
73      for (int i = 0; i < MAX_PAGES - 1; i++) {
74        assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
75        assertFalse(filterMPONE.filterRow());
76        KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
77          Bytes.toBytes(i));
78        assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
79      }
80  
81      /* Only pass PageFilter */
82      rowkey = Bytes.toBytes("z");
83      assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
84      assertFalse(filterMPONE.filterRow());
85      KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(0),
86          Bytes.toBytes(0));
87      assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
88  
89      /* PageFilter will fail now, but should pass because we match yyy */
90      rowkey = Bytes.toBytes("yyy");
91      assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
92      assertFalse(filterMPONE.filterRow());
93      kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(0),
94          Bytes.toBytes(0));
95      assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
96  
97      /* We should filter any row */
98      rowkey = Bytes.toBytes("z");
99      assertTrue(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
100     assertTrue(filterMPONE.filterRow());
101     assertTrue(filterMPONE.filterAllRemaining());
102 
103   }
104 
105   /**
106    * Test "must pass all"
107    * @throws Exception
108    */
109   public void testMPALL() throws Exception {
110     List<Filter> filters = new ArrayList<Filter>();
111     filters.add(new PageFilter(MAX_PAGES));
112     filters.add(new WhileMatchFilter(new PrefixFilter(Bytes.toBytes("yyy"))));
113     Filter filterMPALL =
114       new FilterList(FilterList.Operator.MUST_PASS_ALL, filters);
115     /* Filter must do all below steps:
116      * <ul>
117      * <li>{@link #reset()}</li>
118      * <li>{@link #filterAllRemaining()} -> true indicates scan is over, false, keep going on.</li>
119      * <li>{@link #filterRowKey(byte[],int,int)} -> true to drop this row,
120      * if false, we will also call</li>
121      * <li>{@link #filterKeyValue(org.apache.hadoop.hbase.KeyValue)} -> true to drop this key/value</li>
122      * <li>{@link #filterRow()} -> last chance to drop entire row based on the sequence of
123      * filterValue() calls. Eg: filter a row if it doesn't contain a specified column.
124      * </li>
125      * </ul>
126     */
127     filterMPALL.reset();
128     assertFalse(filterMPALL.filterAllRemaining());
129     byte [] rowkey = Bytes.toBytes("yyyyyyyyy");
130     for (int i = 0; i < MAX_PAGES - 1; i++) {
131       assertFalse(filterMPALL.filterRowKey(rowkey, 0, rowkey.length));
132       KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
133         Bytes.toBytes(i));
134       assertTrue(Filter.ReturnCode.INCLUDE == filterMPALL.filterKeyValue(kv));
135     }
136     filterMPALL.reset();
137     rowkey = Bytes.toBytes("z");
138     assertTrue(filterMPALL.filterRowKey(rowkey, 0, rowkey.length));
139     // Should fail here; row should be filtered out.
140     KeyValue kv = new KeyValue(rowkey, rowkey, rowkey, rowkey);
141     assertTrue(Filter.ReturnCode.NEXT_ROW == filterMPALL.filterKeyValue(kv));
142 
143     // Both filters in Set should be satisfied by now
144     assertTrue(filterMPALL.filterRow());
145   }
146 
147   /**
148    * Test list ordering
149    * @throws Exception
150    */
151   public void testOrdering() throws Exception {
152     List<Filter> filters = new ArrayList<Filter>();
153     filters.add(new PrefixFilter(Bytes.toBytes("yyy")));
154     filters.add(new PageFilter(MAX_PAGES));
155     Filter filterMPONE =
156         new FilterList(FilterList.Operator.MUST_PASS_ONE, filters);
157     /* Filter must do all below steps:
158      * <ul>
159      * <li>{@link #reset()}</li>
160      * <li>{@link #filterAllRemaining()} -> true indicates scan is over, false, keep going on.</li>
161      * <li>{@link #filterRowKey(byte[],int,int)} -> true to drop this row,
162      * if false, we will also call</li>
163      * <li>{@link #filterKeyValue(org.apache.hadoop.hbase.KeyValue)} -> true to drop this key/value</li>
164      * <li>{@link #filterRow()} -> last chance to drop entire row based on the sequence of
165      * filterValue() calls. Eg: filter a row if it doesn't contain a specified column.
166      * </li>
167      * </ul>
168     */
169     filterMPONE.reset();
170     assertFalse(filterMPONE.filterAllRemaining());
171 
172     /* We should be able to fill MAX_PAGES without incrementing page counter */
173     byte [] rowkey = Bytes.toBytes("yyyyyyyy");
174     for (int i = 0; i < MAX_PAGES; i++) {
175       assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
176       KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
177           Bytes.toBytes(i));
178         assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
179       assertFalse(filterMPONE.filterRow());
180     }
181 
182     /* Now let's fill the page filter */
183     rowkey = Bytes.toBytes("xxxxxxx");
184     for (int i = 0; i < MAX_PAGES; i++) {
185       assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
186       KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
187           Bytes.toBytes(i));
188         assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
189       assertFalse(filterMPONE.filterRow());
190     }
191 
192     /* We should still be able to include even though page filter is at max */
193     rowkey = Bytes.toBytes("yyy");
194     for (int i = 0; i < MAX_PAGES; i++) {
195       assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
196       KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
197           Bytes.toBytes(i));
198         assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
199       assertFalse(filterMPONE.filterRow());
200     }
201   }
202 
203   /**
204    * Test serialization
205    * @throws Exception
206    */
207   public void testSerialization() throws Exception {
208     List<Filter> filters = new ArrayList<Filter>();
209     filters.add(new PageFilter(MAX_PAGES));
210     filters.add(new WhileMatchFilter(new PrefixFilter(Bytes.toBytes("yyy"))));
211     Filter filterMPALL =
212       new FilterList(FilterList.Operator.MUST_PASS_ALL, filters);
213 
214     // Decompose filterMPALL to bytes.
215     ByteArrayOutputStream stream = new ByteArrayOutputStream();
216     DataOutputStream out = new DataOutputStream(stream);
217     filterMPALL.write(out);
218     out.close();
219     byte[] buffer = stream.toByteArray();
220 
221     // Recompose filterMPALL.
222     DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
223     FilterList newFilter = new FilterList();
224     newFilter.readFields(in);
225 
226     // TODO: Run TESTS!!!
227   }
228 }