1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.coprocessor.example;
20
21 import com.google.protobuf.RpcCallback;
22 import com.google.protobuf.RpcController;
23 import com.google.protobuf.Service;
24 import org.apache.hadoop.hbase.Coprocessor;
25 import org.apache.hadoop.hbase.CoprocessorEnvironment;
26 import org.apache.hadoop.hbase.KeyValue;
27 import org.apache.hadoop.hbase.client.Scan;
28 import org.apache.hadoop.hbase.coprocessor.CoprocessorException;
29 import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
30 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
31 import org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos;
32 import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
33 import org.apache.hadoop.hbase.protobuf.ResponseConverter;
34 import org.apache.hadoop.hbase.regionserver.InternalScanner;
35 import org.apache.hadoop.hbase.util.Bytes;
36
37 import java.io.IOException;
38 import java.util.ArrayList;
39 import java.util.List;
40
41
42
43
44
45
46
47
48
49 public class RowCountEndpoint extends ExampleProtos.RowCountService
50 implements Coprocessor, CoprocessorService {
51 private RegionCoprocessorEnvironment env;
52
53 public RowCountEndpoint() {
54 }
55
56
57
58
59 @Override
60 public Service getService() {
61 return this;
62 }
63
64
65
66
67 @Override
68 public void getRowCount(RpcController controller, ExampleProtos.CountRequest request,
69 RpcCallback<ExampleProtos.CountResponse> done) {
70 Scan scan = new Scan();
71 scan.setFilter(new FirstKeyOnlyFilter());
72 ExampleProtos.CountResponse response = null;
73 InternalScanner scanner = null;
74 try {
75 scanner = env.getRegion().getScanner(scan);
76 List<KeyValue> results = new ArrayList<KeyValue>();
77 boolean hasMore = false;
78 byte[] lastRow = null;
79 long count = 0;
80 do {
81 hasMore = scanner.next(results);
82 for (KeyValue kv : results) {
83 byte[] currentRow = kv.getRow();
84 if (lastRow == null || !Bytes.equals(lastRow, currentRow)) {
85 lastRow = currentRow;
86 count++;
87 }
88 }
89 results.clear();
90 } while (hasMore);
91
92 response = ExampleProtos.CountResponse.newBuilder()
93 .setCount(count).build();
94 } catch (IOException ioe) {
95 ResponseConverter.setControllerException(controller, ioe);
96 } finally {
97 if (scanner != null) {
98 try {
99 scanner.close();
100 } catch (IOException ignored) {}
101 }
102 }
103 done.run(response);
104 }
105
106
107
108
109 @Override
110 public void getKeyValueCount(RpcController controller, ExampleProtos.CountRequest request,
111 RpcCallback<ExampleProtos.CountResponse> done) {
112 ExampleProtos.CountResponse response = null;
113 InternalScanner scanner = null;
114 try {
115 scanner = env.getRegion().getScanner(new Scan());
116 List<KeyValue> results = new ArrayList<KeyValue>();
117 boolean hasMore = false;
118 long count = 0;
119 do {
120 hasMore = scanner.next(results);
121 for (KeyValue kv : results) {
122 count++;
123 }
124 results.clear();
125 } while (hasMore);
126
127 response = ExampleProtos.CountResponse.newBuilder()
128 .setCount(count).build();
129 } catch (IOException ioe) {
130 ResponseConverter.setControllerException(controller, ioe);
131 } finally {
132 if (scanner != null) {
133 try {
134 scanner.close();
135 } catch (IOException ignored) {}
136 }
137 }
138 done.run(response);
139 }
140
141
142
143
144
145
146
147
148
149
150
151 @Override
152 public void start(CoprocessorEnvironment env) throws IOException {
153 if (env instanceof RegionCoprocessorEnvironment) {
154 this.env = (RegionCoprocessorEnvironment)env;
155 } else {
156 throw new CoprocessorException("Must be loaded on a table region!");
157 }
158 }
159
160 @Override
161 public void stop(CoprocessorEnvironment env) throws IOException {
162
163 }
164 }