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