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.InvalidProtocolBufferException;
23 import org.apache.hadoop.classification.InterfaceAudience;
24 import org.apache.hadoop.classification.InterfaceStability;
25 import org.apache.hadoop.hbase.KeyValue;
26 import org.apache.hadoop.hbase.exceptions.DeserializationException;
27 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
28
29 import java.util.ArrayList;
30
31
32
33
34
35
36
37 @InterfaceAudience.Public
38 @InterfaceStability.Stable
39 public class ColumnPaginationFilter extends FilterBase
40 {
41 private int limit = 0;
42 private int offset = 0;
43 private int count = 0;
44
45 public ColumnPaginationFilter(final int limit, final int offset)
46 {
47 Preconditions.checkArgument(limit >= 0, "limit must be positive %s", limit);
48 Preconditions.checkArgument(offset >= 0, "offset must be positive %s", offset);
49 this.limit = limit;
50 this.offset = offset;
51 }
52
53
54
55
56 public int getLimit() {
57 return limit;
58 }
59
60
61
62
63 public int getOffset() {
64 return offset;
65 }
66
67 @Override
68 public ReturnCode filterKeyValue(KeyValue v)
69 {
70 if(count >= offset + limit)
71 {
72 return ReturnCode.NEXT_ROW;
73 }
74
75 ReturnCode code = count < offset ? ReturnCode.NEXT_COL :
76 ReturnCode.INCLUDE_AND_NEXT_COL;
77 count++;
78 return code;
79 }
80
81 @Override
82 public void reset()
83 {
84 this.count = 0;
85 }
86
87 public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
88 Preconditions.checkArgument(filterArguments.size() == 2,
89 "Expected 2 but got: %s", filterArguments.size());
90 int limit = ParseFilter.convertByteArrayToInt(filterArguments.get(0));
91 int offset = ParseFilter.convertByteArrayToInt(filterArguments.get(1));
92 return new ColumnPaginationFilter(limit, offset);
93 }
94
95
96
97
98 public byte [] toByteArray() {
99 FilterProtos.ColumnPaginationFilter.Builder builder =
100 FilterProtos.ColumnPaginationFilter.newBuilder();
101 builder.setLimit(this.limit);
102 builder.setOffset(this.offset);
103 return builder.build().toByteArray();
104 }
105
106
107
108
109
110
111
112 public static ColumnPaginationFilter parseFrom(final byte [] pbBytes)
113 throws DeserializationException {
114 FilterProtos.ColumnPaginationFilter proto;
115 try {
116 proto = FilterProtos.ColumnPaginationFilter.parseFrom(pbBytes);
117 } catch (InvalidProtocolBufferException e) {
118 throw new DeserializationException(e);
119 }
120 return new ColumnPaginationFilter(proto.getLimit(),proto.getOffset());
121 }
122
123
124
125
126
127
128 boolean areSerializedFieldsEqual(Filter o) {
129 if (o == this) return true;
130 if (!(o instanceof ColumnPaginationFilter)) return false;
131
132 ColumnPaginationFilter other = (ColumnPaginationFilter)o;
133 return this.getLimit() == other.getLimit() && this.getOffset() == other.getOffset();
134 }
135
136 @Override
137 public String toString() {
138 return String.format("%s (%d, %d)", this.getClass().getSimpleName(),
139 this.limit, this.offset);
140 }
141 }