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