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   *
42   * <br/>
43   * {@link Operator#MUST_PASS_ALL} evaluates lazily: evaluation stops as soon as one filter does
44   * not include the KeyValue.
45   *
46   * <br/>
47   * {@link Operator#MUST_PASS_ONE} evaluates non-lazily: all filters are always evaluated.
48   *
49   * <br/>
50   * Defaults to {@link Operator#MUST_PASS_ALL}.
51   * <p>TODO: Fix creation of Configuration on serialization and deserialization.
52   */
53  public class FilterList implements Filter {
54    /** set operator */
55    public static enum Operator {
56      /** !AND */
57      MUST_PASS_ALL,
58      /** !OR */
59      MUST_PASS_ONE
60    }
61  
62    private static final Configuration CONF;
63    static {
64      // We don't know which thread will load this class, so we don't know what
65      // the state of the context classloader will be when this class is loaded.
66      // HBaseConfiguration.create is dependent on the state of the context
67      // classloader of the current thread, so we set it to be the classloader
68      // that was used to load the Filter class to guarantee the consistent
69      // ability to load this class from any thread
70      ClassLoader saveCtxCl = Thread.currentThread().getContextClassLoader();
71      try {
72        Thread.currentThread().setContextClassLoader(
73            Filter.class.getClassLoader());
74        CONF = HBaseConfiguration.create();
75      } finally {
76        Thread.currentThread().setContextClassLoader(saveCtxCl);
77      }
78    }
79    private static final int MAX_LOG_FILTERS = 5;
80    private Operator operator = Operator.MUST_PASS_ALL;
81    private List<Filter> filters = new ArrayList<Filter>();
82    private Filter seekHintFilter = null;
83  
84    /** Reference KeyValue used by {@link #transform(KeyValue)} for validation purpose. */
85    private KeyValue referenceKV = null;
86  
87    /**
88     * When filtering a given KeyValue in {@link #filterKeyValue(KeyValue)},
89     * this stores the transformed KeyValue to be returned by {@link #transform(KeyValue)}.
90     *
91     * Individual filters transformation are applied only when the filter includes the KeyValue.
92     * Transformations are composed in the order specified by {@link #filters}.
93     */
94    private KeyValue transformedKV = null;
95  
96    /**
97     * Default constructor, filters nothing. Required though for RPC
98     * deserialization.
99     */
100   public FilterList() {
101     super();
102   }
103 
104   /**
105    * Constructor that takes a set of {@link Filter}s. The default operator
106    * MUST_PASS_ALL is assumed.
107    *
108    * @param rowFilters list of filters
109    */
110   public FilterList(final List<Filter> rowFilters) {
111     this.filters = rowFilters;
112   }
113 
114   /**
115    * Constructor that takes a var arg number of {@link Filter}s. The fefault operator
116    * MUST_PASS_ALL is assumed.
117    * @param rowFilters
118    */
119   public FilterList(final Filter... rowFilters) {
120     this.filters = Arrays.asList(rowFilters);
121   }
122 
123   /**
124    * Constructor that takes an operator.
125    *
126    * @param operator Operator to process filter set with.
127    */
128   public FilterList(final Operator operator) {
129     this.operator = operator;
130   }
131 
132   /**
133    * Constructor that takes a set of {@link Filter}s and an operator.
134    *
135    * @param operator Operator to process filter set with.
136    * @param rowFilters Set of row filters.
137    */
138   public FilterList(final Operator operator, final List<Filter> rowFilters) {
139     this.filters = rowFilters;
140     this.operator = operator;
141   }
142 
143   /**
144    * Constructor that takes a var arg number of {@link Filter}s and an operator.
145    *
146    * @param operator Operator to process filter set with.
147    * @param rowFilters Filters to use
148    */
149   public FilterList(final Operator operator, final Filter... rowFilters) {
150     this.filters = Arrays.asList(rowFilters);
151     this.operator = operator;
152   }
153 
154   /**
155    * Get the operator.
156    *
157    * @return operator
158    */
159   public Operator getOperator() {
160     return operator;
161   }
162 
163   /**
164    * Get the filters.
165    *
166    * @return filters
167    */
168   public List<Filter> getFilters() {
169     return filters;
170   }
171 
172   /**
173    * Add a filter.
174    *
175    * @param filter another filter
176    */
177   public void addFilter(Filter filter) {
178     this.filters.add(filter);
179   }
180 
181   @Override
182   public void reset() {
183     for (Filter filter : filters) {
184       filter.reset();
185     }
186     seekHintFilter = null;
187   }
188 
189   @Override
190   public boolean filterRowKey(byte[] rowKey, int offset, int length) {
191     boolean flag = (this.operator == Operator.MUST_PASS_ONE) ? true : false;
192     for (Filter filter : filters) {
193       if (this.operator == Operator.MUST_PASS_ALL) {
194         if (filter.filterAllRemaining() ||
195             filter.filterRowKey(rowKey, offset, length)) {
196           flag = true;
197         }
198       } else if (this.operator == Operator.MUST_PASS_ONE) {
199         if (!filter.filterAllRemaining() &&
200             !filter.filterRowKey(rowKey, offset, length)) {
201           flag = false;
202         }
203       }
204     }
205     return flag;
206   }
207 
208   @Override
209   public boolean filterAllRemaining() {
210     for (Filter filter : filters) {
211       if (filter.filterAllRemaining()) {
212         if (operator == Operator.MUST_PASS_ALL) {
213           return true;
214         }
215       } else {
216         if (operator == Operator.MUST_PASS_ONE) {
217           return false;
218         }
219       }
220     }
221     return operator == Operator.MUST_PASS_ONE;
222   }
223 
224   @Override
225   public KeyValue transform(KeyValue v) {
226     // transform() is expected to follow an inclusive filterKeyValue() immediately:
227     if (!v.equals(this.referenceKV)) {
228       throw new IllegalStateException(
229           "Reference KeyValue: " + this.referenceKV + " does not match: " + v);
230      }
231     return this.transformedKV;
232   }
233 
234   @Override
235   public ReturnCode filterKeyValue(KeyValue v) {
236     this.referenceKV = v;
237 
238     // Accumulates successive transformation of every filter that includes the KeyValue:
239     KeyValue transformed = v;
240 
241     ReturnCode rc = operator == Operator.MUST_PASS_ONE?
242         ReturnCode.SKIP: ReturnCode.INCLUDE;
243     for (Filter filter : filters) {
244       if (operator == Operator.MUST_PASS_ALL) {
245         if (filter.filterAllRemaining()) {
246           return ReturnCode.NEXT_ROW;
247         }
248         ReturnCode code = filter.filterKeyValue(v);
249         switch (code) {
250         // Override INCLUDE and continue to evaluate.
251         case INCLUDE_AND_NEXT_COL:
252           rc = ReturnCode.INCLUDE_AND_NEXT_COL;
253         case INCLUDE:
254           transformed = filter.transform(transformed);
255           continue;
256         case SEEK_NEXT_USING_HINT:
257           seekHintFilter = filter;
258           return code;
259         default:
260           return code;
261         }
262       } else if (operator == Operator.MUST_PASS_ONE) {
263         if (filter.filterAllRemaining()) {
264           continue;
265         }
266 
267         ReturnCode code = filter.filterKeyValue(v);
268         switch (code) {
269         case INCLUDE:
270           if (rc != ReturnCode.INCLUDE_AND_NEXT_COL) {
271             rc = ReturnCode.INCLUDE;
272           }
273           transformed = filter.transform(transformed);
274           break;
275         case INCLUDE_AND_NEXT_COL:
276           rc = ReturnCode.INCLUDE_AND_NEXT_COL;
277           transformed = filter.transform(transformed);
278           // must continue here to evaluate all filters
279           break;
280         case NEXT_ROW:
281           break;
282         case SKIP:
283           break;
284         case NEXT_COL:
285           break;
286         case SEEK_NEXT_USING_HINT:
287           break;
288         default:
289           throw new IllegalStateException("Received code is not valid.");
290         }
291       }
292     }
293 
294     // Save the transformed KeyValue for transform():
295     this.transformedKV = transformed;
296 
297     return rc;
298   }
299 
300   @Override
301   public void filterRow(List<KeyValue> kvs) {
302     for (Filter filter : filters) {
303       filter.filterRow(kvs);
304     }
305   }
306 
307   @Override
308   public boolean hasFilterRow() {
309     for (Filter filter : filters) {
310       if(filter.hasFilterRow()) {
311     	return true;
312       }
313     }
314     return false;
315   }
316 
317   @Override
318   public boolean filterRow() {
319     for (Filter filter : filters) {
320       if (operator == Operator.MUST_PASS_ALL) {
321         if (filter.filterRow()) {
322           return true;
323         }
324       } else if (operator == Operator.MUST_PASS_ONE) {
325         if (!filter.filterRow()) {
326           return false;
327         }
328       }
329     }
330     return  operator == Operator.MUST_PASS_ONE;
331   }
332 
333   public void readFields(final DataInput in) throws IOException {
334     byte opByte = in.readByte();
335     operator = Operator.values()[opByte];
336     int size = in.readInt();
337     if (size > 0) {
338       filters = new ArrayList<Filter>(size);
339       for (int i = 0; i < size; i++) {
340         Filter filter = HbaseObjectWritable.readFilter(in, CONF);
341         filters.add(filter);
342       }
343     }
344   }
345 
346   public void write(final DataOutput out) throws IOException {
347     out.writeByte(operator.ordinal());
348     out.writeInt(filters.size());
349     for (Filter filter : filters) {
350       HbaseObjectWritable.writeObject(out, filter, Writable.class, CONF);
351     }
352   }
353 
354   @Override
355   public KeyValue getNextKeyHint(KeyValue currentKV) {
356     KeyValue keyHint = null;
357     if (operator == Operator.MUST_PASS_ALL) {
358       keyHint = seekHintFilter.getNextKeyHint(currentKV);
359       return keyHint;
360     }
361 
362     for (Filter filter : filters) {
363       KeyValue curKeyHint = filter.getNextKeyHint(currentKV);
364       if (curKeyHint == null) {
365         // If we ever don't have a hint and this is must-pass-one, then no hint
366         return null;
367       }
368       if (curKeyHint != null) {
369         // If this is the first hint we find, set it
370         if (keyHint == null) {
371           keyHint = curKeyHint;
372           continue;
373         }
374         if (KeyValue.COMPARATOR.compare(keyHint, curKeyHint) > 0) {
375           // If any condition can pass, we need to keep the min hint
376           keyHint = curKeyHint;
377         }
378       }
379     }
380     return keyHint;
381   }
382 
383   public boolean isFamilyEssential(byte[] name) {
384     for (Filter filter : filters) {
385       if (FilterBase.isFamilyEssential(filter, name)) {
386         return true;
387       }
388     }
389     return false;
390   }
391 
392   @Override
393   public String toString() {
394     return toString(MAX_LOG_FILTERS);
395   }
396 
397   protected String toString(int maxFilters) {
398     int endIndex = this.filters.size() < maxFilters
399         ? this.filters.size() : maxFilters;
400     return String.format("%s %s (%d/%d): %s",
401         this.getClass().getSimpleName(),
402         this.operator == Operator.MUST_PASS_ALL ? "AND" : "OR",
403         endIndex,
404         this.filters.size(),
405         this.filters.subList(0, endIndex).toString());
406   }
407 }