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.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
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 int listSize = filters.size();
173 for (int i=0; i < listSize; i++) {
174 filters.get(i).reset();
175 }
176 seekHintFilter = null;
177 }
178
179 @Override
180 public boolean filterRowKey(byte[] rowKey, int offset, int length) throws IOException {
181 boolean flag = (this.operator == Operator.MUST_PASS_ONE) ? true : false;
182 int listSize = filters.size();
183 for (int i=0; i < listSize; i++) {
184 Filter filter = filters.get(i);
185 if (this.operator == Operator.MUST_PASS_ALL) {
186 if (filter.filterAllRemaining() ||
187 filter.filterRowKey(rowKey, offset, length)) {
188 flag = true;
189 }
190 } else if (this.operator == Operator.MUST_PASS_ONE) {
191 if (!filter.filterAllRemaining() &&
192 !filter.filterRowKey(rowKey, offset, length)) {
193 flag = false;
194 }
195 }
196 }
197 return flag;
198 }
199
200 @Override
201 public boolean filterAllRemaining() throws IOException {
202 int listSize = filters.size();
203 for (int i=0; i < listSize; i++) {
204 Filter filter = filters.get(i);
205 if (filter.filterAllRemaining()) {
206 if (operator == Operator.MUST_PASS_ALL) {
207 return true;
208 }
209 } else {
210 if (operator == Operator.MUST_PASS_ONE) {
211 return false;
212 }
213 }
214 }
215 return operator == Operator.MUST_PASS_ONE;
216 }
217
218 @Override
219 public Cell transformCell(Cell v) throws IOException {
220 return transform(KeyValueUtil.ensureKeyValue(v));
221 }
222
223
224
225
226
227
228
229
230 @Deprecated
231 @Override
232 public KeyValue transform(KeyValue v) throws IOException {
233
234 if (!v.equals(this.referenceKV)) {
235 throw new IllegalStateException(
236 "Reference Cell: " + this.referenceKV + " does not match: " + v);
237 }
238 return KeyValueUtil.ensureKeyValue(this.transformedKV);
239 }
240
241
242 @Override
243 public ReturnCode filterKeyValue(Cell v) throws IOException {
244 this.referenceKV = v;
245
246
247 Cell transformed = v;
248
249 ReturnCode rc = operator == Operator.MUST_PASS_ONE?
250 ReturnCode.SKIP: ReturnCode.INCLUDE;
251 int listSize = filters.size();
252 for (int i=0; i < listSize; i++) {
253 Filter filter = filters.get(i);
254 if (operator == Operator.MUST_PASS_ALL) {
255 if (filter.filterAllRemaining()) {
256 return ReturnCode.NEXT_ROW;
257 }
258 ReturnCode code = filter.filterKeyValue(v);
259 switch (code) {
260
261 case INCLUDE_AND_NEXT_COL:
262 rc = ReturnCode.INCLUDE_AND_NEXT_COL;
263 case INCLUDE:
264 transformed = filter.transformCell(transformed);
265 continue;
266 case SEEK_NEXT_USING_HINT:
267 seekHintFilter = filter;
268 return code;
269 default:
270 return code;
271 }
272 } else if (operator == Operator.MUST_PASS_ONE) {
273 if (filter.filterAllRemaining()) {
274 continue;
275 }
276
277 switch (filter.filterKeyValue(v)) {
278 case INCLUDE:
279 if (rc != ReturnCode.INCLUDE_AND_NEXT_COL) {
280 rc = ReturnCode.INCLUDE;
281 }
282 transformed = filter.transformCell(transformed);
283 break;
284 case INCLUDE_AND_NEXT_COL:
285 rc = ReturnCode.INCLUDE_AND_NEXT_COL;
286 transformed = filter.transformCell(transformed);
287
288 break;
289 case NEXT_ROW:
290 break;
291 case SKIP:
292 break;
293 case NEXT_COL:
294 break;
295 case SEEK_NEXT_USING_HINT:
296 break;
297 default:
298 throw new IllegalStateException("Received code is not valid.");
299 }
300 }
301 }
302
303
304 this.transformedKV = transformed;
305
306 return rc;
307 }
308
309
310
311
312
313
314
315 @Override
316 public void filterRowCells(List<Cell> ignored) throws IOException {
317
318
319 List<KeyValue> kvs = new ArrayList<KeyValue>(ignored.size());
320 int listSize = ignored.size();
321 for (int i=0; i < listSize; i++) {
322 kvs.add(KeyValueUtil.ensureKeyValue(ignored.get(i)));
323 }
324 filterRow(kvs);
325 ignored.clear();
326 ignored.addAll(kvs);
327 }
328
329
330
331
332
333
334 @Override
335 @Deprecated
336 public void filterRow(List<KeyValue> kvs) throws IOException {
337
338
339
340
341 List<Cell> cells = new ArrayList<Cell>(kvs.size());
342 cells.addAll(kvs);
343 int listSize = filters.size();
344 for (int i=0; i < listSize; i++) {
345 filters.get(i).filterRowCells(cells);
346 }
347
348
349 kvs.clear();
350 listSize = cells.size();
351 for (int i=0; i < listSize; i++) {
352 kvs.add(KeyValueUtil.ensureKeyValue(cells.get(i)));
353 }
354 }
355
356 @Override
357 public boolean hasFilterRow() {
358 int listSize = filters.size();
359 for (int i=0; i < listSize; i++) {
360 if(filters.get(i).hasFilterRow()) {
361 return true;
362 }
363 }
364 return false;
365 }
366
367 @Override
368 public boolean filterRow() throws IOException {
369 int listSize = filters.size();
370 for (int i=0; i < listSize; i++) {
371 Filter filter = filters.get(i);
372 if (operator == Operator.MUST_PASS_ALL) {
373 if (filter.filterRow()) {
374 return true;
375 }
376 } else if (operator == Operator.MUST_PASS_ONE) {
377 if (!filter.filterRow()) {
378 return false;
379 }
380 }
381 }
382 return operator == Operator.MUST_PASS_ONE;
383 }
384
385
386
387
388 public byte[] toByteArray() throws IOException {
389 FilterProtos.FilterList.Builder builder =
390 FilterProtos.FilterList.newBuilder();
391 builder.setOperator(FilterProtos.FilterList.Operator.valueOf(operator.name()));
392 for (Filter filter : filters) {
393 builder.addFilters(ProtobufUtil.toFilter(filter));
394 }
395 return builder.build().toByteArray();
396 }
397
398
399
400
401
402
403
404 public static FilterList parseFrom(final byte [] pbBytes)
405 throws DeserializationException {
406 FilterProtos.FilterList proto;
407 try {
408 proto = FilterProtos.FilterList.parseFrom(pbBytes);
409 } catch (InvalidProtocolBufferException e) {
410 throw new DeserializationException(e);
411 }
412
413 List<Filter> rowFilters = new ArrayList<Filter>(proto.getFiltersCount());
414 try {
415 for (FilterProtos.Filter filter : proto.getFiltersList()) {
416 rowFilters.add(ProtobufUtil.toFilter(filter));
417 }
418 } catch (IOException ioe) {
419 throw new DeserializationException(ioe);
420 }
421 return new FilterList(Operator.valueOf(proto.getOperator().name()),rowFilters);
422 }
423
424
425
426
427
428
429 boolean areSerializedFieldsEqual(Filter other) {
430 if (other == this) return true;
431 if (!(other instanceof FilterList)) return false;
432
433 FilterList o = (FilterList)other;
434 return this.getOperator().equals(o.getOperator()) &&
435 ((this.getFilters() == o.getFilters())
436 || this.getFilters().equals(o.getFilters()));
437 }
438
439 @Override
440 @Deprecated
441 public KeyValue getNextKeyHint(KeyValue currentKV) throws IOException {
442 return KeyValueUtil.ensureKeyValue(getNextCellHint((Cell)currentKV));
443 }
444
445 @Override
446 public Cell getNextCellHint(Cell currentKV) throws IOException {
447 Cell keyHint = null;
448 if (operator == Operator.MUST_PASS_ALL) {
449 keyHint = seekHintFilter.getNextCellHint(currentKV);
450 return keyHint;
451 }
452
453
454 int listSize = filters.size();
455 for (int i=0; i < listSize; i++) {
456 Cell curKeyHint = filters.get(i).getNextCellHint(currentKV);
457 if (curKeyHint == null) {
458
459 return null;
460 }
461 if (curKeyHint != null) {
462
463 if (keyHint == null) {
464 keyHint = curKeyHint;
465 continue;
466 }
467 if (KeyValue.COMPARATOR.compare(keyHint, curKeyHint) > 0) {
468 keyHint = curKeyHint;
469 }
470 }
471 }
472 return keyHint;
473 }
474
475 @Override
476 public boolean isFamilyEssential(byte[] name) throws IOException {
477 int listSize = filters.size();
478 for (int i=0; i < listSize; i++) {
479 if (filters.get(i).isFamilyEssential(name)) {
480 return true;
481 }
482 }
483 return false;
484 }
485
486 @Override
487 public void setReversed(boolean reversed) {
488 for (Filter filter : filters) {
489 filter.setReversed(reversed);
490 }
491 this.reversed = reversed;
492 }
493
494 @Override
495 public String toString() {
496 return toString(MAX_LOG_FILTERS);
497 }
498
499 protected String toString(int maxFilters) {
500 int endIndex = this.filters.size() < maxFilters
501 ? this.filters.size() : maxFilters;
502 return String.format("%s %s (%d/%d): %s",
503 this.getClass().getSimpleName(),
504 this.operator == Operator.MUST_PASS_ALL ? "AND" : "OR",
505 endIndex,
506 this.filters.size(),
507 this.filters.subList(0, endIndex).toString());
508 }
509 }