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.hbase.classification.InterfaceAudience;
27  import org.apache.hadoop.hbase.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     int listize = filters.size();
173     for (int i = 0; i < listize; i++) {
174       filters.get(i).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     int listize = filters.size();
183     for (int i = 0; i < listize; i++) {
184       Filter filter = filters.get(i);
185       if (this.operator == Operator.MUST_PASS_ALL) {
186         if (filter.filterAllRemaining() ||
187             filter.filterRowKey(rowKey, offset, length)) {
188           flag =  true;
189         }
190       } else if (this.operator == Operator.MUST_PASS_ONE) {
191         if (!filter.filterAllRemaining() &&
192             !filter.filterRowKey(rowKey, offset, length)) {
193           flag =  false;
194         }
195       }
196     }
197     return flag;
198   }
199 
200   @Override
201   public boolean filterAllRemaining() throws IOException {
202     int listize = filters.size();
203     for (int i = 0; i < listize; i++) {
204       if (filters.get(i).filterAllRemaining()) {
205         if (operator == Operator.MUST_PASS_ALL) {
206           return true;
207         }
208       } else {
209         if (operator == Operator.MUST_PASS_ONE) {
210           return false;
211         }
212       }
213     }
214     return operator == Operator.MUST_PASS_ONE;
215   }
216 
217   @Override
218   public Cell transformCell(Cell v) throws IOException {
219     // transformCell() is expected to follow an inclusive filterKeyValue() immediately:
220     if (!v.equals(this.referenceKV)) {
221       throw new IllegalStateException("Reference Cell: " + this.referenceKV + " does not match: "
222           + v);
223     }
224     return this.transformedKV;
225   }
226 
227   /**
228    * WARNING: please to not override this method.  Instead override {@link #transformCell(Cell)}.
229    *
230    * When removing this, its body should be placed in transformCell.
231    *
232    * This is for transition from 0.94 -> 0.96
233    */
234   @Deprecated
235   @Override
236   public KeyValue transform(KeyValue v) throws IOException {
237     // transform() is expected to follow an inclusive filterKeyValue() immediately:
238     if (!v.equals(this.referenceKV)) {
239       throw new IllegalStateException(
240           "Reference Cell: " + this.referenceKV + " does not match: " + v);
241      }
242     return KeyValueUtil.ensureKeyValue(this.transformedKV);
243   }
244 
245   
246   @Override
247   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="SF_SWITCH_FALLTHROUGH",
248     justification="Intentional")
249   public ReturnCode filterKeyValue(Cell v) throws IOException {
250     this.referenceKV = v;
251 
252     // Accumulates successive transformation of every filter that includes the Cell:
253     Cell transformed = v;
254 
255     ReturnCode rc = operator == Operator.MUST_PASS_ONE?
256         ReturnCode.SKIP: ReturnCode.INCLUDE;
257     int listize = filters.size();
258     for (int i = 0; i < listize; i++) {
259       Filter filter = filters.get(i);
260       if (operator == Operator.MUST_PASS_ALL) {
261         if (filter.filterAllRemaining()) {
262           return ReturnCode.NEXT_ROW;
263         }
264         ReturnCode code = filter.filterKeyValue(v);
265         switch (code) {
266         // Override INCLUDE and continue to evaluate.
267         case INCLUDE_AND_NEXT_COL:
268           rc = ReturnCode.INCLUDE_AND_NEXT_COL; // FindBugs SF_SWITCH_FALLTHROUGH
269         case INCLUDE:
270           transformed = filter.transformCell(transformed);
271           continue;
272         case SEEK_NEXT_USING_HINT:
273           seekHintFilter = filter;
274           return code;
275         default:
276           return code;
277         }
278       } else if (operator == Operator.MUST_PASS_ONE) {
279         if (filter.filterAllRemaining()) {
280           continue;
281         }
282 
283         switch (filter.filterKeyValue(v)) {
284         case INCLUDE:
285           if (rc != ReturnCode.INCLUDE_AND_NEXT_COL) {
286             rc = ReturnCode.INCLUDE;
287           }
288           transformed = filter.transformCell(transformed);
289           break;
290         case INCLUDE_AND_NEXT_COL:
291           rc = ReturnCode.INCLUDE_AND_NEXT_COL;
292           transformed = filter.transformCell(transformed);
293           // must continue here to evaluate all filters
294           break;
295         case NEXT_ROW:
296           break;
297         case SKIP:
298           break;
299         case NEXT_COL:
300           break;
301         case SEEK_NEXT_USING_HINT:
302           break;
303         default:
304           throw new IllegalStateException("Received code is not valid.");
305         }
306       }
307     }
308 
309     // Save the transformed Cell for transform():
310     this.transformedKV = transformed;
311 
312     return rc;
313   }
314 
315   /**
316    * Filters that never filter by modifying the returned List of Cells can
317    * inherit this implementation that does nothing.
318    *
319    * @inheritDoc
320    */
321   @Override
322   public void filterRowCells(List<Cell> cells) throws IOException {
323     int listize = filters.size();
324     for (int i = 0; i < listize; i++) {
325       filters.get(i).filterRowCells(cells);
326     }
327   }
328 
329   @Override
330   public boolean hasFilterRow() {
331     int listize = filters.size();
332     for (int i = 0; i < listize; i++) {
333       if (filters.get(i).hasFilterRow()) {
334         return true;
335       }
336     }
337     return false;
338   }
339 
340   @Override
341   public boolean filterRow() throws IOException {
342     int listize = filters.size();
343     for (int i = 0; i < listize; i++) {
344       Filter filter = filters.get(i);
345       if (operator == Operator.MUST_PASS_ALL) {
346         if (filter.filterRow()) {
347           return true;
348         }
349       } else if (operator == Operator.MUST_PASS_ONE) {
350         if (!filter.filterRow()) {
351           return false;
352         }
353       }
354     }
355     return  operator == Operator.MUST_PASS_ONE;
356   }
357 
358   /**
359    * @return The filter serialized using pb
360    */
361   public byte[] toByteArray() throws IOException {
362     FilterProtos.FilterList.Builder builder =
363       FilterProtos.FilterList.newBuilder();
364     builder.setOperator(FilterProtos.FilterList.Operator.valueOf(operator.name()));
365     int listize = filters.size();
366     for (int i = 0; i < listize; i++) {
367       builder.addFilters(ProtobufUtil.toFilter(filters.get(i)));
368     }
369     return builder.build().toByteArray();
370   }
371 
372   /**
373    * @param pbBytes A pb serialized {@link FilterList} instance
374    * @return An instance of {@link FilterList} made from <code>bytes</code>
375    * @throws DeserializationException
376    * @see #toByteArray
377    */
378   public static FilterList parseFrom(final byte [] pbBytes)
379   throws DeserializationException {
380     FilterProtos.FilterList proto;
381     try {
382       proto = FilterProtos.FilterList.parseFrom(pbBytes);
383     } catch (InvalidProtocolBufferException e) {
384       throw new DeserializationException(e);
385     }
386 
387     List<Filter> rowFilters = new ArrayList<Filter>(proto.getFiltersCount());
388     try {
389       List<org.apache.hadoop.hbase.protobuf.generated.FilterProtos.Filter> filtersList =
390           proto.getFiltersList();
391       int listSize = filtersList.size();
392       for (int i = 0; i < listSize; i++) {
393         rowFilters.add(ProtobufUtil.toFilter(filtersList.get(i)));
394       }
395     } catch (IOException ioe) {
396       throw new DeserializationException(ioe);
397     }
398     return new FilterList(Operator.valueOf(proto.getOperator().name()),rowFilters);
399   }
400 
401   /**
402    * @param other
403    * @return true if and only if the fields of the filter that are serialized
404    * are equal to the corresponding fields in other.  Used for testing.
405    */
406   boolean areSerializedFieldsEqual(Filter other) {
407     if (other == this) return true;
408     if (!(other instanceof FilterList)) return false;
409 
410     FilterList o = (FilterList)other;
411     return this.getOperator().equals(o.getOperator()) &&
412       ((this.getFilters() == o.getFilters())
413       || this.getFilters().equals(o.getFilters()));
414   }
415 
416   @Override
417   @Deprecated
418   public KeyValue getNextKeyHint(KeyValue currentKV) throws IOException {
419     return KeyValueUtil.ensureKeyValue(getNextCellHint((Cell)currentKV));
420   }
421 
422   @Override
423   public Cell getNextCellHint(Cell currentKV) throws IOException {
424     Cell keyHint = null;
425     if (operator == Operator.MUST_PASS_ALL) {
426       keyHint = seekHintFilter.getNextCellHint(currentKV);
427       return keyHint;
428     }
429 
430     // If any condition can pass, we need to keep the min hint
431     int listize = filters.size();
432     for (int i = 0; i < listize; i++) {
433       Cell curKeyHint = filters.get(i).getNextCellHint(currentKV);
434       if (curKeyHint == null) {
435         // If we ever don't have a hint and this is must-pass-one, then no hint
436         return null;
437       }
438       if (curKeyHint != null) {
439         // If this is the first hint we find, set it
440         if (keyHint == null) {
441           keyHint = curKeyHint;
442           continue;
443         }
444         if (KeyValue.COMPARATOR.compare(keyHint, curKeyHint) > 0) {
445           keyHint = curKeyHint;
446         }
447       }
448     }
449     return keyHint;
450   }
451 
452   @Override
453   public boolean isFamilyEssential(byte[] name) throws IOException {
454     int listize = filters.size();
455     for (int i = 0; i < listize; i++) {
456       if (filters.get(i).isFamilyEssential(name)) {
457         return true;
458       }
459     }
460     return false;
461   }
462 
463   @Override
464   public void setReversed(boolean reversed) {
465     int listize = filters.size();
466     for (int i = 0; i < listize; i++) {
467       filters.get(i).setReversed(reversed);
468     }
469     this.reversed = reversed;
470   }
471 
472   @Override
473   public String toString() {
474     return toString(MAX_LOG_FILTERS);
475   }
476 
477   protected String toString(int maxFilters) {
478     int endIndex = this.filters.size() < maxFilters
479         ? this.filters.size() : maxFilters;
480     return String.format("%s %s (%d/%d): %s",
481         this.getClass().getSimpleName(),
482         this.operator == Operator.MUST_PASS_ALL ? "AND" : "OR",
483         endIndex,
484         this.filters.size(),
485         this.filters.subList(0, endIndex).toString());
486   }
487 }