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   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="SF_SWITCH_FALLTHROUGH",
239     justification="Intentional")
240   public ReturnCode filterKeyValue(Cell v) throws IOException {
241     this.referenceKV = v;
242 
243     // Accumulates successive transformation of every filter that includes the Cell:
244     Cell transformed = v;
245 
246     ReturnCode rc = operator == Operator.MUST_PASS_ONE?
247         ReturnCode.SKIP: ReturnCode.INCLUDE;
248     for (Filter filter : filters) {
249       if (operator == Operator.MUST_PASS_ALL) {
250         if (filter.filterAllRemaining()) {
251           return ReturnCode.NEXT_ROW;
252         }
253         ReturnCode code = filter.filterKeyValue(v);
254         switch (code) {
255         // Override INCLUDE and continue to evaluate.
256         case INCLUDE_AND_NEXT_COL:
257           rc = ReturnCode.INCLUDE_AND_NEXT_COL; // FindBugs SF_SWITCH_FALLTHROUGH
258         case INCLUDE:
259           transformed = filter.transformCell(transformed);
260           continue;
261         case SEEK_NEXT_USING_HINT:
262           seekHintFilter = filter;
263           return code;
264         default:
265           return code;
266         }
267       } else if (operator == Operator.MUST_PASS_ONE) {
268         if (filter.filterAllRemaining()) {
269           continue;
270         }
271 
272         switch (filter.filterKeyValue(v)) {
273         case INCLUDE:
274           if (rc != ReturnCode.INCLUDE_AND_NEXT_COL) {
275             rc = ReturnCode.INCLUDE;
276           }
277           transformed = filter.transformCell(transformed);
278           break;
279         case INCLUDE_AND_NEXT_COL:
280           rc = ReturnCode.INCLUDE_AND_NEXT_COL;
281           transformed = filter.transformCell(transformed);
282           // must continue here to evaluate all filters
283           break;
284         case NEXT_ROW:
285           break;
286         case SKIP:
287           break;
288         case NEXT_COL:
289           break;
290         case SEEK_NEXT_USING_HINT:
291           break;
292         default:
293           throw new IllegalStateException("Received code is not valid.");
294         }
295       }
296     }
297 
298     // Save the transformed Cell for transform():
299     this.transformedKV = transformed;
300 
301     return rc;
302   }
303 
304   /**
305    * Filters that never filter by modifying the returned List of Cells can
306    * inherit this implementation that does nothing.
307    *
308    * @inheritDoc
309    */
310   @Override
311   public void filterRowCells(List<Cell> cells) throws IOException {
312     for (Filter filter : filters) {
313       filter.filterRowCells(cells); 
314     }
315   }
316 
317   @Override
318   public boolean hasFilterRow() {
319     for (Filter filter : filters) {
320       if(filter.hasFilterRow()) {
321     	return true;
322       }
323     }
324     return false;
325   }
326 
327   @Override
328   public boolean filterRow() throws IOException {
329     for (Filter filter : filters) {
330       if (operator == Operator.MUST_PASS_ALL) {
331         if (filter.filterRow()) {
332           return true;
333         }
334       } else if (operator == Operator.MUST_PASS_ONE) {
335         if (!filter.filterRow()) {
336           return false;
337         }
338       }
339     }
340     return  operator == Operator.MUST_PASS_ONE;
341   }
342 
343   /**
344    * @return The filter serialized using pb
345    */
346   public byte[] toByteArray() throws IOException {
347     FilterProtos.FilterList.Builder builder =
348       FilterProtos.FilterList.newBuilder();
349     builder.setOperator(FilterProtos.FilterList.Operator.valueOf(operator.name()));
350     for (Filter filter : filters) {
351       builder.addFilters(ProtobufUtil.toFilter(filter));
352     }
353     return builder.build().toByteArray();
354   }
355 
356   /**
357    * @param pbBytes A pb serialized {@link FilterList} instance
358    * @return An instance of {@link FilterList} made from <code>bytes</code>
359    * @throws DeserializationException
360    * @see #toByteArray
361    */
362   public static FilterList parseFrom(final byte [] pbBytes)
363   throws DeserializationException {
364     FilterProtos.FilterList proto;
365     try {
366       proto = FilterProtos.FilterList.parseFrom(pbBytes);
367     } catch (InvalidProtocolBufferException e) {
368       throw new DeserializationException(e);
369     }
370 
371     List<Filter> rowFilters = new ArrayList<Filter>(proto.getFiltersCount());
372     try {
373       for (FilterProtos.Filter filter : proto.getFiltersList()) {
374         rowFilters.add(ProtobufUtil.toFilter(filter));
375       }
376     } catch (IOException ioe) {
377       throw new DeserializationException(ioe);
378     }
379     return new FilterList(Operator.valueOf(proto.getOperator().name()),rowFilters);
380   }
381 
382   /**
383    * @param other
384    * @return true if and only if the fields of the filter that are serialized
385    * are equal to the corresponding fields in other.  Used for testing.
386    */
387   boolean areSerializedFieldsEqual(Filter other) {
388     if (other == this) return true;
389     if (!(other instanceof FilterList)) return false;
390 
391     FilterList o = (FilterList)other;
392     return this.getOperator().equals(o.getOperator()) &&
393       ((this.getFilters() == o.getFilters())
394       || this.getFilters().equals(o.getFilters()));
395   }
396 
397   @Override
398   @Deprecated
399   public KeyValue getNextKeyHint(KeyValue currentKV) throws IOException {
400     return KeyValueUtil.ensureKeyValue(getNextCellHint((Cell)currentKV));
401   }
402 
403   @Override
404   public Cell getNextCellHint(Cell currentKV) throws IOException {
405     Cell keyHint = null;
406     if (operator == Operator.MUST_PASS_ALL) {
407       keyHint = seekHintFilter.getNextCellHint(currentKV);
408       return keyHint;
409     }
410 
411     // If any condition can pass, we need to keep the min hint
412     for (Filter filter : filters) {
413       Cell curKeyHint = filter.getNextCellHint(currentKV);
414       if (curKeyHint == null) {
415         // If we ever don't have a hint and this is must-pass-one, then no hint
416         return null;
417       }
418       if (curKeyHint != null) {
419         // If this is the first hint we find, set it
420         if (keyHint == null) {
421           keyHint = curKeyHint;
422           continue;
423         }
424         if (KeyValue.COMPARATOR.compare(keyHint, curKeyHint) > 0) {
425           keyHint = curKeyHint;
426         }
427       }
428     }
429     return keyHint;
430   }
431 
432   @Override
433   public boolean isFamilyEssential(byte[] name) throws IOException {
434     for (Filter filter : filters) {
435       if (filter.isFamilyEssential(name)) {
436         return true;
437       }
438     }
439     return false;
440   }
441 
442   @Override
443   public void setReversed(boolean reversed) {
444     for (Filter filter : filters) {
445       filter.setReversed(reversed);
446     }
447     this.reversed = reversed;
448   }
449 
450   @Override
451   public String toString() {
452     return toString(MAX_LOG_FILTERS);
453   }
454 
455   protected String toString(int maxFilters) {
456     int endIndex = this.filters.size() < maxFilters
457         ? this.filters.size() : maxFilters;
458     return String.format("%s %s (%d/%d): %s",
459         this.getClass().getSimpleName(),
460         this.operator == Operator.MUST_PASS_ALL ? "AND" : "OR",
461         endIndex,
462         this.filters.size(),
463         this.filters.subList(0, endIndex).toString());
464   }
465 }