HBASE-17936 Refine sum endpoint example in ref guide

Signed-off-by: Michael Stack <stack@apache.org>
This commit is contained in:
Xiang Li 2017-04-18 20:25:37 +08:00 committed by Michael Stack
parent 75d1e0361a
commit d15f75b3cf
1 changed files with 34 additions and 26 deletions

View File

@ -610,7 +610,7 @@ The effect is that the duplicate coprocessor is effectively ignored.
+
[source, java]
----
public class SumEndPoint extends SumService implements Coprocessor, CoprocessorService {
public class SumEndPoint extends Sum.SumService implements Coprocessor, CoprocessorService {
private RegionCoprocessorEnvironment env;
@ -630,31 +630,33 @@ public class SumEndPoint extends SumService implements Coprocessor, CoprocessorS
@Override
public void stop(CoprocessorEnvironment env) throws IOException {
// do mothing
// do nothing
}
@Override
public void getSum(RpcController controller, SumRequest request, RpcCallback done) {
public void getSum(RpcController controller, Sum.SumRequest request, RpcCallback<Sum.SumResponse> done) {
Scan scan = new Scan();
scan.addFamily(Bytes.toBytes(request.getFamily()));
scan.addColumn(Bytes.toBytes(request.getFamily()), Bytes.toBytes(request.getColumn()));
SumResponse response = null;
Sum.SumResponse response = null;
InternalScanner scanner = null;
try {
scanner = env.getRegion().getScanner(scan);
List results = new ArrayList();
List<Cell> results = new ArrayList<>();
boolean hasMore = false;
long sum = 0L;
do {
hasMore = scanner.next(results);
for (Cell cell : results) {
sum = sum + Bytes.toLong(CellUtil.cloneValue(cell));
}
results.clear();
} while (hasMore);
long sum = 0L;
response = SumResponse.newBuilder().setSum(sum).build();
do {
hasMore = scanner.next(results);
for (Cell cell : results) {
sum = sum + Bytes.toLong(CellUtil.cloneValue(cell));
}
results.clear();
} while (hasMore);
response = Sum.SumResponse.newBuilder().setSum(sum).build();
} catch (IOException ioe) {
ResponseConverter.setControllerException(controller, ioe);
} finally {
@ -664,6 +666,7 @@ public class SumEndPoint extends SumService implements Coprocessor, CoprocessorS
} catch (IOException ignored) {}
}
}
done.run(response);
}
}
@ -681,24 +684,29 @@ Table table = connection.getTable(tableName);
//HConnection connection = HConnectionManager.createConnection(conf);
//HTableInterface table = connection.getTable("users");
final SumRequest request = SumRequest.newBuilder().setFamily("salaryDet").setColumn("gross")
.build();
final Sum.SumRequest request = Sum.SumRequest.newBuilder().setFamily("salaryDet").setColumn("gross").build();
try {
Map<byte[], Long> results = table.CoprocessorService (SumService.class, null, null,
new Batch.Call<SumService, Long>() {
@Override
public Long call(SumService aggregate) throws IOException {
BlockingRpcCallback rpcCallback = new BlockingRpcCallback();
aggregate.getSum(null, request, rpcCallback);
SumResponse response = rpcCallback.get();
return response.hasSum() ? response.getSum() : 0L;
Map<byte[], Long> results = table.coprocessorService(
Sum.SumService.class,
null, /* start key */
null, /* end key */
new Batch.Call<Sum.SumService, Long>() {
@Override
public Long call(Sum.SumService aggregate) throws IOException {
BlockingRpcCallback<Sum.SumResponse> rpcCallback = new BlockingRpcCallback<>();
aggregate.getSum(null, request, rpcCallback);
Sum.SumResponse response = rpcCallback.get();
return response.hasSum() ? response.getSum() : 0L;
}
}
});
);
for (Long sum : results.values()) {
System.out.println("Sum = " + sum);
}
} catch (ServiceException e) {
e.printStackTrace();
e.printStackTrace();
} catch (Throwable e) {
e.printStackTrace();
}