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;
20
21 import java.io.IOException;
22 import java.util.ArrayList;
23 import java.util.List;
24
25 import org.apache.commons.logging.Log;
26 import org.apache.commons.logging.LogFactory;
27 import org.apache.hadoop.hbase.Coprocessor;
28 import org.apache.hadoop.hbase.CoprocessorEnvironment;
29 import org.apache.hadoop.hbase.KeyValue;
30 import org.apache.hadoop.hbase.client.Scan;
31 import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.ColumnAggregationService;
32 import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest;
33 import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse;
34 import org.apache.hadoop.hbase.protobuf.ResponseConverter;
35 import org.apache.hadoop.hbase.regionserver.InternalScanner;
36 import org.apache.hadoop.hbase.util.Bytes;
37
38 import com.google.protobuf.RpcCallback;
39 import com.google.protobuf.RpcController;
40 import com.google.protobuf.Service;
41
42
43
44
45
46 public class ColumnAggregationEndpoint extends ColumnAggregationService
47 implements Coprocessor, CoprocessorService {
48 static final Log LOG = LogFactory.getLog(ColumnAggregationEndpoint.class);
49 private RegionCoprocessorEnvironment env = null;
50
51 @Override
52 public Service getService() {
53 return this;
54 }
55
56 @Override
57 public void start(CoprocessorEnvironment env) throws IOException {
58 if (env instanceof RegionCoprocessorEnvironment) {
59 this.env = (RegionCoprocessorEnvironment)env;
60 return;
61 }
62 throw new CoprocessorException("Must be loaded on a table region!");
63 }
64
65 @Override
66 public void stop(CoprocessorEnvironment env) throws IOException {
67
68 }
69
70 @Override
71 public void sum(RpcController controller, SumRequest request, RpcCallback<SumResponse> done) {
72
73 Scan scan = new Scan();
74
75 byte [] family = request.getFamily().toByteArray();
76 byte [] qualifier = request.hasQualifier()? request.getQualifier().toByteArray(): null;
77 if (request.hasQualifier()) {
78 scan.addColumn(family, qualifier);
79 } else {
80 scan.addFamily(family);
81 }
82 int sumResult = 0;
83 InternalScanner scanner = null;
84 try {
85 scanner = this.env.getRegion().getScanner(scan);
86 List<KeyValue> curVals = new ArrayList<KeyValue>();
87 boolean hasMore = false;
88 do {
89 curVals.clear();
90 hasMore = scanner.next(curVals);
91 for (KeyValue kv : curVals) {
92 if (Bytes.equals(qualifier, kv.getQualifier())) {
93 sumResult += Bytes.toInt(kv.getBuffer(), kv.getValueOffset());
94 }
95 }
96 } while (hasMore);
97 } catch (IOException e) {
98 ResponseConverter.setControllerException(controller, e);
99
100 sumResult = -1;
101 LOG.info("Setting sum result to -1 to indicate error", e);
102 } finally {
103 if (scanner != null) {
104 try {
105 scanner.close();
106 } catch (IOException e) {
107 ResponseConverter.setControllerException(controller, e);
108 sumResult = -1;
109 LOG.info("Setting sum result to -1 to indicate error", e);
110 }
111 }
112 }
113 done.run(SumResponse.newBuilder().setSum(sumResult).build());
114 }
115 }