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