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