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 static final int MAX_LOG_FILTERS = 5;
55    private Operator operator = Operator.MUST_PASS_ALL;
56    private List<Filter> filters = new ArrayList<Filter>();
57  
58    /**
59     * Default constructor, filters nothing. Required though for RPC
60     * deserialization.
61     */
62    public FilterList() {
63      super();
64    }
65  
66    /**
67     * Constructor that takes a set of {@link Filter}s. The default operator
68     * MUST_PASS_ALL is assumed.
69     *
70     * @param rowFilters list of filters
71     */
72    public FilterList(final List<Filter> rowFilters) {
73      this.filters = rowFilters;
74    }
75  
76    /**
77     * Constructor that takes a var arg number of {@link Filter}s. The fefault operator
78     * MUST_PASS_ALL is assumed.
79     * @param rowFilters
80     */
81    public FilterList(final Filter... rowFilters) {
82      this.filters = Arrays.asList(rowFilters);
83    }
84  
85    /**
86     * Constructor that takes an operator.
87     *
88     * @param operator Operator to process filter set with.
89     */
90    public FilterList(final Operator operator) {
91      this.operator = operator;
92    }
93  
94    /**
95     * Constructor that takes a set of {@link Filter}s and an operator.
96     *
97     * @param operator Operator to process filter set with.
98     * @param rowFilters Set of row filters.
99     */
100   public FilterList(final Operator operator, final List<Filter> rowFilters) {
101     this.filters = rowFilters;
102     this.operator = operator;
103   }
104 
105   /**
106    * Constructor that takes a var arg number of {@link Filter}s and an operator.
107    *
108    * @param operator Operator to process filter set with.
109    * @param rowFilters Filters to use
110    */
111   public FilterList(final Operator operator, final Filter... rowFilters) {
112     this.filters = Arrays.asList(rowFilters);
113     this.operator = operator;
114   }
115 
116   /**
117    * Get the operator.
118    *
119    * @return operator
120    */
121   public Operator getOperator() {
122     return operator;
123   }
124 
125   /**
126    * Get the filters.
127    *
128    * @return filters
129    */
130   public List<Filter> getFilters() {
131     return filters;
132   }
133 
134   /**
135    * Add a filter.
136    *
137    * @param filter another filter
138    */
139   public void addFilter(Filter filter) {
140     this.filters.add(filter);
141   }
142 
143   @Override
144   public void reset() {
145     for (Filter filter : filters) {
146       filter.reset();
147     }
148   }
149 
150   @Override
151   public boolean filterRowKey(byte[] rowKey, int offset, int length) {
152     boolean flag = (this.operator == Operator.MUST_PASS_ONE) ? true : false;
153     for (Filter filter : filters) {
154       if (this.operator == Operator.MUST_PASS_ALL) {
155         if (filter.filterAllRemaining() ||
156             filter.filterRowKey(rowKey, offset, length)) {
157           flag = true;
158         }
159       } else if (this.operator == Operator.MUST_PASS_ONE) {
160         if (!filter.filterAllRemaining() &&
161             !filter.filterRowKey(rowKey, offset, length)) {
162           flag = false;
163         }
164       }
165     }
166     return flag;
167   }
168 
169   @Override
170   public boolean filterAllRemaining() {
171     for (Filter filter : filters) {
172       if (filter.filterAllRemaining()) {
173         if (operator == Operator.MUST_PASS_ALL) {
174           return true;
175         }
176       } else {
177         if (operator == Operator.MUST_PASS_ONE) {
178           return false;
179         }
180       }
181     }
182     return operator == Operator.MUST_PASS_ONE;
183   }
184 
185   @Override
186   public KeyValue transform(KeyValue v) {
187     KeyValue current = v;
188     for (Filter filter : filters) {
189       current = filter.transform(current);
190     }
191     return current;
192   }
193 
194   @Override
195   public ReturnCode filterKeyValue(KeyValue v) {
196     ReturnCode rc = operator == Operator.MUST_PASS_ONE?
197         ReturnCode.SKIP: ReturnCode.INCLUDE;
198     for (Filter filter : filters) {
199       if (operator == Operator.MUST_PASS_ALL) {
200         if (filter.filterAllRemaining()) {
201           return ReturnCode.NEXT_ROW;
202         }
203         ReturnCode code = filter.filterKeyValue(v);
204         switch (code) {
205         // Override INCLUDE and continue to evaluate.
206         case INCLUDE_AND_NEXT_COL:
207           rc = ReturnCode.INCLUDE_AND_NEXT_COL;
208         case INCLUDE:
209           continue;
210         default:
211           return code;
212         }
213       } else if (operator == Operator.MUST_PASS_ONE) {
214         if (filter.filterAllRemaining()) {
215           continue;
216         }
217 
218         switch (filter.filterKeyValue(v)) {
219         case INCLUDE:
220           if (rc != ReturnCode.INCLUDE_AND_NEXT_COL) {
221             rc = ReturnCode.INCLUDE;
222           }
223           break;
224         case INCLUDE_AND_NEXT_COL:
225           rc = ReturnCode.INCLUDE_AND_NEXT_COL;
226           // must continue here to evaluate all filters
227         case NEXT_ROW:
228         case SKIP:
229           // continue;
230         }
231       }
232     }
233     return rc;
234   }
235 
236   @Override
237   public void filterRow(List<KeyValue> kvs) {
238     for (Filter filter : filters) {
239       filter.filterRow(kvs);
240     }
241   }
242 
243   @Override
244   public boolean hasFilterRow() {
245     for (Filter filter : filters) {
246       if(filter.hasFilterRow()) {
247     	return true;
248       }
249     }
250     return false;
251   }
252 
253   @Override
254   public boolean filterRow() {
255     for (Filter filter : filters) {
256       if (operator == Operator.MUST_PASS_ALL) {
257         if (filter.filterRow()) {
258           return true;
259         }
260       } else if (operator == Operator.MUST_PASS_ONE) {
261         if (!filter.filterRow()) {
262           return false;
263         }
264       }
265     }
266     return  operator == Operator.MUST_PASS_ONE;
267   }
268 
269   public void readFields(final DataInput in) throws IOException {
270     byte opByte = in.readByte();
271     operator = Operator.values()[opByte];
272     int size = in.readInt();
273     if (size > 0) {
274       filters = new ArrayList<Filter>(size);
275       for (int i = 0; i < size; i++) {
276         Filter filter = (Filter)HbaseObjectWritable.readObject(in, conf);
277         filters.add(filter);
278       }
279     }
280   }
281 
282   public void write(final DataOutput out) throws IOException {
283     out.writeByte(operator.ordinal());
284     out.writeInt(filters.size());
285     for (Filter filter : filters) {
286       HbaseObjectWritable.writeObject(out, filter, Writable.class, conf);
287     }
288   }
289 
290   @Override
291   public KeyValue getNextKeyHint(KeyValue currentKV) {
292     KeyValue keyHint = null;
293     for (Filter filter : filters) {
294       KeyValue curKeyHint = filter.getNextKeyHint(currentKV);
295       if (curKeyHint == null && operator == Operator.MUST_PASS_ONE) {
296         // If we ever don't have a hint and this is must-pass-one, then no hint
297         return null;
298       }
299       if (curKeyHint != null) {
300         // If this is the first hint we find, set it
301         if (keyHint == null) {
302           keyHint = curKeyHint;
303           continue;
304         }
305         // There is an existing hint
306         if (operator == Operator.MUST_PASS_ALL &&
307             KeyValue.COMPARATOR.compare(keyHint, curKeyHint) < 0) {
308           // If all conditions must pass, we can keep the max hint
309           keyHint = curKeyHint;
310         } else if (operator == Operator.MUST_PASS_ONE &&
311             KeyValue.COMPARATOR.compare(keyHint, curKeyHint) > 0) {
312           // If any condition can pass, we need to keep the min hint
313           keyHint = curKeyHint;
314         }
315       }
316     }
317     return keyHint;
318   }
319 
320   public boolean isFamilyEssential(byte[] name) {
321     for (Filter filter : filters) {
322       if (FilterBase.isFamilyEssential(filter, name)) {
323         return true;
324       }
325     }
326     return false;
327   }
328 
329   @Override
330   public String toString() {
331     return toString(MAX_LOG_FILTERS);
332   }
333 
334   protected String toString(int maxFilters) {
335     int endIndex = this.filters.size() < maxFilters
336         ? this.filters.size() : maxFilters;
337     return String.format("%s %s (%d/%d): %s",
338         this.getClass().getSimpleName(),
339         this.operator == Operator.MUST_PASS_ALL ? "AND" : "OR",
340         endIndex,
341         this.filters.size(),
342         this.filters.subList(0, endIndex).toString());
343   }
344 }