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.Cell;
25 import org.apache.hadoop.hbase.CellComparator;
26 import org.apache.hadoop.hbase.classification.InterfaceAudience;
27 import org.apache.hadoop.hbase.classification.InterfaceStability;
28 import org.apache.hadoop.hbase.exceptions.DeserializationException;
29 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
30 import org.apache.hadoop.hbase.util.ByteStringer;
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(Cell firstRowCell) {
63
64 int cmp = CellComparator.COMPARATOR.compareRows(firstRowCell, stopRowKey, 0, stopRowKey.length);
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(ByteStringer.wrap(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 }