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.Objects;
022import org.apache.hadoop.hbase.Cell;
023import org.apache.hadoop.hbase.exceptions.DeserializationException;
024import org.apache.yetus.audience.InterfaceAudience;
025
026import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
027
028import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
029import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
030
031/**
032 * A wrapper filter that returns true from {@link #filterAllRemaining()} as soon as the wrapped
033 * filters {@link Filter#filterRowKey(byte[], int, int)},
034 * {@link Filter#filterCell(org.apache.hadoop.hbase.Cell)},
035 * {@link org.apache.hadoop.hbase.filter.Filter#filterRow()} or
036 * {@link org.apache.hadoop.hbase.filter.Filter#filterAllRemaining()} methods returns true.
037 */
038@InterfaceAudience.Public
039public class WhileMatchFilter extends FilterBase {
040  private boolean filterAllRemaining = false;
041  private Filter filter;
042
043  public WhileMatchFilter(Filter filter) {
044    this.filter = filter;
045  }
046
047  public Filter getFilter() {
048    return filter;
049  }
050
051  @Override
052  public void reset() throws IOException {
053    this.filter.reset();
054  }
055
056  private void changeFAR(boolean value) {
057    filterAllRemaining = filterAllRemaining || value;
058  }
059
060  @Override
061  public boolean filterAllRemaining() throws IOException {
062    return this.filterAllRemaining || this.filter.filterAllRemaining();
063  }
064
065  @Override
066  public boolean filterRowKey(byte[] buffer, int offset, int length) throws IOException {
067    boolean value = filter.filterRowKey(buffer, offset, length);
068    changeFAR(value);
069    return value;
070  }
071
072  @Override
073  public boolean filterRowKey(Cell cell) throws IOException {
074    if (filterAllRemaining()) return true;
075    boolean value = filter.filterRowKey(cell);
076    changeFAR(value);
077    return value;
078  }
079
080  @Deprecated
081  @Override
082  public ReturnCode filterKeyValue(final Cell c) throws IOException {
083    return filterCell(c);
084  }
085
086  @Override
087  public ReturnCode filterCell(final Cell c) throws IOException {
088    ReturnCode code = filter.filterCell(c);
089    changeFAR(code != ReturnCode.INCLUDE);
090    return code;
091  }
092
093  @Override
094  public Cell transformCell(Cell v) throws IOException {
095    return filter.transformCell(v);
096  }
097
098  @Override
099  public boolean filterRow() throws IOException {
100    boolean filterRow = this.filter.filterRow();
101    changeFAR(filterRow);
102    return filterRow;
103  }
104
105  @Override
106  public boolean hasFilterRow() {
107    return true;
108  }
109
110  /** Returns The filter serialized using pb */
111  @Override
112  public byte[] toByteArray() throws IOException {
113    FilterProtos.WhileMatchFilter.Builder builder = FilterProtos.WhileMatchFilter.newBuilder();
114    builder.setFilter(ProtobufUtil.toFilter(this.filter));
115    return builder.build().toByteArray();
116  }
117
118  /**
119   * Parse a serialized representation of {@link WhileMatchFilter}
120   * @param pbBytes A pb serialized {@link WhileMatchFilter} instance
121   * @return An instance of {@link WhileMatchFilter} made from <code>bytes</code>
122   * @throws DeserializationException if an error occurred
123   * @see #toByteArray
124   */
125  public static WhileMatchFilter parseFrom(final byte[] pbBytes) throws DeserializationException {
126    FilterProtos.WhileMatchFilter proto;
127    try {
128      proto = FilterProtos.WhileMatchFilter.parseFrom(pbBytes);
129    } catch (InvalidProtocolBufferException e) {
130      throw new DeserializationException(e);
131    }
132    try {
133      return new WhileMatchFilter(ProtobufUtil.toFilter(proto.getFilter()));
134    } catch (IOException ioe) {
135      throw new DeserializationException(ioe);
136    }
137  }
138
139  /**
140   * Return true if and only if the fields of the filter that are serialized are equal to the
141   * corresponding fields in other. Used for testing.
142   */
143  @Override
144  boolean areSerializedFieldsEqual(Filter o) {
145    if (o == this) {
146      return true;
147    }
148    if (!(o instanceof WhileMatchFilter)) {
149      return false;
150    }
151    WhileMatchFilter other = (WhileMatchFilter) o;
152    return getFilter().areSerializedFieldsEqual(other.getFilter());
153  }
154
155  @Override
156  public boolean isFamilyEssential(byte[] name) throws IOException {
157    return filter.isFamilyEssential(name);
158  }
159
160  @Override
161  public String toString() {
162    return this.getClass().getSimpleName() + " " + this.filter.toString();
163  }
164
165  @Override
166  public boolean equals(Object obj) {
167    return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj);
168  }
169
170  @Override
171  public int hashCode() {
172    return Objects.hash(this.filter);
173  }
174}