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.HashSet;
023import java.util.List;
024import java.util.Objects;
025import java.util.Set;
026import org.apache.hadoop.hbase.Cell;
027import org.apache.hadoop.hbase.CellUtil;
028import org.apache.hadoop.hbase.CompareOperator;
029import org.apache.hadoop.hbase.exceptions.DeserializationException;
030import org.apache.hadoop.hbase.util.Bytes;
031import org.apache.yetus.audience.InterfaceAudience;
032
033import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
034import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
035import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
036
037import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
038import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
039
040/**
041 * A filter for adding inter-column timestamp matching Only cells with a correspondingly timestamped
042 * entry in the target column will be retained Not compatible with Scan.setBatch as operations need
043 * full rows for correct filtering
044 */
045@InterfaceAudience.Public
046public class DependentColumnFilter extends CompareFilter {
047
048  protected byte[] columnFamily;
049  protected byte[] columnQualifier;
050  protected boolean dropDependentColumn;
051
052  protected Set<Long> stampSet = new HashSet<>();
053
054  /**
055   * Build a dependent column filter with value checking dependent column varies will be compared
056   * using the supplied compareOp and comparator, for usage of which refer to {@link CompareFilter}
057   * @param family              dependent column family
058   * @param qualifier           dependent column qualifier
059   * @param dropDependentColumn whether the column should be discarded after
060   * @param op                  Value comparison op
061   * @param valueComparator     comparator
062   */
063  public DependentColumnFilter(final byte[] family, final byte[] qualifier,
064    final boolean dropDependentColumn, final CompareOperator op,
065    final ByteArrayComparable valueComparator) {
066    // set up the comparator
067    super(op, valueComparator);
068    this.columnFamily = family;
069    this.columnQualifier = qualifier;
070    this.dropDependentColumn = dropDependentColumn;
071  }
072
073  /**
074   * Constructor for DependentColumn filter. Cells where a Cell from target column with the same
075   * timestamp do not exist will be dropped.
076   * @param family    name of target column family
077   * @param qualifier name of column qualifier
078   */
079  public DependentColumnFilter(final byte[] family, final byte[] qualifier) {
080    this(family, qualifier, false);
081  }
082
083  /**
084   * Constructor for DependentColumn filter. Cells where a Cell from target column with the same
085   * timestamp do not exist will be dropped.
086   * @param family              name of dependent column family
087   * @param qualifier           name of dependent qualifier
088   * @param dropDependentColumn whether the dependent columns Cells should be discarded
089   */
090  public DependentColumnFilter(final byte[] family, final byte[] qualifier,
091    final boolean dropDependentColumn) {
092    this(family, qualifier, dropDependentColumn, CompareOperator.NO_OP, null);
093  }
094
095  /** Returns the column family */
096  public byte[] getFamily() {
097    return this.columnFamily;
098  }
099
100  /** Returns the column qualifier */
101  public byte[] getQualifier() {
102    return this.columnQualifier;
103  }
104
105  /** Returns true if we should drop the dependent column, false otherwise */
106  public boolean dropDependentColumn() {
107    return this.dropDependentColumn;
108  }
109
110  public boolean getDropDependentColumn() {
111    return this.dropDependentColumn;
112  }
113
114  @Override
115  public boolean filterAllRemaining() {
116    return false;
117  }
118
119  @Override
120  public ReturnCode filterCell(final Cell c) {
121    // Check if the column and qualifier match
122    if (!CellUtil.matchingColumn(c, this.columnFamily, this.columnQualifier)) {
123      // include non-matches for the time being, they'll be discarded afterwards
124      return ReturnCode.INCLUDE;
125    }
126    // If it doesn't pass the op, skip it
127    if (comparator != null && compareValue(getCompareOperator(), comparator, c))
128      return ReturnCode.SKIP;
129
130    stampSet.add(c.getTimestamp());
131    if (dropDependentColumn) {
132      return ReturnCode.SKIP;
133    }
134    return ReturnCode.INCLUDE;
135  }
136
137  @Override
138  public void filterRowCells(List<Cell> kvs) {
139    kvs.removeIf(kv -> !stampSet.contains(kv.getTimestamp()));
140  }
141
142  @Override
143  public boolean hasFilterRow() {
144    return true;
145  }
146
147  @Override
148  public boolean filterRow() {
149    return false;
150  }
151
152  @Override
153  public void reset() {
154    stampSet.clear();
155  }
156
157  public static Filter createFilterFromArguments(ArrayList<byte[]> filterArguments) {
158    Preconditions.checkArgument(
159      filterArguments.size() == 2 || filterArguments.size() == 3 || filterArguments.size() == 5,
160      "Expected 2, 3 or 5 but got: %s", filterArguments.size());
161    if (filterArguments.size() == 2) {
162      byte[] family = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0));
163      byte[] qualifier = ParseFilter.removeQuotesFromByteArray(filterArguments.get(1));
164      return new DependentColumnFilter(family, qualifier);
165
166    } else if (filterArguments.size() == 3) {
167      byte[] family = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0));
168      byte[] qualifier = ParseFilter.removeQuotesFromByteArray(filterArguments.get(1));
169      boolean dropDependentColumn = ParseFilter.convertByteArrayToBoolean(filterArguments.get(2));
170      return new DependentColumnFilter(family, qualifier, dropDependentColumn);
171
172    } else if (filterArguments.size() == 5) {
173      byte[] family = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0));
174      byte[] qualifier = ParseFilter.removeQuotesFromByteArray(filterArguments.get(1));
175      boolean dropDependentColumn = ParseFilter.convertByteArrayToBoolean(filterArguments.get(2));
176      CompareOperator op = ParseFilter.createCompareOperator(filterArguments.get(3));
177      ByteArrayComparable comparator =
178        ParseFilter.createComparator(ParseFilter.removeQuotesFromByteArray(filterArguments.get(4)));
179      return new DependentColumnFilter(family, qualifier, dropDependentColumn, op, comparator);
180    } else {
181      throw new IllegalArgumentException("Expected 2, 3 or 5 but got: " + filterArguments.size());
182    }
183  }
184
185  /** Returns The filter serialized using pb */
186  @Override
187  public byte[] toByteArray() {
188    FilterProtos.DependentColumnFilter.Builder builder =
189      FilterProtos.DependentColumnFilter.newBuilder();
190    builder.setCompareFilter(super.convert());
191    if (this.columnFamily != null) {
192      builder.setColumnFamily(UnsafeByteOperations.unsafeWrap(this.columnFamily));
193    }
194    if (this.columnQualifier != null) {
195      builder.setColumnQualifier(UnsafeByteOperations.unsafeWrap(this.columnQualifier));
196    }
197    builder.setDropDependentColumn(this.dropDependentColumn);
198    return builder.build().toByteArray();
199  }
200
201  /**
202   * Parse a seralized representation of {@link DependentColumnFilter}
203   * @param pbBytes A pb serialized {@link DependentColumnFilter} instance
204   * @return An instance of {@link DependentColumnFilter} made from <code>bytes</code>
205   * @throws DeserializationException if an error occurred
206   * @see #toByteArray
207   */
208  public static DependentColumnFilter parseFrom(final byte[] pbBytes)
209    throws DeserializationException {
210    FilterProtos.DependentColumnFilter proto;
211    try {
212      proto = FilterProtos.DependentColumnFilter.parseFrom(pbBytes);
213    } catch (InvalidProtocolBufferException e) {
214      throw new DeserializationException(e);
215    }
216    final CompareOperator valueCompareOp =
217      CompareOperator.valueOf(proto.getCompareFilter().getCompareOp().name());
218    ByteArrayComparable valueComparator = null;
219    try {
220      if (proto.getCompareFilter().hasComparator()) {
221        valueComparator = ProtobufUtil.toComparator(proto.getCompareFilter().getComparator());
222      }
223    } catch (IOException ioe) {
224      throw new DeserializationException(ioe);
225    }
226    return new DependentColumnFilter(
227      proto.hasColumnFamily() ? proto.getColumnFamily().toByteArray() : null,
228      proto.hasColumnQualifier() ? proto.getColumnQualifier().toByteArray() : null,
229      proto.getDropDependentColumn(), valueCompareOp, valueComparator);
230  }
231
232  /**
233   * Returns true if and only if the fields of the filter that are serialized are equal to the
234   * corresponding fields in other. Used for testing.
235   */
236  @edu.umd.cs.findbugs.annotations.SuppressWarnings(
237      value = "RCN_REDUNDANT_NULLCHECK_OF_NONNULL_VALUE")
238  @Override
239  boolean areSerializedFieldsEqual(Filter o) {
240    if (o == this) {
241      return true;
242    }
243    if (!(o instanceof DependentColumnFilter)) {
244      return false;
245    }
246    DependentColumnFilter other = (DependentColumnFilter) o;
247    return other != null && super.areSerializedFieldsEqual(other)
248      && Bytes.equals(this.getFamily(), other.getFamily())
249      && Bytes.equals(this.getQualifier(), other.getQualifier())
250      && this.dropDependentColumn() == other.dropDependentColumn();
251  }
252
253  @Override
254  public String toString() {
255    return String.format("%s (%s, %s, %s, %s, %s)", this.getClass().getSimpleName(),
256      Bytes.toStringBinary(this.columnFamily), Bytes.toStringBinary(this.columnQualifier),
257      this.dropDependentColumn, this.op.name(),
258      this.comparator != null ? Bytes.toStringBinary(this.comparator.getValue()) : "null");
259  }
260
261  @Override
262  public boolean equals(Object obj) {
263    return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj);
264  }
265
266  @Override
267  public int hashCode() {
268    return Objects.hash(Bytes.hashCode(getFamily()), Bytes.hashCode(getQualifier()),
269      dropDependentColumn(), getComparator(), getCompareOperator());
270  }
271}