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