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