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.List;
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 * This is a Filter wrapper class which is used in the server side. Some filter related hooks can be
033 * defined in this wrapper. The only way to create a FilterWrapper instance is passing a client side
034 * Filter instance through {@link org.apache.hadoop.hbase.client.Scan#getFilter()}.
035 */
036@InterfaceAudience.Private
037final public class FilterWrapper extends Filter {
038  Filter filter = null;
039
040  public FilterWrapper(Filter filter) {
041    if (null == filter) {
042      // ensure the filter instance is not null
043      throw new NullPointerException("Cannot create FilterWrapper with null Filter");
044    }
045    this.filter = filter;
046  }
047
048  /** Returns The filter serialized using pb */
049  @Override
050  public byte[] toByteArray() throws IOException {
051    FilterProtos.FilterWrapper.Builder builder = FilterProtos.FilterWrapper.newBuilder();
052    builder.setFilter(ProtobufUtil.toFilter(this.filter));
053    return builder.build().toByteArray();
054  }
055
056  /**
057   * @param pbBytes A pb serialized {@link FilterWrapper} instance
058   * @return An instance of {@link FilterWrapper} made from <code>bytes</code>
059   * @throws org.apache.hadoop.hbase.exceptions.DeserializationException
060   * @see #toByteArray
061   */
062  public static FilterWrapper parseFrom(final byte[] pbBytes) throws DeserializationException {
063    FilterProtos.FilterWrapper proto;
064    try {
065      proto = FilterProtos.FilterWrapper.parseFrom(pbBytes);
066    } catch (InvalidProtocolBufferException e) {
067      throw new DeserializationException(e);
068    }
069    try {
070      return new FilterWrapper(ProtobufUtil.toFilter(proto.getFilter()));
071    } catch (IOException ioe) {
072      throw new DeserializationException(ioe);
073    }
074  }
075
076  @Override
077  public void reset() throws IOException {
078    this.filter.reset();
079  }
080
081  @Override
082  public boolean filterAllRemaining() throws IOException {
083    return this.filter.filterAllRemaining();
084  }
085
086  @Override
087  public boolean filterRow() throws IOException {
088    return this.filter.filterRow();
089  }
090
091  @Override
092  public Cell getNextCellHint(Cell currentCell) throws IOException {
093    return this.filter.getNextCellHint(currentCell);
094  }
095
096  @Override
097  public boolean filterRowKey(byte[] buffer, int offset, int length) throws IOException {
098    // No call to this.
099    if (filterAllRemaining()) return true;
100    return this.filter.filterRowKey(buffer, offset, length);
101  }
102
103  @Override
104  public boolean filterRowKey(Cell cell) throws IOException {
105    if (filterAllRemaining()) return true;
106    return this.filter.filterRowKey(cell);
107  }
108
109  @Deprecated
110  @Override
111  public ReturnCode filterKeyValue(final Cell c) throws IOException {
112    return filterCell(c);
113  }
114
115  @Override
116  public ReturnCode filterCell(final Cell c) throws IOException {
117    return this.filter.filterCell(c);
118  }
119
120  @Override
121  public Cell transformCell(Cell v) throws IOException {
122    return this.filter.transformCell(v);
123  }
124
125  @Override
126  public boolean hasFilterRow() {
127    return this.filter.hasFilterRow();
128  }
129
130  @Override
131  public void filterRowCells(List<Cell> kvs) throws IOException {
132    filterRowCellsWithRet(kvs);
133  }
134
135  public enum FilterRowRetCode {
136    NOT_CALLED,
137    INCLUDE, // corresponds to filter.filterRow() returning false
138    EXCLUDE, // corresponds to filter.filterRow() returning true
139    INCLUDE_THIS_FAMILY // exclude other families
140  }
141
142  public FilterRowRetCode filterRowCellsWithRet(List<Cell> kvs) throws IOException {
143    // To fix HBASE-6429,
144    // Filter with filterRow() returning true is incompatible with scan with limit
145    // 1. hasFilterRow() returns true, if either filterRow() or filterRow(kvs) is implemented.
146    // 2. filterRow() is merged with filterRow(kvs),
147    // so that to make all those row related filtering stuff in the same function.
148    this.filter.filterRowCells(kvs);
149    if (!kvs.isEmpty()) {
150      if (this.filter.filterRow()) {
151        kvs.clear();
152        return FilterRowRetCode.EXCLUDE;
153      }
154      return FilterRowRetCode.INCLUDE;
155    }
156    return FilterRowRetCode.NOT_CALLED;
157  }
158
159  @Override
160  public boolean isFamilyEssential(byte[] name) throws IOException {
161    return filter.isFamilyEssential(name);
162  }
163
164  /**
165   * @param o the other filter to compare with
166   * @return true if and only if the fields of the filter that are serialized are equal to the
167   *         corresponding fields in other. Used for testing.
168   */
169  @Override
170  boolean areSerializedFieldsEqual(Filter o) {
171    if (o == this) return true;
172    if (!(o instanceof FilterWrapper)) return false;
173
174    FilterWrapper other = (FilterWrapper) o;
175    return this.filter.areSerializedFieldsEqual(other.filter);
176  }
177}