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