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.classification.InterfaceAudience;
26 import org.apache.hadoop.classification.InterfaceStability;
27 import org.apache.hadoop.hbase.KeyValue;
28 import org.apache.hadoop.hbase.exceptions.DeserializationException;
29 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
30 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
31
32 import com.google.protobuf.InvalidProtocolBufferException;
33
34
35
36
37
38
39
40
41 @InterfaceAudience.Private
42 @InterfaceStability.Evolving
43 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 @Override
101 public KeyValue getNextKeyHint(KeyValue currentKV) throws IOException {
102 return this.filter.getNextKeyHint(currentKV);
103 }
104
105 @Override
106 public boolean filterRowKey(byte[] buffer, int offset, int length) throws IOException {
107 return this.filter.filterRowKey(buffer, offset, length);
108 }
109
110 @Override
111 public ReturnCode filterKeyValue(KeyValue v) throws IOException {
112 return this.filter.filterKeyValue(v);
113 }
114
115 @Override
116 public KeyValue transform(KeyValue v) throws IOException {
117 return this.filter.transform(v);
118 }
119
120 @Override
121 public boolean hasFilterRow() {
122 return this.filter.hasFilterRow();
123 }
124
125 @Override
126 public void filterRow(List<KeyValue> kvs) throws IOException {
127
128
129
130
131
132 this.filter.filterRow(kvs);
133 if (!kvs.isEmpty() && this.filter.filterRow()) {
134 kvs.clear();
135 }
136 }
137
138 @Override
139 public boolean isFamilyEssential(byte[] name) throws IOException {
140 return filter.isFamilyEssential(name);
141 }
142
143
144
145
146
147
148 boolean areSerializedFieldsEqual(Filter o) {
149 if (o == this) return true;
150 if (!(o instanceof FilterWrapper)) return false;
151
152 FilterWrapper other = (FilterWrapper)o;
153 return this.filter.areSerializedFieldsEqual(other.filter);
154 }
155 }