001/**
002 *
003 * Licensed to the Apache Software Foundation (ASF) under one
004 * or more contributor license agreements.  See the NOTICE file
005 * distributed with this work for additional information
006 * regarding copyright ownership.  The ASF licenses this file
007 * to you under the Apache License, Version 2.0 (the
008 * "License"); you may not use this file except in compliance
009 * with the License.  You may obtain a copy of the License at
010 *
011 *     http://www.apache.org/licenses/LICENSE-2.0
012 *
013 * Unless required by applicable law or agreed to in writing, software
014 * distributed under the License is distributed on an "AS IS" BASIS,
015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
016 * See the License for the specific language governing permissions and
017 * limitations under the License.
018 */
019
020package org.apache.hadoop.hbase.filter;
021
022import java.io.IOException;
023import java.util.ArrayList;
024
025import org.apache.hadoop.hbase.Cell;
026import org.apache.hadoop.hbase.CellUtil;
027import org.apache.hadoop.hbase.CompareOperator;
028import org.apache.hadoop.hbase.PrivateCellUtil;
029import org.apache.yetus.audience.InterfaceAudience;
030import org.apache.hadoop.hbase.exceptions.DeserializationException;
031import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
032import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
033import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
034import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
035import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
036import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
037import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.CompareType;
038import org.apache.hadoop.hbase.util.Bytes;
039
040import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
041
042/**
043 * This filter is used to filter cells based on value. It takes a {@link CompareFilter.CompareOp}
044 * operator (equal, greater, not equal, etc), and either a byte [] value or
045 * a ByteArrayComparable.
046 * <p>
047 * If we have a byte [] value then we just do a lexicographic compare. For
048 * example, if passed value is 'b' and cell has 'a' and the compare operator
049 * is LESS, then we will filter out this cell (return true).  If this is not
050 * sufficient (eg you want to deserialize a long and then compare it to a fixed
051 * long value), then you can pass in your own comparator instead.
052 * <p>
053 * You must also specify a family and qualifier.  Only the value of this column
054 * will be tested. When using this filter on a 
055 * {@link org.apache.hadoop.hbase.CellScanner} with specified
056 * inputs, the column to be tested should also be added as input (otherwise
057 * the filter will regard the column as missing).
058 * <p>
059 * To prevent the entire row from being emitted if the column is not found
060 * on a row, use {@link #setFilterIfMissing}.
061 * Otherwise, if the column is found, the entire row will be emitted only if
062 * the value passes.  If the value fails, the row will be filtered out.
063 * <p>
064 * In order to test values of previous versions (timestamps), set
065 * {@link #setLatestVersionOnly} to false. The default is true, meaning that
066 * only the latest version's value is tested and all previous versions are ignored.
067 * <p>
068 * To filter based on the value of all scanned columns, use {@link ValueFilter}.
069 */
070@InterfaceAudience.Public
071public class SingleColumnValueFilter extends FilterBase {
072
073  protected byte [] columnFamily;
074  protected byte [] columnQualifier;
075  protected CompareOperator op;
076  protected org.apache.hadoop.hbase.filter.ByteArrayComparable comparator;
077  protected boolean foundColumn = false;
078  protected boolean matchedColumn = false;
079  protected boolean filterIfMissing = false;
080  protected boolean latestVersionOnly = true;
081
082  /**
083   * Constructor for binary compare of the value of a single column.  If the
084   * column is found and the condition passes, all columns of the row will be
085   * emitted.  If the condition fails, the row will not be emitted.
086   * <p>
087   * Use the filterIfColumnMissing flag to set whether the rest of the columns
088   * in a row will be emitted if the specified column to check is not found in
089   * the row.
090   *
091   * @param family name of column family
092   * @param qualifier name of column qualifier
093   * @param compareOp operator
094   * @param value value to compare column values against
095   * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use
096   * {@link #SingleColumnValueFilter(byte[], byte[], CompareOperator, byte[])} instead.
097   */
098  @Deprecated
099  public SingleColumnValueFilter(final byte [] family, final byte [] qualifier,
100      final CompareOp compareOp, final byte[] value) {
101    this(family, qualifier, CompareOperator.valueOf(compareOp.name()),
102      new org.apache.hadoop.hbase.filter.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 op operator
117   * @param value value to compare column values against
118   */
119  public SingleColumnValueFilter(final byte [] family, final byte [] qualifier,
120                                 final CompareOperator op, final byte[] value) {
121    this(family, qualifier, op,
122      new org.apache.hadoop.hbase.filter.BinaryComparator(value));
123  }
124
125  /**
126   * Constructor for binary compare of the value of a single column.  If the
127   * column is found and the condition passes, all columns of the row will be
128   * emitted.  If the condition fails, the row will not be emitted.
129   * <p>
130   * Use the filterIfColumnMissing flag to set whether the rest of the columns
131   * in a row will be emitted if the specified column to check is not found in
132   * the row.
133   *
134   * @param family name of column family
135   * @param qualifier name of column qualifier
136   * @param compareOp operator
137   * @param comparator Comparator to use.
138   * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use
139   * {@link #SingleColumnValueFilter(byte[], byte[], CompareOperator, ByteArrayComparable)} instead.
140   */
141  @Deprecated
142  public SingleColumnValueFilter(final byte [] family, final byte [] qualifier,
143      final CompareOp compareOp,
144      final org.apache.hadoop.hbase.filter.ByteArrayComparable comparator) {
145    this(family, qualifier, CompareOperator.valueOf(compareOp.name()), comparator);
146  }
147
148  /**
149   * Constructor for binary compare of the value of a single column.  If the
150   * column is found and the condition passes, all columns of the row will be
151   * emitted.  If the condition fails, the row will not be emitted.
152   * <p>
153   * Use the filterIfColumnMissing flag to set whether the rest of the columns
154   * in a row will be emitted if the specified column to check is not found in
155   * the row.
156   *
157   * @param family name of column family
158   * @param qualifier name of column qualifier
159   * @param op operator
160   * @param comparator Comparator to use.
161   */
162  public SingleColumnValueFilter(final byte [] family, final byte [] qualifier,
163      final CompareOperator op,
164      final org.apache.hadoop.hbase.filter.ByteArrayComparable comparator) {
165    this.columnFamily = family;
166    this.columnQualifier = qualifier;
167    this.op = op;
168    this.comparator = comparator;
169  }
170
171  /**
172   * Constructor for protobuf deserialization only.
173   * @param family
174   * @param qualifier
175   * @param compareOp
176   * @param comparator
177   * @param filterIfMissing
178   * @param latestVersionOnly
179   * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use
180   * {@link #SingleColumnValueFilter(byte[], byte[], CompareOperator, ByteArrayComparable,
181   *   boolean, boolean)} instead.
182   */
183  @Deprecated
184  protected SingleColumnValueFilter(final byte[] family, final byte[] qualifier,
185      final CompareOp compareOp, org.apache.hadoop.hbase.filter.ByteArrayComparable comparator,
186      final boolean filterIfMissing,
187      final boolean latestVersionOnly) {
188    this(family, qualifier, CompareOperator.valueOf(compareOp.name()), comparator, filterIfMissing,
189      latestVersionOnly);
190  }
191
192  /**
193   * Constructor for protobuf deserialization only.
194   * @param family
195   * @param qualifier
196   * @param op
197   * @param comparator
198   * @param filterIfMissing
199   * @param latestVersionOnly
200   */
201  protected SingleColumnValueFilter(final byte[] family, final byte[] qualifier,
202      final CompareOperator op, org.apache.hadoop.hbase.filter.ByteArrayComparable comparator,
203       final boolean filterIfMissing, final boolean latestVersionOnly) {
204    this(family, qualifier, op, comparator);
205    this.filterIfMissing = filterIfMissing;
206    this.latestVersionOnly = latestVersionOnly;
207  }
208
209  /**
210   * @return operator
211   * @deprecated  since 2.0.0. Will be removed in 3.0.0. Use {@link #getCompareOperator()} instead.
212   */
213  @Deprecated
214  public CompareOp getOperator() {
215    return CompareOp.valueOf(op.name());
216  }
217
218  public CompareOperator getCompareOperator() {
219    return op;
220  }
221
222  /**
223   * @return the comparator
224   */
225  public org.apache.hadoop.hbase.filter.ByteArrayComparable getComparator() {
226    return comparator;
227  }
228
229  /**
230   * @return the family
231   */
232  public byte[] getFamily() {
233    return columnFamily;
234  }
235
236  /**
237   * @return the qualifier
238   */
239  public byte[] getQualifier() {
240    return columnQualifier;
241  }
242
243  @Override
244  public boolean filterRowKey(Cell cell) throws IOException {
245    // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
246    return false;
247  }
248
249  @Deprecated
250  @Override
251  public ReturnCode filterKeyValue(final Cell c) {
252    return filterCell(c);
253  }
254
255  @Override
256  public ReturnCode filterCell(final Cell c) {
257    // System.out.println("REMOVE KEY=" + keyValue.toString() + ", value=" + Bytes.toString(keyValue.getValue()));
258    if (this.matchedColumn) {
259      // We already found and matched the single column, all keys now pass
260      return ReturnCode.INCLUDE;
261    } else if (this.latestVersionOnly && this.foundColumn) {
262      // We found but did not match the single column, skip to next row
263      return ReturnCode.NEXT_ROW;
264    }
265    if (!CellUtil.matchingColumn(c, this.columnFamily, this.columnQualifier)) {
266      return ReturnCode.INCLUDE;
267    }
268    foundColumn = true;
269    if (filterColumnValue(c)) {
270      return this.latestVersionOnly? ReturnCode.NEXT_ROW: ReturnCode.INCLUDE;
271    }
272    this.matchedColumn = true;
273    return ReturnCode.INCLUDE;
274  }
275
276  private boolean filterColumnValue(final Cell cell) {
277    int compareResult = PrivateCellUtil.compareValue(cell, this.comparator);
278    return CompareFilter.compare(this.op, compareResult);
279  }
280
281  @Override
282  public boolean filterRow() {
283    // If column was found, return false if it was matched, true if it was not
284    // If column not found, return true if we filter if missing, false if not
285    return this.foundColumn? !this.matchedColumn: this.filterIfMissing;
286  }
287  
288  @Override
289  public boolean hasFilterRow() {
290    return true;
291  }
292
293  @Override
294  public void reset() {
295    foundColumn = false;
296    matchedColumn = false;
297  }
298
299  /**
300   * Get whether entire row should be filtered if column is not found.
301   * @return true if row should be skipped if column not found, false if row
302   * should be let through anyways
303   */
304  public boolean getFilterIfMissing() {
305    return filterIfMissing;
306  }
307
308  /**
309   * Set whether entire row should be filtered if column is not found.
310   * <p>
311   * If true, the entire row will be skipped if the column is not found.
312   * <p>
313   * If false, the row will pass if the column is not found.  This is default.
314   * @param filterIfMissing flag
315   */
316  public void setFilterIfMissing(boolean filterIfMissing) {
317    this.filterIfMissing = filterIfMissing;
318  }
319
320  /**
321   * Get whether only the latest version of the column value should be compared.
322   * If true, the row will be returned if only the latest version of the column
323   * value matches. If false, the row will be returned if any version of the
324   * column value matches. The default is true.
325   * @return return value
326   */
327  public boolean getLatestVersionOnly() {
328    return latestVersionOnly;
329  }
330
331  /**
332   * Set whether only the latest version of the column value should be compared.
333   * If true, the row will be returned if only the latest version of the column
334   * value matches. If false, the row will be returned if any version of the
335   * column value matches. The default is true.
336   * @param latestVersionOnly flag
337   */
338  public void setLatestVersionOnly(boolean latestVersionOnly) {
339    this.latestVersionOnly = latestVersionOnly;
340  }
341
342  public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
343    Preconditions.checkArgument(filterArguments.size() == 4 || filterArguments.size() == 6,
344                                "Expected 4 or 6 but got: %s", filterArguments.size());
345    byte [] family = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0));
346    byte [] qualifier = ParseFilter.removeQuotesFromByteArray(filterArguments.get(1));
347    CompareOperator op = ParseFilter.createCompareOperator(filterArguments.get(2));
348    org.apache.hadoop.hbase.filter.ByteArrayComparable comparator = ParseFilter.createComparator(
349      ParseFilter.removeQuotesFromByteArray(filterArguments.get(3)));
350
351    if (comparator instanceof RegexStringComparator ||
352        comparator instanceof SubstringComparator) {
353      if (op != CompareOperator.EQUAL &&
354          op != CompareOperator.NOT_EQUAL) {
355        throw new IllegalArgumentException ("A regexstring comparator and substring comparator " +
356                                            "can only be used with EQUAL and NOT_EQUAL");
357      }
358    }
359
360    SingleColumnValueFilter filter = new SingleColumnValueFilter(family, qualifier,
361                                                                 op, comparator);
362
363    if (filterArguments.size() == 6) {
364      boolean filterIfMissing = ParseFilter.convertByteArrayToBoolean(filterArguments.get(4));
365      boolean latestVersionOnly = ParseFilter.convertByteArrayToBoolean(filterArguments.get(5));
366      filter.setFilterIfMissing(filterIfMissing);
367      filter.setLatestVersionOnly(latestVersionOnly);
368    }
369    return filter;
370  }
371
372  FilterProtos.SingleColumnValueFilter convert() {
373    FilterProtos.SingleColumnValueFilter.Builder builder =
374      FilterProtos.SingleColumnValueFilter.newBuilder();
375    if (this.columnFamily != null) {
376      builder.setColumnFamily(UnsafeByteOperations.unsafeWrap(this.columnFamily));
377    }
378    if (this.columnQualifier != null) {
379      builder.setColumnQualifier(UnsafeByteOperations.unsafeWrap(this.columnQualifier));
380    }
381    HBaseProtos.CompareType compareOp = CompareType.valueOf(this.op.name());
382    builder.setCompareOp(compareOp);
383    builder.setComparator(ProtobufUtil.toComparator(this.comparator));
384    builder.setFilterIfMissing(this.filterIfMissing);
385    builder.setLatestVersionOnly(this.latestVersionOnly);
386
387    return builder.build();
388  }
389
390  /**
391   * @return The filter serialized using pb
392   */
393  @Override
394  public byte [] toByteArray() {
395    return convert().toByteArray();
396  }
397
398  /**
399   * @param pbBytes A pb serialized {@link SingleColumnValueFilter} instance
400   * @return An instance of {@link SingleColumnValueFilter} made from <code>bytes</code>
401   * @throws org.apache.hadoop.hbase.exceptions.DeserializationException
402   * @see #toByteArray
403   */
404  public static SingleColumnValueFilter parseFrom(final byte [] pbBytes)
405  throws DeserializationException {
406    FilterProtos.SingleColumnValueFilter proto;
407    try {
408      proto = FilterProtos.SingleColumnValueFilter.parseFrom(pbBytes);
409    } catch (InvalidProtocolBufferException e) {
410      throw new DeserializationException(e);
411    }
412
413    final CompareOperator compareOp =
414      CompareOperator.valueOf(proto.getCompareOp().name());
415    final org.apache.hadoop.hbase.filter.ByteArrayComparable comparator;
416    try {
417      comparator = ProtobufUtil.toComparator(proto.getComparator());
418    } catch (IOException ioe) {
419      throw new DeserializationException(ioe);
420    }
421
422    return new SingleColumnValueFilter(proto.hasColumnFamily() ? proto.getColumnFamily()
423        .toByteArray() : null, proto.hasColumnQualifier() ? proto.getColumnQualifier()
424        .toByteArray() : null, compareOp, comparator, proto.getFilterIfMissing(), proto
425        .getLatestVersionOnly());
426  }
427
428  /**
429   * @return true if and only if the fields of the filter that are serialized
430   * are equal to the corresponding fields in other.  Used for testing.
431   */
432  @Override
433  boolean areSerializedFieldsEqual(Filter o) {
434    if (o == this) return true;
435    if (!(o instanceof SingleColumnValueFilter)) return false;
436
437    SingleColumnValueFilter other = (SingleColumnValueFilter)o;
438    return Bytes.equals(this.getFamily(), other.getFamily())
439      && Bytes.equals(this.getQualifier(), other.getQualifier())
440      && this.op.equals(other.op)
441      && this.getComparator().areSerializedFieldsEqual(other.getComparator())
442      && this.getFilterIfMissing() == other.getFilterIfMissing()
443      && this.getLatestVersionOnly() == other.getLatestVersionOnly();
444  }
445
446  /**
447   * The only CF this filter needs is given column family. So, it's the only essential
448   * column in whole scan. If filterIfMissing == false, all families are essential,
449   * because of possibility of skipping the rows without any data in filtered CF.
450   */
451  @Override
452  public boolean isFamilyEssential(byte[] name) {
453    return !this.filterIfMissing || Bytes.equals(name, this.columnFamily);
454  }
455
456  @Override
457  public String toString() {
458    return String.format("%s (%s, %s, %s, %s)",
459        this.getClass().getSimpleName(), Bytes.toStringBinary(this.columnFamily),
460        Bytes.toStringBinary(this.columnQualifier), this.op.name(),
461        Bytes.toStringBinary(this.comparator.getValue()));
462  }
463}