-
Notifications
You must be signed in to change notification settings - Fork 107
Add partial result to AggregateCursor continuation #3254
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
6c1ad10
855ae27
5c2da3f
c62bffa
06d8e65
0950385
b275ea6
c953df8
df09e54
195b68c
97a3cdc
4810d22
2948826
0d2d050
9c4c50f
2eade46
e76fe89
9540c23
89cdd6c
413d480
047de80
bb1bd6a
255f19b
eaa0d78
a1422b0
7e3187a
3de4fcc
b88775b
8b9703c
14d157f
7abfe4b
31588aa
6148359
fe6dd12
6c2ee73
2e59b19
830158c
c91bc64
cd63dc5
5e05cdf
39d92a0
1add4ca
57bf49d
cfd3b88
e875d82
4cda3e8
93ea4ec
e98d3ba
f811e35
62d2e14
d9a1798
bb4525e
6004807
be6887c
f39e470
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -22,12 +22,19 @@ | |
|
||
import com.apple.foundationdb.annotation.API; | ||
import com.apple.foundationdb.async.AsyncUtil; | ||
import com.apple.foundationdb.record.ByteArrayContinuation; | ||
import com.apple.foundationdb.record.RecordCoreException; | ||
import com.apple.foundationdb.record.RecordCursor; | ||
import com.apple.foundationdb.record.RecordCursorContinuation; | ||
import com.apple.foundationdb.record.RecordCursorProto; | ||
import com.apple.foundationdb.record.RecordCursorResult; | ||
import com.apple.foundationdb.record.RecordCursorVisitor; | ||
import com.apple.foundationdb.record.query.plan.plans.QueryResult; | ||
import com.apple.foundationdb.record.query.plan.plans.RecordQueryStreamingAggregationPlan; | ||
import com.apple.foundationdb.tuple.ByteArrayUtil2; | ||
import com.google.common.base.Verify; | ||
import com.google.protobuf.ByteString; | ||
import com.google.protobuf.InvalidProtocolBufferException; | ||
import com.google.protobuf.Message; | ||
|
||
import javax.annotation.Nonnull; | ||
|
@@ -55,26 +62,37 @@ public class AggregateCursor<M extends Message> implements RecordCursor<QueryRes | |
// Previous non-empty record processed by this cursor | ||
@Nullable | ||
private RecordCursorResult<QueryResult> previousValidResult; | ||
@Nonnull | ||
private RecordCursorContinuation previousContinuationInGroup; | ||
@Nullable | ||
private RecordCursorProto.PartialAggregationResult partialAggregationResult; | ||
@Nonnull | ||
private final RecordQueryStreamingAggregationPlan.SerializationMode serializationMode; | ||
|
||
public AggregateCursor(@Nonnull RecordCursor<QueryResult> inner, | ||
@Nonnull final StreamGrouping<M> streamGrouping) { | ||
@Nonnull final StreamGrouping<M> streamGrouping, | ||
@Nonnull RecordCursorContinuation continuation, | ||
@Nonnull final RecordQueryStreamingAggregationPlan.SerializationMode serializationMode) { | ||
this.inner = inner; | ||
this.streamGrouping = streamGrouping; | ||
this.serializationMode = serializationMode; | ||
this.previousContinuationInGroup = continuation; | ||
} | ||
|
||
@Nonnull | ||
@Override | ||
public CompletableFuture<RecordCursorResult<QueryResult>> onNext() { | ||
if (previousResult != null && !previousResult.hasNext()) { | ||
// we are done | ||
return CompletableFuture.completedFuture(RecordCursorResult.exhausted()); | ||
return CompletableFuture.completedFuture(RecordCursorResult.withoutNextValue(new AggregateCursorContinuation(previousResult.getContinuation(), streamGrouping.getPartialAggregationResult(), serializationMode), | ||
previousResult.getNoNextReason())); | ||
} | ||
|
||
return AsyncUtil.whileTrue(() -> inner.onNext().thenApply(innerResult -> { | ||
previousResult = innerResult; | ||
if (!innerResult.hasNext()) { | ||
if (!isNoRecords()) { | ||
streamGrouping.finalizeGroup(); | ||
partialAggregationResult = streamGrouping.finalizeGroup(); | ||
} | ||
return false; | ||
} else { | ||
|
@@ -83,45 +101,73 @@ public CompletableFuture<RecordCursorResult<QueryResult>> onNext() { | |
if (!groupBreak) { | ||
// previousValidResult is the last row before group break, it sets the continuation | ||
previousValidResult = innerResult; | ||
previousContinuationInGroup = previousValidResult.getContinuation(); | ||
} | ||
return (!groupBreak); | ||
} | ||
}), getExecutor()).thenApply(vignore -> { | ||
if (isNoRecords()) { | ||
// Edge case where there are no records at all | ||
return RecordCursorResult.exhausted(); | ||
// either innerResult.hasNext() = false; or groupBreak = true | ||
if (Verify.verifyNotNull(previousResult).hasNext()) { | ||
// in this case groupBreak = true, return aggregated result and continuation, partialAggregationResult = null | ||
// previousValidResult = null happens when 1st row of current scan != last row of last scan, results in groupBreak = true and previousValidResult = null | ||
RecordCursorContinuation c = new AggregateCursorContinuation(previousContinuationInGroup, serializationMode); | ||
|
||
/* | ||
* Update the previousValidResult to the next continuation even though it hasn't been returned. This is to return the correct continuation when there are single-element groups. | ||
* Below is an example that shows how continuation(previousValidResult) moves: | ||
* Initial: previousResult = null, previousValidResult = null | ||
row0 groupKey0 groupBreak = False previousValidResult = row0 previousResult = row0 | ||
row1 groupKey0 groupBreak = False previousValidResult = row1 previousResult = row1 | ||
row2 groupKey1 groupBreak = True previousValidResult = row1 previousResult = row2 | ||
* returns result (groupKey0, continuation = row1), and set previousValidResult = row2 | ||
* | ||
* Now there are 2 scenarios, 1) the current iteration continues; 2) the current iteration stops | ||
* In scenario 1, the iteration continues, it gets to row3: | ||
row3 groupKey2 groupBreak = True previousValidResult = row2 previousResult = row3 | ||
* returns result (groupKey1, continuation = row2), and set previousValidResult = row3 | ||
* | ||
* In scenario 2, a new iteration starts from row2 (because the last returned continuation = row1), and set initial previousResult = null, previousValidResult = null: | ||
row2 groupKey1 groupBreak = False previousValidResult = row2 previousResult = row2 | ||
* (Note that because a new iteration starts, groupBreak = False for row2.) | ||
row3 groupKey2 groupBreak = True previousValidResult = row2 previousResult = row3 | ||
* returns result (groupKey1, continuation = row2), and set previousValidResult = row3 | ||
* | ||
* Both scenarios returns the correct result, and continuation are both set to row3 in the end, row2 is scanned twice if a new iteration starts. | ||
*/ | ||
previousValidResult = previousResult; | ||
previousContinuationInGroup = Verify.verifyNotNull(previousValidResult).getContinuation(); | ||
return RecordCursorResult.withNextValue(QueryResult.ofComputed(streamGrouping.getCompletedGroupResult()), c); | ||
} else { | ||
// innerResult.hasNext() = false | ||
if (Verify.verifyNotNull(previousResult).getNoNextReason() == NoNextReason.SOURCE_EXHAUSTED) { | ||
// exhausted | ||
if (previousValidResult == null && partialAggregationResult == null) { | ||
return RecordCursorResult.exhausted(); | ||
} else { | ||
RecordCursorContinuation c = new AggregateCursorContinuation(previousContinuationInGroup, serializationMode); | ||
previousValidResult = previousResult; | ||
previousContinuationInGroup = Verify.verifyNotNull(previousValidResult).getContinuation(); | ||
return RecordCursorResult.withNextValue(QueryResult.ofComputed(streamGrouping.getCompletedGroupResult()), c); | ||
} | ||
} else { | ||
// stopped in the middle of a group | ||
RecordCursorContinuation currentContinuation = new AggregateCursorContinuation(Verify.verifyNotNull(previousResult).getContinuation(), partialAggregationResult, serializationMode); | ||
previousValidResult = previousResult; | ||
previousContinuationInGroup = Verify.verifyNotNull(previousValidResult).getContinuation(); | ||
if (serializationMode == RecordQueryStreamingAggregationPlan.SerializationMode.TO_NEW) { | ||
return RecordCursorResult.withoutNextValue(currentContinuation, Verify.verifyNotNull(previousResult).getNoNextReason()); | ||
} else { | ||
// for TO_OLD, return {groupKey: partialResult from row x to y}, next time will return {groupKey: partialResult from y+1 to next stop point} | ||
return RecordCursorResult.withNextValue(QueryResult.ofComputed(streamGrouping.getCompletedGroupResult()), currentContinuation); | ||
alecgrieser marked this conversation as resolved.
Show resolved
Hide resolved
|
||
} | ||
} | ||
} | ||
// Use the last valid result for the continuation as we need non-terminal one here. | ||
RecordCursorContinuation continuation = Verify.verifyNotNull(previousValidResult).getContinuation(); | ||
/* | ||
* Update the previousValidResult to the next continuation even though it hasn't been returned. This is to return the correct continuation when there are single-element groups. | ||
* Below is an example that shows how continuation(previousValidResult) moves: | ||
* Initial: previousResult = null, previousValidResult = null | ||
row0 groupKey0 groupBreak = False previousValidResult = row0 previousResult = row0 | ||
row1 groupKey0 groupBreak = False previousValidResult = row1 previousResult = row1 | ||
row2 groupKey1 groupBreak = True previousValidResult = row1 previousResult = row2 | ||
* returns result (groupKey0, continuation = row1), and set previousValidResult = row2 | ||
* | ||
* Now there are 2 scenarios, 1) the current iteration continues; 2) the current iteration stops | ||
* In scenario 1, the iteration continues, it gets to row3: | ||
row3 groupKey2 groupBreak = True previousValidResult = row2 previousResult = row3 | ||
* returns result (groupKey1, continuation = row2), and set previousValidResult = row3 | ||
* | ||
* In scenario 2, a new iteration starts from row2 (because the last returned continuation = row1), and set initial previousResult = null, previousValidResult = null: | ||
row2 groupKey1 groupBreak = False previousValidResult = row2 previousResult = row2 | ||
* (Note that because a new iteration starts, groupBreak = False for row2.) | ||
row3 groupKey2 groupBreak = True previousValidResult = row2 previousResult = row3 | ||
* returns result (groupKey1, continuation = row2), and set previousValidResult = row3 | ||
* | ||
* Both scenarios returns the correct result, and continuation are both set to row3 in the end, row2 is scanned twice if a new iteration starts. | ||
*/ | ||
previousValidResult = previousResult; | ||
return RecordCursorResult.withNextValue(QueryResult.ofComputed(streamGrouping.getCompletedGroupResult()), continuation); | ||
}); | ||
} | ||
|
||
|
||
|
||
private boolean isNoRecords() { | ||
return ((previousValidResult == null) && (!Verify.verifyNotNull(previousResult).hasNext())); | ||
return ((previousValidResult == null) && (!Verify.verifyNotNull(previousResult).hasNext()) && (streamGrouping.getPartialAggregationResult() == null)); | ||
} | ||
|
||
@Override | ||
|
@@ -147,4 +193,84 @@ public boolean accept(@Nonnull RecordCursorVisitor visitor) { | |
} | ||
return visitor.visitLeave(this); | ||
} | ||
|
||
public static class AggregateCursorContinuation implements RecordCursorContinuation { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. A comment generally about the approach: this isn't quite the way that I thought this was going to go. My assumption had been that we were going to modify the continuation here so that we always return an I do think the alternative presented here (that is, the inner continuation is associated with the last value in a group that was returned and It would be nice if we were a little forward looking and could handle a future continuation of that format. I could also see us wanting to transition to that approach in this PR now is I suppose debatable. |
||
@Nonnull | ||
private final RecordCursorContinuation innerContinuation; | ||
|
||
@Nullable | ||
private final RecordCursorProto.PartialAggregationResult partialAggregationResult; | ||
|
||
@Nullable | ||
private RecordCursorProto.AggregateCursorContinuation cachedProto; | ||
|
||
private final RecordQueryStreamingAggregationPlan.SerializationMode serializationMode; | ||
|
||
public AggregateCursorContinuation(@Nonnull RecordCursorContinuation innerContinuation, @Nullable RecordCursorProto.PartialAggregationResult partialAggregationResult, final RecordQueryStreamingAggregationPlan.SerializationMode serializationMode) { | ||
this.innerContinuation = innerContinuation; | ||
this.partialAggregationResult = partialAggregationResult; | ||
this.serializationMode = serializationMode; | ||
} | ||
|
||
public AggregateCursorContinuation(@Nonnull RecordCursorContinuation other, RecordQueryStreamingAggregationPlan.SerializationMode serializationMode) { | ||
this(other, null, serializationMode); | ||
} | ||
|
||
@Nonnull | ||
@Override | ||
public ByteString toByteString() { | ||
if (serializationMode == RecordQueryStreamingAggregationPlan.SerializationMode.TO_OLD) { | ||
return innerContinuation.toByteString(); | ||
} else { | ||
return isEnd() ? ByteString.EMPTY : toProto().toByteString(); | ||
} | ||
} | ||
|
||
@Nullable | ||
@Override | ||
public byte[] toBytes() { | ||
ByteString byteString = toByteString(); | ||
return byteString.isEmpty() ? null : byteString.toByteArray(); | ||
} | ||
|
||
@Override | ||
public boolean isEnd() { | ||
return innerContinuation.isEnd(); | ||
alecgrieser marked this conversation as resolved.
Show resolved
Hide resolved
|
||
} | ||
|
||
@Nullable | ||
public byte[] getInnerContinuation() { | ||
return innerContinuation.toBytes(); | ||
} | ||
|
||
@Nullable | ||
public RecordCursorProto.PartialAggregationResult getPartialAggregationResult() { | ||
return partialAggregationResult; | ||
} | ||
|
||
@Nonnull | ||
private RecordCursorProto.AggregateCursorContinuation toProto() { | ||
if (cachedProto == null) { | ||
RecordCursorProto.AggregateCursorContinuation.Builder cachedProtoBuilder = RecordCursorProto.AggregateCursorContinuation.newBuilder().setContinuation(innerContinuation.toByteString()); | ||
if (partialAggregationResult != null) { | ||
cachedProtoBuilder.setPartialAggregationResults(partialAggregationResult); | ||
} | ||
cachedProto = cachedProtoBuilder.build(); | ||
} | ||
return cachedProto; | ||
} | ||
|
||
public static AggregateCursorContinuation fromRawBytes(@Nonnull byte[] rawBytes, RecordQueryStreamingAggregationPlan.SerializationMode serializationMode) { | ||
if (serializationMode == RecordQueryStreamingAggregationPlan.SerializationMode.TO_OLD) { | ||
return new AggregateCursorContinuation(ByteArrayContinuation.fromNullable(rawBytes), serializationMode); | ||
} | ||
try { | ||
RecordCursorProto.AggregateCursorContinuation continuationProto = RecordCursorProto.AggregateCursorContinuation.parseFrom(rawBytes); | ||
return new AggregateCursorContinuation(ByteArrayContinuation.fromNullable(continuationProto.getContinuation().toByteArray()), continuationProto.hasPartialAggregationResults() ? continuationProto.getPartialAggregationResults() : null, serializationMode); | ||
} catch (InvalidProtocolBufferException ipbe) { | ||
throw new RecordCoreException("error parsing continuation", ipbe) | ||
.addLogInfo("raw_bytes", ByteArrayUtil2.loggable(rawBytes)); | ||
} | ||
} | ||
} | ||
} |
Uh oh!
There was an error while loading. Please reload this page.