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 java.io.IOException;
23 import java.util.List;
24
25 import org.apache.hadoop.hbase.classification.InterfaceAudience;
26 import org.apache.hadoop.hbase.Cell;
27 import org.apache.hadoop.hbase.KeyValue;
28 import org.apache.hadoop.hbase.KeyValueUtil;
29 import org.apache.hadoop.hbase.exceptions.DeserializationException;
30 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
31 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
32
33 import com.google.protobuf.InvalidProtocolBufferException;
34
35
36
37
38
39
40
41
42 @InterfaceAudience.Private
43 final public class FilterWrapper extends Filter {
44 Filter filter = null;
45
46 public FilterWrapper( Filter filter ) {
47 if (null == filter) {
48
49 throw new NullPointerException("Cannot create FilterWrapper with null Filter");
50 }
51 this.filter = filter;
52 }
53
54
55
56
57 public byte[] toByteArray() throws IOException {
58 FilterProtos.FilterWrapper.Builder builder =
59 FilterProtos.FilterWrapper.newBuilder();
60 builder.setFilter(ProtobufUtil.toFilter(this.filter));
61 return builder.build().toByteArray();
62 }
63
64
65
66
67
68
69
70 public static FilterWrapper parseFrom(final byte [] pbBytes)
71 throws DeserializationException {
72 FilterProtos.FilterWrapper proto;
73 try {
74 proto = FilterProtos.FilterWrapper.parseFrom(pbBytes);
75 } catch (InvalidProtocolBufferException e) {
76 throw new DeserializationException(e);
77 }
78 try {
79 return new FilterWrapper(ProtobufUtil.toFilter(proto.getFilter()));
80 } catch (IOException ioe) {
81 throw new DeserializationException(ioe);
82 }
83 }
84
85 @Override
86 public void reset() throws IOException {
87 this.filter.reset();
88 }
89
90 @Override
91 public boolean filterAllRemaining() throws IOException {
92 return this.filter.filterAllRemaining();
93 }
94
95 @Override
96 public boolean filterRow() throws IOException {
97 return this.filter.filterRow();
98 }
99
100
101
102
103 @Override
104 @Deprecated
105 public KeyValue getNextKeyHint(KeyValue currentKV) throws IOException {
106
107 return KeyValueUtil.ensureKeyValue(this.filter.getNextCellHint((Cell)currentKV));
108 }
109
110 @Override
111 public Cell getNextCellHint(Cell currentKV) throws IOException {
112 return this.filter.getNextCellHint(currentKV);
113 }
114
115 @Override
116 public boolean filterRowKey(byte[] buffer, int offset, int length) throws IOException {
117 return this.filter.filterRowKey(buffer, offset, length);
118 }
119
120 @Override
121 public ReturnCode filterKeyValue(Cell v) throws IOException {
122 return this.filter.filterKeyValue(v);
123 }
124
125 @Override
126 public Cell transformCell(Cell v) throws IOException {
127 return this.filter.transformCell(v);
128 }
129
130
131
132
133
134
135 @Override
136 @Deprecated
137 public KeyValue transform(KeyValue currentKV) throws IOException {
138
139 return KeyValueUtil.ensureKeyValue(this.filter.transformCell(currentKV));
140 }
141
142 @Override
143 public boolean hasFilterRow() {
144 return this.filter.hasFilterRow();
145 }
146
147 @Override
148 public void filterRowCells(List<Cell> kvs) throws IOException {
149 filterRowCellsWithRet(kvs);
150 }
151
152 public enum FilterRowRetCode {
153 NOT_CALLED,
154 INCLUDE,
155 EXCLUDE
156 }
157 public FilterRowRetCode filterRowCellsWithRet(List<Cell> kvs) throws IOException {
158
159
160
161
162
163 this.filter.filterRowCells(kvs);
164 if (!kvs.isEmpty()) {
165 if (this.filter.filterRow()) {
166 kvs.clear();
167 return FilterRowRetCode.EXCLUDE;
168 }
169 return FilterRowRetCode.INCLUDE;
170 }
171 return FilterRowRetCode.NOT_CALLED;
172 }
173
174 @Override
175 public boolean isFamilyEssential(byte[] name) throws IOException {
176 return filter.isFamilyEssential(name);
177 }
178
179
180
181
182
183
184 boolean areSerializedFieldsEqual(Filter o) {
185 if (o == this) return true;
186 if (!(o instanceof FilterWrapper)) return false;
187
188 FilterWrapper other = (FilterWrapper)o;
189 return this.filter.areSerializedFieldsEqual(other.filter);
190 }
191 }