View Javadoc

1   /**
2    * Copyright 2010 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 org.apache.hadoop.conf.Configuration;
23  import org.apache.hadoop.hbase.HBaseConfiguration;
24  import org.apache.hadoop.hbase.KeyValue;
25  import org.apache.hadoop.hbase.io.HbaseObjectWritable;
26  import org.apache.hadoop.io.Writable;
27  
28  import java.io.DataInput;
29  import java.io.DataOutput;
30  import java.io.IOException;
31  import java.util.ArrayList;
32  import java.util.Arrays;
33  import java.util.List;
34  
35  /**
36   * Implementation of {@link Filter} that represents an ordered List of Filters
37   * which will be evaluated with a specified boolean operator {@link Operator#MUST_PASS_ALL}
38   * (<code>!AND</code>) or {@link Operator#MUST_PASS_ONE} (<code>!OR</code>).
39   * Since you can use Filter Lists as children of Filter Lists, you can create a
40   * hierarchy of filters to be evaluated.
41   * Defaults to {@link Operator#MUST_PASS_ALL}.
42   * <p>TODO: Fix creation of Configuration on serialization and deserialization.
43   */
44  public class FilterList implements Filter {
45    /** set operator */
46    public static enum Operator {
47      /** !AND */
48      MUST_PASS_ALL,
49      /** !OR */
50      MUST_PASS_ONE
51    }
52  
53    private static final Configuration conf = HBaseConfiguration.create();
54    private Operator operator = Operator.MUST_PASS_ALL;
55    private List<Filter> filters = new ArrayList<Filter>();
56  
57    /**
58     * Default constructor, filters nothing. Required though for RPC
59     * deserialization.
60     */
61    public FilterList() {
62      super();
63    }
64  
65    /**
66     * Constructor that takes a set of {@link Filter}s. The default operator
67     * MUST_PASS_ALL is assumed.
68     *
69     * @param rowFilters list of filters
70     */
71    public FilterList(final List<Filter> rowFilters) {
72      this.filters = rowFilters;
73    }
74  
75    /**
76     * Constructor that takes a var arg number of {@link Filter}s. The fefault operator
77     * MUST_PASS_ALL is assumed.
78     * @param rowFilters
79     */
80    public FilterList(final Filter... rowFilters) {
81      this.filters = Arrays.asList(rowFilters);
82    }
83  
84    /**
85     * Constructor that takes an operator.
86     *
87     * @param operator Operator to process filter set with.
88     */
89    public FilterList(final Operator operator) {
90      this.operator = operator;
91    }
92  
93    /**
94     * Constructor that takes a set of {@link Filter}s and an operator.
95     *
96     * @param operator Operator to process filter set with.
97     * @param rowFilters Set of row filters.
98     */
99    public FilterList(final Operator operator, final List<Filter> rowFilters) {
100     this.filters = rowFilters;
101     this.operator = operator;
102   }
103 
104   /**
105    * Constructor that takes a var arg number of {@link Filter}s and an operator.
106    *
107    * @param operator Operator to process filter set with.
108    * @param rowFilters Filters to use
109    */
110   public FilterList(final Operator operator, final Filter... rowFilters) {
111     this.filters = Arrays.asList(rowFilters);
112     this.operator = operator;
113   }
114 
115   /**
116    * Get the operator.
117    *
118    * @return operator
119    */
120   public Operator getOperator() {
121     return operator;
122   }
123 
124   /**
125    * Get the filters.
126    *
127    * @return filters
128    */
129   public List<Filter> getFilters() {
130     return filters;
131   }
132 
133   /**
134    * Add a filter.
135    *
136    * @param filter another filter
137    */
138   public void addFilter(Filter filter) {
139     this.filters.add(filter);
140   }
141 
142   @Override
143   public void reset() {
144     for (Filter filter : filters) {
145       filter.reset();
146     }
147   }
148 
149   @Override
150   public boolean filterRowKey(byte[] rowKey, int offset, int length) {
151     for (Filter filter : filters) {
152       if (this.operator == Operator.MUST_PASS_ALL) {
153         if (filter.filterAllRemaining() ||
154             filter.filterRowKey(rowKey, offset, length)) {
155           return true;
156         }
157       } else if (this.operator == Operator.MUST_PASS_ONE) {
158         if (!filter.filterAllRemaining() &&
159             !filter.filterRowKey(rowKey, offset, length)) {
160           return false;
161         }
162       }
163     }
164     return this.operator == Operator.MUST_PASS_ONE;
165   }
166 
167   @Override
168   public boolean filterAllRemaining() {
169     for (Filter filter : filters) {
170       if (filter.filterAllRemaining()) {
171         if (operator == Operator.MUST_PASS_ALL) {
172           return true;
173         }
174       } else {
175         if (operator == Operator.MUST_PASS_ONE) {
176           return false;
177         }
178       }
179     }
180     return operator == Operator.MUST_PASS_ONE;
181   }
182 
183   @Override
184   public KeyValue transform(KeyValue v) {
185     KeyValue current = v;
186     for (Filter filter : filters) {
187       current = filter.transform(current);
188     }
189     return current;
190   }
191 
192   @Override
193   public ReturnCode filterKeyValue(KeyValue v) {
194     ReturnCode rc = operator == Operator.MUST_PASS_ONE?
195         ReturnCode.SKIP: ReturnCode.INCLUDE;
196     for (Filter filter : filters) {
197       if (operator == Operator.MUST_PASS_ALL) {
198         if (filter.filterAllRemaining()) {
199           return ReturnCode.NEXT_ROW;
200         }
201         ReturnCode code = filter.filterKeyValue(v);
202         switch (code) {
203         case INCLUDE:
204           continue;
205         case NEXT_ROW:
206         case SKIP:
207           return ReturnCode.SKIP;
208         default:
209           return code;
210         }
211       } else if (operator == Operator.MUST_PASS_ONE) {
212         if (filter.filterAllRemaining()) {
213           continue;
214         }
215 
216         switch (filter.filterKeyValue(v)) {
217         case INCLUDE:
218           rc = ReturnCode.INCLUDE;
219           // must continue here to evaluate all filters
220         case NEXT_ROW:
221         case SKIP:
222           // continue;
223         }
224       }
225     }
226     return rc;
227   }
228 
229   @Override
230   public void filterRow(List<KeyValue> kvs) {
231     for (Filter filter : filters) {
232       filter.filterRow(kvs);
233     }
234   }
235 
236   @Override
237   public boolean hasFilterRow() {
238     for (Filter filter : filters) {
239       if(filter.hasFilterRow()) {
240     	return true;
241       }
242     }
243     return false;
244   }
245 
246   @Override
247   public boolean filterRow() {
248     for (Filter filter : filters) {
249       if (operator == Operator.MUST_PASS_ALL) {
250         if (filter.filterAllRemaining() || filter.filterRow()) {
251           return true;
252         }
253       } else if (operator == Operator.MUST_PASS_ONE) {
254         if (!filter.filterAllRemaining()
255             && !filter.filterRow()) {
256           return false;
257         }
258       }
259     }
260     return  operator == Operator.MUST_PASS_ONE;
261   }
262 
263   public void readFields(final DataInput in) throws IOException {
264     byte opByte = in.readByte();
265     operator = Operator.values()[opByte];
266     int size = in.readInt();
267     if (size > 0) {
268       filters = new ArrayList<Filter>(size);
269       for (int i = 0; i < size; i++) {
270         Filter filter = (Filter)HbaseObjectWritable.readObject(in, conf);
271         filters.add(filter);
272       }
273     }
274   }
275 
276   public void write(final DataOutput out) throws IOException {
277     out.writeByte(operator.ordinal());
278     out.writeInt(filters.size());
279     for (Filter filter : filters) {
280       HbaseObjectWritable.writeObject(out, filter, Writable.class, conf);
281     }
282   }
283 
284   @Override
285   public KeyValue getNextKeyHint(KeyValue currentKV) {
286     KeyValue keyHint = null;
287     for (Filter filter : filters) {
288       KeyValue curKeyHint = filter.getNextKeyHint(currentKV);
289       if (curKeyHint == null && operator == Operator.MUST_PASS_ONE) {
290         // If we ever don't have a hint and this is must-pass-one, then no hint
291         return null;
292       }
293       if (curKeyHint != null) {
294         // If this is the first hint we find, set it
295         if (keyHint == null) {
296           keyHint = curKeyHint;
297           continue;
298         }
299         // There is an existing hint
300         if (operator == Operator.MUST_PASS_ALL &&
301             KeyValue.COMPARATOR.compare(keyHint, curKeyHint) < 0) {
302           // If all conditions must pass, we can keep the max hint
303           keyHint = curKeyHint;
304         } else if (operator == Operator.MUST_PASS_ONE &&
305             KeyValue.COMPARATOR.compare(keyHint, curKeyHint) > 0) {
306           // If any condition can pass, we need to keep the min hint
307           keyHint = curKeyHint;
308         }
309       }
310     }
311     return keyHint;
312   }
313 }