1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.client.coprocessor;
21
22 import com.google.protobuf.ByteString;
23 import com.google.protobuf.Message;
24 import org.apache.commons.logging.Log;
25 import org.apache.commons.logging.LogFactory;
26 import org.apache.hadoop.classification.InterfaceAudience;
27 import org.apache.hadoop.classification.InterfaceStability;
28 import org.apache.hadoop.conf.Configuration;
29 import org.apache.hadoop.hbase.HConstants;
30 import org.apache.hadoop.hbase.KeyValue;
31 import org.apache.hadoop.hbase.client.HTable;
32 import org.apache.hadoop.hbase.client.Result;
33 import org.apache.hadoop.hbase.client.ResultScanner;
34 import org.apache.hadoop.hbase.client.Scan;
35 import org.apache.hadoop.hbase.coprocessor.ColumnInterpreter;
36 import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
37 import org.apache.hadoop.hbase.ipc.ServerRpcController;
38 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
39 import org.apache.hadoop.hbase.protobuf.generated.AggregateProtos.AggregateArgument;
40 import org.apache.hadoop.hbase.protobuf.generated.AggregateProtos.AggregateResponse;
41 import org.apache.hadoop.hbase.protobuf.generated.AggregateProtos.AggregateService;
42 import org.apache.hadoop.hbase.util.Bytes;
43 import org.apache.hadoop.hbase.util.Pair;
44
45 import java.io.IOException;
46 import java.nio.ByteBuffer;
47 import java.util.ArrayList;
48 import java.util.List;
49 import java.util.Map;
50 import java.util.NavigableMap;
51 import java.util.NavigableSet;
52 import java.util.TreeMap;
53 import java.util.concurrent.atomic.AtomicLong;
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74 @InterfaceAudience.Public
75 @InterfaceStability.Evolving
76 public class AggregationClient {
77
78 private static final Log log = LogFactory.getLog(AggregationClient.class);
79 Configuration conf;
80
81
82
83
84
85 public AggregationClient(Configuration cfg) {
86 this.conf = cfg;
87 }
88
89
90
91
92
93
94
95
96
97
98
99
100
101 public <R, S, P extends Message, Q extends Message, T extends Message>
102 R max(final byte[] tableName, final ColumnInterpreter<R, S, P, Q, T> ci,
103 final Scan scan) throws Throwable {
104 final AggregateArgument requestArg = validateArgAndGetPB(scan, ci);
105 class MaxCallBack implements Batch.Callback<R> {
106 R max = null;
107
108 R getMax() {
109 return max;
110 }
111
112 @Override
113 public synchronized void update(byte[] region, byte[] row, R result) {
114 max = (max == null || (result != null && ci.compare(max, result) < 0)) ? result : max;
115 }
116 }
117 MaxCallBack aMaxCallBack = new MaxCallBack();
118 HTable table = null;
119 try {
120 table = new HTable(conf, tableName);
121 table.coprocessorService(AggregateService.class, scan.getStartRow(),
122 scan.getStopRow(), new Batch.Call<AggregateService, R>() {
123 @Override
124 public R call(AggregateService instance) throws IOException {
125 ServerRpcController controller = new ServerRpcController();
126 BlockingRpcCallback<AggregateResponse> rpcCallback =
127 new BlockingRpcCallback<AggregateResponse>();
128 instance.getMax(controller, requestArg, rpcCallback);
129 AggregateResponse response = rpcCallback.get();
130 if (controller.failedOnException()) {
131 throw controller.getFailedOn();
132 }
133 if (response.getFirstPartCount() > 0) {
134 ByteString b = response.getFirstPart(0);
135 Q q = ProtobufUtil.getParsedGenericInstance(ci.getClass(), 3, b);
136 return ci.getCellValueFromProto(q);
137 }
138 return null;
139 }
140 }, aMaxCallBack);
141 } finally {
142 if (table != null) {
143 table.close();
144 }
145 }
146 return aMaxCallBack.getMax();
147 }
148
149 private void validateParameters(Scan scan) throws IOException {
150 if (scan == null
151 || (Bytes.equals(scan.getStartRow(), scan.getStopRow()) && !Bytes
152 .equals(scan.getStartRow(), HConstants.EMPTY_START_ROW))
153 || ((Bytes.compareTo(scan.getStartRow(), scan.getStopRow()) > 0) &&
154 !Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW))) {
155 throw new IOException(
156 "Agg client Exception: Startrow should be smaller than Stoprow");
157 } else if (scan.getFamilyMap().size() != 1) {
158 throw new IOException("There must be only one family.");
159 }
160 }
161
162
163
164
165
166
167
168
169
170
171
172 public <R, S, P extends Message, Q extends Message, T extends Message>
173 R min(final byte[] tableName, final ColumnInterpreter<R, S, P, Q, T> ci,
174 final Scan scan) throws Throwable {
175 final AggregateArgument requestArg = validateArgAndGetPB(scan, ci);
176 class MinCallBack implements Batch.Callback<R> {
177
178 private R min = null;
179
180 public R getMinimum() {
181 return min;
182 }
183
184 @Override
185 public synchronized void update(byte[] region, byte[] row, R result) {
186 min = (min == null || (result != null && ci.compare(result, min) < 0)) ? result : min;
187 }
188 }
189 MinCallBack minCallBack = new MinCallBack();
190 HTable table = null;
191 try {
192 table = new HTable(conf, tableName);
193 table.coprocessorService(AggregateService.class, scan.getStartRow(),
194 scan.getStopRow(), new Batch.Call<AggregateService, R>() {
195
196 @Override
197 public R call(AggregateService instance) throws IOException {
198 ServerRpcController controller = new ServerRpcController();
199 BlockingRpcCallback<AggregateResponse> rpcCallback =
200 new BlockingRpcCallback<AggregateResponse>();
201 instance.getMin(controller, requestArg, rpcCallback);
202 AggregateResponse response = rpcCallback.get();
203 if (controller.failedOnException()) {
204 throw controller.getFailedOn();
205 }
206 if (response.getFirstPartCount() > 0) {
207 ByteString b = response.getFirstPart(0);
208 Q q = ProtobufUtil.getParsedGenericInstance(ci.getClass(), 3, b);
209 return ci.getCellValueFromProto(q);
210 }
211 return null;
212 }
213 }, minCallBack);
214 } finally {
215 if (table != null) {
216 table.close();
217 }
218 }
219 log.debug("Min fom all regions is: " + minCallBack.getMinimum());
220 return minCallBack.getMinimum();
221 }
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236 public <R, S, P extends Message, Q extends Message, T extends Message>
237 long rowCount(final byte[] tableName,
238 final ColumnInterpreter<R, S, P, Q, T> ci, final Scan scan) throws Throwable {
239 final AggregateArgument requestArg = validateArgAndGetPB(scan, ci);
240 class RowNumCallback implements Batch.Callback<Long> {
241 private final AtomicLong rowCountL = new AtomicLong(0);
242
243 public long getRowNumCount() {
244 return rowCountL.get();
245 }
246
247 @Override
248 public void update(byte[] region, byte[] row, Long result) {
249 rowCountL.addAndGet(result.longValue());
250 }
251 }
252 RowNumCallback rowNum = new RowNumCallback();
253 HTable table = null;
254 try {
255 table = new HTable(conf, tableName);
256 table.coprocessorService(AggregateService.class, scan.getStartRow(),
257 scan.getStopRow(), new Batch.Call<AggregateService, Long>() {
258 @Override
259 public Long call(AggregateService instance) throws IOException {
260 ServerRpcController controller = new ServerRpcController();
261 BlockingRpcCallback<AggregateResponse> rpcCallback =
262 new BlockingRpcCallback<AggregateResponse>();
263 instance.getRowNum(controller, requestArg, rpcCallback);
264 AggregateResponse response = rpcCallback.get();
265 if (controller.failedOnException()) {
266 throw controller.getFailedOn();
267 }
268 byte[] bytes = getBytesFromResponse(response.getFirstPart(0));
269 ByteBuffer bb = ByteBuffer.allocate(8).put(bytes);
270 bb.rewind();
271 return bb.getLong();
272 }
273 }, rowNum);
274 } finally {
275 if (table != null) {
276 table.close();
277 }
278 }
279 return rowNum.getRowNumCount();
280 }
281
282
283
284
285
286
287
288
289
290
291 public <R, S, P extends Message, Q extends Message, T extends Message>
292 S sum(final byte[] tableName, final ColumnInterpreter<R, S, P, Q, T> ci,
293 final Scan scan) throws Throwable {
294 final AggregateArgument requestArg = validateArgAndGetPB(scan, ci);
295
296 class SumCallBack implements Batch.Callback<S> {
297 S sumVal = null;
298
299 public S getSumResult() {
300 return sumVal;
301 }
302
303 @Override
304 public synchronized void update(byte[] region, byte[] row, S result) {
305 sumVal = ci.add(sumVal, result);
306 }
307 }
308 SumCallBack sumCallBack = new SumCallBack();
309 HTable table = null;
310 try {
311 table = new HTable(conf, tableName);
312 table.coprocessorService(AggregateService.class, scan.getStartRow(),
313 scan.getStopRow(), new Batch.Call<AggregateService, S>() {
314 @Override
315 public S call(AggregateService instance) throws IOException {
316 ServerRpcController controller = new ServerRpcController();
317 BlockingRpcCallback<AggregateResponse> rpcCallback =
318 new BlockingRpcCallback<AggregateResponse>();
319 instance.getSum(controller, requestArg, rpcCallback);
320 AggregateResponse response = rpcCallback.get();
321 if (controller.failedOnException()) {
322 throw controller.getFailedOn();
323 }
324 if (response.getFirstPartCount() == 0) {
325 return null;
326 }
327 ByteString b = response.getFirstPart(0);
328 T t = ProtobufUtil.getParsedGenericInstance(ci.getClass(), 4, b);
329 S s = ci.getPromotedValueFromProto(t);
330 return s;
331 }
332 }, sumCallBack);
333 } finally {
334 if (table != null) {
335 table.close();
336 }
337 }
338 return sumCallBack.getSumResult();
339 }
340
341
342
343
344
345
346
347
348
349 private <R, S, P extends Message, Q extends Message, T extends Message>
350 Pair<S, Long> getAvgArgs(final byte[] tableName,
351 final ColumnInterpreter<R, S, P, Q, T> ci, final Scan scan) throws Throwable {
352 final AggregateArgument requestArg = validateArgAndGetPB(scan, ci);
353 class AvgCallBack implements Batch.Callback<Pair<S, Long>> {
354 S sum = null;
355 Long rowCount = 0l;
356
357 public Pair<S, Long> getAvgArgs() {
358 return new Pair<S, Long>(sum, rowCount);
359 }
360
361 @Override
362 public synchronized void update(byte[] region, byte[] row, Pair<S, Long> result) {
363 sum = ci.add(sum, result.getFirst());
364 rowCount += result.getSecond();
365 }
366 }
367 AvgCallBack avgCallBack = new AvgCallBack();
368 HTable table = null;
369 try {
370 table = new HTable(conf, tableName);
371 table.coprocessorService(AggregateService.class, scan.getStartRow(),
372 scan.getStopRow(),
373 new Batch.Call<AggregateService, Pair<S, Long>>() {
374 @Override
375 public Pair<S, Long> call(AggregateService instance)
376 throws IOException {
377 ServerRpcController controller = new ServerRpcController();
378 BlockingRpcCallback<AggregateResponse> rpcCallback =
379 new BlockingRpcCallback<AggregateResponse>();
380 instance.getAvg(controller, requestArg, rpcCallback);
381 AggregateResponse response = rpcCallback.get();
382 if (controller.failedOnException()) {
383 throw controller.getFailedOn();
384 }
385 Pair<S,Long> pair = new Pair<S, Long>(null, 0L);
386 if (response.getFirstPartCount() == 0) {
387 return pair;
388 }
389 ByteString b = response.getFirstPart(0);
390 T t = ProtobufUtil.getParsedGenericInstance(ci.getClass(), 4, b);
391 S s = ci.getPromotedValueFromProto(t);
392 pair.setFirst(s);
393 ByteBuffer bb = ByteBuffer.allocate(8).put(
394 getBytesFromResponse(response.getSecondPart()));
395 bb.rewind();
396 pair.setSecond(bb.getLong());
397 return pair;
398 }
399 }, avgCallBack);
400 } finally {
401 if (table != null) {
402 table.close();
403 }
404 }
405 return avgCallBack.getAvgArgs();
406 }
407
408
409
410
411
412
413
414
415
416
417
418
419
420 public <R, S, P extends Message, Q extends Message, T extends Message>
421 double avg(final byte[] tableName,
422 final ColumnInterpreter<R, S, P, Q, T> ci, Scan scan) throws Throwable {
423 Pair<S, Long> p = getAvgArgs(tableName, ci, scan);
424 return ci.divideForAvg(p.getFirst(), p.getSecond());
425 }
426
427
428
429
430
431
432
433
434
435
436
437
438 private <R, S, P extends Message, Q extends Message, T extends Message>
439 Pair<List<S>, Long> getStdArgs(final byte[] tableName,
440 final ColumnInterpreter<R, S, P, Q, T> ci, final Scan scan) throws Throwable {
441 final AggregateArgument requestArg = validateArgAndGetPB(scan, ci);
442 class StdCallback implements Batch.Callback<Pair<List<S>, Long>> {
443 long rowCountVal = 0l;
444 S sumVal = null, sumSqVal = null;
445
446 public Pair<List<S>, Long> getStdParams() {
447 List<S> l = new ArrayList<S>();
448 l.add(sumVal);
449 l.add(sumSqVal);
450 Pair<List<S>, Long> p = new Pair<List<S>, Long>(l, rowCountVal);
451 return p;
452 }
453
454 @Override
455 public synchronized void update(byte[] region, byte[] row, Pair<List<S>, Long> result) {
456 if (result.getFirst().size() > 0) {
457 sumVal = ci.add(sumVal, result.getFirst().get(0));
458 sumSqVal = ci.add(sumSqVal, result.getFirst().get(1));
459 rowCountVal += result.getSecond();
460 }
461 }
462 }
463 StdCallback stdCallback = new StdCallback();
464 HTable table = null;
465 try {
466 table = new HTable(conf, tableName);
467 table.coprocessorService(AggregateService.class, scan.getStartRow(),
468 scan.getStopRow(),
469 new Batch.Call<AggregateService, Pair<List<S>, Long>>() {
470 @Override
471 public Pair<List<S>, Long> call(AggregateService instance)
472 throws IOException {
473 ServerRpcController controller = new ServerRpcController();
474 BlockingRpcCallback<AggregateResponse> rpcCallback =
475 new BlockingRpcCallback<AggregateResponse>();
476 instance.getStd(controller, requestArg, rpcCallback);
477 AggregateResponse response = rpcCallback.get();
478 if (controller.failedOnException()) {
479 throw controller.getFailedOn();
480 }
481 Pair<List<S>,Long> pair =
482 new Pair<List<S>, Long>(new ArrayList<S>(), 0L);
483 if (response.getFirstPartCount() == 0) {
484 return pair;
485 }
486 List<S> list = new ArrayList<S>();
487 for (int i = 0; i < response.getFirstPartCount(); i++) {
488 ByteString b = response.getFirstPart(i);
489 T t = ProtobufUtil.getParsedGenericInstance(ci.getClass(), 4, b);
490 S s = ci.getPromotedValueFromProto(t);
491 list.add(s);
492 }
493 pair.setFirst(list);
494 ByteBuffer bb = ByteBuffer.allocate(8).put(
495 getBytesFromResponse(response.getSecondPart()));
496 bb.rewind();
497 pair.setSecond(bb.getLong());
498 return pair;
499 }
500 }, stdCallback);
501 } finally {
502 if (table != null) {
503 table.close();
504 }
505 }
506 return stdCallback.getStdParams();
507 }
508
509
510
511
512
513
514
515
516
517
518
519
520
521 public <R, S, P extends Message, Q extends Message, T extends Message>
522 double std(final byte[] tableName, ColumnInterpreter<R, S, P, Q, T> ci,
523 Scan scan) throws Throwable {
524 Pair<List<S>, Long> p = getStdArgs(tableName, ci, scan);
525 double res = 0d;
526 double avg = ci.divideForAvg(p.getFirst().get(0), p.getSecond());
527 double avgOfSumSq = ci.divideForAvg(p.getFirst().get(1), p.getSecond());
528 res = avgOfSumSq - (avg) * (avg);
529 res = Math.pow(res, 0.5);
530 return res;
531 }
532
533
534
535
536
537
538
539
540
541
542
543
544
545 private <R, S, P extends Message, Q extends Message, T extends Message>
546 Pair<NavigableMap<byte[], List<S>>, List<S>>
547 getMedianArgs(final byte[] tableName,
548 final ColumnInterpreter<R, S, P, Q, T> ci, final Scan scan) throws Throwable {
549 final AggregateArgument requestArg = validateArgAndGetPB(scan, ci);
550 final NavigableMap<byte[], List<S>> map =
551 new TreeMap<byte[], List<S>>(Bytes.BYTES_COMPARATOR);
552 class StdCallback implements Batch.Callback<List<S>> {
553 S sumVal = null, sumWeights = null;
554
555 public Pair<NavigableMap<byte[], List<S>>, List<S>> getMedianParams() {
556 List<S> l = new ArrayList<S>();
557 l.add(sumVal);
558 l.add(sumWeights);
559 Pair<NavigableMap<byte[], List<S>>, List<S>> p =
560 new Pair<NavigableMap<byte[], List<S>>, List<S>>(map, l);
561 return p;
562 }
563
564 @Override
565 public synchronized void update(byte[] region, byte[] row, List<S> result) {
566 map.put(row, result);
567 sumVal = ci.add(sumVal, result.get(0));
568 sumWeights = ci.add(sumWeights, result.get(1));
569 }
570 }
571 StdCallback stdCallback = new StdCallback();
572 HTable table = null;
573 try {
574 table = new HTable(conf, tableName);
575 table.coprocessorService(AggregateService.class, scan.getStartRow(),
576 scan.getStopRow(), new Batch.Call<AggregateService, List<S>>() {
577 @Override
578 public List<S> call(AggregateService instance) throws IOException {
579 ServerRpcController controller = new ServerRpcController();
580 BlockingRpcCallback<AggregateResponse> rpcCallback =
581 new BlockingRpcCallback<AggregateResponse>();
582 instance.getMedian(controller, requestArg, rpcCallback);
583 AggregateResponse response = rpcCallback.get();
584 if (controller.failedOnException()) {
585 throw controller.getFailedOn();
586 }
587
588 List<S> list = new ArrayList<S>();
589 for (int i = 0; i < response.getFirstPartCount(); i++) {
590 ByteString b = response.getFirstPart(i);
591 T t = ProtobufUtil.getParsedGenericInstance(ci.getClass(), 4, b);
592 S s = ci.getPromotedValueFromProto(t);
593 list.add(s);
594 }
595 return list;
596 }
597
598 }, stdCallback);
599 } finally {
600 if (table != null) {
601 table.close();
602 }
603 }
604 return stdCallback.getMedianParams();
605 }
606
607
608
609
610
611
612
613
614
615
616
617 public <R, S, P extends Message, Q extends Message, T extends Message>
618 R median(final byte[] tableName, ColumnInterpreter<R, S, P, Q, T> ci,
619 Scan scan) throws Throwable {
620 Pair<NavigableMap<byte[], List<S>>, List<S>> p = getMedianArgs(tableName, ci, scan);
621 byte[] startRow = null;
622 byte[] colFamily = scan.getFamilies()[0];
623 NavigableSet<byte[]> quals = scan.getFamilyMap().get(colFamily);
624 NavigableMap<byte[], List<S>> map = p.getFirst();
625 S sumVal = p.getSecond().get(0);
626 S sumWeights = p.getSecond().get(1);
627 double halfSumVal = ci.divideForAvg(sumVal, 2L);
628 double movingSumVal = 0;
629 boolean weighted = false;
630 if (quals.size() > 1) {
631 weighted = true;
632 halfSumVal = ci.divideForAvg(sumWeights, 2L);
633 }
634
635 for (Map.Entry<byte[], List<S>> entry : map.entrySet()) {
636 S s = weighted ? entry.getValue().get(1) : entry.getValue().get(0);
637 double newSumVal = movingSumVal + ci.divideForAvg(s, 1L);
638 if (newSumVal > halfSumVal) break;
639 movingSumVal = newSumVal;
640 startRow = entry.getKey();
641 }
642
643 Scan scan2 = new Scan(scan);
644
645 if (startRow != null) scan2.setStartRow(startRow);
646 HTable table = null;
647 ResultScanner scanner = null;
648 try {
649 table = new HTable(conf, tableName);
650 int cacheSize = scan2.getCaching();
651 if (!scan2.getCacheBlocks() || scan2.getCaching() < 2) {
652 scan2.setCacheBlocks(true);
653 cacheSize = 5;
654 scan2.setCaching(cacheSize);
655 }
656 scanner = table.getScanner(scan2);
657 Result[] results = null;
658 byte[] qualifier = quals.pollFirst();
659
660 byte[] weightQualifier = weighted ? quals.pollLast() : qualifier;
661 R value = null;
662 do {
663 results = scanner.next(cacheSize);
664 if (results != null && results.length > 0) {
665 for (int i = 0; i < results.length; i++) {
666 Result r = results[i];
667
668 KeyValue kv = r.getColumnLatest(colFamily, weightQualifier);
669 R newValue = ci.getValue(colFamily, weightQualifier, kv);
670 S s = ci.castToReturnType(newValue);
671 double newSumVal = movingSumVal + ci.divideForAvg(s, 1L);
672
673 if (newSumVal > halfSumVal) {
674 return value;
675 }
676 movingSumVal = newSumVal;
677 kv = r.getColumnLatest(colFamily, qualifier);
678 value = ci.getValue(colFamily, qualifier, kv);
679 }
680 }
681 } while (results != null && results.length > 0);
682 } finally {
683 if (scanner != null) {
684 scanner.close();
685 }
686 if (table != null) {
687 table.close();
688 }
689 }
690 return null;
691 }
692
693 <R, S, P extends Message, Q extends Message, T extends Message> AggregateArgument
694 validateArgAndGetPB(Scan scan, ColumnInterpreter<R,S,P,Q,T> ci)
695 throws IOException {
696 validateParameters(scan);
697 final AggregateArgument.Builder requestBuilder =
698 AggregateArgument.newBuilder();
699 requestBuilder.setInterpreterClassName(ci.getClass().getCanonicalName());
700 P columnInterpreterSpecificData = null;
701 if ((columnInterpreterSpecificData = ci.getRequestData())
702 != null) {
703 requestBuilder.setInterpreterSpecificBytes(columnInterpreterSpecificData.toByteString());
704 }
705 requestBuilder.setScan(ProtobufUtil.toScan(scan));
706 return requestBuilder.build();
707 }
708
709 byte[] getBytesFromResponse(ByteString response) {
710 ByteBuffer bb = response.asReadOnlyByteBuffer();
711 bb.rewind();
712 byte[] bytes;
713 if (bb.hasArray()) {
714 bytes = bb.array();
715 } else {
716 bytes = response.toByteArray();
717 }
718 return bytes;
719 }
720 }