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