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