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