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
24 import java.io.IOException;
25 import java.util.ArrayList;
26 import java.util.Objects;
27
28 import org.apache.hadoop.hbase.Cell;
29 import org.apache.hadoop.hbase.classification.InterfaceAudience;
30 import org.apache.hadoop.hbase.classification.InterfaceStability;
31 import org.apache.hadoop.hbase.exceptions.DeserializationException;
32 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
33
34
35
36
37
38
39
40
41
42
43
44
45 @InterfaceAudience.Public
46 @InterfaceStability.Stable
47 public class PageFilter extends FilterBase {
48 private long pageSize = Long.MAX_VALUE;
49 private int rowsAccepted = 0;
50
51
52
53
54
55
56 public PageFilter(final long pageSize) {
57 Preconditions.checkArgument(pageSize >= 0, "must be positive %s", pageSize);
58 this.pageSize = pageSize;
59 }
60
61 public long getPageSize() {
62 return pageSize;
63 }
64
65 @Override
66 public ReturnCode filterKeyValue(Cell ignored) throws IOException {
67 return ReturnCode.INCLUDE;
68 }
69
70
71
72 @Override
73 public Cell transformCell(Cell v) {
74 return v;
75 }
76
77 @Override
78 public boolean filterAllRemaining() {
79 return this.rowsAccepted >= this.pageSize;
80 }
81
82 @Override
83 public boolean filterRow() {
84 this.rowsAccepted++;
85 return this.rowsAccepted > this.pageSize;
86 }
87
88 @Override
89 public boolean hasFilterRow() {
90 return true;
91 }
92
93 public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
94 Preconditions.checkArgument(filterArguments.size() == 1,
95 "Expected 1 but got: %s", filterArguments.size());
96 long pageSize = ParseFilter.convertByteArrayToLong(filterArguments.get(0));
97 return new PageFilter(pageSize);
98 }
99
100
101
102
103 @Override
104 public byte [] toByteArray() {
105 FilterProtos.PageFilter.Builder builder =
106 FilterProtos.PageFilter.newBuilder();
107 builder.setPageSize(this.pageSize);
108 return builder.build().toByteArray();
109 }
110
111
112
113
114
115
116
117 public static PageFilter parseFrom(final byte [] pbBytes)
118 throws DeserializationException {
119 FilterProtos.PageFilter proto;
120 try {
121 proto = FilterProtos.PageFilter.parseFrom(pbBytes);
122 } catch (InvalidProtocolBufferException e) {
123 throw new DeserializationException(e);
124 }
125 return new PageFilter(proto.getPageSize());
126 }
127
128
129
130
131
132
133 @Override
134 boolean areSerializedFieldsEqual(Filter o) {
135 if (o == this) return true;
136 if (!(o instanceof PageFilter)) return false;
137
138 PageFilter other = (PageFilter)o;
139 return this.getPageSize() == other.getPageSize();
140 }
141
142 @Override
143 public String toString() {
144 return this.getClass().getSimpleName() + " " + this.pageSize;
145 }
146
147 @Override
148 public boolean equals(Object obj) {
149 return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj);
150 }
151
152 @Override
153 public int hashCode() {
154 return Objects.hash(this.pageSize);
155 }
156 }