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