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