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