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.hbase.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    @Override
58    public byte[] toByteArray() throws IOException {
59      FilterProtos.FilterWrapper.Builder builder =
60        FilterProtos.FilterWrapper.newBuilder();
61      builder.setFilter(ProtobufUtil.toFilter(this.filter));
62      return builder.build().toByteArray();
63    }
64  
65    /**
66     * @param pbBytes A pb serialized {@link FilterWrapper} instance
67     * @return An instance of {@link FilterWrapper} made from <code>bytes</code>
68     * @throws org.apache.hadoop.hbase.exceptions.DeserializationException
69     * @see #toByteArray
70     */
71    public static FilterWrapper parseFrom(final byte [] pbBytes)
72    throws DeserializationException {
73      FilterProtos.FilterWrapper proto;
74      try {
75        proto = FilterProtos.FilterWrapper.parseFrom(pbBytes);
76      } catch (InvalidProtocolBufferException e) {
77        throw new DeserializationException(e);
78      }
79      try {
80        return new FilterWrapper(ProtobufUtil.toFilter(proto.getFilter()));
81      } catch (IOException ioe) {
82        throw new DeserializationException(ioe);
83      }
84    }
85  
86    @Override
87    public void reset() throws IOException {
88      this.filter.reset();
89    }
90  
91    @Override
92    public boolean filterAllRemaining() throws IOException {
93      return this.filter.filterAllRemaining();
94    }
95  
96    @Override
97    public boolean filterRow() throws IOException {
98      return this.filter.filterRow();
99    }
100 
101   /**
102    * This method is deprecated and you should override Cell getNextKeyHint(Cell) instead.
103    */
104   @Override
105   @Deprecated
106   public KeyValue getNextKeyHint(KeyValue currentKV) throws IOException {
107     // This will never get called.
108     return KeyValueUtil.ensureKeyValue(this.filter.getNextCellHint((Cell)currentKV));
109   }
110 
111   @Override
112   public Cell getNextCellHint(Cell currentKV) throws IOException {
113     return this.filter.getNextCellHint(currentKV);
114   }
115 
116   @Override
117   public boolean filterRowKey(byte[] buffer, int offset, int length) throws IOException {
118     return this.filter.filterRowKey(buffer, offset, length);
119   }
120 
121   @Override
122   public ReturnCode filterKeyValue(Cell v) throws IOException {
123     return this.filter.filterKeyValue(v);
124   }
125 
126   @Override
127   public Cell transformCell(Cell v) throws IOException {
128     return this.filter.transformCell(v);
129   }
130 
131   /**
132    * WARNING: please to not override this method.  Instead override {@link #transformCell(Cell)}.
133    *
134    * This is for transition from 0.94 -&gt; 0.96
135    */
136   @Override
137   @Deprecated
138   public KeyValue transform(KeyValue currentKV) throws IOException {
139     // This will never get called.
140     return KeyValueUtil.ensureKeyValue(this.filter.transformCell(currentKV));
141   }
142 
143   @Override
144   public boolean hasFilterRow() {
145     return this.filter.hasFilterRow();
146   }
147 
148   @Override
149   public void filterRowCells(List<Cell> kvs) throws IOException {
150     filterRowCellsWithRet(kvs);
151   }
152 
153   public enum FilterRowRetCode {
154     NOT_CALLED,
155     INCLUDE,     // corresponds to filter.filterRow() returning false
156     EXCLUDE      // corresponds to filter.filterRow() returning true
157   }
158   public FilterRowRetCode filterRowCellsWithRet(List<Cell> kvs) throws IOException {
159     //To fix HBASE-6429,
160     //Filter with filterRow() returning true is incompatible with scan with limit
161     //1. hasFilterRow() returns true, if either filterRow() or filterRow(kvs) is implemented.
162     //2. filterRow() is merged with filterRow(kvs),
163     //so that to make all those row related filtering stuff in the same function.
164     this.filter.filterRowCells(kvs);
165     if (!kvs.isEmpty()) {
166       if (this.filter.filterRow()) {
167         kvs.clear();
168         return FilterRowRetCode.EXCLUDE;
169       }
170       return FilterRowRetCode.INCLUDE;
171     }
172     return FilterRowRetCode.NOT_CALLED;
173   }
174 
175   @Override
176   public boolean isFamilyEssential(byte[] name) throws IOException {
177     return filter.isFamilyEssential(name);
178   }
179 
180   /**
181    * @param other
182    * @return true if and only if the fields of the filter that are serialized
183    * are equal to the corresponding fields in other.  Used for testing.
184    */
185   @Override
186   boolean areSerializedFieldsEqual(Filter o) {
187     if (o == this) return true;
188     if (!(o instanceof FilterWrapper)) return false;
189 
190     FilterWrapper other = (FilterWrapper)o;
191     return this.filter.areSerializedFieldsEqual(other.filter);
192   }
193 }