View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.filter;
20  
21  import java.io.IOException;
22  import java.util.ArrayList;
23  import java.util.Arrays;
24  import java.util.List;
25  
26  import org.apache.hadoop.classification.InterfaceAudience;
27  import org.apache.hadoop.classification.InterfaceStability;
28  import org.apache.hadoop.hbase.Cell;
29  import org.apache.hadoop.hbase.CellComparator;
30  import org.apache.hadoop.hbase.KeyValue;
31  import org.apache.hadoop.hbase.KeyValueUtil;
32  import org.apache.hadoop.hbase.exceptions.DeserializationException;
33  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
34  import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
35  
36  import com.google.protobuf.InvalidProtocolBufferException;
37  
38  /**
39   * Implementation of {@link Filter} that represents an ordered List of Filters
40   * which will be evaluated with a specified boolean operator {@link Operator#MUST_PASS_ALL}
41   * (<code>AND</code>) or {@link Operator#MUST_PASS_ONE} (<code>OR</code>).
42   * Since you can use Filter Lists as children of Filter Lists, you can create a
43   * hierarchy of filters to be evaluated.
44   *
45   * <br/>
46   * {@link Operator#MUST_PASS_ALL} evaluates lazily: evaluation stops as soon as one filter does
47   * not include the KeyValue.
48   *
49   * <br/>
50   * {@link Operator#MUST_PASS_ONE} evaluates non-lazily: all filters are always evaluated.
51   *
52   * <br/>
53   * Defaults to {@link Operator#MUST_PASS_ALL}.
54   */
55  @InterfaceAudience.Public
56  @InterfaceStability.Stable
57  final public class FilterList extends Filter {
58    /** set operator */
59    @InterfaceAudience.Public
60    @InterfaceStability.Stable
61    public static enum Operator {
62      /** !AND */
63      MUST_PASS_ALL,
64      /** !OR */
65      MUST_PASS_ONE
66    }
67  
68    private static final int MAX_LOG_FILTERS = 5;
69    private Operator operator = Operator.MUST_PASS_ALL;
70    private List<Filter> filters = new ArrayList<Filter>();
71    private Filter seekHintFilter = null;
72  
73    /** Reference Cell used by {@link #transformCell(Cell)} for validation purpose. */
74    private Cell referenceCell = null;
75  
76    /**
77     * When filtering a given Cell in {@link #filterKeyValue(Cell)},
78     * this stores the transformed Cell to be returned by {@link #transformCell(Cell)}.
79     *
80     * Individual filters transformation are applied only when the filter includes the Cell.
81     * Transformations are composed in the order specified by {@link #filters}.
82     */
83    private Cell transformedCell = null;
84  
85    /**
86     * Constructor that takes a set of {@link Filter}s. The default operator
87     * MUST_PASS_ALL is assumed.
88     *
89     * @param rowFilters list of filters
90     */
91    public FilterList(final List<Filter> rowFilters) {
92      if (rowFilters instanceof ArrayList) {
93        this.filters = rowFilters;
94      } else {
95        this.filters = new ArrayList<Filter>(rowFilters);
96      }
97    }
98  
99    /**
100    * Constructor that takes a var arg number of {@link Filter}s. The fefault operator
101    * MUST_PASS_ALL is assumed.
102    * @param rowFilters
103    */
104   public FilterList(final Filter... rowFilters) {
105     this.filters = new ArrayList<Filter>(Arrays.asList(rowFilters));
106   }
107 
108   /**
109    * Constructor that takes an operator.
110    *
111    * @param operator Operator to process filter set with.
112    */
113   public FilterList(final Operator operator) {
114     this.operator = operator;
115   }
116 
117   /**
118    * Constructor that takes a set of {@link Filter}s and an operator.
119    *
120    * @param operator Operator to process filter set with.
121    * @param rowFilters Set of row filters.
122    */
123   public FilterList(final Operator operator, final List<Filter> rowFilters) {
124     this.filters = new ArrayList<Filter>(rowFilters);
125     this.operator = operator;
126   }
127 
128   /**
129    * Constructor that takes a var arg number of {@link Filter}s and an operator.
130    *
131    * @param operator Operator to process filter set with.
132    * @param rowFilters Filters to use
133    */
134   public FilterList(final Operator operator, final Filter... rowFilters) {
135     this.filters = new ArrayList<Filter>(Arrays.asList(rowFilters));
136     this.operator = operator;
137   }
138 
139   /**
140    * Get the operator.
141    *
142    * @return operator
143    */
144   public Operator getOperator() {
145     return operator;
146   }
147 
148   /**
149    * Get the filters.
150    *
151    * @return filters
152    */
153   public List<Filter> getFilters() {
154     return filters;
155   }
156 
157   /**
158    * Add a filter.
159    *
160    * @param filter another filter
161    */
162   public void addFilter(Filter filter) {
163     if (this.isReversed() != filter.isReversed()) {
164       throw new IllegalArgumentException(
165           "Filters in the list must have the same reversed flag, this.reversed="
166               + this.isReversed());
167     }
168     this.filters.add(filter);
169   }
170 
171   @Override
172   public void reset() throws IOException {
173     for (Filter filter : filters) {
174       filter.reset();
175     }
176     seekHintFilter = null;
177   }
178 
179   @Override
180   public boolean filterRowKey(byte[] rowKey, int offset, int length) throws IOException {
181     boolean flag = (this.operator == Operator.MUST_PASS_ONE) ? true : false;
182     for (Filter filter : filters) {
183       if (this.operator == Operator.MUST_PASS_ALL) {
184         if (filter.filterAllRemaining() ||
185             filter.filterRowKey(rowKey, offset, length)) {
186           flag =  true;
187         }
188       } else if (this.operator == Operator.MUST_PASS_ONE) {
189         if (!filter.filterAllRemaining() &&
190             !filter.filterRowKey(rowKey, offset, length)) {
191           flag =  false;
192         }
193       }
194     }
195     return flag;
196   }
197 
198   @Override
199   public boolean filterAllRemaining() throws IOException {
200     for (Filter filter : filters) {
201       if (filter.filterAllRemaining()) {
202         if (operator == Operator.MUST_PASS_ALL) {
203           return true;
204         }
205       } else {
206         if (operator == Operator.MUST_PASS_ONE) {
207           return false;
208         }
209       }
210     }
211     return operator == Operator.MUST_PASS_ONE;
212   }
213 
214   @Override
215   public Cell transformCell(Cell c) throws IOException {
216     if (!CellComparator.equals(c, referenceCell)) {
217       throw new IllegalStateException("Reference Cell: " + this.referenceCell + " does not match: "
218           + c);
219     }
220     return this.transformedCell;
221   }
222 
223   /**
224    * WARNING: please to not override this method.  Instead override {@link #transformCell(Cell)}.
225    *
226    * When removing this, its body should be placed in transformCell.
227    *
228    * This is for transition from 0.94 -> 0.96
229    */
230   @Deprecated
231   @Override
232   public KeyValue transform(KeyValue v) throws IOException {
233     // transform() is expected to follow an inclusive filterKeyValue() immediately:
234     if (!v.equals(this.referenceCell)) {
235       throw new IllegalStateException(
236           "Reference Cell: " + this.referenceCell + " does not match: " + v);
237      }
238     return KeyValueUtil.ensureKeyValue(this.transformedCell);
239   }
240 
241   
242   @Override
243   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="SF_SWITCH_FALLTHROUGH",
244     justification="Intentional")
245   public ReturnCode filterKeyValue(Cell c) throws IOException {
246     this.referenceCell = c;
247 
248     // Accumulates successive transformation of every filter that includes the Cell:
249     Cell transformed = c;
250 
251     ReturnCode rc = operator == Operator.MUST_PASS_ONE?
252         ReturnCode.SKIP: ReturnCode.INCLUDE;
253     for (Filter filter : filters) {
254       if (operator == Operator.MUST_PASS_ALL) {
255         if (filter.filterAllRemaining()) {
256           return ReturnCode.NEXT_ROW;
257         }
258         ReturnCode code = filter.filterKeyValue(c);
259         switch (code) {
260         // Override INCLUDE and continue to evaluate.
261         case INCLUDE_AND_NEXT_COL:
262           rc = ReturnCode.INCLUDE_AND_NEXT_COL; // FindBugs SF_SWITCH_FALLTHROUGH
263         case INCLUDE:
264           transformed = filter.transformCell(transformed);
265           continue;
266         case SEEK_NEXT_USING_HINT:
267           seekHintFilter = filter;
268           return code;
269         default:
270           return code;
271         }
272       } else if (operator == Operator.MUST_PASS_ONE) {
273         if (filter.filterAllRemaining()) {
274           continue;
275         }
276 
277         switch (filter.filterKeyValue(c)) {
278         case INCLUDE:
279           if (rc != ReturnCode.INCLUDE_AND_NEXT_COL) {
280             rc = ReturnCode.INCLUDE;
281           }
282           transformed = filter.transformCell(transformed);
283           break;
284         case INCLUDE_AND_NEXT_COL:
285           rc = ReturnCode.INCLUDE_AND_NEXT_COL;
286           transformed = filter.transformCell(transformed);
287           // must continue here to evaluate all filters
288           break;
289         case NEXT_ROW:
290           break;
291         case SKIP:
292           break;
293         case NEXT_COL:
294           break;
295         case SEEK_NEXT_USING_HINT:
296           break;
297         default:
298           throw new IllegalStateException("Received code is not valid.");
299         }
300       }
301     }
302 
303     // Save the transformed Cell for transform():
304     this.transformedCell = transformed;
305 
306     return rc;
307   }
308 
309   /**
310    * Filters that never filter by modifying the returned List of Cells can
311    * inherit this implementation that does nothing.
312    *
313    * @inheritDoc
314    */
315   @Override
316   public void filterRowCells(List<Cell> cells) throws IOException {
317     for (Filter filter : filters) {
318       filter.filterRowCells(cells); 
319     }
320   }
321 
322   @Override
323   public boolean hasFilterRow() {
324     for (Filter filter : filters) {
325       if(filter.hasFilterRow()) {
326     	return true;
327       }
328     }
329     return false;
330   }
331 
332   @Override
333   public boolean filterRow() throws IOException {
334     for (Filter filter : filters) {
335       if (operator == Operator.MUST_PASS_ALL) {
336         if (filter.filterRow()) {
337           return true;
338         }
339       } else if (operator == Operator.MUST_PASS_ONE) {
340         if (!filter.filterRow()) {
341           return false;
342         }
343       }
344     }
345     return  operator == Operator.MUST_PASS_ONE;
346   }
347 
348   /**
349    * @return The filter serialized using pb
350    */
351   public byte[] toByteArray() throws IOException {
352     FilterProtos.FilterList.Builder builder =
353       FilterProtos.FilterList.newBuilder();
354     builder.setOperator(FilterProtos.FilterList.Operator.valueOf(operator.name()));
355     for (Filter filter : filters) {
356       builder.addFilters(ProtobufUtil.toFilter(filter));
357     }
358     return builder.build().toByteArray();
359   }
360 
361   /**
362    * @param pbBytes A pb serialized {@link FilterList} instance
363    * @return An instance of {@link FilterList} made from <code>bytes</code>
364    * @throws DeserializationException
365    * @see #toByteArray
366    */
367   public static FilterList parseFrom(final byte [] pbBytes)
368   throws DeserializationException {
369     FilterProtos.FilterList proto;
370     try {
371       proto = FilterProtos.FilterList.parseFrom(pbBytes);
372     } catch (InvalidProtocolBufferException e) {
373       throw new DeserializationException(e);
374     }
375 
376     List<Filter> rowFilters = new ArrayList<Filter>(proto.getFiltersCount());
377     try {
378       for (FilterProtos.Filter filter : proto.getFiltersList()) {
379         rowFilters.add(ProtobufUtil.toFilter(filter));
380       }
381     } catch (IOException ioe) {
382       throw new DeserializationException(ioe);
383     }
384     return new FilterList(Operator.valueOf(proto.getOperator().name()),rowFilters);
385   }
386 
387   /**
388    * @param other
389    * @return true if and only if the fields of the filter that are serialized
390    * are equal to the corresponding fields in other.  Used for testing.
391    */
392   boolean areSerializedFieldsEqual(Filter other) {
393     if (other == this) return true;
394     if (!(other instanceof FilterList)) return false;
395 
396     FilterList o = (FilterList)other;
397     return this.getOperator().equals(o.getOperator()) &&
398       ((this.getFilters() == o.getFilters())
399       || this.getFilters().equals(o.getFilters()));
400   }
401 
402   @Override
403   @Deprecated
404   public KeyValue getNextKeyHint(KeyValue currentKV) throws IOException {
405     return KeyValueUtil.ensureKeyValue(getNextCellHint((Cell)currentKV));
406   }
407 
408   @Override
409   public Cell getNextCellHint(Cell currentCell) throws IOException {
410     Cell keyHint = null;
411     if (operator == Operator.MUST_PASS_ALL) {
412       keyHint = seekHintFilter.getNextCellHint(currentCell);
413       return keyHint;
414     }
415 
416     // If any condition can pass, we need to keep the min hint
417     for (Filter filter : filters) {
418       Cell curKeyHint = filter.getNextCellHint(currentCell);
419       if (curKeyHint == null) {
420         // If we ever don't have a hint and this is must-pass-one, then no hint
421         return null;
422       }
423       if (curKeyHint != null) {
424         // If this is the first hint we find, set it
425         if (keyHint == null) {
426           keyHint = curKeyHint;
427           continue;
428         }
429         if (KeyValue.COMPARATOR.compare(keyHint, curKeyHint) > 0) {
430           keyHint = curKeyHint;
431         }
432       }
433     }
434     return keyHint;
435   }
436 
437   @Override
438   public boolean isFamilyEssential(byte[] name) throws IOException {
439     for (Filter filter : filters) {
440       if (filter.isFamilyEssential(name)) {
441         return true;
442       }
443     }
444     return false;
445   }
446 
447   @Override
448   public void setReversed(boolean reversed) {
449     for (Filter filter : filters) {
450       filter.setReversed(reversed);
451     }
452     this.reversed = reversed;
453   }
454 
455   @Override
456   public String toString() {
457     return toString(MAX_LOG_FILTERS);
458   }
459 
460   protected String toString(int maxFilters) {
461     int endIndex = this.filters.size() < maxFilters
462         ? this.filters.size() : maxFilters;
463     return String.format("%s %s (%d/%d): %s",
464         this.getClass().getSimpleName(),
465         this.operator == Operator.MUST_PASS_ALL ? "AND" : "OR",
466         endIndex,
467         this.filters.size(),
468         this.filters.subList(0, endIndex).toString());
469   }
470 }