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