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.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
45
46
47
48
49
50
51
52
53
54 @InterfaceAudience.Public
55 @InterfaceStability.Stable
56 final public class FilterList extends Filter {
57
58 @InterfaceAudience.Public
59 @InterfaceStability.Stable
60 public static enum Operator {
61
62 MUST_PASS_ALL,
63
64 MUST_PASS_ONE
65 }
66
67 private static final int MAX_LOG_FILTERS = 5;
68 private Operator operator = Operator.MUST_PASS_ALL;
69 private List<Filter> filters = new ArrayList<Filter>();
70 private Filter seekHintFilter = null;
71
72
73 private Cell referenceKV = null;
74
75
76
77
78
79
80
81
82 private Cell transformedKV = null;
83
84
85
86
87
88
89
90 public FilterList(final List<Filter> rowFilters) {
91 if (rowFilters instanceof ArrayList) {
92 this.filters = rowFilters;
93 } else {
94 this.filters = new ArrayList<Filter>(rowFilters);
95 }
96 }
97
98
99
100
101
102
103 public FilterList(final Filter... rowFilters) {
104 this.filters = new ArrayList<Filter>(Arrays.asList(rowFilters));
105 }
106
107
108
109
110
111
112 public FilterList(final Operator operator) {
113 this.operator = operator;
114 }
115
116
117
118
119
120
121
122 public FilterList(final Operator operator, final List<Filter> rowFilters) {
123 this.filters = new ArrayList<Filter>(rowFilters);
124 this.operator = operator;
125 }
126
127
128
129
130
131
132
133 public FilterList(final Operator operator, final Filter... rowFilters) {
134 this.filters = new ArrayList<Filter>(Arrays.asList(rowFilters));
135 this.operator = operator;
136 }
137
138
139
140
141
142
143 public Operator getOperator() {
144 return operator;
145 }
146
147
148
149
150
151
152 public List<Filter> getFilters() {
153 return filters;
154 }
155
156
157
158
159
160
161 public void addFilter(Filter filter) {
162 if (this.isReversed() != filter.isReversed()) {
163 throw new IllegalArgumentException(
164 "Filters in the list must have the same reversed flag, this.reversed="
165 + this.isReversed());
166 }
167 this.filters.add(filter);
168 }
169
170 @Override
171 public void reset() throws IOException {
172 for (Filter filter : filters) {
173 filter.reset();
174 }
175 seekHintFilter = null;
176 }
177
178 @Override
179 public boolean filterRowKey(byte[] rowKey, int offset, int length) throws IOException {
180 boolean flag = (this.operator == Operator.MUST_PASS_ONE) ? true : false;
181 for (Filter filter : filters) {
182 if (this.operator == Operator.MUST_PASS_ALL) {
183 if (filter.filterAllRemaining() ||
184 filter.filterRowKey(rowKey, offset, length)) {
185 flag = true;
186 }
187 } else if (this.operator == Operator.MUST_PASS_ONE) {
188 if (!filter.filterAllRemaining() &&
189 !filter.filterRowKey(rowKey, offset, length)) {
190 flag = false;
191 }
192 }
193 }
194 return flag;
195 }
196
197 @Override
198 public boolean filterAllRemaining() throws IOException {
199 for (Filter filter : filters) {
200 if (filter.filterAllRemaining()) {
201 if (operator == Operator.MUST_PASS_ALL) {
202 return true;
203 }
204 } else {
205 if (operator == Operator.MUST_PASS_ONE) {
206 return false;
207 }
208 }
209 }
210 return operator == Operator.MUST_PASS_ONE;
211 }
212
213 @Override
214 public Cell transformCell(Cell v) throws IOException {
215 return transform(KeyValueUtil.ensureKeyValue(v));
216 }
217
218
219
220
221
222
223
224
225 @Deprecated
226 @Override
227 public KeyValue transform(KeyValue v) throws IOException {
228
229 if (!v.equals(this.referenceKV)) {
230 throw new IllegalStateException(
231 "Reference Cell: " + this.referenceKV + " does not match: " + v);
232 }
233 return KeyValueUtil.ensureKeyValue(this.transformedKV);
234 }
235
236
237 @Override
238 public ReturnCode filterKeyValue(Cell v) throws IOException {
239 this.referenceKV = v;
240
241
242 Cell transformed = v;
243
244 ReturnCode rc = operator == Operator.MUST_PASS_ONE?
245 ReturnCode.SKIP: ReturnCode.INCLUDE;
246 for (Filter filter : filters) {
247 if (operator == Operator.MUST_PASS_ALL) {
248 if (filter.filterAllRemaining()) {
249 return ReturnCode.NEXT_ROW;
250 }
251 ReturnCode code = filter.filterKeyValue(v);
252 switch (code) {
253
254 case INCLUDE_AND_NEXT_COL:
255 rc = ReturnCode.INCLUDE_AND_NEXT_COL;
256 case INCLUDE:
257 transformed = filter.transformCell(transformed);
258 continue;
259 case SEEK_NEXT_USING_HINT:
260 seekHintFilter = filter;
261 return code;
262 default:
263 return code;
264 }
265 } else if (operator == Operator.MUST_PASS_ONE) {
266 if (filter.filterAllRemaining()) {
267 continue;
268 }
269
270 switch (filter.filterKeyValue(v)) {
271 case INCLUDE:
272 if (rc != ReturnCode.INCLUDE_AND_NEXT_COL) {
273 rc = ReturnCode.INCLUDE;
274 }
275 transformed = filter.transformCell(transformed);
276 break;
277 case INCLUDE_AND_NEXT_COL:
278 rc = ReturnCode.INCLUDE_AND_NEXT_COL;
279 transformed = filter.transformCell(transformed);
280
281 break;
282 case NEXT_ROW:
283 break;
284 case SKIP:
285 break;
286 case NEXT_COL:
287 break;
288 case SEEK_NEXT_USING_HINT:
289 break;
290 default:
291 throw new IllegalStateException("Received code is not valid.");
292 }
293 }
294 }
295
296
297 this.transformedKV = transformed;
298
299 return rc;
300 }
301
302
303
304
305
306
307
308 @Override
309 public void filterRowCells(List<Cell> ignored) throws IOException {
310
311
312 List<KeyValue> kvs = new ArrayList<KeyValue>(ignored.size());
313 for (Cell c : ignored) {
314 kvs.add(KeyValueUtil.ensureKeyValue(c));
315 }
316 filterRow(kvs);
317 ignored.clear();
318 ignored.addAll(kvs);
319 }
320
321
322
323
324
325
326 @Override
327 @Deprecated
328 public void filterRow(List<KeyValue> kvs) throws IOException {
329
330
331
332
333 List<Cell> cells = new ArrayList<Cell>(kvs.size());
334 cells.addAll(kvs);
335 for (Filter filter : filters) {
336 filter.filterRowCells(cells);
337 }
338
339
340 kvs.clear();
341 for (Cell c : cells) {
342 kvs.add(KeyValueUtil.ensureKeyValue(c));
343 }
344 }
345
346 @Override
347 public boolean hasFilterRow() {
348 for (Filter filter : filters) {
349 if(filter.hasFilterRow()) {
350 return true;
351 }
352 }
353 return false;
354 }
355
356 @Override
357 public boolean filterRow() throws IOException {
358 for (Filter filter : filters) {
359 if (operator == Operator.MUST_PASS_ALL) {
360 if (filter.filterRow()) {
361 return true;
362 }
363 } else if (operator == Operator.MUST_PASS_ONE) {
364 if (!filter.filterRow()) {
365 return false;
366 }
367 }
368 }
369 return operator == Operator.MUST_PASS_ONE;
370 }
371
372
373
374
375 public byte[] toByteArray() throws IOException {
376 FilterProtos.FilterList.Builder builder =
377 FilterProtos.FilterList.newBuilder();
378 builder.setOperator(FilterProtos.FilterList.Operator.valueOf(operator.name()));
379 for (Filter filter : filters) {
380 builder.addFilters(ProtobufUtil.toFilter(filter));
381 }
382 return builder.build().toByteArray();
383 }
384
385
386
387
388
389
390
391 public static FilterList parseFrom(final byte [] pbBytes)
392 throws DeserializationException {
393 FilterProtos.FilterList proto;
394 try {
395 proto = FilterProtos.FilterList.parseFrom(pbBytes);
396 } catch (InvalidProtocolBufferException e) {
397 throw new DeserializationException(e);
398 }
399
400 List<Filter> rowFilters = new ArrayList<Filter>(proto.getFiltersCount());
401 try {
402 for (FilterProtos.Filter filter : proto.getFiltersList()) {
403 rowFilters.add(ProtobufUtil.toFilter(filter));
404 }
405 } catch (IOException ioe) {
406 throw new DeserializationException(ioe);
407 }
408 return new FilterList(Operator.valueOf(proto.getOperator().name()),rowFilters);
409 }
410
411
412
413
414
415
416 boolean areSerializedFieldsEqual(Filter other) {
417 if (other == this) return true;
418 if (!(other instanceof FilterList)) return false;
419
420 FilterList o = (FilterList)other;
421 return this.getOperator().equals(o.getOperator()) &&
422 ((this.getFilters() == o.getFilters())
423 || this.getFilters().equals(o.getFilters()));
424 }
425
426 @Override
427 @Deprecated
428 public KeyValue getNextKeyHint(KeyValue currentKV) throws IOException {
429 return KeyValueUtil.ensureKeyValue(getNextCellHint((Cell)currentKV));
430 }
431
432 @Override
433 public Cell getNextCellHint(Cell currentKV) throws IOException {
434 Cell keyHint = null;
435 if (operator == Operator.MUST_PASS_ALL) {
436 keyHint = seekHintFilter.getNextCellHint(currentKV);
437 return keyHint;
438 }
439
440
441 for (Filter filter : filters) {
442 Cell curKeyHint = filter.getNextCellHint(currentKV);
443 if (curKeyHint == null) {
444
445 return null;
446 }
447 if (curKeyHint != null) {
448
449 if (keyHint == null) {
450 keyHint = curKeyHint;
451 continue;
452 }
453 if (KeyValue.COMPARATOR.compare(keyHint, curKeyHint) > 0) {
454 keyHint = curKeyHint;
455 }
456 }
457 }
458 return keyHint;
459 }
460
461 @Override
462 public boolean isFamilyEssential(byte[] name) throws IOException {
463 for (Filter filter : filters) {
464 if (filter.isFamilyEssential(name)) {
465 return true;
466 }
467 }
468 return false;
469 }
470
471 @Override
472 public void setReversed(boolean reversed) {
473 for (Filter filter : filters) {
474 filter.setReversed(reversed);
475 }
476 this.reversed = reversed;
477 }
478
479 @Override
480 public String toString() {
481 return toString(MAX_LOG_FILTERS);
482 }
483
484 protected String toString(int maxFilters) {
485 int endIndex = this.filters.size() < maxFilters
486 ? this.filters.size() : maxFilters;
487 return String.format("%s %s (%d/%d): %s",
488 this.getClass().getSimpleName(),
489 this.operator == Operator.MUST_PASS_ALL ? "AND" : "OR",
490 endIndex,
491 this.filters.size(),
492 this.filters.subList(0, endIndex).toString());
493 }
494 }