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
62
63 @Override
64 public Cell transformCell(Cell v) {
65 return v;
66 }
67
68 public boolean filterRowKey(byte[] buffer, int offset, int length) {
69 if (buffer == null) {
70
71 if (this.stopRowKey == null) {
72 return true;
73 }
74 return false;
75 }
76
77 int cmp = Bytes.compareTo(stopRowKey, 0, stopRowKey.length,
78 buffer, offset, length);
79
80 if(cmp < 0) {
81 done = true;
82 }
83 return done;
84 }
85
86 public boolean filterAllRemaining() {
87 return done;
88 }
89
90 public static Filter createFilterFromArguments (ArrayList<byte []> filterArguments) {
91 Preconditions.checkArgument(filterArguments.size() == 1,
92 "Expected 1 but got: %s", filterArguments.size());
93 byte [] stopRowKey = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0));
94 return new InclusiveStopFilter(stopRowKey);
95 }
96
97
98
99
100 public byte [] toByteArray() {
101 FilterProtos.InclusiveStopFilter.Builder builder =
102 FilterProtos.InclusiveStopFilter.newBuilder();
103 if (this.stopRowKey != null) builder.setStopRowKey(ByteStringer.wrap(this.stopRowKey));
104 return builder.build().toByteArray();
105 }
106
107
108
109
110
111
112
113 public static InclusiveStopFilter parseFrom(final byte [] pbBytes)
114 throws DeserializationException {
115 FilterProtos.InclusiveStopFilter proto;
116 try {
117 proto = FilterProtos.InclusiveStopFilter.parseFrom(pbBytes);
118 } catch (InvalidProtocolBufferException e) {
119 throw new DeserializationException(e);
120 }
121 return new InclusiveStopFilter(proto.hasStopRowKey()?proto.getStopRowKey().toByteArray():null);
122 }
123
124
125
126
127
128
129 boolean areSerializedFieldsEqual(Filter o) {
130 if (o == this) return true;
131 if (!(o instanceof InclusiveStopFilter)) return false;
132
133 InclusiveStopFilter other = (InclusiveStopFilter)o;
134 return Bytes.equals(this.getStopRowKey(), other.getStopRowKey());
135 }
136
137 @Override
138 public String toString() {
139 return this.getClass().getSimpleName() + " " + Bytes.toStringBinary(this.stopRowKey);
140 }
141 }