1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.filter;
20
21 import java.io.IOException;
22 import java.util.ArrayList;
23 import java.util.Arrays;
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.conf.Configuration;
29 import org.apache.hadoop.hbase.HBaseConfiguration;
30 import org.apache.hadoop.hbase.KeyValue;
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 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
35
36 import com.google.protobuf.InvalidProtocolBufferException;
37
38
39
40
41
42
43
44
45
46
47 @InterfaceAudience.Public
48 @InterfaceStability.Stable
49 public class FilterList extends Filter {
50
51 public static enum Operator {
52
53 MUST_PASS_ALL,
54
55 MUST_PASS_ONE
56 }
57
58 private static final Configuration conf = HBaseConfiguration.create();
59 private static final int MAX_LOG_FILTERS = 5;
60 private Operator operator = Operator.MUST_PASS_ALL;
61 private List<Filter> filters = new ArrayList<Filter>();
62
63
64
65
66
67
68
69 public FilterList(final List<Filter> rowFilters) {
70 if (rowFilters instanceof ArrayList) {
71 this.filters = rowFilters;
72 } else {
73 this.filters = new ArrayList<Filter>(rowFilters);
74 }
75 }
76
77
78
79
80
81
82 public FilterList(final Filter... rowFilters) {
83 this.filters = new ArrayList<Filter>(Arrays.asList(rowFilters));
84 }
85
86
87
88
89
90
91 public FilterList(final Operator operator) {
92 this.operator = operator;
93 }
94
95
96
97
98
99
100
101 public FilterList(final Operator operator, final List<Filter> rowFilters) {
102 this.filters = new ArrayList<Filter>(rowFilters);
103 this.operator = operator;
104 }
105
106
107
108
109
110
111
112 public FilterList(final Operator operator, final Filter... rowFilters) {
113 this.filters = new ArrayList<Filter>(Arrays.asList(rowFilters));
114 this.operator = operator;
115 }
116
117
118
119
120
121
122 public Operator getOperator() {
123 return operator;
124 }
125
126
127
128
129
130
131 public List<Filter> getFilters() {
132 return filters;
133 }
134
135
136
137
138
139
140 public void addFilter(Filter filter) {
141 this.filters.add(filter);
142 }
143
144 @Override
145 public void reset() throws IOException {
146 for (Filter filter : filters) {
147 filter.reset();
148 }
149 }
150
151 @Override
152 public boolean filterRowKey(byte[] rowKey, int offset, int length) throws IOException {
153 for (Filter filter : filters) {
154 if (this.operator == Operator.MUST_PASS_ALL) {
155 if (filter.filterAllRemaining() ||
156 filter.filterRowKey(rowKey, offset, length)) {
157 return true;
158 }
159 } else if (this.operator == Operator.MUST_PASS_ONE) {
160 if (!filter.filterAllRemaining() &&
161 !filter.filterRowKey(rowKey, offset, length)) {
162 return false;
163 }
164 }
165 }
166 return this.operator == Operator.MUST_PASS_ONE;
167 }
168
169 @Override
170 public boolean filterAllRemaining() throws IOException {
171 for (Filter filter : filters) {
172 if (filter.filterAllRemaining()) {
173 if (operator == Operator.MUST_PASS_ALL) {
174 return true;
175 }
176 } else {
177 if (operator == Operator.MUST_PASS_ONE) {
178 return false;
179 }
180 }
181 }
182 return operator == Operator.MUST_PASS_ONE;
183 }
184
185 @Override
186 public KeyValue transform(KeyValue v) throws IOException {
187 KeyValue current = v;
188 for (Filter filter : filters) {
189 current = filter.transform(current);
190 }
191 return current;
192 }
193
194 @Override
195 public ReturnCode filterKeyValue(KeyValue v) throws IOException {
196 ReturnCode rc = operator == Operator.MUST_PASS_ONE?
197 ReturnCode.SKIP: ReturnCode.INCLUDE;
198 for (Filter filter : filters) {
199 if (operator == Operator.MUST_PASS_ALL) {
200 if (filter.filterAllRemaining()) {
201 return ReturnCode.NEXT_ROW;
202 }
203 ReturnCode code = filter.filterKeyValue(v);
204 switch (code) {
205
206 case INCLUDE_AND_NEXT_COL:
207 rc = ReturnCode.INCLUDE_AND_NEXT_COL;
208 case INCLUDE:
209 continue;
210 default:
211 return code;
212 }
213 } else if (operator == Operator.MUST_PASS_ONE) {
214 if (filter.filterAllRemaining()) {
215 continue;
216 }
217
218 switch (filter.filterKeyValue(v)) {
219 case INCLUDE:
220 if (rc != ReturnCode.INCLUDE_AND_NEXT_COL) {
221 rc = ReturnCode.INCLUDE;
222 }
223 break;
224 case INCLUDE_AND_NEXT_COL:
225 rc = ReturnCode.INCLUDE_AND_NEXT_COL;
226
227 break;
228 case NEXT_ROW:
229 break;
230 case SKIP:
231
232 break;
233 case NEXT_COL:
234 break;
235 case SEEK_NEXT_USING_HINT:
236 break;
237 default:
238 throw new IllegalStateException("Received code is not valid.");
239 }
240 }
241 }
242 return rc;
243 }
244
245 @Override
246 public void filterRow(List<KeyValue> kvs) throws IOException {
247 for (Filter filter : filters) {
248 filter.filterRow(kvs);
249 }
250 }
251
252 @Override
253 public boolean hasFilterRow() {
254 for (Filter filter : filters) {
255 if(filter.hasFilterRow()) {
256 return true;
257 }
258 }
259 return false;
260 }
261
262 @Override
263 public boolean filterRow() throws IOException {
264 for (Filter filter : filters) {
265 if (operator == Operator.MUST_PASS_ALL) {
266 if (filter.filterRow()) {
267 return true;
268 }
269 } else if (operator == Operator.MUST_PASS_ONE) {
270 if (!filter.filterRow()) {
271 return false;
272 }
273 }
274 }
275 return operator == Operator.MUST_PASS_ONE;
276 }
277
278
279
280
281 public byte[] toByteArray() throws IOException {
282 FilterProtos.FilterList.Builder builder =
283 FilterProtos.FilterList.newBuilder();
284 builder.setOperator(FilterProtos.FilterList.Operator.valueOf(operator.name()));
285 for (Filter filter : filters) {
286 builder.addFilters(ProtobufUtil.toFilter(filter));
287 }
288 return builder.build().toByteArray();
289 }
290
291
292
293
294
295
296
297 public static FilterList parseFrom(final byte [] pbBytes)
298 throws DeserializationException {
299 FilterProtos.FilterList proto;
300 try {
301 proto = FilterProtos.FilterList.parseFrom(pbBytes);
302 } catch (InvalidProtocolBufferException e) {
303 throw new DeserializationException(e);
304 }
305
306 List<Filter> rowFilters = new ArrayList<Filter>(proto.getFiltersCount());
307 try {
308 for (HBaseProtos.Filter filter : proto.getFiltersList()) {
309 rowFilters.add(ProtobufUtil.toFilter(filter));
310 }
311 } catch (IOException ioe) {
312 throw new DeserializationException(ioe);
313 }
314 return new FilterList(Operator.valueOf(proto.getOperator().name()),rowFilters);
315 }
316
317
318
319
320
321
322 boolean areSerializedFieldsEqual(Filter o) {
323 if (o == this) return true;
324 if (!(o instanceof FilterList)) return false;
325
326 FilterList other = (FilterList)o;
327 return this.getOperator().equals(other.getOperator()) &&
328 ((this.getFilters() == other.getFilters())
329 || this.getFilters().equals(other.getFilters()));
330 }
331
332 @Override
333 public KeyValue getNextKeyHint(KeyValue currentKV) throws IOException {
334 KeyValue keyHint = null;
335 for (Filter filter : filters) {
336 KeyValue curKeyHint = filter.getNextKeyHint(currentKV);
337 if (curKeyHint == null && operator == Operator.MUST_PASS_ONE) {
338
339 return null;
340 }
341 if (curKeyHint != null) {
342
343 if (keyHint == null) {
344 keyHint = curKeyHint;
345 continue;
346 }
347
348 if (operator == Operator.MUST_PASS_ALL &&
349 KeyValue.COMPARATOR.compare(keyHint, curKeyHint) < 0) {
350
351 keyHint = curKeyHint;
352 } else if (operator == Operator.MUST_PASS_ONE &&
353 KeyValue.COMPARATOR.compare(keyHint, curKeyHint) > 0) {
354
355 keyHint = curKeyHint;
356 }
357 }
358 }
359 return keyHint;
360 }
361
362 @Override
363 public boolean isFamilyEssential(byte[] name) throws IOException {
364 for (Filter filter : filters) {
365 if (filter.isFamilyEssential(name)) {
366 return true;
367 }
368 }
369 return false;
370 }
371
372 @Override
373 public String toString() {
374 return toString(MAX_LOG_FILTERS);
375 }
376
377 protected String toString(int maxFilters) {
378 int endIndex = this.filters.size() < maxFilters
379 ? this.filters.size() : maxFilters;
380 return String.format("%s %s (%d/%d): %s",
381 this.getClass().getSimpleName(),
382 this.operator == Operator.MUST_PASS_ALL ? "AND" : "OR",
383 endIndex,
384 this.filters.size(),
385 this.filters.subList(0, endIndex).toString());
386 }
387 }