1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.filter;
20
21 import java.io.IOException;
22 import java.util.ArrayList;
23 import java.util.Arrays;
24 import java.util.Collections;
25 import java.util.List;
26 import java.util.Objects;
27
28 import com.google.protobuf.InvalidProtocolBufferException;
29 import org.apache.hadoop.hbase.Cell;
30 import org.apache.hadoop.hbase.KeyValue;
31 import org.apache.hadoop.hbase.KeyValueUtil;
32 import org.apache.hadoop.hbase.classification.InterfaceAudience;
33 import org.apache.hadoop.hbase.classification.InterfaceStability;
34 import org.apache.hadoop.hbase.exceptions.DeserializationException;
35 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
36 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
37
38
39
40
41
42
43
44
45
46
47
48 @InterfaceAudience.Public
49 @InterfaceStability.Stable
50 final public class FilterList extends Filter {
51
52
53 @InterfaceAudience.Public
54 @InterfaceStability.Stable
55 public enum Operator {
56
57 MUST_PASS_ALL,
58
59 MUST_PASS_ONE
60 }
61
62 private Operator operator = Operator.MUST_PASS_ALL;
63 private FilterListBase filterListBase;
64
65
66
67
68
69
70 public FilterList(final Operator operator, final List<Filter> filters) {
71 if (operator == Operator.MUST_PASS_ALL) {
72 filterListBase = new FilterListWithAND(filters);
73 } else if (operator == Operator.MUST_PASS_ONE) {
74 filterListBase = new FilterListWithOR(filters);
75 } else {
76 throw new IllegalArgumentException("Invalid operator: " + operator);
77 }
78 this.operator = operator;
79 }
80
81
82
83
84
85
86 public FilterList(final List<Filter> filters) {
87 this(Operator.MUST_PASS_ALL, filters);
88 }
89
90
91
92
93
94
95 public FilterList(final Filter... filters) {
96 this(Operator.MUST_PASS_ALL, Arrays.asList(filters));
97 }
98
99
100
101
102
103 public FilterList(final Operator operator) {
104 this(operator, new ArrayList<Filter>());
105 }
106
107
108
109
110
111
112 public FilterList(final Operator operator, final Filter... filters) {
113 this(operator, Arrays.asList(filters));
114 }
115
116
117
118
119
120 public Operator getOperator() {
121 return operator;
122 }
123
124
125
126
127
128 public List<Filter> getFilters() {
129 return filterListBase.getFilters();
130 }
131
132 public int size() {
133 return filterListBase.size();
134 }
135
136 public void addFilter(List<Filter> filters) {
137 filterListBase.addFilterLists(filters);
138 }
139
140
141
142
143
144 public void addFilter(Filter filter) {
145 addFilter(Collections.singletonList(filter));
146 }
147
148 @Override
149 public void reset() throws IOException {
150 filterListBase.reset();
151 }
152
153 @Override
154 public boolean filterRowKey(byte[] rowKey, int offset, int length) throws IOException {
155 return filterListBase.filterRowKey(rowKey, offset, length);
156 }
157
158 @Override
159 public boolean filterAllRemaining() throws IOException {
160 return filterListBase.filterAllRemaining();
161 }
162
163 @Override
164 public Cell transformCell(Cell c) throws IOException {
165 return filterListBase.transformCell(c);
166 }
167
168
169
170
171
172
173 @Deprecated
174 @Override
175 public KeyValue transform(KeyValue v) throws IOException {
176 return KeyValueUtil.ensureKeyValue(transformCell((Cell) v));
177 }
178
179 @Override
180 public ReturnCode filterKeyValue(Cell c) throws IOException {
181 return filterListBase.filterKeyValue(c);
182 }
183
184
185
186
187
188 @Override
189 public void filterRowCells(List<Cell> cells) throws IOException {
190 filterListBase.filterRowCells(cells);
191 }
192
193 @Override
194 public boolean hasFilterRow() {
195 return filterListBase.hasFilterRow();
196 }
197
198 @Override
199 public boolean filterRow() throws IOException {
200 return filterListBase.filterRow();
201 }
202
203
204
205
206 @Override
207 public byte[] toByteArray() throws IOException {
208 FilterProtos.FilterList.Builder builder = FilterProtos.FilterList.newBuilder();
209 builder.setOperator(FilterProtos.FilterList.Operator.valueOf(operator.name()));
210 ArrayList<Filter> filters = filterListBase.getFilters();
211 for (int i = 0, n = filters.size(); i < n; i++) {
212 builder.addFilters(ProtobufUtil.toFilter(filters.get(i)));
213 }
214 return builder.build().toByteArray();
215 }
216
217
218
219
220
221
222
223 public static FilterList parseFrom(final byte[] pbBytes) throws DeserializationException {
224 FilterProtos.FilterList proto;
225 try {
226 proto = FilterProtos.FilterList.parseFrom(pbBytes);
227 } catch (InvalidProtocolBufferException e) {
228 throw new DeserializationException(e);
229 }
230
231 List<Filter> rowFilters = new ArrayList<Filter>(proto.getFiltersCount());
232 try {
233 List<FilterProtos.Filter> filtersList = proto.getFiltersList();
234 for (int i = 0, n = filtersList.size(); i < n; i++) {
235 rowFilters.add(ProtobufUtil.toFilter(filtersList.get(i)));
236 }
237 } catch (IOException ioe) {
238 throw new DeserializationException(ioe);
239 }
240 return new FilterList(Operator.valueOf(proto.getOperator().name()), rowFilters);
241 }
242
243
244
245
246
247
248 @Override
249 boolean areSerializedFieldsEqual(Filter other) {
250 if (other == this) return true;
251 if (!(other instanceof FilterList)) return false;
252
253 FilterList o = (FilterList) other;
254 return this.getOperator().equals(o.getOperator())
255 && ((this.getFilters() == o.getFilters()) || this.getFilters().equals(o.getFilters()));
256 }
257
258 @Override
259 @Deprecated
260 public KeyValue getNextKeyHint(KeyValue currentKV) throws IOException {
261 return KeyValueUtil.ensureKeyValue(getNextCellHint((Cell) currentKV));
262 }
263
264 @Override
265 public Cell getNextCellHint(Cell currentCell) throws IOException {
266 return this.filterListBase.getNextCellHint(currentCell);
267 }
268
269 @Override
270 public boolean isFamilyEssential(byte[] name) throws IOException {
271 return this.filterListBase.isFamilyEssential(name);
272 }
273
274 @Override
275 public void setReversed(boolean reversed) {
276 this.reversed = reversed;
277 this.filterListBase.setReversed(reversed);
278 }
279
280 @Override
281 public boolean isReversed() {
282 assert this.reversed == this.filterListBase.isReversed();
283 return this.reversed;
284 }
285
286
287
288
289
290 @Deprecated
291 public String toString(int maxFilters) {
292 return this.filterListBase.toString();
293 }
294
295 @Override
296 public String toString() {
297 return this.filterListBase.toString();
298 }
299
300 @Override
301 public boolean equals(Object obj) {
302 return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj);
303 }
304
305 @Override
306 public int hashCode() {
307 return Objects.hash(getOperator(), getFilters());
308 }
309 }