1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.filter;
20
21 import java.util.ArrayList;
22 import java.util.Objects;
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.KeyValueUtil;
29 import org.apache.hadoop.hbase.exceptions.DeserializationException;
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 ColumnPaginationFilter extends FilterBase
45 {
46 private int limit = 0;
47 private int offset = -1;
48 private byte[] columnOffset = null;
49 private int count = 0;
50
51
52
53
54
55
56
57
58
59
60 public ColumnPaginationFilter(final int limit, final int offset)
61 {
62 Preconditions.checkArgument(limit >= 0, "limit must be positive %s", limit);
63 Preconditions.checkArgument(offset >= 0, "offset must be positive %s", offset);
64 this.limit = limit;
65 this.offset = offset;
66 }
67
68
69
70
71
72
73
74
75
76
77
78 public ColumnPaginationFilter(final int limit, final byte[] columnOffset) {
79 Preconditions.checkArgument(limit >= 0, "limit must be positive %s", limit);
80 Preconditions.checkArgument(columnOffset != null,
81 "columnOffset must be non-null %s",
82 columnOffset);
83 this.limit = limit;
84 this.columnOffset = columnOffset;
85 }
86
87
88
89
90 public int getLimit() {
91 return limit;
92 }
93
94
95
96
97 public int getOffset() {
98 return offset;
99 }
100
101
102
103
104 public byte[] getColumnOffset() {
105 return columnOffset;
106 }
107
108 @Override
109 public ReturnCode filterKeyValue(Cell v)
110 {
111 if (columnOffset != null) {
112 if (count >= limit) {
113 return ReturnCode.NEXT_ROW;
114 }
115 byte[] buffer = v.getQualifierArray();
116 if (buffer == null) {
117 return ReturnCode.SEEK_NEXT_USING_HINT;
118 }
119 int cmp = 0;
120
121 if (count == 0) {
122 cmp = Bytes.compareTo(buffer,
123 v.getQualifierOffset(),
124 v.getQualifierLength(),
125 this.columnOffset,
126 0,
127 this.columnOffset.length);
128 }
129 if (cmp < 0) {
130 return ReturnCode.SEEK_NEXT_USING_HINT;
131 } else {
132 count++;
133 return ReturnCode.INCLUDE_AND_NEXT_COL;
134 }
135 } else {
136 if (count >= offset + limit) {
137 return ReturnCode.NEXT_ROW;
138 }
139
140 ReturnCode code = count < offset ? ReturnCode.NEXT_COL :
141 ReturnCode.INCLUDE_AND_NEXT_COL;
142 count++;
143 return code;
144 }
145 }
146
147
148
149 @Override
150 public Cell transformCell(Cell v) {
151 return v;
152 }
153
154 @Override
155 public Cell getNextCellHint(Cell kv) {
156 return KeyValueUtil.createFirstOnRow(
157 kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(), kv.getFamilyArray(),
158 kv.getFamilyOffset(), kv.getFamilyLength(), columnOffset, 0, columnOffset.length);
159 }
160
161 @Override
162 public void reset()
163 {
164 this.count = 0;
165 }
166
167 public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
168 Preconditions.checkArgument(filterArguments.size() == 2,
169 "Expected 2 but got: %s", filterArguments.size());
170 int limit = ParseFilter.convertByteArrayToInt(filterArguments.get(0));
171 int offset = ParseFilter.convertByteArrayToInt(filterArguments.get(1));
172 return new ColumnPaginationFilter(limit, offset);
173 }
174
175
176
177
178 @Override
179 public byte [] toByteArray() {
180 FilterProtos.ColumnPaginationFilter.Builder builder =
181 FilterProtos.ColumnPaginationFilter.newBuilder();
182 builder.setLimit(this.limit);
183 if (this.offset >= 0) {
184 builder.setOffset(this.offset);
185 }
186 if (this.columnOffset != null) {
187 builder.setColumnOffset(ByteStringer.wrap(this.columnOffset));
188 }
189 return builder.build().toByteArray();
190 }
191
192
193
194
195
196
197
198 public static ColumnPaginationFilter parseFrom(final byte [] pbBytes)
199 throws DeserializationException {
200 FilterProtos.ColumnPaginationFilter proto;
201 try {
202 proto = FilterProtos.ColumnPaginationFilter.parseFrom(pbBytes);
203 } catch (InvalidProtocolBufferException e) {
204 throw new DeserializationException(e);
205 }
206 if (proto.hasColumnOffset()) {
207 return new ColumnPaginationFilter(proto.getLimit(),
208 proto.getColumnOffset().toByteArray());
209 }
210 return new ColumnPaginationFilter(proto.getLimit(),proto.getOffset());
211 }
212
213
214
215
216
217
218 @Override
219 boolean areSerializedFieldsEqual(Filter o) {
220 if (o == this) return true;
221 if (!(o instanceof ColumnPaginationFilter)) return false;
222
223 ColumnPaginationFilter other = (ColumnPaginationFilter)o;
224 if (this.columnOffset != null) {
225 return this.getLimit() == other.getLimit() &&
226 Bytes.equals(this.getColumnOffset(), other.getColumnOffset());
227 }
228 return this.getLimit() == other.getLimit() && this.getOffset() == other.getOffset();
229 }
230
231 @Override
232 public String toString() {
233 if (this.columnOffset != null) {
234 return (this.getClass().getSimpleName() + "(" + this.limit + ", " +
235 Bytes.toStringBinary(this.columnOffset) + ")");
236 }
237 return String.format("%s (%d, %d)", this.getClass().getSimpleName(),
238 this.limit, this.offset);
239 }
240
241 @Override
242 public boolean equals(Object obj) {
243 return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj);
244 }
245
246 @Override
247 public int hashCode() {
248 return columnOffset == null ? Objects.hash(this.limit, this.offset) :
249 Objects.hash(this.limit, Bytes.hashCode(this.columnOffset));
250 }
251 }