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 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 public FilterList() {
85 super();
86 }
87
88
89
90
91
92
93
94 public FilterList(final List<Filter> rowFilters) {
95 this.filters = rowFilters;
96 }
97
98
99
100
101
102
103 public FilterList(final Filter... rowFilters) {
104 this.filters = 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 = 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 = 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 this.filters.add(filter);
163 }
164
165 @Override
166 public void reset() {
167 for (Filter filter : filters) {
168 filter.reset();
169 }
170 seekHintFilter = null;
171 }
172
173 @Override
174 public boolean filterRowKey(byte[] rowKey, int offset, int length) {
175 boolean flag = (this.operator == Operator.MUST_PASS_ONE) ? true : false;
176 for (Filter filter : filters) {
177 if (this.operator == Operator.MUST_PASS_ALL) {
178 if (filter.filterAllRemaining() ||
179 filter.filterRowKey(rowKey, offset, length)) {
180 flag = true;
181 }
182 } else if (this.operator == Operator.MUST_PASS_ONE) {
183 if (!filter.filterAllRemaining() &&
184 !filter.filterRowKey(rowKey, offset, length)) {
185 flag = false;
186 }
187 }
188 }
189 return flag;
190 }
191
192 @Override
193 public boolean filterAllRemaining() {
194 for (Filter filter : filters) {
195 if (filter.filterAllRemaining()) {
196 if (operator == Operator.MUST_PASS_ALL) {
197 return true;
198 }
199 } else {
200 if (operator == Operator.MUST_PASS_ONE) {
201 return false;
202 }
203 }
204 }
205 return operator == Operator.MUST_PASS_ONE;
206 }
207
208 @Override
209 public KeyValue transform(KeyValue v) {
210
211 if (!v.equals(this.referenceKV)) {
212 throw new IllegalStateException(
213 "Reference KeyValue: " + this.referenceKV + " does not match: " + v);
214 }
215 return this.transformedKV;
216 }
217
218 @Override
219 public ReturnCode filterKeyValue(KeyValue v) {
220 this.referenceKV = v;
221
222
223 KeyValue transformed = v;
224
225 ReturnCode rc = operator == Operator.MUST_PASS_ONE?
226 ReturnCode.SKIP: ReturnCode.INCLUDE;
227 for (Filter filter : filters) {
228 if (operator == Operator.MUST_PASS_ALL) {
229 if (filter.filterAllRemaining()) {
230 return ReturnCode.NEXT_ROW;
231 }
232 ReturnCode code = filter.filterKeyValue(v);
233 switch (code) {
234
235 case INCLUDE_AND_NEXT_COL:
236 rc = ReturnCode.INCLUDE_AND_NEXT_COL;
237 case INCLUDE:
238 transformed = filter.transform(transformed);
239 continue;
240 case SEEK_NEXT_USING_HINT:
241 seekHintFilter = filter;
242 return code;
243 default:
244 return code;
245 }
246 } else if (operator == Operator.MUST_PASS_ONE) {
247 if (filter.filterAllRemaining()) {
248 continue;
249 }
250
251 ReturnCode code = filter.filterKeyValue(v);
252 switch (code) {
253 case INCLUDE:
254 if (rc != ReturnCode.INCLUDE_AND_NEXT_COL) {
255 rc = ReturnCode.INCLUDE;
256 }
257 transformed = filter.transform(transformed);
258 break;
259 case INCLUDE_AND_NEXT_COL:
260 rc = ReturnCode.INCLUDE_AND_NEXT_COL;
261 transformed = filter.transform(transformed);
262
263 break;
264 case NEXT_ROW:
265 break;
266 case SKIP:
267 break;
268 case NEXT_COL:
269 break;
270 case SEEK_NEXT_USING_HINT:
271 break;
272 default:
273 throw new IllegalStateException("Received code is not valid.");
274 }
275 }
276 }
277
278
279 this.transformedKV = transformed;
280
281 return rc;
282 }
283
284 @Override
285 public void filterRow(List<KeyValue> kvs) {
286 for (Filter filter : filters) {
287 filter.filterRow(kvs);
288 }
289 }
290
291 @Override
292 public boolean hasFilterRow() {
293 for (Filter filter : filters) {
294 if(filter.hasFilterRow()) {
295 return true;
296 }
297 }
298 return false;
299 }
300
301 @Override
302 public boolean filterRow() {
303 for (Filter filter : filters) {
304 if (operator == Operator.MUST_PASS_ALL) {
305 if (filter.filterRow()) {
306 return true;
307 }
308 } else if (operator == Operator.MUST_PASS_ONE) {
309 if (!filter.filterRow()) {
310 return false;
311 }
312 }
313 }
314 return operator == Operator.MUST_PASS_ONE;
315 }
316
317 public void readFields(final DataInput in) throws IOException {
318 byte opByte = in.readByte();
319 operator = Operator.values()[opByte];
320 int size = in.readInt();
321 if (size > 0) {
322 filters = new ArrayList<Filter>(size);
323 for (int i = 0; i < size; i++) {
324 Filter filter = (Filter)HbaseObjectWritable.readObject(in, conf);
325 filters.add(filter);
326 }
327 }
328 }
329
330 public void write(final DataOutput out) throws IOException {
331 out.writeByte(operator.ordinal());
332 out.writeInt(filters.size());
333 for (Filter filter : filters) {
334 HbaseObjectWritable.writeObject(out, filter, Writable.class, conf);
335 }
336 }
337
338 @Override
339 public KeyValue getNextKeyHint(KeyValue currentKV) {
340 KeyValue keyHint = null;
341 if (operator == Operator.MUST_PASS_ALL) {
342 keyHint = seekHintFilter.getNextKeyHint(currentKV);
343 return keyHint;
344 }
345
346 for (Filter filter : filters) {
347 KeyValue curKeyHint = filter.getNextKeyHint(currentKV);
348 if (curKeyHint == null) {
349
350 return null;
351 }
352 if (curKeyHint != null) {
353
354 if (keyHint == null) {
355 keyHint = curKeyHint;
356 continue;
357 }
358 if (KeyValue.COMPARATOR.compare(keyHint, curKeyHint) > 0) {
359
360 keyHint = curKeyHint;
361 }
362 }
363 }
364 return keyHint;
365 }
366
367 public boolean isFamilyEssential(byte[] name) {
368 for (Filter filter : filters) {
369 if (FilterBase.isFamilyEssential(filter, name)) {
370 return true;
371 }
372 }
373 return false;
374 }
375
376 @Override
377 public String toString() {
378 return toString(MAX_LOG_FILTERS);
379 }
380
381 protected String toString(int maxFilters) {
382 int endIndex = this.filters.size() < maxFilters
383 ? this.filters.size() : maxFilters;
384 return String.format("%s %s (%d/%d): %s",
385 this.getClass().getSimpleName(),
386 this.operator == Operator.MUST_PASS_ALL ? "AND" : "OR",
387 endIndex,
388 this.filters.size(),
389 this.filters.subList(0, endIndex).toString());
390 }
391 }