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