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  
20  package org.apache.hadoop.hbase.filter;
21  
22  import java.io.IOException;
23  import java.util.List;
24  
25  import org.apache.hadoop.hbase.classification.InterfaceAudience;
26  import org.apache.hadoop.hbase.classification.InterfaceStability;
27  import org.apache.hadoop.hbase.Cell;
28  import org.apache.hadoop.hbase.KeyValue;
29  import org.apache.hadoop.hbase.exceptions.DeserializationException;
30  
31  /**
32   * Interface for row and column filters directly applied within the regionserver.
33   *
34   * A filter can expect the following call sequence:
35   * <ul>
36   *   <li> {@link #reset()} : reset the filter state before filtering a new row. </li>
37   *   <li> {@link #filterAllRemaining()}: true means row scan is over; false means keep going. </li>
38   *   <li> {@link #filterRowKey(byte[],int,int)}: true means drop this row; false means include.</li>
39   *   <li> {@link #filterKeyValue(Cell)}: decides whether to include or exclude this KeyValue.
40   *        See {@link ReturnCode}. </li>
41   *   <li> {@link #transform(KeyValue)}: if the KeyValue is included, let the filter transform the
42   *        KeyValue. </li>
43   *   <li> {@link #filterRowCells(List)}: allows direct modification of the final list to be submitted
44   *   <li> {@link #filterRow()}: last chance to drop entire row based on the sequence of
45   *        filter calls. Eg: filter a row if it doesn't contain a specified column. </li>
46   * </ul>
47   *
48   * Filter instances are created one per region/scan.  This abstract class replaces
49   * the old RowFilterInterface.
50   *
51   * When implementing your own filters, consider inheriting {@link FilterBase} to help
52   * you reduce boilerplate.
53   *
54   * @see FilterBase
55   */
56  @InterfaceAudience.Public
57  @InterfaceStability.Stable
58  public abstract class Filter {
59    protected transient boolean reversed;
60    /**
61     * Reset the state of the filter between rows.
62     * 
63     * Concrete implementers can signal a failure condition in their code by throwing an
64     * {@link IOException}.
65     * 
66     * @throws IOException in case an I/O or an filter specific failure needs to be signaled.
67     */
68    abstract public void reset() throws IOException;
69  
70    /**
71     * Filters a row based on the row key. If this returns true, the entire row will be excluded. If
72     * false, each KeyValue in the row will be passed to {@link #filterKeyValue(Cell)} below.
73     * 
74     * Concrete implementers can signal a failure condition in their code by throwing an
75     * {@link IOException}.
76     * 
77     * @param buffer buffer containing row key
78     * @param offset offset into buffer where row key starts
79     * @param length length of the row key
80     * @return true, remove entire row, false, include the row (maybe).
81     * @throws IOException in case an I/O or an filter specific failure needs to be signaled.
82     */
83    abstract public boolean filterRowKey(byte[] buffer, int offset, int length) throws IOException;
84  
85    /**
86     * If this returns true, the scan will terminate.
87     * 
88     * Concrete implementers can signal a failure condition in their code by throwing an
89     * {@link IOException}.
90     * 
91     * @return true to end scan, false to continue.
92     * @throws IOException in case an I/O or an filter specific failure needs to be signaled.
93     */
94    abstract public boolean filterAllRemaining() throws IOException;
95  
96    /**
97     * A way to filter based on the column family, column qualifier and/or the column value. Return
98     * code is described below. This allows filters to filter only certain number of columns, then
99     * terminate without matching ever column.
100    * 
101    * If filterRowKey returns true, filterKeyValue needs to be consistent with it.
102    * 
103    * filterKeyValue can assume that filterRowKey has already been called for the row.
104    * 
105    * If your filter returns <code>ReturnCode.NEXT_ROW</code>, it should return
106    * <code>ReturnCode.NEXT_ROW</code> until {@link #reset()} is called just in case the caller calls
107    * for the next row.
108    * 
109    * Concrete implementers can signal a failure condition in their code by throwing an
110    * {@link IOException}.
111    * 
112    * @param v the Cell in question
113    * @return code as described below
114    * @throws IOException in case an I/O or an filter specific failure needs to be signaled.
115    * @see Filter.ReturnCode
116    */
117   abstract public ReturnCode filterKeyValue(final Cell v) throws IOException;
118 
119   /**
120    * Give the filter a chance to transform the passed KeyValue. If the Cell is changed a new
121    * Cell object must be returned.
122    * 
123    * @see org.apache.hadoop.hbase.KeyValue#shallowCopy()
124    *      The transformed KeyValue is what is eventually returned to the client. Most filters will
125    *      return the passed KeyValue unchanged.
126    * @see org.apache.hadoop.hbase.filter.KeyOnlyFilter#transform(KeyValue) for an example of a
127    *      transformation.
128    * 
129    *      Concrete implementers can signal a failure condition in their code by throwing an
130    *      {@link IOException}.
131    * 
132    * @param v the KeyValue in question
133    * @return the changed KeyValue
134    * @throws IOException in case an I/O or an filter specific failure needs to be signaled.
135    */
136   abstract public Cell transformCell(final Cell v) throws IOException;
137 
138   /**
139    * WARNING: please to not override this method.  Instead override {@link #transformCell(Cell)}.
140    * This is for transition from 0.94 -&gt; 0.96
141    **/
142   @Deprecated // use Cell transformCell(final Cell)
143   abstract public KeyValue transform(final KeyValue currentKV) throws IOException;
144  
145   
146   /**
147    * Return codes for filterValue().
148    */
149   @InterfaceAudience.Public
150   @InterfaceStability.Stable
151   public enum ReturnCode {
152     /**
153      * Include the Cell
154      */
155     INCLUDE,
156     /**
157      * Include the Cell and seek to the next column skipping older versions.
158      */
159     INCLUDE_AND_NEXT_COL,
160     /**
161      * Skip this Cell
162      */
163     SKIP,
164     /**
165      * Skip this column. Go to the next column in this row.
166      */
167     NEXT_COL,
168     /**
169      * Done with columns, skip to next row. Note that filterRow() will
170      * still be called.
171      */
172     NEXT_ROW,
173     /**
174      * Seek to next key which is given as hint by the filter.
175      */
176     SEEK_NEXT_USING_HINT,
177 }
178 
179   /**
180    * Chance to alter the list of Cells to be submitted. Modifications to the list will carry on
181    * 
182    * Concrete implementers can signal a failure condition in their code by throwing an
183    * {@link IOException}.
184    * 
185    * @param kvs the list of Cells to be filtered
186    * @throws IOException in case an I/O or an filter specific failure needs to be signaled.
187    */
188   abstract public void filterRowCells(List<Cell> kvs) throws IOException;
189 
190   /**
191    * Primarily used to check for conflicts with scans(such as scans that do not read a full row at a
192    * time).
193    * 
194    * @return True if this filter actively uses filterRowCells(List) or filterRow().
195    */
196   abstract public boolean hasFilterRow();
197 
198   /**
199    * Last chance to veto row based on previous {@link #filterKeyValue(Cell)} calls. The filter
200    * needs to retain state then return a particular value for this call if they wish to exclude a
201    * row if a certain column is missing (for example).
202    * 
203    * Concrete implementers can signal a failure condition in their code by throwing an
204    * {@link IOException}.
205    * 
206    * @return true to exclude row, false to include row.
207    * @throws IOException in case an I/O or an filter specific failure needs to be signaled.
208    */
209   abstract public boolean filterRow() throws IOException;
210 
211   /**
212    * @param currentKV
213    * @return KeyValue which must be next seeked. return null if the filter is not sure which key to
214    *         seek to next.
215    * @throws IOException
216    * Function is Deprecated. Use {@link #getNextCellHint(Cell)} instead.
217    */
218   @Deprecated
219   abstract public KeyValue getNextKeyHint(final KeyValue currentKV) throws IOException;
220 
221   /**
222    * If the filter returns the match code SEEK_NEXT_USING_HINT, then it should also tell which is
223    * the next key it must seek to. After receiving the match code SEEK_NEXT_USING_HINT, the
224    * QueryMatcher would call this function to find out which key it must next seek to.
225    * 
226    * Concrete implementers can signal a failure condition in their code by throwing an
227    * {@link IOException}.
228    * 
229    * @return KeyValue which must be next seeked. return null if the filter is not sure which key to
230    *         seek to next.
231    * @throws IOException in case an I/O or an filter specific failure needs to be signaled.
232    */
233   abstract public Cell getNextCellHint(final Cell currentKV) throws IOException;
234 
235   /**
236    * Check that given column family is essential for filter to check row. Most filters always return
237    * true here. But some could have more sophisticated logic which could significantly reduce
238    * scanning process by not even touching columns until we are 100% sure that it's data is needed
239    * in result.
240    * 
241    * Concrete implementers can signal a failure condition in their code by throwing an
242    * {@link IOException}.
243    * 
244    * @throws IOException in case an I/O or an filter specific failure needs to be signaled.
245    */
246   abstract public boolean isFamilyEssential(byte[] name) throws IOException;
247 
248   /**
249    * TODO: JAVADOC
250    * 
251    * Concrete implementers can signal a failure condition in their code by throwing an
252    * {@link IOException}.
253    * 
254    * @return The filter serialized using pb
255    * @throws IOException in case an I/O or an filter specific failure needs to be signaled.
256    */
257   abstract public byte[] toByteArray() throws IOException;
258 
259   /**
260    * 
261    * Concrete implementers can signal a failure condition in their code by throwing an
262    * {@link IOException}.
263    * 
264    * @param pbBytes A pb serialized {@link Filter} instance
265    * @return An instance of {@link Filter} made from <code>bytes</code>
266    * @throws DeserializationException
267    * @see #toByteArray
268    */
269   public static Filter parseFrom(final byte [] pbBytes) throws DeserializationException {
270     throw new DeserializationException(
271       "parseFrom called on base Filter, but should be called on derived type");
272   }
273 
274   /**
275    * Concrete implementers can signal a failure condition in their code by throwing an
276    * {@link IOException}.
277    * 
278    * @param other
279    * @return true if and only if the fields of the filter that are serialized are equal to the
280    *         corresponding fields in other. Used for testing.
281    * @throws IOException in case an I/O or an filter specific failure needs to be signaled.
282    */
283   abstract boolean areSerializedFieldsEqual(Filter other);
284 
285   /**
286    * alter the reversed scan flag
287    * @param reversed flag
288    */
289   public void setReversed(boolean reversed) {
290     this.reversed = reversed;
291   }
292 
293   public boolean isReversed() {
294     return this.reversed;
295   }
296 }