1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
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.hbase.classification.InterfaceAudience;
27 import org.apache.hadoop.hbase.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
39
40
41
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
51 throw new NullPointerException("Cannot create FilterWrapper with null Filter");
52 }
53 this.filter = filter;
54 }
55
56
57
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
68
69
70
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
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
113
114 @Override
115 public Cell getNextCellHint(Cell currentKV) throws IOException {
116
117
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
134
135 return transform(KeyValueUtil.ensureKeyValue(v));
136 }
137
138
139
140
141
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,
162 EXCLUDE
163 }
164 public FilterRowRetCode filterRowCellsWithRet(List<Cell> kvs) throws IOException {
165
166
167
168
169
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
182
183
184
185
186 @Override
187 @Deprecated
188 public void filterRow(List<KeyValue> kvs) throws IOException {
189
190
191 throw new UnsupportedOperationException("filterRow(List<KeyValue>) should never be called");
192 }
193
194 @Override
195 public boolean isFamilyEssential(byte[] name) throws IOException {
196 return filter.isFamilyEssential(name);
197 }
198
199
200
201
202
203
204 boolean areSerializedFieldsEqual(Filter o) {
205 if (o == this) return true;
206 if (!(o instanceof FilterWrapper)) return false;
207
208 FilterWrapper other = (FilterWrapper)o;
209 return this.filter.areSerializedFieldsEqual(other.filter);
210 }
211 }