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