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.nio.ByteBuffer;
23 import java.util.ArrayList;
24 import java.util.List;
25 import java.util.NavigableSet;
26
27 import org.apache.commons.logging.Log;
28 import org.apache.commons.logging.LogFactory;
29 import org.apache.hadoop.classification.InterfaceAudience;
30 import org.apache.hadoop.classification.InterfaceStability;
31 import org.apache.hadoop.hbase.Coprocessor;
32 import org.apache.hadoop.hbase.CoprocessorEnvironment;
33 import org.apache.hadoop.hbase.KeyValue;
34 import org.apache.hadoop.hbase.client.Scan;
35 import org.apache.hadoop.hbase.client.coprocessor.AggregationClient;
36 import org.apache.hadoop.hbase.exceptions.CoprocessorException;
37 import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
38 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
39 import org.apache.hadoop.hbase.protobuf.ResponseConverter;
40 import org.apache.hadoop.hbase.protobuf.generated.AggregateProtos.AggregateArgument;
41 import org.apache.hadoop.hbase.protobuf.generated.AggregateProtos.AggregateResponse;
42 import org.apache.hadoop.hbase.protobuf.generated.AggregateProtos.AggregateService;
43 import org.apache.hadoop.hbase.regionserver.InternalScanner;
44
45 import com.google.protobuf.ByteString;
46 import com.google.protobuf.Message;
47 import com.google.protobuf.RpcCallback;
48 import com.google.protobuf.RpcController;
49 import com.google.protobuf.Service;
50
51
52
53
54
55
56
57
58
59
60
61
62
63 @InterfaceAudience.Public
64 @InterfaceStability.Evolving
65 public class AggregateImplementation<T, S, P extends Message, Q extends Message, R extends Message>
66 extends AggregateService implements CoprocessorService, Coprocessor {
67 protected static final Log log = LogFactory.getLog(AggregateImplementation.class);
68 private RegionCoprocessorEnvironment env;
69
70
71
72
73
74
75
76
77 @Override
78 public void getMax(RpcController controller, AggregateArgument request,
79 RpcCallback<AggregateResponse> done) {
80 InternalScanner scanner = null;
81 AggregateResponse response = null;
82 T max = null;
83 try {
84 ColumnInterpreter<T, S, P, Q, R> ci = constructColumnInterpreterFromRequest(request);
85 T temp;
86 Scan scan = ProtobufUtil.toScan(request.getScan());
87 scanner = env.getRegion().getScanner(scan);
88 List<KeyValue> results = new ArrayList<KeyValue>();
89 byte[] colFamily = scan.getFamilies()[0];
90 NavigableSet<byte[]> qualifiers = scan.getFamilyMap().get(colFamily);
91 byte[] qualifier = null;
92 if (qualifiers != null && !qualifiers.isEmpty()) {
93 qualifier = qualifiers.pollFirst();
94 }
95
96 boolean hasMoreRows = false;
97 do {
98 hasMoreRows = scanner.next(results);
99 for (KeyValue kv : results) {
100 temp = ci.getValue(colFamily, qualifier, kv);
101 max = (max == null || (temp != null && ci.compare(temp, max) > 0)) ? temp : max;
102 }
103 results.clear();
104 } while (hasMoreRows);
105 if (max != null) {
106 AggregateResponse.Builder builder = AggregateResponse.newBuilder();
107 builder.addFirstPart(ci.getProtoForCellType(max).toByteString());
108 response = builder.build();
109 }
110 } catch (IOException e) {
111 ResponseConverter.setControllerException(controller, e);
112 } finally {
113 if (scanner != null) {
114 try {
115 scanner.close();
116 } catch (IOException ignored) {}
117 }
118 }
119 log.info("Maximum from this region is "
120 + env.getRegion().getRegionNameAsString() + ": " + max);
121 done.run(response);
122 }
123
124
125
126
127
128
129
130
131 @Override
132 public void getMin(RpcController controller, AggregateArgument request,
133 RpcCallback<AggregateResponse> done) {
134 AggregateResponse response = null;
135 InternalScanner scanner = null;
136 T min = null;
137 try {
138 ColumnInterpreter<T, S, P, Q, R> ci = constructColumnInterpreterFromRequest(request);
139 T temp;
140 Scan scan = ProtobufUtil.toScan(request.getScan());
141 scanner = env.getRegion().getScanner(scan);
142 List<KeyValue> results = new ArrayList<KeyValue>();
143 byte[] colFamily = scan.getFamilies()[0];
144 NavigableSet<byte[]> qualifiers = scan.getFamilyMap().get(colFamily);
145 byte[] qualifier = null;
146 if (qualifiers != null && !qualifiers.isEmpty()) {
147 qualifier = qualifiers.pollFirst();
148 }
149 boolean hasMoreRows = false;
150 do {
151 hasMoreRows = scanner.next(results);
152 for (KeyValue kv : results) {
153 temp = ci.getValue(colFamily, qualifier, kv);
154 min = (min == null || (temp != null && ci.compare(temp, min) < 0)) ? temp : min;
155 }
156 results.clear();
157 } while (hasMoreRows);
158 if (min != null) {
159 response = AggregateResponse.newBuilder().addFirstPart(
160 ci.getProtoForCellType(min).toByteString()).build();
161 }
162 } catch (IOException e) {
163 ResponseConverter.setControllerException(controller, e);
164 } finally {
165 if (scanner != null) {
166 try {
167 scanner.close();
168 } catch (IOException ignored) {}
169 }
170 }
171 log.info("Minimum from this region is "
172 + env.getRegion().getRegionNameAsString() + ": " + min);
173 done.run(response);
174 }
175
176
177
178
179
180
181
182
183 @Override
184 public void getSum(RpcController controller, AggregateArgument request,
185 RpcCallback<AggregateResponse> done) {
186 AggregateResponse response = null;
187 InternalScanner scanner = null;
188 long sum = 0l;
189 try {
190 ColumnInterpreter<T, S, P, Q, R> ci = constructColumnInterpreterFromRequest(request);
191 S sumVal = null;
192 T temp;
193 Scan scan = ProtobufUtil.toScan(request.getScan());
194 scanner = env.getRegion().getScanner(scan);
195 byte[] colFamily = scan.getFamilies()[0];
196 NavigableSet<byte[]> qualifiers = scan.getFamilyMap().get(colFamily);
197 byte[] qualifier = null;
198 if (qualifiers != null && !qualifiers.isEmpty()) {
199 qualifier = qualifiers.pollFirst();
200 }
201 List<KeyValue> results = new ArrayList<KeyValue>();
202 boolean hasMoreRows = false;
203 do {
204 hasMoreRows = scanner.next(results);
205 for (KeyValue kv : results) {
206 temp = ci.getValue(colFamily, qualifier, kv);
207 if (temp != null)
208 sumVal = ci.add(sumVal, ci.castToReturnType(temp));
209 }
210 results.clear();
211 } while (hasMoreRows);
212 if (sumVal != null) {
213 response = AggregateResponse.newBuilder().addFirstPart(
214 ci.getProtoForPromotedType(sumVal).toByteString()).build();
215 }
216 } catch (IOException e) {
217 ResponseConverter.setControllerException(controller, e);
218 } finally {
219 if (scanner != null) {
220 try {
221 scanner.close();
222 } catch (IOException ignored) {}
223 }
224 }
225 log.debug("Sum from this region is "
226 + env.getRegion().getRegionNameAsString() + ": " + sum);
227 done.run(response);
228 }
229
230
231
232
233
234
235 @Override
236 public void getRowNum(RpcController controller, AggregateArgument request,
237 RpcCallback<AggregateResponse> done) {
238 AggregateResponse response = null;
239 long counter = 0l;
240 List<KeyValue> results = new ArrayList<KeyValue>();
241 InternalScanner scanner = null;
242 try {
243 Scan scan = ProtobufUtil.toScan(request.getScan());
244 byte[] colFamily = scan.getFamilies()[0];
245 NavigableSet<byte[]> qualifiers = scan.getFamilyMap().get(colFamily);
246 byte[] qualifier = null;
247 if (qualifiers != null && !qualifiers.isEmpty()) {
248 qualifier = qualifiers.pollFirst();
249 }
250 if (scan.getFilter() == null && qualifier == null)
251 scan.setFilter(new FirstKeyOnlyFilter());
252 scanner = env.getRegion().getScanner(scan);
253 boolean hasMoreRows = false;
254 do {
255 hasMoreRows = scanner.next(results);
256 if (results.size() > 0) {
257 counter++;
258 }
259 results.clear();
260 } while (hasMoreRows);
261 ByteBuffer bb = ByteBuffer.allocate(8).putLong(counter);
262 bb.rewind();
263 response = AggregateResponse.newBuilder().addFirstPart(
264 ByteString.copyFrom(bb)).build();
265 } catch (IOException e) {
266 ResponseConverter.setControllerException(controller, e);
267 } finally {
268 if (scanner != null) {
269 try {
270 scanner.close();
271 } catch (IOException ignored) {}
272 }
273 }
274 log.info("Row counter from this region is "
275 + env.getRegion().getRegionNameAsString() + ": " + counter);
276 done.run(response);
277 }
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292 @Override
293 public void getAvg(RpcController controller, AggregateArgument request,
294 RpcCallback<AggregateResponse> done) {
295 AggregateResponse response = null;
296 InternalScanner scanner = null;
297 try {
298 ColumnInterpreter<T, S, P, Q, R> ci = constructColumnInterpreterFromRequest(request);
299 S sumVal = null;
300 Long rowCountVal = 0l;
301 Scan scan = ProtobufUtil.toScan(request.getScan());
302 scanner = env.getRegion().getScanner(scan);
303 byte[] colFamily = scan.getFamilies()[0];
304 NavigableSet<byte[]> qualifiers = scan.getFamilyMap().get(colFamily);
305 byte[] qualifier = null;
306 if (qualifiers != null && !qualifiers.isEmpty()) {
307 qualifier = qualifiers.pollFirst();
308 }
309 List<KeyValue> results = new ArrayList<KeyValue>();
310 boolean hasMoreRows = false;
311
312 do {
313 results.clear();
314 hasMoreRows = scanner.next(results);
315 for (KeyValue kv : results) {
316 sumVal = ci.add(sumVal, ci.castToReturnType(ci.getValue(colFamily,
317 qualifier, kv)));
318 }
319 rowCountVal++;
320 } while (hasMoreRows);
321 if (sumVal != null) {
322 ByteString first = ci.getProtoForPromotedType(sumVal).toByteString();
323 AggregateResponse.Builder pair = AggregateResponse.newBuilder();
324 pair.addFirstPart(first);
325 ByteBuffer bb = ByteBuffer.allocate(8).putLong(rowCountVal);
326 bb.rewind();
327 pair.setSecondPart(ByteString.copyFrom(bb));
328 response = pair.build();
329 }
330 } catch (IOException e) {
331 ResponseConverter.setControllerException(controller, e);
332 } finally {
333 if (scanner != null) {
334 try {
335 scanner.close();
336 } catch (IOException ignored) {}
337 }
338 }
339 done.run(response);
340 }
341
342
343
344
345
346
347
348
349
350
351 @Override
352 public void getStd(RpcController controller, AggregateArgument request,
353 RpcCallback<AggregateResponse> done) {
354 InternalScanner scanner = null;
355 AggregateResponse response = null;
356 try {
357 ColumnInterpreter<T, S, P, Q, R> ci = constructColumnInterpreterFromRequest(request);
358 S sumVal = null, sumSqVal = null, tempVal = null;
359 long rowCountVal = 0l;
360 Scan scan = ProtobufUtil.toScan(request.getScan());
361 scanner = env.getRegion().getScanner(scan);
362 byte[] colFamily = scan.getFamilies()[0];
363 NavigableSet<byte[]> qualifiers = scan.getFamilyMap().get(colFamily);
364 byte[] qualifier = null;
365 if (qualifiers != null && !qualifiers.isEmpty()) {
366 qualifier = qualifiers.pollFirst();
367 }
368 List<KeyValue> results = new ArrayList<KeyValue>();
369
370 boolean hasMoreRows = false;
371
372 do {
373 tempVal = null;
374 hasMoreRows = scanner.next(results);
375 for (KeyValue kv : results) {
376 tempVal = ci.add(tempVal, ci.castToReturnType(ci.getValue(colFamily,
377 qualifier, kv)));
378 }
379 results.clear();
380 sumVal = ci.add(sumVal, tempVal);
381 sumSqVal = ci.add(sumSqVal, ci.multiply(tempVal, tempVal));
382 rowCountVal++;
383 } while (hasMoreRows);
384 if (sumVal != null) {
385 ByteString first_sumVal = ci.getProtoForPromotedType(sumVal).toByteString();
386 ByteString first_sumSqVal = ci.getProtoForPromotedType(sumSqVal).toByteString();
387 AggregateResponse.Builder pair = AggregateResponse.newBuilder();
388 pair.addFirstPart(first_sumVal);
389 pair.addFirstPart(first_sumSqVal);
390 ByteBuffer bb = ByteBuffer.allocate(8).putLong(rowCountVal);
391 bb.rewind();
392 pair.setSecondPart(ByteString.copyFrom(bb));
393 response = pair.build();
394 }
395 } catch (IOException e) {
396 ResponseConverter.setControllerException(controller, e);
397 } finally {
398 if (scanner != null) {
399 try {
400 scanner.close();
401 } catch (IOException ignored) {}
402 }
403 }
404 done.run(response);
405 }
406
407
408
409
410
411
412
413
414
415 @Override
416 public void getMedian(RpcController controller, AggregateArgument request,
417 RpcCallback<AggregateResponse> done) {
418 AggregateResponse response = null;
419 InternalScanner scanner = null;
420 try {
421 ColumnInterpreter<T, S, P, Q, R> ci = constructColumnInterpreterFromRequest(request);
422 S sumVal = null, sumWeights = null, tempVal = null, tempWeight = null;
423 Scan scan = ProtobufUtil.toScan(request.getScan());
424 scanner = env.getRegion().getScanner(scan);
425 byte[] colFamily = scan.getFamilies()[0];
426 NavigableSet<byte[]> qualifiers = scan.getFamilyMap().get(colFamily);
427 byte[] valQualifier = null, weightQualifier = null;
428 if (qualifiers != null && !qualifiers.isEmpty()) {
429 valQualifier = qualifiers.pollFirst();
430
431 weightQualifier = qualifiers.pollLast();
432 }
433 List<KeyValue> results = new ArrayList<KeyValue>();
434
435 boolean hasMoreRows = false;
436
437 do {
438 tempVal = null;
439 tempWeight = null;
440 hasMoreRows = scanner.next(results);
441 for (KeyValue kv : results) {
442 tempVal = ci.add(tempVal, ci.castToReturnType(ci.getValue(colFamily,
443 valQualifier, kv)));
444 if (weightQualifier != null) {
445 tempWeight = ci.add(tempWeight,
446 ci.castToReturnType(ci.getValue(colFamily, weightQualifier, kv)));
447 }
448 }
449 results.clear();
450 sumVal = ci.add(sumVal, tempVal);
451 sumWeights = ci.add(sumWeights, tempWeight);
452 } while (hasMoreRows);
453 ByteString first_sumVal = ci.getProtoForPromotedType(sumVal).toByteString();
454 S s = sumWeights == null ? ci.castToReturnType(ci.getMinValue()) : sumWeights;
455 ByteString first_sumWeights = ci.getProtoForPromotedType(s).toByteString();
456 AggregateResponse.Builder pair = AggregateResponse.newBuilder();
457 pair.addFirstPart(first_sumVal);
458 pair.addFirstPart(first_sumWeights);
459 response = pair.build();
460 } catch (IOException e) {
461 ResponseConverter.setControllerException(controller, e);
462 } finally {
463 if (scanner != null) {
464 try {
465 scanner.close();
466 } catch (IOException ignored) {}
467 }
468 }
469 done.run(response);
470 }
471
472 @SuppressWarnings("unchecked")
473 ColumnInterpreter<T,S,P,Q,R> constructColumnInterpreterFromRequest(
474 AggregateArgument request) throws IOException {
475 String className = request.getInterpreterClassName();
476 Class<?> cls;
477 try {
478 cls = Class.forName(className);
479 ColumnInterpreter<T,S,P,Q,R> ci = (ColumnInterpreter<T, S, P, Q, R>) cls.newInstance();
480 if (request.hasInterpreterSpecificBytes()) {
481 ByteString b = request.getInterpreterSpecificBytes();
482 P initMsg = ProtobufUtil.getParsedGenericInstance(ci.getClass(), 2, b);
483 ci.initialize(initMsg);
484 }
485 return ci;
486 } catch (ClassNotFoundException e) {
487 throw new IOException(e);
488 } catch (InstantiationException e) {
489 throw new IOException(e);
490 } catch (IllegalAccessException e) {
491 throw new IOException(e);
492 }
493 }
494
495 @Override
496 public Service getService() {
497 return this;
498 }
499
500
501
502
503
504
505
506
507
508
509
510 @Override
511 public void start(CoprocessorEnvironment env) throws IOException {
512 if (env instanceof RegionCoprocessorEnvironment) {
513 this.env = (RegionCoprocessorEnvironment)env;
514 } else {
515 throw new CoprocessorException("Must be loaded on a table region!");
516 }
517 }
518
519 @Override
520 public void stop(CoprocessorEnvironment env) throws IOException {
521
522 }
523
524 }