Skip to content

Commit 47ac3eb

Browse files
author
Hernan Gelaf-Romer
committed
Merge branch 'hubspot-2.6' of github.com:HubSpot/hbase into hubspot-2.6
2 parents 397a1dc + e2684cc commit 47ac3eb

File tree

14 files changed

+1250
-109
lines changed

14 files changed

+1250
-109
lines changed

CLAUDE.md

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,5 @@
1+
When in the hubspot-2.6 branch, or sub-branches of that branch, use mvn11 instead of mvn, and use the hadoop-3.0 maven profile
2+
3+
When in the master branch, or sub-branches of that branch, use mvn17 instead of mvn, and do not use the hadoop-3.0 maven profile.
4+
5+
Always run the spotless:apply maven target after making changes.

hbase-endpoint/pom.xml

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -115,6 +115,10 @@
115115
<type>test-jar</type>
116116
<scope>test</scope>
117117
</dependency>
118+
<dependency>
119+
<groupId>com.github.stephenc.findbugs</groupId>
120+
<artifactId>findbugs-annotations</artifactId>
121+
</dependency>
118122
<dependency>
119123
<groupId>org.bouncycastle</groupId>
120124
<artifactId>bcprov-jdk18on</artifactId>

hbase-endpoint/src/main/java/org/apache/hadoop/hbase/coprocessor/AggregateImplementation.java

Lines changed: 92 additions & 47 deletions
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@
2424
import com.google.protobuf.RpcCallback;
2525
import com.google.protobuf.RpcController;
2626
import com.google.protobuf.Service;
27+
import edu.umd.cs.findbugs.annotations.Nullable;
2728
import java.io.IOException;
2829
import java.lang.reflect.InvocationTargetException;
2930
import java.nio.ByteBuffer;
@@ -32,6 +33,7 @@
3233
import java.util.Collections;
3334
import java.util.List;
3435
import java.util.NavigableSet;
36+
import java.util.function.Function;
3537
import org.apache.commons.io.IOUtils;
3638
import org.apache.hadoop.hbase.Cell;
3739
import org.apache.hadoop.hbase.CoprocessorEnvironment;
@@ -82,7 +84,7 @@ public void getMax(RpcController controller, AggregateRequest request,
8284
AggregateResponse response = null;
8385
PartialResultContext partialResultContext = new PartialResultContext();
8486
T max = null;
85-
boolean hasMoreRows = false;
87+
boolean hasMoreRows = true;
8688
try {
8789
ColumnInterpreter<T, S, P, Q, R> ci = constructColumnInterpreterFromRequest(request);
8890
T temp;
@@ -109,12 +111,8 @@ public void getMax(RpcController controller, AggregateRequest request,
109111
postScanPartialResultUpdate(results, partialResultContext);
110112
results.clear();
111113
} while (hasMoreRows);
112-
if (max != null) {
113-
AggregateResponse.Builder builder = AggregateResponse.newBuilder();
114-
builder.addFirstPart(ci.getProtoForCellType(max).toByteString());
115-
setPartialResultResponse(builder, request, hasMoreRows, partialResultContext);
116-
response = builder.build();
117-
}
114+
response = singlePartResponse(request, hasMoreRows, partialResultContext, max,
115+
ci::getProtoForCellType);
118116
} catch (IOException e) {
119117
CoprocessorRpcUtils.setControllerException(controller, e);
120118
} finally {
@@ -142,7 +140,7 @@ public void getMin(RpcController controller, AggregateRequest request,
142140
InternalScanner scanner = null;
143141
PartialResultContext partialResultContext = new PartialResultContext();
144142
T min = null;
145-
boolean hasMoreRows = false;
143+
boolean hasMoreRows = true;
146144
try {
147145
ColumnInterpreter<T, S, P, Q, R> ci = constructColumnInterpreterFromRequest(request);
148146
T temp;
@@ -168,12 +166,8 @@ public void getMin(RpcController controller, AggregateRequest request,
168166
postScanPartialResultUpdate(results, partialResultContext);
169167
results.clear();
170168
} while (hasMoreRows);
171-
if (min != null) {
172-
AggregateResponse.Builder responseBuilder =
173-
AggregateResponse.newBuilder().addFirstPart(ci.getProtoForCellType(min).toByteString());
174-
setPartialResultResponse(responseBuilder, request, hasMoreRows, partialResultContext);
175-
response = responseBuilder.build();
176-
}
169+
response = singlePartResponse(request, hasMoreRows, partialResultContext, min,
170+
ci::getProtoForCellType);
177171
} catch (IOException e) {
178172
CoprocessorRpcUtils.setControllerException(controller, e);
179173
} finally {
@@ -201,7 +195,7 @@ public void getSum(RpcController controller, AggregateRequest request,
201195
InternalScanner scanner = null;
202196
PartialResultContext partialResultContext = new PartialResultContext();
203197
long sum = 0L;
204-
boolean hasMoreRows = false;
198+
boolean hasMoreRows = true;
205199
try {
206200
ColumnInterpreter<T, S, P, Q, R> ci = constructColumnInterpreterFromRequest(request);
207201
S sumVal = null;
@@ -230,12 +224,8 @@ public void getSum(RpcController controller, AggregateRequest request,
230224
postScanPartialResultUpdate(results, partialResultContext);
231225
results.clear();
232226
} while (hasMoreRows);
233-
if (sumVal != null) {
234-
AggregateResponse.Builder responseBuilder = AggregateResponse.newBuilder()
235-
.addFirstPart(ci.getProtoForPromotedType(sumVal).toByteString());
236-
setPartialResultResponse(responseBuilder, request, hasMoreRows, partialResultContext);
237-
response = responseBuilder.build();
238-
}
227+
response = singlePartResponse(request, hasMoreRows, partialResultContext, sumVal,
228+
ci::getProtoForPromotedType);
239229
} catch (IOException e) {
240230
CoprocessorRpcUtils.setControllerException(controller, e);
241231
} finally {
@@ -262,7 +252,7 @@ public void getRowNum(RpcController controller, AggregateRequest request,
262252
List<Cell> results = new ArrayList<>();
263253
InternalScanner scanner = null;
264254
PartialResultContext partialResultContext = new PartialResultContext();
265-
boolean hasMoreRows = false;
255+
boolean hasMoreRows = true;
266256
try {
267257
Scan scan = ProtobufUtil.toScan(request.getScan());
268258
byte[][] colFamilies = scan.getFamilies();
@@ -290,10 +280,8 @@ public void getRowNum(RpcController controller, AggregateRequest request,
290280
} while (hasMoreRows);
291281
ByteBuffer bb = ByteBuffer.allocate(8).putLong(counter);
292282
bb.rewind();
293-
AggregateResponse.Builder responseBuilder =
294-
AggregateResponse.newBuilder().addFirstPart(ByteString.copyFrom(bb));
295-
setPartialResultResponse(responseBuilder, request, hasMoreRows, partialResultContext);
296-
response = responseBuilder.build();
283+
response = responseBuilder(request, hasMoreRows, partialResultContext)
284+
.addFirstPart(ByteString.copyFrom(bb)).build();
297285
} catch (IOException e) {
298286
CoprocessorRpcUtils.setControllerException(controller, e);
299287
} finally {
@@ -337,7 +325,7 @@ public void getAvg(RpcController controller, AggregateRequest request,
337325
qualifier = qualifiers.pollFirst();
338326
}
339327
List<Cell> results = new ArrayList<>();
340-
boolean hasMoreRows = false;
328+
boolean hasMoreRows = true;
341329

342330
do {
343331
results.clear();
@@ -353,14 +341,25 @@ public void getAvg(RpcController controller, AggregateRequest request,
353341
rowCountVal++;
354342
postScanPartialResultUpdate(results, partialResultContext);
355343
} while (hasMoreRows);
356-
if (sumVal != null) {
357-
ByteString first = ci.getProtoForPromotedType(sumVal).toByteString();
344+
345+
if (sumVal != null && !request.getClientSupportsPartialResult()) {
358346
AggregateResponse.Builder pair = AggregateResponse.newBuilder();
347+
ByteString first = ci.getProtoForPromotedType(sumVal).toByteString();
359348
pair.addFirstPart(first);
360349
ByteBuffer bb = ByteBuffer.allocate(8).putLong(rowCountVal);
361350
bb.rewind();
362351
pair.setSecondPart(ByteString.copyFrom(bb));
363-
setPartialResultResponse(pair, request, hasMoreRows, partialResultContext);
352+
response = pair.build();
353+
} else if (request.getClientSupportsPartialResult()) {
354+
AggregateResponse.Builder pair =
355+
responseBuilder(request, hasMoreRows, partialResultContext);
356+
if (sumVal != null) {
357+
ByteString first = ci.getProtoForPromotedType(sumVal).toByteString();
358+
pair.addFirstPart(first);
359+
ByteBuffer bb = ByteBuffer.allocate(8).putLong(rowCountVal);
360+
bb.rewind();
361+
pair.setSecondPart(ByteString.copyFrom(bb));
362+
}
364363
response = pair.build();
365364
}
366365
} catch (IOException e) {
@@ -402,7 +401,7 @@ public void getStd(RpcController controller, AggregateRequest request,
402401
}
403402
List<Cell> results = new ArrayList<>();
404403

405-
boolean hasMoreRows = false;
404+
boolean hasMoreRows = true;
406405

407406
do {
408407
if (shouldBreakForThrottling(request, scan, partialResultContext)) {
@@ -421,16 +420,29 @@ public void getStd(RpcController controller, AggregateRequest request,
421420
sumSqVal = ci.add(sumSqVal, ci.multiply(tempVal, tempVal));
422421
rowCountVal++;
423422
} while (hasMoreRows);
424-
if (sumVal != null) {
423+
424+
if (sumVal != null && !request.getClientSupportsPartialResult()) {
425+
AggregateResponse.Builder pair = AggregateResponse.newBuilder();
425426
ByteString first_sumVal = ci.getProtoForPromotedType(sumVal).toByteString();
426427
ByteString first_sumSqVal = ci.getProtoForPromotedType(sumSqVal).toByteString();
427-
AggregateResponse.Builder pair = AggregateResponse.newBuilder();
428428
pair.addFirstPart(first_sumVal);
429429
pair.addFirstPart(first_sumSqVal);
430430
ByteBuffer bb = ByteBuffer.allocate(8).putLong(rowCountVal);
431431
bb.rewind();
432432
pair.setSecondPart(ByteString.copyFrom(bb));
433-
setPartialResultResponse(pair, request, hasMoreRows, partialResultContext);
433+
response = pair.build();
434+
} else if (request.getClientSupportsPartialResult()) {
435+
AggregateResponse.Builder pair =
436+
responseBuilder(request, hasMoreRows, partialResultContext);
437+
if (sumVal != null) {
438+
ByteString first_sumVal = ci.getProtoForPromotedType(sumVal).toByteString();
439+
ByteString first_sumSqVal = ci.getProtoForPromotedType(sumSqVal).toByteString();
440+
pair.addFirstPart(first_sumVal);
441+
pair.addFirstPart(first_sumSqVal);
442+
ByteBuffer bb = ByteBuffer.allocate(8).putLong(rowCountVal);
443+
bb.rewind();
444+
pair.setSecondPart(ByteString.copyFrom(bb));
445+
}
434446
response = pair.build();
435447
}
436448
} catch (IOException e) {
@@ -471,7 +483,7 @@ public void getMedian(RpcController controller, AggregateRequest request,
471483
}
472484
List<Cell> results = new ArrayList<>();
473485

474-
boolean hasMoreRows = false;
486+
boolean hasMoreRows = true;
475487
do {
476488
if (shouldBreakForThrottling(request, scan, partialResultContext)) {
477489
break;
@@ -493,14 +505,26 @@ public void getMedian(RpcController controller, AggregateRequest request,
493505
sumVal = ci.add(sumVal, tempVal);
494506
sumWeights = ci.add(sumWeights, tempWeight);
495507
} while (hasMoreRows);
496-
ByteString first_sumVal = ci.getProtoForPromotedType(sumVal).toByteString();
497-
S s = sumWeights == null ? ci.castToReturnType(ci.getMinValue()) : sumWeights;
498-
ByteString first_sumWeights = ci.getProtoForPromotedType(s).toByteString();
499-
AggregateResponse.Builder pair = AggregateResponse.newBuilder();
500-
pair.addFirstPart(first_sumVal);
501-
pair.addFirstPart(first_sumWeights);
502-
setPartialResultResponse(pair, request, hasMoreRows, partialResultContext);
503-
response = pair.build();
508+
if (sumVal != null && !request.getClientSupportsPartialResult()) {
509+
AggregateResponse.Builder pair = AggregateResponse.newBuilder();
510+
ByteString first_sumVal = ci.getProtoForPromotedType(sumVal).toByteString();
511+
S s = sumWeights == null ? ci.castToReturnType(ci.getMinValue()) : sumWeights;
512+
ByteString first_sumWeights = ci.getProtoForPromotedType(s).toByteString();
513+
pair.addFirstPart(first_sumVal);
514+
pair.addFirstPart(first_sumWeights);
515+
response = pair.build();
516+
} else if (request.getClientSupportsPartialResult()) {
517+
AggregateResponse.Builder pair =
518+
responseBuilder(request, hasMoreRows, partialResultContext);
519+
if (sumVal != null) {
520+
ByteString first_sumVal = ci.getProtoForPromotedType(sumVal).toByteString();
521+
S s = sumWeights == null ? ci.castToReturnType(ci.getMinValue()) : sumWeights;
522+
ByteString first_sumWeights = ci.getProtoForPromotedType(s).toByteString();
523+
pair.addFirstPart(first_sumVal);
524+
pair.addFirstPart(first_sumWeights);
525+
}
526+
response = pair.build();
527+
}
504528
} catch (IOException e) {
505529
CoprocessorRpcUtils.setControllerException(controller, e);
506530
} finally {
@@ -558,20 +582,41 @@ private void postScanPartialResultUpdate(List<Cell> results, PartialResultContex
558582
}
559583
}
560584

561-
private void setPartialResultResponse(AggregateResponse.Builder builder, AggregateRequest request,
562-
boolean hasMoreRows, PartialResultContext context) throws IOException {
563-
// If we encountered an RpcThrottlingException, tell the client the partial result we've
564-
// accumulated so far, and what row to start scanning at in order to finish the scan.
585+
@Nullable
586+
private <ACC, RES extends Message> AggregateResponse singlePartResponse(AggregateRequest request,
587+
boolean hasMoreRows, PartialResultContext partialResultContext, ACC acc,
588+
Function<ACC, RES> toRes) {
589+
AggregateResponse response = null;
590+
if (acc != null && !request.getClientSupportsPartialResult()) {
591+
ByteString first = toRes.apply(acc).toByteString();
592+
response = AggregateResponse.newBuilder().addFirstPart(first).build();
593+
} else if (request.getClientSupportsPartialResult()) {
594+
AggregateResponse.Builder responseBuilder =
595+
responseBuilder(request, hasMoreRows, partialResultContext);
596+
if (acc != null) {
597+
responseBuilder.addFirstPart(toRes.apply(acc).toByteString());
598+
}
599+
response = responseBuilder.build();
600+
}
601+
return response;
602+
}
603+
604+
private AggregateResponse.Builder responseBuilder(AggregateRequest request, boolean hasMoreRows,
605+
PartialResultContext context) {
606+
AggregateResponse.Builder builder = AggregateResponse.newBuilder();
565607
if (request.getClientSupportsPartialResult() && hasMoreRows) {
566608
if (context.lastRowSuccessfullyProcessedArray != null) {
567609
byte[] lastRowSuccessfullyProcessed = Arrays.copyOfRange(
568610
context.lastRowSuccessfullyProcessedArray, context.lastRowSuccessfullyProcessedOffset,
569611
context.lastRowSuccessfullyProcessedOffset + context.lastRowSuccessfullyProcessedLength);
570612
builder.setNextChunkStartRow(ByteString.copyFrom(
571613
ClientUtil.calculateTheClosestNextRowKeyForPrefix(lastRowSuccessfullyProcessed)));
614+
} else {
615+
builder.setNextChunkStartRow(request.getScan().getStartRow());
572616
}
573617
builder.setWaitIntervalMs(context.waitIntervalMs);
574618
}
619+
return builder;
575620
}
576621

577622
@SuppressWarnings("unchecked")

0 commit comments

Comments
 (0)