001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.filter;
019
020import java.io.IOException;
021import java.util.ArrayList;
022import java.util.Objects;
023import org.apache.hadoop.hbase.Cell;
024import org.apache.hadoop.hbase.CellUtil;
025import org.apache.hadoop.hbase.CompareOperator;
026import org.apache.hadoop.hbase.PrivateCellUtil;
027import org.apache.hadoop.hbase.exceptions.DeserializationException;
028import org.apache.hadoop.hbase.util.Bytes;
029import org.apache.yetus.audience.InterfaceAudience;
030
031import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
032import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
033import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
034
035import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
036import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
037import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
038
039/**
040 * Different from {@link SingleColumnValueFilter} which returns an <b>entire</b> row when specified
041 * condition is matched, {@link ColumnValueFilter} return the matched cell only.
042 * <p>
043 * This filter is used to filter cells based on column and value. It takes a
044 * {@link org.apache.hadoop.hbase.CompareOperator} operator (<, <=, =, !=, >, >=), and and a
045 * {@link ByteArrayComparable} comparator.
046 */
047@InterfaceAudience.Public
048public class ColumnValueFilter extends FilterBase {
049  private final byte[] family;
050  private final byte[] qualifier;
051  private final CompareOperator op;
052  private final ByteArrayComparable comparator;
053
054  // This flag is used to speed up seeking cells when matched column is found, such that following
055  // columns in the same row can be skipped faster by NEXT_ROW instead of NEXT_COL.
056  private boolean columnFound = false;
057
058  public ColumnValueFilter(final byte[] family, final byte[] qualifier, final CompareOperator op,
059    final byte[] value) {
060    this(family, qualifier, op, new BinaryComparator(value));
061  }
062
063  public ColumnValueFilter(final byte[] family, final byte[] qualifier, final CompareOperator op,
064    final ByteArrayComparable comparator) {
065    this.family = Preconditions.checkNotNull(family, "family should not be null.");
066    this.qualifier = qualifier == null ? new byte[0] : qualifier;
067    this.op = Preconditions.checkNotNull(op, "CompareOperator should not be null");
068    this.comparator = Preconditions.checkNotNull(comparator, "Comparator should not be null");
069  }
070
071  /**
072   * n
073   */
074  public CompareOperator getCompareOperator() {
075    return op;
076  }
077
078  /**
079   * @return the comparator
080   */
081  public ByteArrayComparable getComparator() {
082    return comparator;
083  }
084
085  /**
086   * @return the column family
087   */
088  public byte[] getFamily() {
089    return family;
090  }
091
092  /**
093   * @return the qualifier
094   */
095  public byte[] getQualifier() {
096    return qualifier;
097  }
098
099  @Override
100  public void reset() throws IOException {
101    columnFound = false;
102  }
103
104  @Override
105  public boolean filterRowKey(Cell cell) throws IOException {
106    return false;
107  }
108
109  @Override
110  public ReturnCode filterCell(Cell c) throws IOException {
111    // 1. Check column match
112    if (!CellUtil.matchingColumn(c, this.family, this.qualifier)) {
113      return columnFound ? ReturnCode.NEXT_ROW : ReturnCode.NEXT_COL;
114    }
115    // Column found
116    columnFound = true;
117    // 2. Check value match:
118    // True means filter out, just skip this cell, else include it.
119    return compareValue(getCompareOperator(), getComparator(), c)
120      ? ReturnCode.SKIP
121      : ReturnCode.INCLUDE;
122  }
123
124  /**
125   * This method is used to determine a cell should be included or filtered out.
126   * @param op         one of operators {@link CompareOperator}
127   * @param comparator comparator used to compare cells.
128   * @param cell       cell to be compared.
129   * @return true means cell should be filtered out, included otherwise.
130   */
131  private boolean compareValue(final CompareOperator op, final ByteArrayComparable comparator,
132    final Cell cell) {
133    if (op == CompareOperator.NO_OP) {
134      return true;
135    }
136    int compareResult = PrivateCellUtil.compareValue(cell, comparator);
137    return CompareFilter.compare(op, compareResult);
138  }
139
140  /**
141   * Creating this filter by reflection, it is used by {@link ParseFilter},
142   * @param filterArguments arguments for creating a ColumnValueFilter
143   * @return a ColumnValueFilter
144   */
145  public static Filter createFilterFromArguments(ArrayList<byte[]> filterArguments) {
146    Preconditions.checkArgument(filterArguments.size() == 4, "Expect 4 arguments: %s",
147      filterArguments.size());
148    byte[] family = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0));
149    byte[] qualifier = ParseFilter.removeQuotesFromByteArray(filterArguments.get(1));
150    CompareOperator operator = ParseFilter.createCompareOperator(filterArguments.get(2));
151    ByteArrayComparable comparator =
152      ParseFilter.createComparator(ParseFilter.removeQuotesFromByteArray(filterArguments.get(3)));
153
154    if (comparator instanceof RegexStringComparator || comparator instanceof SubstringComparator) {
155      if (operator != CompareOperator.EQUAL && operator != CompareOperator.NOT_EQUAL) {
156        throw new IllegalArgumentException("A regexstring comparator and substring comparator "
157          + "can only be used with EQUAL and NOT_EQUAL");
158      }
159    }
160
161    return new ColumnValueFilter(family, qualifier, operator, comparator);
162  }
163
164  /**
165   * @return A pb instance to represent this instance.
166   */
167  FilterProtos.ColumnValueFilter convert() {
168    FilterProtos.ColumnValueFilter.Builder builder = FilterProtos.ColumnValueFilter.newBuilder();
169
170    builder.setFamily(UnsafeByteOperations.unsafeWrap(this.family));
171    builder.setQualifier(UnsafeByteOperations.unsafeWrap(this.qualifier));
172    builder.setCompareOp(HBaseProtos.CompareType.valueOf(this.op.name()));
173    builder.setComparator(ProtobufUtil.toComparator(this.comparator));
174
175    return builder.build();
176  }
177
178  /**
179   * Parse protobuf bytes to a ColumnValueFilter
180   * @param pbBytes pbBytes
181   * @return a ColumnValueFilter
182   * @throws DeserializationException deserialization exception
183   */
184  public static ColumnValueFilter parseFrom(final byte[] pbBytes) throws DeserializationException {
185    FilterProtos.ColumnValueFilter proto;
186    try {
187      proto = FilterProtos.ColumnValueFilter.parseFrom(pbBytes);
188    } catch (InvalidProtocolBufferException e) {
189      throw new DeserializationException(e);
190    }
191
192    final CompareOperator compareOp = CompareOperator.valueOf(proto.getCompareOp().name());
193    final ByteArrayComparable comparator;
194    try {
195      comparator = ProtobufUtil.toComparator(proto.getComparator());
196    } catch (IOException ioe) {
197      throw new DeserializationException(ioe);
198    }
199
200    return new ColumnValueFilter(proto.getFamily().toByteArray(),
201      proto.getQualifier().toByteArray(), compareOp, comparator);
202  }
203
204  @Override
205  public byte[] toByteArray() throws IOException {
206    return convert().toByteArray();
207  }
208
209  @Override
210  boolean areSerializedFieldsEqual(Filter o) {
211    if (o == this) {
212      return true;
213    } else if (!(o instanceof ColumnValueFilter)) {
214      return false;
215    }
216
217    ColumnValueFilter other = (ColumnValueFilter) o;
218    return Bytes.equals(this.getFamily(), other.getFamily())
219      && Bytes.equals(this.getQualifier(), other.getQualifier())
220      && this.getCompareOperator().equals(other.getCompareOperator())
221      && this.getComparator().areSerializedFieldsEqual(other.getComparator());
222  }
223
224  @Override
225  public boolean isFamilyEssential(byte[] name) throws IOException {
226    return Bytes.equals(name, this.family);
227  }
228
229  @Override
230  public String toString() {
231    return String.format("%s (%s, %s, %s, %s)", getClass().getSimpleName(),
232      Bytes.toStringBinary(this.family), Bytes.toStringBinary(this.qualifier), this.op.name(),
233      Bytes.toStringBinary(this.comparator.getValue()));
234  }
235
236  @Override
237  public boolean equals(Object obj) {
238    return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj);
239  }
240
241  @Override
242  public int hashCode() {
243    return Objects.hash(Bytes.hashCode(getFamily()), Bytes.hashCode(getQualifier()),
244      getCompareOperator(), getComparator());
245  }
246}