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