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.ArrayList;
24  
25  import org.apache.hadoop.hbase.util.ByteStringer;
26  import org.apache.commons.logging.Log;
27  import org.apache.commons.logging.LogFactory;
28  import org.apache.hadoop.hbase.classification.InterfaceAudience;
29  import org.apache.hadoop.hbase.classification.InterfaceStability;
30  import org.apache.hadoop.hbase.Cell;
31  import org.apache.hadoop.hbase.CellUtil;
32  import org.apache.hadoop.hbase.exceptions.DeserializationException;
33  import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
34  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
35  import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
36  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
37  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CompareType;
38  import org.apache.hadoop.hbase.util.Bytes;
39  
40  import com.google.common.base.Preconditions;
41  import com.google.protobuf.InvalidProtocolBufferException;
42  
43  /**
44   * This filter is used to filter cells based on value. It takes a {@link CompareFilter.CompareOp}
45   * operator (equal, greater, not equal, etc), and either a byte [] value or
46   * a ByteArrayComparable.
47   * <p>
48   * If we have a byte [] value then we just do a lexicographic compare. For
49   * example, if passed value is 'b' and cell has 'a' and the compare operator
50   * is LESS, then we will filter out this cell (return true).  If this is not
51   * sufficient (eg you want to deserialize a long and then compare it to a fixed
52   * long value), then you can pass in your own comparator instead.
53   * <p>
54   * You must also specify a family and qualifier.  Only the value of this column
55   * will be tested. When using this filter on a 
56   * {@link org.apache.hadoop.hbase.CellScanner} with specified
57   * inputs, the column to be tested should also be added as input (otherwise
58   * the filter will regard the column as missing).
59   * <p>
60   * To prevent the entire row from being emitted if the column is not found
61   * on a row, use {@link #setFilterIfMissing}.
62   * Otherwise, if the column is found, the entire row will be emitted only if
63   * the value passes.  If the value fails, the row will be filtered out.
64   * <p>
65   * In order to test values of previous versions (timestamps), set
66   * {@link #setLatestVersionOnly} to false. The default is true, meaning that
67   * only the latest version's value is tested and all previous versions are ignored.
68   * <p>
69   * To filter based on the value of all scanned columns, use {@link ValueFilter}.
70   */
71  @InterfaceAudience.Public
72  @InterfaceStability.Stable
73  public class SingleColumnValueFilter extends FilterBase {
74    static final Log LOG = LogFactory.getLog(SingleColumnValueFilter.class);
75  
76    protected byte [] columnFamily;
77    protected byte [] columnQualifier;
78    protected CompareOp compareOp;
79    protected ByteArrayComparable comparator;
80    protected boolean foundColumn = false;
81    protected boolean matchedColumn = false;
82    protected boolean filterIfMissing = false;
83    protected boolean latestVersionOnly = true;
84  
85    /**
86     * Constructor for binary compare of the value of a single column.  If the
87     * column is found and the condition passes, all columns of the row will be
88     * emitted.  If the condition fails, the row will not be emitted.
89     * <p>
90     * Use the filterIfColumnMissing flag to set whether the rest of the columns
91     * in a row will be emitted if the specified column to check is not found in
92     * the row.
93     *
94     * @param family name of column family
95     * @param qualifier name of column qualifier
96     * @param compareOp operator
97     * @param value value to compare column values against
98     */
99    public SingleColumnValueFilter(final byte [] family, final byte [] qualifier,
100       final CompareOp compareOp, final byte[] value) {
101     this(family, qualifier, compareOp, new BinaryComparator(value));
102   }
103 
104   /**
105    * Constructor for binary compare of the value of a single column.  If the
106    * column is found and the condition passes, all columns of the row will be
107    * emitted.  If the condition fails, the row will not be emitted.
108    * <p>
109    * Use the filterIfColumnMissing flag to set whether the rest of the columns
110    * in a row will be emitted if the specified column to check is not found in
111    * the row.
112    *
113    * @param family name of column family
114    * @param qualifier name of column qualifier
115    * @param compareOp operator
116    * @param comparator Comparator to use.
117    */
118   public SingleColumnValueFilter(final byte [] family, final byte [] qualifier,
119       final CompareOp compareOp, final ByteArrayComparable comparator) {
120     this.columnFamily = family;
121     this.columnQualifier = qualifier;
122     this.compareOp = compareOp;
123     this.comparator = comparator;
124   }
125 
126   /**
127    * Constructor for protobuf deserialization only.
128    * @param family
129    * @param qualifier
130    * @param compareOp
131    * @param comparator
132    * @param filterIfMissing
133    * @param latestVersionOnly
134    */
135   protected SingleColumnValueFilter(final byte[] family, final byte[] qualifier,
136       final CompareOp compareOp, ByteArrayComparable comparator, final boolean filterIfMissing,
137       final boolean latestVersionOnly) {
138     this(family, qualifier, compareOp, comparator);
139     this.filterIfMissing = filterIfMissing;
140     this.latestVersionOnly = latestVersionOnly;
141   }
142 
143   /**
144    * @return operator
145    */
146   public CompareOp getOperator() {
147     return compareOp;
148   }
149 
150   /**
151    * @return the comparator
152    */
153   public ByteArrayComparable getComparator() {
154     return comparator;
155   }
156 
157   /**
158    * @return the family
159    */
160   public byte[] getFamily() {
161     return columnFamily;
162   }
163 
164   /**
165    * @return the qualifier
166    */
167   public byte[] getQualifier() {
168     return columnQualifier;
169   }
170 
171   @Override
172   public ReturnCode filterKeyValue(Cell c) {
173     if (this.matchedColumn) {
174       // We already found and matched the single column, all keys now pass
175       return ReturnCode.INCLUDE;
176     } else if (this.latestVersionOnly && this.foundColumn) {
177       // We found but did not match the single column, skip to next row
178       return ReturnCode.NEXT_ROW;
179     }
180     if (!CellUtil.matchingColumn(c, this.columnFamily, this.columnQualifier)) {
181       return ReturnCode.INCLUDE;
182     }
183     foundColumn = true;
184     if (filterColumnValue(c.getValueArray(), c.getValueOffset(), c.getValueLength())) {
185       return this.latestVersionOnly? ReturnCode.NEXT_ROW: ReturnCode.INCLUDE;
186     }
187     this.matchedColumn = true;
188     return ReturnCode.INCLUDE;
189   }
190 
191   // Override here explicitly as the method in super class FilterBase might do a KeyValue recreate.
192   // See HBASE-12068
193   @Override
194   public Cell transformCell(Cell v) {
195     return v;
196   }
197 
198   private boolean filterColumnValue(final byte [] data, final int offset,
199       final int length) {
200     int compareResult = this.comparator.compareTo(data, offset, length);
201     switch (this.compareOp) {
202     case LESS:
203       return compareResult <= 0;
204     case LESS_OR_EQUAL:
205       return compareResult < 0;
206     case EQUAL:
207       return compareResult != 0;
208     case NOT_EQUAL:
209       return compareResult == 0;
210     case GREATER_OR_EQUAL:
211       return compareResult > 0;
212     case GREATER:
213       return compareResult >= 0;
214     default:
215       throw new RuntimeException("Unknown Compare op " + compareOp.name());
216     }
217   }
218 
219   public boolean filterRow() {
220     // If column was found, return false if it was matched, true if it was not
221     // If column not found, return true if we filter if missing, false if not
222     return this.foundColumn? !this.matchedColumn: this.filterIfMissing;
223   }
224   
225   public boolean hasFilterRow() {
226     return true;
227   }
228 
229   public void reset() {
230     foundColumn = false;
231     matchedColumn = false;
232   }
233 
234   /**
235    * Get whether entire row should be filtered if column is not found.
236    * @return true if row should be skipped if column not found, false if row
237    * should be let through anyways
238    */
239   public boolean getFilterIfMissing() {
240     return filterIfMissing;
241   }
242 
243   /**
244    * Set whether entire row should be filtered if column is not found.
245    * <p>
246    * If true, the entire row will be skipped if the column is not found.
247    * <p>
248    * If false, the row will pass if the column is not found.  This is default.
249    * @param filterIfMissing flag
250    */
251   public void setFilterIfMissing(boolean filterIfMissing) {
252     this.filterIfMissing = filterIfMissing;
253   }
254 
255   /**
256    * Get whether only the latest version of the column value should be compared.
257    * If true, the row will be returned if only the latest version of the column
258    * value matches. If false, the row will be returned if any version of the
259    * column value matches. The default is true.
260    * @return return value
261    */
262   public boolean getLatestVersionOnly() {
263     return latestVersionOnly;
264   }
265 
266   /**
267    * Set whether only the latest version of the column value should be compared.
268    * If true, the row will be returned if only the latest version of the column
269    * value matches. If false, the row will be returned if any version of the
270    * column value matches. The default is true.
271    * @param latestVersionOnly flag
272    */
273   public void setLatestVersionOnly(boolean latestVersionOnly) {
274     this.latestVersionOnly = latestVersionOnly;
275   }
276 
277   public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
278     Preconditions.checkArgument(filterArguments.size() == 4 || filterArguments.size() == 6,
279                                 "Expected 4 or 6 but got: %s", filterArguments.size());
280     byte [] family = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0));
281     byte [] qualifier = ParseFilter.removeQuotesFromByteArray(filterArguments.get(1));
282     CompareOp compareOp = ParseFilter.createCompareOp(filterArguments.get(2));
283     ByteArrayComparable comparator = ParseFilter.createComparator(
284       ParseFilter.removeQuotesFromByteArray(filterArguments.get(3)));
285 
286     if (comparator instanceof RegexStringComparator ||
287         comparator instanceof SubstringComparator) {
288       if (compareOp != CompareOp.EQUAL &&
289           compareOp != CompareOp.NOT_EQUAL) {
290         throw new IllegalArgumentException ("A regexstring comparator and substring comparator " +
291                                             "can only be used with EQUAL and NOT_EQUAL");
292       }
293     }
294 
295     SingleColumnValueFilter filter = new SingleColumnValueFilter(family, qualifier,
296                                                                  compareOp, comparator);
297 
298     if (filterArguments.size() == 6) {
299       boolean filterIfMissing = ParseFilter.convertByteArrayToBoolean(filterArguments.get(4));
300       boolean latestVersionOnly = ParseFilter.convertByteArrayToBoolean(filterArguments.get(5));
301       filter.setFilterIfMissing(filterIfMissing);
302       filter.setLatestVersionOnly(latestVersionOnly);
303     }
304     return filter;
305   }
306 
307   FilterProtos.SingleColumnValueFilter convert() {
308     FilterProtos.SingleColumnValueFilter.Builder builder =
309       FilterProtos.SingleColumnValueFilter.newBuilder();
310     if (this.columnFamily != null) {
311       builder.setColumnFamily(ByteStringer.wrap(this.columnFamily));
312     }
313     if (this.columnQualifier != null) {
314       builder.setColumnQualifier(ByteStringer.wrap(this.columnQualifier));
315     }
316     HBaseProtos.CompareType compareOp = CompareType.valueOf(this.compareOp.name());
317     builder.setCompareOp(compareOp);
318     builder.setComparator(ProtobufUtil.toComparator(this.comparator));
319     builder.setFilterIfMissing(this.filterIfMissing);
320     builder.setLatestVersionOnly(this.latestVersionOnly);
321 
322     return builder.build();
323   }
324 
325   /**
326    * @return The filter serialized using pb
327    */
328   public byte [] toByteArray() {
329     return convert().toByteArray();
330   }
331 
332   /**
333    * @param pbBytes A pb serialized {@link SingleColumnValueFilter} instance
334    * @return An instance of {@link SingleColumnValueFilter} made from <code>bytes</code>
335    * @throws org.apache.hadoop.hbase.exceptions.DeserializationException
336    * @see #toByteArray
337    */
338   public static SingleColumnValueFilter parseFrom(final byte [] pbBytes)
339   throws DeserializationException {
340     FilterProtos.SingleColumnValueFilter proto;
341     try {
342       proto = FilterProtos.SingleColumnValueFilter.parseFrom(pbBytes);
343     } catch (InvalidProtocolBufferException e) {
344       throw new DeserializationException(e);
345     }
346 
347     final CompareOp compareOp =
348       CompareOp.valueOf(proto.getCompareOp().name());
349     final ByteArrayComparable comparator;
350     try {
351       comparator = ProtobufUtil.toComparator(proto.getComparator());
352     } catch (IOException ioe) {
353       throw new DeserializationException(ioe);
354     }
355 
356     return new SingleColumnValueFilter(proto.hasColumnFamily() ? proto.getColumnFamily()
357         .toByteArray() : null, proto.hasColumnQualifier() ? proto.getColumnQualifier()
358         .toByteArray() : null, compareOp, comparator, proto.getFilterIfMissing(), proto
359         .getLatestVersionOnly());
360   }
361 
362   /**
363    * @param other
364    * @return true if and only if the fields of the filter that are serialized
365    * are equal to the corresponding fields in other.  Used for testing.
366    */
367   boolean areSerializedFieldsEqual(Filter o) {
368     if (o == this) return true;
369     if (!(o instanceof SingleColumnValueFilter)) return false;
370 
371     SingleColumnValueFilter other = (SingleColumnValueFilter)o;
372     return Bytes.equals(this.getFamily(), other.getFamily())
373       && Bytes.equals(this.getQualifier(), other.getQualifier())
374       && this.compareOp.equals(other.compareOp)
375       && this.getComparator().areSerializedFieldsEqual(other.getComparator())
376       && this.getFilterIfMissing() == other.getFilterIfMissing()
377       && this.getLatestVersionOnly() == other.getLatestVersionOnly();
378   }
379 
380   /**
381    * The only CF this filter needs is given column family. So, it's the only essential
382    * column in whole scan. If filterIfMissing == false, all families are essential,
383    * because of possibility of skipping the rows without any data in filtered CF.
384    */
385   public boolean isFamilyEssential(byte[] name) {
386     return !this.filterIfMissing || Bytes.equals(name, this.columnFamily);
387   }
388 
389   @Override
390   public String toString() {
391     return String.format("%s (%s, %s, %s, %s)",
392         this.getClass().getSimpleName(), Bytes.toStringBinary(this.columnFamily),
393         Bytes.toStringBinary(this.columnQualifier), this.compareOp.name(),
394         Bytes.toStringBinary(this.comparator.getValue()));
395   }
396 }