View Javadoc

1   /*
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  
20  package org.apache.hadoop.hbase.filter;
21  
22  import java.io.IOException;
23  
24  import org.apache.hadoop.hbase.Cell;
25  import org.apache.hadoop.hbase.classification.InterfaceAudience;
26  import org.apache.hadoop.hbase.classification.InterfaceStability;
27  import org.apache.hadoop.hbase.exceptions.DeserializationException;
28  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
29  import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
30  
31  import com.google.protobuf.InvalidProtocolBufferException;
32  
33  /**
34   * A wrapper filter that returns true from {@link #filterAllRemaining()} as soon
35   * as the wrapped filters {@link Filter#filterRowKey(byte[], int, int)},
36   * {@link Filter#filterKeyValue(org.apache.hadoop.hbase.Cell)},
37   * {@link org.apache.hadoop.hbase.filter.Filter#filterRow()} or
38   * {@link org.apache.hadoop.hbase.filter.Filter#filterAllRemaining()} methods
39   * returns true.
40   */
41  @InterfaceAudience.Public
42  @InterfaceStability.Stable
43  public class WhileMatchFilter extends FilterBase {
44    private boolean filterAllRemaining = false;
45    private Filter filter;
46  
47    public WhileMatchFilter(Filter filter) {
48      this.filter = filter;
49    }
50  
51    public Filter getFilter() {
52      return filter;
53    }
54  
55    public void reset() throws IOException {
56      this.filter.reset();
57    }
58  
59    private void changeFAR(boolean value) {
60      filterAllRemaining = filterAllRemaining || value;
61    }
62  
63    @Override
64    public boolean filterAllRemaining() throws IOException {
65      return this.filterAllRemaining || this.filter.filterAllRemaining();
66    }
67  
68    @Override
69    public boolean filterRowKey(byte[] buffer, int offset, int length) throws IOException {
70      boolean value = filter.filterRowKey(buffer, offset, length);
71      changeFAR(value);
72      return value;
73    }
74  
75    @Override
76    public boolean filterRowKey(Cell cell) throws IOException {
77      if (filterAllRemaining()) return true;
78      boolean value = filter.filterRowKey(cell);
79      changeFAR(value);
80      return value;
81    }
82
83    @Override
84    public ReturnCode filterKeyValue(Cell v) throws IOException {
85      ReturnCode c = filter.filterKeyValue(v);
86      changeFAR(c != ReturnCode.INCLUDE);
87      return c;
88    }
89
90    @Override
91    public Cell transformCell(Cell v) throws IOException {
92      return filter.transformCell(v);
93    }
94
95    @Override
96    public boolean filterRow() throws IOException {
97      boolean filterRow = this.filter.filterRow();
98      changeFAR(filterRow);
99      return filterRow;
100   }
101
102   @Override
103   public boolean hasFilterRow() {
104     return true;
105   }
106
107   /**
108    * @return The filter serialized using pb
109    */
110   public byte[] toByteArray() throws IOException {
111     FilterProtos.WhileMatchFilter.Builder builder =
112       FilterProtos.WhileMatchFilter.newBuilder();
113     builder.setFilter(ProtobufUtil.toFilter(this.filter));
114     return builder.build().toByteArray();
115   }
116
117   /**
118    * @param pbBytes A pb serialized {@link WhileMatchFilter} instance
119    * @return An instance of {@link WhileMatchFilter} made from <code>bytes</code>
120    * @throws org.apache.hadoop.hbase.exceptions.DeserializationException
121    * @see #toByteArray
122    */
123   public static WhileMatchFilter parseFrom(final byte [] pbBytes)
124   throws DeserializationException {
125     FilterProtos.WhileMatchFilter proto;
126     try {
127       proto = FilterProtos.WhileMatchFilter.parseFrom(pbBytes);
128     } catch (InvalidProtocolBufferException e) {
129       throw new DeserializationException(e);
130     }
131     try {
132       return new WhileMatchFilter(ProtobufUtil.toFilter(proto.getFilter()));
133     } catch (IOException ioe) {
134       throw new DeserializationException(ioe);
135     }
136   }
137
138   /**
139    * @param other
140    * @return true if and only if the fields of the filter that are serialized
141    * are equal to the corresponding fields in other.  Used for testing.
142    */
143   boolean areSerializedFieldsEqual(Filter o) {
144     if (o == this) return true;
145     if (!(o instanceof WhileMatchFilter)) return false;
146
147     WhileMatchFilter other = (WhileMatchFilter)o;
148     return getFilter().areSerializedFieldsEqual(other.getFilter());
149   }
150
151   public boolean isFamilyEssential(byte[] name) throws IOException {
152     return filter.isFamilyEssential(name);
153   }
154
155   @Override
156   public String toString() {
157     return this.getClass().getSimpleName() + " " + this.filter.toString();
158   }
159 }