View Javadoc

1   /*
2    * Copyright The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.filter;
21  
22  import java.io.IOException;
23  import java.util.List;
24  
25  import org.apache.hadoop.classification.InterfaceAudience;
26  import org.apache.hadoop.hbase.Cell;
27  import org.apache.hadoop.hbase.KeyValue;
28  import org.apache.hadoop.hbase.KeyValueUtil;
29  import org.apache.hadoop.hbase.exceptions.DeserializationException;
30  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
31  import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
32  
33  import com.google.protobuf.InvalidProtocolBufferException;
34  
35  /**
36   * This is a Filter wrapper class which is used in the server side. Some filter
37   * related hooks can be defined in this wrapper. The only way to create a
38   * FilterWrapper instance is passing a client side Filter instance through
39   * {@link org.apache.hadoop.hbase.client.Scan#getFilter()}.
40   * 
41   */
42  @InterfaceAudience.Private
43  final public class FilterWrapper extends Filter {
44    Filter filter = null;
45  
46    public FilterWrapper( Filter filter ) {
47      if (null == filter) {
48        // ensure the filter instance is not null
49        throw new NullPointerException("Cannot create FilterWrapper with null Filter");
50      }
51      this.filter = filter;
52    }
53  
54    /**
55     * @return The filter serialized using pb
56     */
57    public byte[] toByteArray() throws IOException {
58      FilterProtos.FilterWrapper.Builder builder =
59        FilterProtos.FilterWrapper.newBuilder();
60      builder.setFilter(ProtobufUtil.toFilter(this.filter));
61      return builder.build().toByteArray();
62    }
63  
64    /**
65     * @param pbBytes A pb serialized {@link FilterWrapper} instance
66     * @return An instance of {@link FilterWrapper} made from <code>bytes</code>
67     * @throws org.apache.hadoop.hbase.exceptions.DeserializationException
68     * @see #toByteArray
69     */
70    public static FilterWrapper parseFrom(final byte [] pbBytes)
71    throws DeserializationException {
72      FilterProtos.FilterWrapper proto;
73      try {
74        proto = FilterProtos.FilterWrapper.parseFrom(pbBytes);
75      } catch (InvalidProtocolBufferException e) {
76        throw new DeserializationException(e);
77      }
78      try {
79        return new FilterWrapper(ProtobufUtil.toFilter(proto.getFilter()));
80      } catch (IOException ioe) {
81        throw new DeserializationException(ioe);
82      }
83    }
84  
85    @Override
86    public void reset() throws IOException {
87      this.filter.reset();
88    }
89  
90    @Override
91    public boolean filterAllRemaining() throws IOException {
92      return this.filter.filterAllRemaining();
93    }
94  
95    @Override
96    public boolean filterRow() throws IOException {
97      return this.filter.filterRow();
98    }
99  
100   /**
101    * This method is deprecated and you should override Cell getNextKeyHint(Cell) instead.
102    */
103   @Override
104   @Deprecated
105   public KeyValue getNextKeyHint(KeyValue currentKV) throws IOException {
106     return KeyValueUtil.ensureKeyValue(this.filter.getNextCellHint((Cell)currentKV));
107   }
108 
109   /**
110    * Old filter wrapper descendants will implement KV getNextKeyHint(KV) so we should call it.
111    */
112   @Override
113   public Cell getNextCellHint(Cell currentCell) throws IOException {
114     // Old filters based off of this class will override KeyValue getNextKeyHint(KeyValue).
115     // Thus to maintain compatibility we need to call the old version.
116     return this.getNextKeyHint(KeyValueUtil.ensureKeyValue(currentCell));
117   }
118 
119   @Override
120   public boolean filterRowKey(byte[] buffer, int offset, int length) throws IOException {
121     return this.filter.filterRowKey(buffer, offset, length);
122   }
123 
124   @Override
125   public ReturnCode filterKeyValue(Cell v) throws IOException {
126     return this.filter.filterKeyValue(v);
127   }
128 
129   @Override
130   public Cell transformCell(Cell v) throws IOException {
131     // Old filters based off of this class will override KeyValue transform(KeyValue).
132     // Thus to maintain compatibility we need to call the old version.
133     return transform(KeyValueUtil.ensureKeyValue(v));
134   }
135 
136   /**
137    * WARNING: please to not override this method.  Instead override {@link #transformCell(Cell)}.
138    *
139    * This is for transition from 0.94 -> 0.96
140    */
141   @Override
142   @Deprecated
143   public KeyValue transform(KeyValue currentKV) throws IOException {
144     return KeyValueUtil.ensureKeyValue(this.filter.transformCell(currentKV));
145   }
146 
147   @Override
148   public boolean hasFilterRow() {
149     return this.filter.hasFilterRow();
150   }
151 
152   @Override
153   public void filterRowCells(List<Cell> kvs) throws IOException {
154     filterRowCellsWithRet(kvs);
155   }
156 
157   public enum FilterRowRetCode {
158     NOT_CALLED,
159     INCLUDE,     // corresponds to filter.filterRow() returning false
160     EXCLUDE      // corresponds to filter.filterRow() returning true
161   }
162   public FilterRowRetCode filterRowCellsWithRet(List<Cell> kvs) throws IOException {
163     //To fix HBASE-6429,
164     //Filter with filterRow() returning true is incompatible with scan with limit
165     //1. hasFilterRow() returns true, if either filterRow() or filterRow(kvs) is implemented.
166     //2. filterRow() is merged with filterRow(kvs),
167     //so that to make all those row related filtering stuff in the same function.
168     this.filter.filterRowCells(kvs);
169     if (!kvs.isEmpty()) {
170       if (this.filter.filterRow()) {
171         kvs.clear();
172         return FilterRowRetCode.EXCLUDE;
173       }
174       return FilterRowRetCode.INCLUDE;
175     }
176     return FilterRowRetCode.NOT_CALLED;
177   }
178 
179   @Override
180   public boolean isFamilyEssential(byte[] name) throws IOException {
181     return filter.isFamilyEssential(name);
182   }
183 
184   /**
185    * @param other
186    * @return true if and only if the fields of the filter that are serialized
187    * are equal to the corresponding fields in other.  Used for testing.
188    */
189   boolean areSerializedFieldsEqual(Filter o) {
190     if (o == this) return true;
191     if (!(o instanceof FilterWrapper)) return false;
192 
193     FilterWrapper other = (FilterWrapper)o;
194     return this.filter.areSerializedFieldsEqual(other.filter);
195   }
196 }