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 public class FilterList implements Filter {
45
46 public static enum Operator {
47
48 MUST_PASS_ALL,
49
50 MUST_PASS_ONE
51 }
52
53 private static final Configuration conf = HBaseConfiguration.create();
54 private static final int MAX_LOG_FILTERS = 5;
55 private Operator operator = Operator.MUST_PASS_ALL;
56 private List<Filter> filters = new ArrayList<Filter>();
57
58
59
60
61
62 public FilterList() {
63 super();
64 }
65
66
67
68
69
70
71
72 public FilterList(final List<Filter> rowFilters) {
73 this.filters = rowFilters;
74 }
75
76
77
78
79
80
81 public FilterList(final Filter... rowFilters) {
82 this.filters = Arrays.asList(rowFilters);
83 }
84
85
86
87
88
89
90 public FilterList(final Operator operator) {
91 this.operator = operator;
92 }
93
94
95
96
97
98
99
100 public FilterList(final Operator operator, final List<Filter> rowFilters) {
101 this.filters = rowFilters;
102 this.operator = operator;
103 }
104
105
106
107
108
109
110
111 public FilterList(final Operator operator, final Filter... rowFilters) {
112 this.filters = Arrays.asList(rowFilters);
113 this.operator = operator;
114 }
115
116
117
118
119
120
121 public Operator getOperator() {
122 return operator;
123 }
124
125
126
127
128
129
130 public List<Filter> getFilters() {
131 return filters;
132 }
133
134
135
136
137
138
139 public void addFilter(Filter filter) {
140 this.filters.add(filter);
141 }
142
143 @Override
144 public void reset() {
145 for (Filter filter : filters) {
146 filter.reset();
147 }
148 }
149
150 @Override
151 public boolean filterRowKey(byte[] rowKey, int offset, int length) {
152 for (Filter filter : filters) {
153 if (this.operator == Operator.MUST_PASS_ALL) {
154 if (filter.filterAllRemaining() ||
155 filter.filterRowKey(rowKey, offset, length)) {
156 return true;
157 }
158 } else if (this.operator == Operator.MUST_PASS_ONE) {
159 if (!filter.filterAllRemaining() &&
160 !filter.filterRowKey(rowKey, offset, length)) {
161 return false;
162 }
163 }
164 }
165 return this.operator == Operator.MUST_PASS_ONE;
166 }
167
168 @Override
169 public boolean filterAllRemaining() {
170 for (Filter filter : filters) {
171 if (filter.filterAllRemaining()) {
172 if (operator == Operator.MUST_PASS_ALL) {
173 return true;
174 }
175 } else {
176 if (operator == Operator.MUST_PASS_ONE) {
177 return false;
178 }
179 }
180 }
181 return operator == Operator.MUST_PASS_ONE;
182 }
183
184 @Override
185 public KeyValue transform(KeyValue v) {
186 KeyValue current = v;
187 for (Filter filter : filters) {
188 current = filter.transform(current);
189 }
190 return current;
191 }
192
193 @Override
194 public ReturnCode filterKeyValue(KeyValue v) {
195 ReturnCode rc = operator == Operator.MUST_PASS_ONE?
196 ReturnCode.SKIP: ReturnCode.INCLUDE;
197 for (Filter filter : filters) {
198 if (operator == Operator.MUST_PASS_ALL) {
199 if (filter.filterAllRemaining()) {
200 return ReturnCode.NEXT_ROW;
201 }
202 ReturnCode code = filter.filterKeyValue(v);
203 switch (code) {
204
205 case INCLUDE_AND_NEXT_COL:
206 rc = ReturnCode.INCLUDE_AND_NEXT_COL;
207 case INCLUDE:
208 continue;
209 default:
210 return code;
211 }
212 } else if (operator == Operator.MUST_PASS_ONE) {
213 if (filter.filterAllRemaining()) {
214 continue;
215 }
216
217 switch (filter.filterKeyValue(v)) {
218 case INCLUDE:
219 if (rc != ReturnCode.INCLUDE_AND_NEXT_COL) {
220 rc = ReturnCode.INCLUDE;
221 }
222 break;
223 case INCLUDE_AND_NEXT_COL:
224 rc = ReturnCode.INCLUDE_AND_NEXT_COL;
225
226 case NEXT_ROW:
227 case SKIP:
228
229 }
230 }
231 }
232 return rc;
233 }
234
235 @Override
236 public void filterRow(List<KeyValue> kvs) {
237 for (Filter filter : filters) {
238 filter.filterRow(kvs);
239 }
240 }
241
242 @Override
243 public boolean hasFilterRow() {
244 for (Filter filter : filters) {
245 if(filter.hasFilterRow()) {
246 return true;
247 }
248 }
249 return false;
250 }
251
252 @Override
253 public boolean filterRow() {
254 for (Filter filter : filters) {
255 if (operator == Operator.MUST_PASS_ALL) {
256 if (filter.filterRow()) {
257 return true;
258 }
259 } else if (operator == Operator.MUST_PASS_ONE) {
260 if (!filter.filterRow()) {
261 return false;
262 }
263 }
264 }
265 return operator == Operator.MUST_PASS_ONE;
266 }
267
268 public void readFields(final DataInput in) throws IOException {
269 byte opByte = in.readByte();
270 operator = Operator.values()[opByte];
271 int size = in.readInt();
272 if (size > 0) {
273 filters = new ArrayList<Filter>(size);
274 for (int i = 0; i < size; i++) {
275 Filter filter = (Filter)HbaseObjectWritable.readObject(in, conf);
276 filters.add(filter);
277 }
278 }
279 }
280
281 public void write(final DataOutput out) throws IOException {
282 out.writeByte(operator.ordinal());
283 out.writeInt(filters.size());
284 for (Filter filter : filters) {
285 HbaseObjectWritable.writeObject(out, filter, Writable.class, conf);
286 }
287 }
288
289 @Override
290 public KeyValue getNextKeyHint(KeyValue currentKV) {
291 KeyValue keyHint = null;
292 for (Filter filter : filters) {
293 KeyValue curKeyHint = filter.getNextKeyHint(currentKV);
294 if (curKeyHint == null && operator == Operator.MUST_PASS_ONE) {
295
296 return null;
297 }
298 if (curKeyHint != null) {
299
300 if (keyHint == null) {
301 keyHint = curKeyHint;
302 continue;
303 }
304
305 if (operator == Operator.MUST_PASS_ALL &&
306 KeyValue.COMPARATOR.compare(keyHint, curKeyHint) < 0) {
307
308 keyHint = curKeyHint;
309 } else if (operator == Operator.MUST_PASS_ONE &&
310 KeyValue.COMPARATOR.compare(keyHint, curKeyHint) > 0) {
311
312 keyHint = curKeyHint;
313 }
314 }
315 }
316 return keyHint;
317 }
318
319 public boolean isFamilyEssential(byte[] name) {
320 for (Filter filter : filters) {
321 if (FilterBase.isFamilyEssential(filter, name)) {
322 return true;
323 }
324 }
325 return false;
326 }
327
328 @Override
329 public String toString() {
330 return toString(MAX_LOG_FILTERS);
331 }
332
333 protected String toString(int maxFilters) {
334 int endIndex = this.filters.size() < maxFilters
335 ? this.filters.size() : maxFilters;
336 return String.format("%s %s (%d/%d): %s",
337 this.getClass().getSimpleName(),
338 this.operator == Operator.MUST_PASS_ALL ? "AND" : "OR",
339 endIndex,
340 this.filters.size(),
341 this.filters.subList(0, endIndex).toString());
342 }
343 }