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