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.util.ArrayList;
23
24 import com.google.protobuf.HBaseZeroCopyByteString;
25 import org.apache.hadoop.classification.InterfaceAudience;
26 import org.apache.hadoop.classification.InterfaceStability;
27 import org.apache.hadoop.hbase.Cell;
28 import org.apache.hadoop.hbase.exceptions.DeserializationException;
29 import org.apache.hadoop.hbase.filter.Filter.ReturnCode;
30 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
31 import org.apache.hadoop.hbase.util.Bytes;
32
33 import com.google.common.base.Preconditions;
34 import com.google.protobuf.InvalidProtocolBufferException;
35
36
37
38
39
40
41
42 @InterfaceAudience.Public
43 @InterfaceStability.Stable
44 public class InclusiveStopFilter extends FilterBase {
45 private byte [] stopRowKey;
46 private boolean done = false;
47
48 public InclusiveStopFilter(final byte [] stopRowKey) {
49 this.stopRowKey = stopRowKey;
50 }
51
52 public byte[] getStopRowKey() {
53 return this.stopRowKey;
54 }
55
56 @Override
57 public ReturnCode filterKeyValue(Cell v) {
58 if (done) return ReturnCode.NEXT_ROW;
59 return ReturnCode.INCLUDE;
60 }
61
62 public boolean filterRowKey(byte[] buffer, int offset, int length) {
63 if (buffer == null) {
64
65 if (this.stopRowKey == null) {
66 return true;
67 }
68 return false;
69 }
70
71 int cmp = Bytes.compareTo(stopRowKey, 0, stopRowKey.length,
72 buffer, offset, length);
73
74 if(cmp < 0) {
75 done = true;
76 }
77 return done;
78 }
79
80 public boolean filterAllRemaining() {
81 return done;
82 }
83
84 public static Filter createFilterFromArguments (ArrayList<byte []> filterArguments) {
85 Preconditions.checkArgument(filterArguments.size() == 1,
86 "Expected 1 but got: %s", filterArguments.size());
87 byte [] stopRowKey = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0));
88 return new InclusiveStopFilter(stopRowKey);
89 }
90
91
92
93
94 public byte [] toByteArray() {
95 FilterProtos.InclusiveStopFilter.Builder builder =
96 FilterProtos.InclusiveStopFilter.newBuilder();
97 if (this.stopRowKey != null) builder.setStopRowKey(HBaseZeroCopyByteString.wrap(this.stopRowKey));
98 return builder.build().toByteArray();
99 }
100
101
102
103
104
105
106
107 public static InclusiveStopFilter parseFrom(final byte [] pbBytes)
108 throws DeserializationException {
109 FilterProtos.InclusiveStopFilter proto;
110 try {
111 proto = FilterProtos.InclusiveStopFilter.parseFrom(pbBytes);
112 } catch (InvalidProtocolBufferException e) {
113 throw new DeserializationException(e);
114 }
115 return new InclusiveStopFilter(proto.hasStopRowKey()?proto.getStopRowKey().toByteArray():null);
116 }
117
118
119
120
121
122
123 boolean areSerializedFieldsEqual(Filter o) {
124 if (o == this) return true;
125 if (!(o instanceof InclusiveStopFilter)) return false;
126
127 InclusiveStopFilter other = (InclusiveStopFilter)o;
128 return Bytes.equals(this.getStopRowKey(), other.getStopRowKey());
129 }
130
131 @Override
132 public String toString() {
133 return this.getClass().getSimpleName() + " " + Bytes.toStringBinary(this.stopRowKey);
134 }
135 }