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.List;
33
34
35
36
37
38
39
40
41
42
43 public class FilterList implements Filter {
44
45 public static enum Operator {
46
47 MUST_PASS_ALL,
48
49 MUST_PASS_ONE
50 }
51
52 private static final Configuration conf = HBaseConfiguration.create();
53 private Operator operator = Operator.MUST_PASS_ALL;
54 private List<Filter> filters = new ArrayList<Filter>();
55
56
57
58
59
60 public FilterList() {
61 super();
62 }
63
64
65
66
67
68
69
70 public FilterList(final List<Filter> rowFilters) {
71 this.filters = rowFilters;
72 }
73
74
75
76
77
78
79 public FilterList(final Operator operator) {
80 this.operator = operator;
81 }
82
83
84
85
86
87
88
89 public FilterList(final Operator operator, final List<Filter> rowFilters) {
90 this.filters = rowFilters;
91 this.operator = operator;
92 }
93
94
95
96
97
98
99 public Operator getOperator() {
100 return operator;
101 }
102
103
104
105
106
107
108 public List<Filter> getFilters() {
109 return filters;
110 }
111
112
113
114
115
116
117 public void addFilter(Filter filter) {
118 this.filters.add(filter);
119 }
120
121 @Override
122 public void reset() {
123 for (Filter filter : filters) {
124 filter.reset();
125 }
126 }
127
128 @Override
129 public boolean filterRowKey(byte[] rowKey, int offset, int length) {
130 for (Filter filter : filters) {
131 if (this.operator == Operator.MUST_PASS_ALL) {
132 if (filter.filterAllRemaining() ||
133 filter.filterRowKey(rowKey, offset, length)) {
134 return true;
135 }
136 } else if (this.operator == Operator.MUST_PASS_ONE) {
137 if (!filter.filterAllRemaining() &&
138 !filter.filterRowKey(rowKey, offset, length)) {
139 return false;
140 }
141 }
142 }
143 return this.operator == Operator.MUST_PASS_ONE;
144 }
145
146 @Override
147 public boolean filterAllRemaining() {
148 for (Filter filter : filters) {
149 if (filter.filterAllRemaining()) {
150 if (operator == Operator.MUST_PASS_ALL) {
151 return true;
152 }
153 } else {
154 if (operator == Operator.MUST_PASS_ONE) {
155 return false;
156 }
157 }
158 }
159 return operator == Operator.MUST_PASS_ONE;
160 }
161
162 @Override
163 public ReturnCode filterKeyValue(KeyValue v) {
164 for (Filter filter : filters) {
165 if (operator == Operator.MUST_PASS_ALL) {
166 if (filter.filterAllRemaining()) {
167 return ReturnCode.NEXT_ROW;
168 }
169 switch (filter.filterKeyValue(v)) {
170 case INCLUDE:
171 continue;
172 case NEXT_ROW:
173 case SKIP:
174 return ReturnCode.SKIP;
175 }
176 } else if (operator == Operator.MUST_PASS_ONE) {
177 if (filter.filterAllRemaining()) {
178 continue;
179 }
180
181 switch (filter.filterKeyValue(v)) {
182 case INCLUDE:
183 return ReturnCode.INCLUDE;
184 case NEXT_ROW:
185 case SKIP:
186
187 }
188 }
189 }
190 return operator == Operator.MUST_PASS_ONE?
191 ReturnCode.SKIP: ReturnCode.INCLUDE;
192 }
193
194 @Override
195 public void filterRow(List<KeyValue> kvs) {
196 for (Filter filter : filters) {
197 filter.filterRow(kvs);
198 }
199 }
200
201 @Override
202 public boolean hasFilterRow() {
203 for (Filter filter : filters) {
204 if(filter.hasFilterRow()) {
205 return true;
206 }
207 }
208 return false;
209 }
210
211 @Override
212 public boolean filterRow() {
213 for (Filter filter : filters) {
214 if (operator == Operator.MUST_PASS_ALL) {
215 if (filter.filterAllRemaining() || filter.filterRow()) {
216 return true;
217 }
218 } else if (operator == Operator.MUST_PASS_ONE) {
219 if (!filter.filterAllRemaining()
220 && !filter.filterRow()) {
221 return false;
222 }
223 }
224 }
225 return operator == Operator.MUST_PASS_ONE;
226 }
227
228 public void readFields(final DataInput in) throws IOException {
229 byte opByte = in.readByte();
230 operator = Operator.values()[opByte];
231 int size = in.readInt();
232 if (size > 0) {
233 filters = new ArrayList<Filter>(size);
234 for (int i = 0; i < size; i++) {
235 Filter filter = (Filter)HbaseObjectWritable.readObject(in, conf);
236 filters.add(filter);
237 }
238 }
239 }
240
241 public void write(final DataOutput out) throws IOException {
242 out.writeByte(operator.ordinal());
243 out.writeInt(filters.size());
244 for (Filter filter : filters) {
245 HbaseObjectWritable.writeObject(out, filter, Writable.class, conf);
246 }
247 }
248 }