From 6c1ad10afc52a9b934acd5708b6c90d44d8d6c9a Mon Sep 17 00:00:00 2001 From: Pengpeng Lu Date: Mon, 24 Feb 2025 15:43:45 -0800 Subject: [PATCH 01/14] save --- fdb-relational-core/src/main/proto/continuation.proto | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/fdb-relational-core/src/main/proto/continuation.proto b/fdb-relational-core/src/main/proto/continuation.proto index 4b5dfae0d3..0e396dca22 100644 --- a/fdb-relational-core/src/main/proto/continuation.proto +++ b/fdb-relational-core/src/main/proto/continuation.proto @@ -48,6 +48,13 @@ message ContinuationProto { optional Reason reason = 6; } +message PartialAggregationResult { + optional bytes cursor = 1; + optional PValue groupingKey = 2; + optional int64 scanned_rows = 3; + optional PQueryResult partial_result = 4; // for example, current avg value +} + message TypedQueryArgument { optional com.apple.foundationdb.record.planprotos.PType type = 1; optional int32 literals_table_index = 2; // index of the corresponding entry in the literals table for the statement From 855ae272f06fa0c544417d25fcadc0dd48567ea9 Mon Sep 17 00:00:00 2001 From: Pengpeng Lu Date: Wed, 26 Feb 2025 11:56:58 -0800 Subject: [PATCH 02/14] sync --- .../cursors/aggregate/AggregateCursor.java | 98 ++++++++++++------- .../RecordQueryStreamingAggregationPlan.java | 2 +- .../query/FDBStreamAggregationTest.java | 75 ++++++++++++-- .../src/main/proto/continuation.proto | 1 - 4 files changed, 133 insertions(+), 43 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/AggregateCursor.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/AggregateCursor.java index ad000a659a..eef330b8bd 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/AggregateCursor.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/AggregateCursor.java @@ -22,6 +22,7 @@ import com.apple.foundationdb.annotation.API; import com.apple.foundationdb.async.AsyncUtil; +import com.apple.foundationdb.record.ByteArrayContinuation; import com.apple.foundationdb.record.RecordCursor; import com.apple.foundationdb.record.RecordCursorContinuation; import com.apple.foundationdb.record.RecordCursorResult; @@ -57,13 +58,20 @@ public class AggregateCursor implements RecordCursor previousValidResult; + // last row in last group, is null if the current group is the first group + @Nullable + private RecordCursorResult lastInLastGroup; + @Nullable + byte[] continuation; public AggregateCursor(@Nonnull RecordCursor inner, @Nonnull final StreamGrouping streamGrouping, - boolean isCreateDefaultOnEmpty) { + boolean isCreateDefaultOnEmpty, + @Nullable byte[] continuation) { this.inner = inner; this.streamGrouping = streamGrouping; this.isCreateDefaultOnEmpty = isCreateDefaultOnEmpty; + this.continuation = continuation; } @Nonnull @@ -71,63 +79,83 @@ public AggregateCursor(@Nonnull RecordCursor inner, public CompletableFuture> onNext() { if (previousResult != null && !previousResult.hasNext()) { // we are done - return CompletableFuture.completedFuture(RecordCursorResult.exhausted()); + return CompletableFuture.completedFuture(RecordCursorResult.withoutNextValue(previousResult.getContinuation(), + previousResult.getNoNextReason())); } return AsyncUtil.whileTrue(() -> inner.onNext().thenApply(innerResult -> { previousResult = innerResult; if (!innerResult.hasNext()) { if (!isNoRecords() || (isCreateDefaultOnEmpty && streamGrouping.isResultOnEmpty())) { + // the method streamGrouping.finalizeGroup() computes previousCompleteResult and resets the accumulator streamGrouping.finalizeGroup(); } return false; } else { final QueryResult queryResult = Objects.requireNonNull(innerResult.get()); boolean groupBreak = streamGrouping.apply(queryResult); - if (!groupBreak) { + if (groupBreak) { + lastInLastGroup = previousValidResult; + } else { // previousValidResult is the last row before group break, it sets the continuation previousValidResult = innerResult; } return (!groupBreak); } }), getExecutor()).thenApply(vignore -> { - if (isNoRecords()) { - // Edge case where there are no records at all - if (isCreateDefaultOnEmpty && streamGrouping.isResultOnEmpty()) { - return RecordCursorResult.withNextValue(QueryResult.ofComputed(streamGrouping.getCompletedGroupResult()), RecordCursorStartContinuation.START); + // either innerResult.hasNext() = false; or groupBreak = true + if (Verify.verifyNotNull(previousResult).hasNext()) { + // in this case groupBreak = true, return aggregated result and continuation + 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); + } else { + if (Verify.verifyNotNull(previousResult).getNoNextReason() == NoNextReason.SOURCE_EXHAUSTED) { + if (previousValidResult == null) { + return RecordCursorResult.exhausted(); + } else { + RecordCursorContinuation continuation = previousValidResult.getContinuation(); + previousValidResult = previousResult; + return RecordCursorResult.withNextValue(QueryResult.ofComputed(streamGrouping.getCompletedGroupResult()), continuation); + } } else { - return RecordCursorResult.exhausted(); + RecordCursorContinuation currentContinuation; + // in the current scan, if current group is the first group, set the continuation to the start of the current scan + // otherwise set the continuation to the last row in the last group + if (lastInLastGroup == null) { + currentContinuation = continuation == null ? RecordCursorStartContinuation.START : ByteArrayContinuation.fromNullable(continuation); + } else { + currentContinuation = lastInLastGroup.getContinuation(); + } + previousValidResult = lastInLastGroup; + return RecordCursorResult.withoutNextValue(currentContinuation, Verify.verifyNotNull(previousResult).getNoNextReason()); } } - // 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())); } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/plans/RecordQueryStreamingAggregationPlan.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/plans/RecordQueryStreamingAggregationPlan.java index 4342b4b65d..323be325ef 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/plans/RecordQueryStreamingAggregationPlan.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/plans/RecordQueryStreamingAggregationPlan.java @@ -148,7 +148,7 @@ public RecordCursor executePlan(@Nonnull FDBRec (FDBRecordStoreBase)store, context, inner.getAlias()); - return new AggregateCursor<>(innerCursor, streamGrouping, isCreateDefaultOnEmpty) + return new AggregateCursor<>(innerCursor, streamGrouping, isCreateDefaultOnEmpty, continuation) .skipThenLimit(executeProperties.getSkip(), executeProperties.getReturnedRowLimit()); } diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBStreamAggregationTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBStreamAggregationTest.java index 42e4105005..98fda6dba1 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBStreamAggregationTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBStreamAggregationTest.java @@ -20,11 +20,17 @@ package com.apple.foundationdb.record.provider.foundationdb.query; +import com.apple.foundationdb.record.ByteScanLimiterFactory; import com.apple.foundationdb.record.EvaluationContext; import com.apple.foundationdb.record.ExecuteProperties; +import com.apple.foundationdb.record.ExecuteState; import com.apple.foundationdb.record.RecordCursor; +import com.apple.foundationdb.record.RecordCursorContinuation; +import com.apple.foundationdb.record.RecordCursorEndContinuation; import com.apple.foundationdb.record.RecordCursorResult; +import com.apple.foundationdb.record.RecordCursorStartContinuation; import com.apple.foundationdb.record.RecordMetaData; +import com.apple.foundationdb.record.RecordScanLimiterFactory; import com.apple.foundationdb.record.TestRecords1Proto; import com.apple.foundationdb.record.provider.foundationdb.FDBRecordContext; import com.apple.foundationdb.record.query.plan.ScanComparisons; @@ -49,11 +55,13 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; import javax.annotation.Nonnull; +import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -305,6 +313,34 @@ void aggregateNoRecordsNoGroupNoAggregate(final boolean useNestedResult, final i } } + @Test + void aggregateHitScanLimitReached() { + try (final var context = openContext()) { + openSimpleRecordStore(context, NO_HOOK); + + final var plan = + new AggregationPlanBuilder(recordStore.getRecordMetaData(), "MySimpleRecord") + .withAggregateValue("num_value_2", value -> new NumericAggregationValue.Sum(NumericAggregationValue.PhysicalOperator.SUM_I, value)) + .withGroupCriterion("str_value_indexed") + .build(false); + + // In the testing data, there are 2 groups, each group has 3 rows. + // recordScanLimit = 5: scans 3 rows, and the 4th scan hits SCAN_LIMIT_REACHED + // although the first group contains exactly 3 rows, we don't know we've finished the first group before we get to the 4th row, so nothing is returned, continuation is back to START + RecordCursorContinuation continuation1 = executePlanWithRecordScanLimit(plan, 5, null, null); + Assertions.assertEquals(RecordCursorStartContinuation.START, continuation1); + // recordScanLimit = 6: scans 4 rows, and the 5th scan hits SCAN_LIMIT_REACHED, we know that we've finished the 1st group, aggregated result is returned + RecordCursorContinuation continuation2 = executePlanWithRecordScanLimit(plan, 6, continuation1.toBytes(), resultOf("0", 3)); + // continue with recordScanLimit = 5, scans 3 rows and hits SCAN_LIMIT_REACHED + // again, we don't know that we've finished the 2nd group, nothing is returned, continuation is back to where the scan starts + RecordCursorContinuation continuation3 = executePlanWithRecordScanLimit(plan, 5, continuation2.toBytes(), null); + Assertions.assertArrayEquals(continuation2.toBytes(), continuation3.toBytes()); + // finish the 2nd group, aggregated result is returned, exhausted the source + RecordCursorContinuation continuation4 = executePlanWithRecordScanLimit(plan, 6, continuation3.toBytes(), resultOf("1", 12)); + Assertions.assertEquals(RecordCursorEndContinuation.END, continuation4); + } + } + private static Stream provideArguments() { // (boolean, rowLimit) // setting rowLimit = 0 is equivalent to no limit @@ -334,11 +370,17 @@ private void populateDB(final int numRecords) throws Exception { } @Nonnull - private RecordCursor executePlan(final RecordQueryPlan plan, final int rowLimit, final byte[] continuation) { + private RecordCursor executePlan(final RecordQueryPlan plan, final int rowLimit, final int recordScanLimit, final byte[] continuation) { final var types = plan.getDynamicTypes(); final var typeRepository = TypeRepository.newBuilder().addAllTypes(types).build(); + ExecuteState executeState; + if (recordScanLimit > 0) { + executeState = new ExecuteState(RecordScanLimiterFactory.enforce(recordScanLimit), ByteScanLimiterFactory.tracking()); + } else { + executeState = ExecuteState.NO_LIMITS; + } ExecuteProperties executeProperties = ExecuteProperties.SERIAL_EXECUTE; - executeProperties = executeProperties.setReturnedRowLimit(rowLimit); + executeProperties = executeProperties.setReturnedRowLimit(rowLimit).setState(executeState); try { return plan.executePlan(recordStore, EvaluationContext.forTypeRepository(typeRepository), continuation, executeProperties); } catch (final Throwable t) { @@ -346,11 +388,32 @@ private RecordCursor executePlan(final RecordQueryPlan plan, final } } + private RecordCursorContinuation executePlanWithRecordScanLimit(final RecordQueryPlan plan, final int recordScanLimit, byte[] continuation, @Nullable List expectedResult) { + List queryResults = new LinkedList<>(); + RecordCursor currentCursor = executePlan(plan, 0, recordScanLimit, continuation); + RecordCursorResult currentCursorResult; + RecordCursorContinuation cursorContinuation; + while (true) { + currentCursorResult = currentCursor.getNext(); + cursorContinuation = currentCursorResult.getContinuation(); + if (!currentCursorResult.hasNext()) { + break; + } + queryResults.add(currentCursorResult.get()); + } + if (expectedResult == null) { + Assertions.assertTrue(queryResults.isEmpty()); + } else { + assertResults(this::assertResultFlattened, queryResults, expectedResult); + } + return cursorContinuation; + } + private List executePlanWithRowLimit(final RecordQueryPlan plan, final int rowLimit) { byte[] continuation = null; List queryResults = new LinkedList<>(); while (true) { - RecordCursor currentCursor = executePlan(plan, rowLimit, continuation); + RecordCursor currentCursor = executePlan(plan, rowLimit, 0, continuation); RecordCursorResult currentCursorResult; while (true) { currentCursorResult = currentCursor.getNext(); @@ -369,7 +432,7 @@ private List executePlanWithRowLimit(final RecordQueryPlan plan, fi private void assertResults(@Nonnull final BiConsumer> checkConsumer, @Nonnull final List actual, @Nonnull final List... expected) { Assertions.assertEquals(expected.length, actual.size()); - for (var i = 0 ; i < actual.size() ; i++) { + for (var i = 0; i < actual.size(); i++) { checkConsumer.accept(actual.get(i), expected[i]); } } @@ -385,7 +448,7 @@ private void assertResultFlattened(final QueryResult actual, final List expec final var resultFields = resultFieldsBuilder.build(); Assertions.assertEquals(resultFields.size(), expected.size()); - for (var i = 0 ; i < resultFields.size() ; i++) { + for (var i = 0; i < resultFields.size(); i++) { Assertions.assertEquals(expected.get(i), resultFields.get(i)); } } @@ -416,7 +479,7 @@ private void assertResultNested(final QueryResult actual, final List expected final var resultFields = resultFieldsBuilder.build(); Assertions.assertEquals(resultFields.size(), expected.size()); - for (var i = 0 ; i < resultFields.size() ; i++) { + for (var i = 0; i < resultFields.size(); i++) { Assertions.assertEquals(expected.get(i), resultFields.get(i)); } } diff --git a/fdb-relational-core/src/main/proto/continuation.proto b/fdb-relational-core/src/main/proto/continuation.proto index 0e396dca22..a6fae9d9c6 100644 --- a/fdb-relational-core/src/main/proto/continuation.proto +++ b/fdb-relational-core/src/main/proto/continuation.proto @@ -49,7 +49,6 @@ message ContinuationProto { } message PartialAggregationResult { - optional bytes cursor = 1; optional PValue groupingKey = 2; optional int64 scanned_rows = 3; optional PQueryResult partial_result = 4; // for example, current avg value From 5c2da3f8c8effcaa8691f94a4ba80f67d9f6f431 Mon Sep 17 00:00:00 2001 From: Pengpeng Lu Date: Wed, 26 Feb 2025 15:01:30 -0800 Subject: [PATCH 03/14] save --- .../foundationdb/record/EvaluationContext.java | 14 ++++++++++++++ .../record/RecordCursorContinuation.java | 1 + .../cursors/aggregate/AggregateCursor.java | 10 +++++++++- .../record/cursors/aggregate/StreamGrouping.java | 6 ++++++ .../query/plan/cascades/values/Accumulator.java | 4 ++++ .../query/plan/cascades/values/CountValue.java | 13 +++++++++++++ .../cascades/values/NumericAggregationValue.java | 16 +++++++++++++++- .../cascades/values/RecordConstructorValue.java | 7 +++++++ .../src/main/proto/record_query_plan.proto | 5 +++++ .../relational/recordlayer/ContinuationImpl.java | 15 +++++++++++++++ .../relational/recordlayer/query/QueryPlan.java | 3 ++- .../src/main/proto/continuation.proto | 7 +------ 12 files changed, 92 insertions(+), 9 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/EvaluationContext.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/EvaluationContext.java index 71ef331cdc..bef638ceba 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/EvaluationContext.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/EvaluationContext.java @@ -24,6 +24,7 @@ import com.apple.foundationdb.record.logging.LogMessageKeys; import com.apple.foundationdb.record.query.plan.cascades.CorrelationIdentifier; import com.apple.foundationdb.record.query.plan.cascades.typing.TypeRepository; +import com.apple.foundationdb.record.planprotos.PartialAggregationResult; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -45,6 +46,9 @@ public class EvaluationContext { @Nonnull private final TypeRepository typeRepository; + @Nullable + private final PartialAggregationResult partialAggregationResultProto; + public static final EvaluationContext EMPTY = new EvaluationContext(Bindings.EMPTY_BINDINGS, TypeRepository.EMPTY_SCHEMA); /** @@ -57,8 +61,13 @@ public static EvaluationContext empty() { } private EvaluationContext(@Nonnull Bindings bindings, @Nonnull TypeRepository typeRepository) { + this(bindings, typeRepository, null); + } + + private EvaluationContext(@Nonnull Bindings bindings, @Nonnull TypeRepository typeRepository, @Nullable PartialAggregationResult proto) { this.bindings = bindings; this.typeRepository = typeRepository; + this.partialAggregationResultProto = proto; } /** @@ -89,6 +98,11 @@ public static EvaluationContext forTypeRepository(@Nonnull TypeRepository typeRe return new EvaluationContext(Bindings.EMPTY_BINDINGS, typeRepository); } + @Nonnull + public static EvaluationContext forBindingsAndTypeRepositoryAndPartialAggregationResult(@Nonnull Bindings bindings, @Nonnull TypeRepository typeRepository, @Nullable PartialAggregationResult proto) { + return new EvaluationContext(bindings, typeRepository, proto); + } + /** * Create a new EvaluationContext with a single binding. * diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/RecordCursorContinuation.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/RecordCursorContinuation.java index 5f10476d36..9f2f666283 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/RecordCursorContinuation.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/RecordCursorContinuation.java @@ -21,6 +21,7 @@ package com.apple.foundationdb.record; import com.apple.foundationdb.annotation.API; +import com.apple.foundationdb.record.planprotos.PartialAggregationResult; import com.google.protobuf.ByteString; import javax.annotation.Nonnull; diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/AggregateCursor.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/AggregateCursor.java index eef330b8bd..9bce7d3c8e 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/AggregateCursor.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/AggregateCursor.java @@ -28,6 +28,7 @@ import com.apple.foundationdb.record.RecordCursorResult; import com.apple.foundationdb.record.RecordCursorStartContinuation; import com.apple.foundationdb.record.RecordCursorVisitor; +import com.apple.foundationdb.record.cursors.RecursiveUnionCursor; import com.apple.foundationdb.record.query.plan.plans.QueryResult; import com.google.common.base.Verify; import com.google.protobuf.Message; @@ -105,7 +106,7 @@ public CompletableFuture> onNext() { }), getExecutor()).thenApply(vignore -> { // either innerResult.hasNext() = false; or groupBreak = true if (Verify.verifyNotNull(previousResult).hasNext()) { - // in this case groupBreak = true, return aggregated result and continuation + // in this case groupBreak = true, return aggregated result and continuation, partialAggregationResult = null 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. @@ -132,7 +133,9 @@ public CompletableFuture> onNext() { previousValidResult = previousResult; return RecordCursorResult.withNextValue(QueryResult.ofComputed(streamGrouping.getCompletedGroupResult()), continuation); } else { + // innerResult.hasNext() = false, might stop in the middle of a group if (Verify.verifyNotNull(previousResult).getNoNextReason() == NoNextReason.SOURCE_EXHAUSTED) { + // exhausted if (previousValidResult == null) { return RecordCursorResult.exhausted(); } else { @@ -141,6 +144,9 @@ public CompletableFuture> onNext() { return RecordCursorResult.withNextValue(QueryResult.ofComputed(streamGrouping.getCompletedGroupResult()), continuation); } } else { + // stopped in the middle of a group + ContinuationImpl. + /* RecordCursorContinuation currentContinuation; // in the current scan, if current group is the first group, set the continuation to the start of the current scan // otherwise set the continuation to the last row in the last group @@ -149,6 +155,8 @@ public CompletableFuture> onNext() { } else { currentContinuation = lastInLastGroup.getContinuation(); } + + */ previousValidResult = lastInLastGroup; return RecordCursorResult.withoutNextValue(currentContinuation, Verify.verifyNotNull(previousResult).getNoNextReason()); } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/StreamGrouping.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/StreamGrouping.java index 721a920657..8d5388b3a4 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/StreamGrouping.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/StreamGrouping.java @@ -23,6 +23,7 @@ import com.apple.foundationdb.record.Bindings; import com.apple.foundationdb.record.EvaluationContext; import com.apple.foundationdb.record.RecordCursorResult; +import com.apple.foundationdb.record.planprotos.PartialAggregationResult; import com.apple.foundationdb.record.provider.foundationdb.FDBRecordStoreBase; import com.apple.foundationdb.record.query.plan.cascades.CorrelationIdentifier; import com.apple.foundationdb.record.query.plan.cascades.values.Accumulator; @@ -197,4 +198,9 @@ private Object evalGroupingKey(@Nullable final Object currentObject) { public boolean isResultOnEmpty() { return groupingKeyValue == null; } + + @Nullable + public PartialAggregationResult getPartialAggregationResult() { + return accumulator.getPartialAggregationResult(); + } } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/Accumulator.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/Accumulator.java index e75e6d9e69..1ecb7a2da1 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/Accumulator.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/Accumulator.java @@ -20,6 +20,8 @@ package com.apple.foundationdb.record.query.plan.cascades.values; +import com.apple.foundationdb.record.planprotos.PartialAggregationResult; + import javax.annotation.Nullable; /** @@ -29,4 +31,6 @@ public interface Accumulator { void accumulate(@Nullable Object currentObject); @Nullable Object finish(); + + @Nullable PartialAggregationResult getPartialAggregationResult(); } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/CountValue.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/CountValue.java index 4adc95d892..6b2567668b 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/CountValue.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/CountValue.java @@ -32,6 +32,7 @@ import com.apple.foundationdb.record.planprotos.PCountValue; import com.apple.foundationdb.record.planprotos.PCountValue.PPhysicalOperator; import com.apple.foundationdb.record.planprotos.PValue; +import com.apple.foundationdb.record.planprotos.PartialAggregationResult; import com.apple.foundationdb.record.provider.foundationdb.FDBRecordStoreBase; import com.apple.foundationdb.record.query.plan.cascades.AliasMap; import com.apple.foundationdb.record.query.plan.cascades.BuiltInFunction; @@ -345,6 +346,18 @@ public void accumulate(@Nullable final Object currentObject) { public Object finish() { return physicalOperator.evalPartialToFinal(state); } + + @Nullable + @Override + public PartialAggregationResult getPartialAggregationResult() { + if (state == null) { + return null; + } + return PartialAggregationResult.newBuilder() + .setPhysicalOperatorName(physicalOperator.name()) + // (TODO) store state, maybe a string? + .build(); + } } /** diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/NumericAggregationValue.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/NumericAggregationValue.java index 9c8a0073fc..f1f2f4dfaa 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/NumericAggregationValue.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/NumericAggregationValue.java @@ -37,6 +37,7 @@ import com.apple.foundationdb.record.planprotos.PNumericAggregationValue.PPhysicalOperator; import com.apple.foundationdb.record.planprotos.PNumericAggregationValue.PSum; import com.apple.foundationdb.record.planprotos.PValue; +import com.apple.foundationdb.record.planprotos.PartialAggregationResult; import com.apple.foundationdb.record.provider.foundationdb.FDBRecordStoreBase; import com.apple.foundationdb.record.provider.foundationdb.indexes.BitmapValueIndexMaintainer; import com.apple.foundationdb.record.query.plan.cascades.AliasMap; @@ -58,6 +59,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; +import com.google.protobuf.Any; import com.google.protobuf.Message; import javax.annotation.Nonnull; @@ -862,5 +864,17 @@ public void accumulate(@Nullable final Object currentObject) { public Object finish() { return physicalOperator.evalPartialToFinal(state); } - } + + @Nullable + @Override + public PartialAggregationResult getPartialAggregationResult() { + if (state == null) { + return null; + } + return PartialAggregationResult.newBuilder() + .setPhysicalOperatorName(physicalOperator.name()) + // (TODO) store state, maybe a string? + .build(); + } + } } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/RecordConstructorValue.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/RecordConstructorValue.java index ea0cafaa0e..50be4a91cc 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/RecordConstructorValue.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/RecordConstructorValue.java @@ -29,6 +29,7 @@ import com.apple.foundationdb.record.PlanSerializationContext; import com.apple.foundationdb.record.planprotos.PRecordConstructorValue; import com.apple.foundationdb.record.planprotos.PValue; +import com.apple.foundationdb.record.planprotos.PartialAggregationResult; import com.apple.foundationdb.record.provider.foundationdb.FDBRecordStoreBase; import com.apple.foundationdb.record.provider.foundationdb.FDBRecordVersion; import com.apple.foundationdb.record.query.plan.cascades.AliasMap; @@ -374,6 +375,12 @@ private List buildAccumulators() { } return childAccumulatorsBuilder.build(); } + + @Nullable + @Override + public PartialAggregationResult getPartialAggregationResult() { + return null; + } }; } diff --git a/fdb-record-layer-core/src/main/proto/record_query_plan.proto b/fdb-record-layer-core/src/main/proto/record_query_plan.proto index 4e93153c64..a8323102f4 100644 --- a/fdb-record-layer-core/src/main/proto/record_query_plan.proto +++ b/fdb-record-layer-core/src/main/proto/record_query_plan.proto @@ -1781,4 +1781,9 @@ message PRecursiveUnionQueryPlan { optional PPhysicalQuantifier recursiveStateQuantifier = 2; optional string initialTempTableAlias = 3; optional string recursiveTempTableAlias = 4; +} + +message PartialAggregationResult { + optional string physical_operator_name = 1; + optional google.protobuf.Any state = 2; } \ No newline at end of file diff --git a/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/ContinuationImpl.java b/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/ContinuationImpl.java index 7503d25d0b..c0bc379969 100644 --- a/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/ContinuationImpl.java +++ b/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/ContinuationImpl.java @@ -23,6 +23,7 @@ import com.apple.foundationdb.annotation.API; import com.apple.foundationdb.record.RecordCursorContinuation; +import com.apple.foundationdb.record.planprotos.PartialAggregationResult; import com.apple.foundationdb.relational.api.Continuation; import com.apple.foundationdb.relational.api.exceptions.ErrorCode; import com.apple.foundationdb.relational.api.exceptions.RelationalException; @@ -81,6 +82,10 @@ public byte[] getExecutionState() { } } + public PartialAggregationResult getPartialAggregationResult() { + return proto.getPartialAggregationResult(); + } + @Override public Reason getReason() { if (proto.hasReason()) { @@ -176,6 +181,16 @@ public static Continuation fromRecordCursorContinuation(RecordCursorContinuation return cursorContinuation.isEnd() ? END : new ContinuationImpl(cursorContinuation.toBytes()); } + public static Continuation fromRecordCursorContinuationAndPartialAggregationResult(@Nonnull RecordCursorContinuation cursorContinuation, @Nullable PartialAggregationResult partialAggregationResultProto) { + if (cursorContinuation.isEnd()) { + return END; + } + ContinuationProto.Builder builder = ContinuationProto.newBuilder().setVersion(CURRENT_VERSION); + builder.setExecutionState(ByteString.copyFrom(Objects.requireNonNull(cursorContinuation.toBytes()))) + .setPartialAggregationResult(partialAggregationResultProto); + return new ContinuationImpl(builder.build()); + } + /** * Deserialize and parse a continuation. This would create a continuation from a previously serialized byte array. * @param bytes the serialized continuation diff --git a/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/QueryPlan.java b/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/QueryPlan.java index d196162add..f00f4c096f 100644 --- a/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/QueryPlan.java +++ b/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/QueryPlan.java @@ -238,7 +238,6 @@ public RelationalResultSet executeInternal(@Nonnull final ExecutionContext execu final String schemaName = conn.getSchema(); try (RecordLayerSchema recordLayerSchema = conn.getRecordLayerDatabase().loadSchema(schemaName)) { final var evaluationContext = queryExecutionParameters.getEvaluationContext(); - final var typedEvaluationContext = EvaluationContext.forBindingsAndTypeRepository(evaluationContext.getBindings(), typeRepository); final ContinuationImpl parsedContinuation; try { parsedContinuation = ContinuationImpl.parseContinuation(queryExecutionParameters.getContinuation()); @@ -246,6 +245,8 @@ public RelationalResultSet executeInternal(@Nonnull final ExecutionContext execu executionContext.metricCollector.increment(RelationalMetric.RelationalCount.CONTINUATION_REJECTED); throw ExceptionUtil.toRelationalException(ipbe); } + final var typedEvaluationContext = EvaluationContext.forBindingsAndTypeRepositoryAndPartialAggregationResult(evaluationContext.getBindings(), typeRepository, parsedContinuation.getPartialAggregationResult()); + if (queryExecutionParameters.isForExplain()) { return executeExplain(parsedContinuation); } else { diff --git a/fdb-relational-core/src/main/proto/continuation.proto b/fdb-relational-core/src/main/proto/continuation.proto index a6fae9d9c6..17fdc48004 100644 --- a/fdb-relational-core/src/main/proto/continuation.proto +++ b/fdb-relational-core/src/main/proto/continuation.proto @@ -46,12 +46,7 @@ message ContinuationProto { // The serialized package optional CompiledStatement compiled_statement = 5; optional Reason reason = 6; -} - -message PartialAggregationResult { - optional PValue groupingKey = 2; - optional int64 scanned_rows = 3; - optional PQueryResult partial_result = 4; // for example, current avg value + optional com.apple.foundationdb.record.planprotos.PartialAggregationResult partial_aggregation_result = 7; } message TypedQueryArgument { From c62bffaed2b62a312f8c08fb3f2a43021d2fd91f Mon Sep 17 00:00:00 2001 From: Pengpeng Lu Date: Thu, 27 Feb 2025 10:10:07 -0800 Subject: [PATCH 04/14] store null state --- .../foundationdb/record/EvaluationContext.java | 5 +++++ .../record/RecordCursorContinuation.java | 5 +++++ .../cursors/aggregate/AggregateCursor.java | 7 +++++-- .../cursors/aggregate/StreamGrouping.java | 1 + .../foundationdb/KeyValueCursorBase.java | 18 +++++++++++++++++- .../plan/cascades/values/Accumulator.java | 2 ++ .../query/plan/cascades/values/CountValue.java | 9 +++++++++ .../values/NumericAggregationValue.java | 9 +++++++++ .../values/RecordConstructorValue.java | 4 ++++ .../src/main/proto/continuation.proto | 0 10 files changed, 57 insertions(+), 3 deletions(-) rename {fdb-relational-core => fdb-record-layer-core}/src/main/proto/continuation.proto (100%) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/EvaluationContext.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/EvaluationContext.java index bef638ceba..e83978ad7a 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/EvaluationContext.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/EvaluationContext.java @@ -115,6 +115,11 @@ public static EvaluationContext forBinding(@Nonnull String bindingName, @Nullabl return new EvaluationContext(Bindings.newBuilder().set(bindingName, value).build(), TypeRepository.EMPTY_SCHEMA); } + @Nullable + public PartialAggregationResult getPartialAggregationResult() { + return partialAggregationResultProto; + } + /** * Retrieve the mapping from parameter names to values associated with * this context. diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/RecordCursorContinuation.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/RecordCursorContinuation.java index 9f2f666283..45158451ec 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/RecordCursorContinuation.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/RecordCursorContinuation.java @@ -74,6 +74,11 @@ default ByteString toByteString() { return bytes == null ? ByteString.EMPTY : ByteString.copyFrom(bytes); } + @Nullable + default PartialAggregationResult getPartialAggregationResult() { + return null; + } + /** * Return whether this continuation is an "end continuation", i.e., represents that the iteration has reached * its end and would not produce more results even if restarted. If {@code isEnd()} returns {@code true}, then diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/AggregateCursor.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/AggregateCursor.java index 9bce7d3c8e..d711ae1def 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/AggregateCursor.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/AggregateCursor.java @@ -29,8 +29,11 @@ import com.apple.foundationdb.record.RecordCursorStartContinuation; import com.apple.foundationdb.record.RecordCursorVisitor; import com.apple.foundationdb.record.cursors.RecursiveUnionCursor; +import com.apple.foundationdb.record.provider.foundationdb.KeyValueCursorBase; import com.apple.foundationdb.record.query.plan.plans.QueryResult; +import com.apple.foundationdb.relational.continuation.ContinuationProto; import com.google.common.base.Verify; +import com.google.protobuf.ByteString; import com.google.protobuf.Message; import javax.annotation.Nonnull; @@ -145,7 +148,6 @@ public CompletableFuture> onNext() { } } else { // stopped in the middle of a group - ContinuationImpl. /* RecordCursorContinuation currentContinuation; // in the current scan, if current group is the first group, set the continuation to the start of the current scan @@ -157,7 +159,8 @@ public CompletableFuture> onNext() { } */ - previousValidResult = lastInLastGroup; + RecordCursorContinuation currentContinuation = ((KeyValueCursorBase.Continuation) previousValidResult.getContinuation()).withPartialAggregationResult(streamGrouping.getPartialAggregationResult()); + previousValidResult = previousResult; return RecordCursorResult.withoutNextValue(currentContinuation, Verify.verifyNotNull(previousResult).getNoNextReason()); } } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/StreamGrouping.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/StreamGrouping.java index 8d5388b3a4..59d07cb6d2 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/StreamGrouping.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/StreamGrouping.java @@ -109,6 +109,7 @@ public StreamGrouping(@Nullable final Value groupingKeyValue, this.groupingKeyValue = groupingKeyValue; this.aggregateValue = aggregateValue; this.accumulator = aggregateValue.createAccumulator(context.getTypeRepository()); + this.accumulator.setInitialState(context.getPartialAggregationResult()); this.store = store; this.context = context; this.alias = alias; diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/KeyValueCursorBase.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/KeyValueCursorBase.java index ec5f7b9d17..bda2c016d3 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/KeyValueCursorBase.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/KeyValueCursorBase.java @@ -39,6 +39,7 @@ import com.apple.foundationdb.record.cursors.AsyncIteratorCursor; import com.apple.foundationdb.record.cursors.BaseCursor; import com.apple.foundationdb.record.cursors.CursorLimitManager; +import com.apple.foundationdb.record.planprotos.PartialAggregationResult; import com.apple.foundationdb.subspace.Subspace; import com.apple.foundationdb.tuple.Tuple; import com.google.protobuf.ByteString; @@ -134,18 +135,29 @@ private RecordCursorContinuation continuationHelper() { return new Continuation(lastKey, prefixLength); } - private static class Continuation implements RecordCursorContinuation { + public static class Continuation implements RecordCursorContinuation { @Nullable private final byte[] lastKey; private final int prefixLength; + @Nullable + private final PartialAggregationResult partialAggregationResult; public Continuation(@Nullable final byte[] lastKey, final int prefixLength) { + // Note that doing this without a full copy is dangerous if the array is ever mutated. + // Currently, this never happens and the only thing that changes is which array lastKey points to. + // However, if logic in KeyValueCursor or KeyValue changes, this could break continuations. + // To resolve it, we could resort to doing a full copy here, although that's somewhat expensive. + this(lastKey, prefixLength, null); + } + + private Continuation(@Nullable final byte[] lastKey, final int prefixLength, @Nullable PartialAggregationResult partialAggregationResult) { // Note that doing this without a full copy is dangerous if the array is ever mutated. // Currently, this never happens and the only thing that changes is which array lastKey points to. // However, if logic in KeyValueCursor or KeyValue changes, this could break continuations. // To resolve it, we could resort to doing a full copy here, although that's somewhat expensive. this.lastKey = lastKey; this.prefixLength = prefixLength; + this.partialAggregationResult = partialAggregationResult; } @Override @@ -171,6 +183,10 @@ public byte[] toBytes() { } return Arrays.copyOfRange(lastKey, prefixLength, lastKey.length); } + + public Continuation withPartialAggregationResult(@Nullable PartialAggregationResult partialAggregationResult) { + return new Continuation(lastKey, prefixLength, partialAggregationResult); + } } /** diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/Accumulator.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/Accumulator.java index 1ecb7a2da1..21a57eb742 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/Accumulator.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/Accumulator.java @@ -33,4 +33,6 @@ public interface Accumulator { @Nullable Object finish(); @Nullable PartialAggregationResult getPartialAggregationResult(); + + void setInitialState(@Nullable PartialAggregationResult partialAggregationResult); } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/CountValue.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/CountValue.java index 6b2567668b..b80e0d4fa5 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/CountValue.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/CountValue.java @@ -358,6 +358,15 @@ public PartialAggregationResult getPartialAggregationResult() { // (TODO) store state, maybe a string? .build(); } + + @Override + public void setInitialState(@Nullable PartialAggregationResult partialAggregationResult) { + if (partialAggregationResult != null) { + // check physical operator name are the same + // check this.state == null + this.state = partialAggregationResult.getState(); + } + } } /** diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/NumericAggregationValue.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/NumericAggregationValue.java index f1f2f4dfaa..4f15278028 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/NumericAggregationValue.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/NumericAggregationValue.java @@ -876,5 +876,14 @@ public PartialAggregationResult getPartialAggregationResult() { // (TODO) store state, maybe a string? .build(); } + + @Override + public void setInitialState(@Nullable PartialAggregationResult partialAggregationResult) { + if (partialAggregationResult != null) { + // check physical operator name are the same + // check this.state == null + this.state = partialAggregationResult.getState(); + } + } } } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/RecordConstructorValue.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/RecordConstructorValue.java index 50be4a91cc..74e5f82323 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/RecordConstructorValue.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/RecordConstructorValue.java @@ -381,6 +381,10 @@ private List buildAccumulators() { public PartialAggregationResult getPartialAggregationResult() { return null; } + + @Override + public void setInitialState(@Nullable PartialAggregationResult partialAggregationResult) { + } }; } diff --git a/fdb-relational-core/src/main/proto/continuation.proto b/fdb-record-layer-core/src/main/proto/continuation.proto similarity index 100% rename from fdb-relational-core/src/main/proto/continuation.proto rename to fdb-record-layer-core/src/main/proto/continuation.proto From 06d8e65a8753cf402685514b185cbd072cbe3d98 Mon Sep 17 00:00:00 2001 From: Pengpeng Lu Date: Mon, 3 Mar 2025 15:42:18 -0800 Subject: [PATCH 05/14] work except exhausted --- .../record/EvaluationContext.java | 1 + .../cursors/aggregate/AggregateCursor.java | 20 +++-- .../cursors/aggregate/StreamGrouping.java | 33 +++++-- .../foundationdb/KeyValueCursorBase.java | 7 +- .../plan/cascades/values/Accumulator.java | 11 ++- .../plan/cascades/values/CountValue.java | 20 +++-- .../values/NumericAggregationValue.java | 86 +++++++++++++++++-- .../values/RecordConstructorValue.java | 26 +++++- .../src/main/proto/continuation.proto | 2 +- .../src/main/proto/record_query_plan.proto | 10 ++- .../query/FDBStreamAggregationTest.java | 33 ++++--- 11 files changed, 198 insertions(+), 51 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/EvaluationContext.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/EvaluationContext.java index e83978ad7a..4c9c385868 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/EvaluationContext.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/EvaluationContext.java @@ -22,6 +22,7 @@ import com.apple.foundationdb.annotation.API; import com.apple.foundationdb.record.logging.LogMessageKeys; +import com.apple.foundationdb.record.planprotos.PValue; import com.apple.foundationdb.record.query.plan.cascades.CorrelationIdentifier; import com.apple.foundationdb.record.query.plan.cascades.typing.TypeRepository; import com.apple.foundationdb.record.planprotos.PartialAggregationResult; diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/AggregateCursor.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/AggregateCursor.java index d711ae1def..2777778da7 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/AggregateCursor.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/AggregateCursor.java @@ -26,14 +26,11 @@ import com.apple.foundationdb.record.RecordCursor; import com.apple.foundationdb.record.RecordCursorContinuation; import com.apple.foundationdb.record.RecordCursorResult; -import com.apple.foundationdb.record.RecordCursorStartContinuation; import com.apple.foundationdb.record.RecordCursorVisitor; -import com.apple.foundationdb.record.cursors.RecursiveUnionCursor; +import com.apple.foundationdb.record.planprotos.PartialAggregationResult; import com.apple.foundationdb.record.provider.foundationdb.KeyValueCursorBase; import com.apple.foundationdb.record.query.plan.plans.QueryResult; -import com.apple.foundationdb.relational.continuation.ContinuationProto; import com.google.common.base.Verify; -import com.google.protobuf.ByteString; import com.google.protobuf.Message; import javax.annotation.Nonnull; @@ -66,6 +63,8 @@ public class AggregateCursor implements RecordCursor lastInLastGroup; @Nullable + private PartialAggregationResult partialAggregationResult; + @Nullable byte[] continuation; public AggregateCursor(@Nonnull RecordCursor inner, @@ -89,14 +88,17 @@ public CompletableFuture> onNext() { return AsyncUtil.whileTrue(() -> inner.onNext().thenApply(innerResult -> { previousResult = innerResult; + System.out.println("scan innerResult hasNext:" + innerResult.hasNext()); if (!innerResult.hasNext()) { + System.out.println("cursor noNextReason:" + innerResult.getNoNextReason()); if (!isNoRecords() || (isCreateDefaultOnEmpty && streamGrouping.isResultOnEmpty())) { // the method streamGrouping.finalizeGroup() computes previousCompleteResult and resets the accumulator - streamGrouping.finalizeGroup(); + partialAggregationResult = streamGrouping.finalizeGroup(); } return false; } else { final QueryResult queryResult = Objects.requireNonNull(innerResult.get()); + System.out.println("inner result:" + innerResult.get().getMessage()); boolean groupBreak = streamGrouping.apply(queryResult); if (groupBreak) { lastInLastGroup = previousValidResult; @@ -110,7 +112,9 @@ public CompletableFuture> onNext() { // either innerResult.hasNext() = false; or groupBreak = true if (Verify.verifyNotNull(previousResult).hasNext()) { // in this case groupBreak = true, return aggregated result and continuation, partialAggregationResult = null - RecordCursorContinuation continuation = Verify.verifyNotNull(previousValidResult).getContinuation(); + // previousValidResult = null happens when 1st row of current scan != last row of last scan, results in groupBreak = true and previousValidResult = null + RecordCursorContinuation c = previousValidResult == null ? ByteArrayContinuation.fromNullable(continuation) : 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: @@ -134,7 +138,7 @@ public CompletableFuture> onNext() { * 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); + return RecordCursorResult.withNextValue(QueryResult.ofComputed(streamGrouping.getCompletedGroupResult()), c); } else { // innerResult.hasNext() = false, might stop in the middle of a group if (Verify.verifyNotNull(previousResult).getNoNextReason() == NoNextReason.SOURCE_EXHAUSTED) { @@ -159,7 +163,7 @@ public CompletableFuture> onNext() { } */ - RecordCursorContinuation currentContinuation = ((KeyValueCursorBase.Continuation) previousValidResult.getContinuation()).withPartialAggregationResult(streamGrouping.getPartialAggregationResult()); + RecordCursorContinuation currentContinuation = ((KeyValueCursorBase.Continuation) previousValidResult.getContinuation()).withPartialAggregationResult(partialAggregationResult); previousValidResult = previousResult; return RecordCursorResult.withoutNextValue(currentContinuation, Verify.verifyNotNull(previousResult).getNoNextReason()); } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/StreamGrouping.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/StreamGrouping.java index 59d07cb6d2..58be9d3122 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/StreamGrouping.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/StreamGrouping.java @@ -22,6 +22,8 @@ import com.apple.foundationdb.record.Bindings; import com.apple.foundationdb.record.EvaluationContext; +import com.apple.foundationdb.record.PlanHashable; +import com.apple.foundationdb.record.PlanSerializationContext; import com.apple.foundationdb.record.RecordCursorResult; import com.apple.foundationdb.record.planprotos.PartialAggregationResult; import com.apple.foundationdb.record.provider.foundationdb.FDBRecordStoreBase; @@ -29,6 +31,9 @@ import com.apple.foundationdb.record.query.plan.cascades.values.Accumulator; import com.apple.foundationdb.record.query.plan.cascades.values.AggregateValue; import com.apple.foundationdb.record.query.plan.cascades.values.Value; +import com.apple.foundationdb.record.query.plan.serialization.DefaultPlanSerializationRegistry; +import com.google.protobuf.DynamicMessage; +import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.Message; import javax.annotation.Nonnull; @@ -86,6 +91,10 @@ public class StreamGrouping { @Nonnull private final Value completeResultValue; + @Nonnull + private static PlanSerializationContext serializationContext = new PlanSerializationContext(DefaultPlanSerializationRegistry.INSTANCE, + PlanHashable.CURRENT_FOR_CONTINUATION); + /** * Create a new group aggregator. * @@ -109,7 +118,9 @@ public StreamGrouping(@Nullable final Value groupingKeyValue, this.groupingKeyValue = groupingKeyValue; this.aggregateValue = aggregateValue; this.accumulator = aggregateValue.createAccumulator(context.getTypeRepository()); - this.accumulator.setInitialState(context.getPartialAggregationResult()); + if (context.getPartialAggregationResult() != null) { + this.accumulator.setInitialState(context.getPartialAggregationResult().getAccumulatorStatesList()); + } this.store = store; this.context = context; this.alias = alias; @@ -163,26 +174,36 @@ public Object getCompletedGroupResult() { private boolean isGroupBreak(final Object currentGroup, final Object nextGroup) { if (currentGroup == null) { + if (context.getPartialAggregationResult() != null) { + try { + this.currentGroup = DynamicMessage.parseFrom(((Message) nextGroup).getDescriptorForType(), context.getPartialAggregationResult().getGroupKey()); + return (!this.currentGroup.equals(nextGroup)); + } catch (InvalidProtocolBufferException e) { + throw new RuntimeException(e); + } + } return false; } else { return (!currentGroup.equals(nextGroup)); } } - public void finalizeGroup() { - finalizeGroup(null); + public PartialAggregationResult finalizeGroup() { + return finalizeGroup(null); } - private void finalizeGroup(Object nextGroup) { + private PartialAggregationResult finalizeGroup(Object nextGroup) { final EvaluationContext nestedContext = context.childBuilder() .setBinding(groupingKeyAlias, currentGroup) .setBinding(aggregateAlias, accumulator.finish()) .build(context.getTypeRepository()); previousCompleteResult = completeResultValue.eval(store, nestedContext); + PartialAggregationResult result = currentGroup == null ? null : getPartialAggregationResult((Message) currentGroup); currentGroup = nextGroup; // "Reset" the accumulator by creating a fresh one. accumulator = aggregateValue.createAccumulator(context.getTypeRepository()); + return result; } private void accumulate(@Nullable Object currentObject) { @@ -201,7 +222,7 @@ public boolean isResultOnEmpty() { } @Nullable - public PartialAggregationResult getPartialAggregationResult() { - return accumulator.getPartialAggregationResult(); + public PartialAggregationResult getPartialAggregationResult(@Nonnull Message groupingKey) { + return accumulator.getPartialAggregationResult(groupingKey, serializationContext); } } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/KeyValueCursorBase.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/KeyValueCursorBase.java index bda2c016d3..4521999453 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/KeyValueCursorBase.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/KeyValueCursorBase.java @@ -40,6 +40,7 @@ import com.apple.foundationdb.record.cursors.BaseCursor; import com.apple.foundationdb.record.cursors.CursorLimitManager; import com.apple.foundationdb.record.planprotos.PartialAggregationResult; +import com.apple.foundationdb.relational.continuation.ContinuationProto; import com.apple.foundationdb.subspace.Subspace; import com.apple.foundationdb.tuple.Tuple; import com.google.protobuf.ByteString; @@ -181,7 +182,11 @@ public byte[] toBytes() { if (lastKey == null) { return null; } - return Arrays.copyOfRange(lastKey, prefixLength, lastKey.length); + ContinuationProto proto = ContinuationProto.newBuilder() + .setExecutionState(ByteString.copyFrom(Arrays.copyOfRange(lastKey, prefixLength, lastKey.length))) + .setPartialAggregationResults(partialAggregationResult) + .build(); + return proto.toByteArray(); } public Continuation withPartialAggregationResult(@Nullable PartialAggregationResult partialAggregationResult) { diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/Accumulator.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/Accumulator.java index 21a57eb742..5ba4db8e06 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/Accumulator.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/Accumulator.java @@ -20,9 +20,15 @@ package com.apple.foundationdb.record.query.plan.cascades.values; +import com.apple.foundationdb.record.PlanSerializationContext; +import com.apple.foundationdb.record.RecordMetaDataProto; +import com.apple.foundationdb.record.planprotos.AccumulatorState; import com.apple.foundationdb.record.planprotos.PartialAggregationResult; +import com.google.protobuf.Message; +import javax.annotation.Nonnull; import javax.annotation.Nullable; +import java.util.List; /** * An aggregate accumulator. @@ -32,7 +38,8 @@ public interface Accumulator { @Nullable Object finish(); - @Nullable PartialAggregationResult getPartialAggregationResult(); + @Nullable + PartialAggregationResult getPartialAggregationResult(Message groupingKey, PlanSerializationContext serializationContext); - void setInitialState(@Nullable PartialAggregationResult partialAggregationResult); + void setInitialState(@Nonnull List accumulatorStates); } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/CountValue.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/CountValue.java index b80e0d4fa5..d40b69c24a 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/CountValue.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/CountValue.java @@ -29,6 +29,7 @@ import com.apple.foundationdb.record.PlanSerializationContext; import com.apple.foundationdb.record.RecordCoreException; import com.apple.foundationdb.record.metadata.IndexTypes; +import com.apple.foundationdb.record.planprotos.AccumulatorState; import com.apple.foundationdb.record.planprotos.PCountValue; import com.apple.foundationdb.record.planprotos.PCountValue.PPhysicalOperator; import com.apple.foundationdb.record.planprotos.PValue; @@ -46,6 +47,7 @@ import com.google.common.base.Verify; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; +import com.google.protobuf.Any; import com.google.protobuf.Message; import javax.annotation.Nonnull; @@ -349,23 +351,23 @@ public Object finish() { @Nullable @Override - public PartialAggregationResult getPartialAggregationResult() { + public PartialAggregationResult getPartialAggregationResult(Message groupingKey, PlanSerializationContext serializationContext) { if (state == null) { return null; } return PartialAggregationResult.newBuilder() - .setPhysicalOperatorName(physicalOperator.name()) - // (TODO) store state, maybe a string? + .setGroupKey(groupingKey.toByteString()) + .addAccumulatorStates(AccumulatorState.newBuilder() + .setPhysicalOperatorName(physicalOperator.name()) + .addState(String.valueOf(state))) .build(); } @Override - public void setInitialState(@Nullable PartialAggregationResult partialAggregationResult) { - if (partialAggregationResult != null) { - // check physical operator name are the same - // check this.state == null - this.state = partialAggregationResult.getState(); - } + public void setInitialState(@Nonnull List accumulatorStates) { + // check physical operator name are the same + // check this.state == null + this.state = Long.parseLong(accumulatorStates.get(0).getState(0)); } } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/NumericAggregationValue.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/NumericAggregationValue.java index 4f15278028..1aee645a12 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/NumericAggregationValue.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/NumericAggregationValue.java @@ -29,6 +29,7 @@ import com.apple.foundationdb.record.PlanSerializationContext; import com.apple.foundationdb.record.RecordCoreArgumentException; import com.apple.foundationdb.record.metadata.IndexTypes; +import com.apple.foundationdb.record.planprotos.AccumulatorState; import com.apple.foundationdb.record.planprotos.PNumericAggregationValue; import com.apple.foundationdb.record.planprotos.PNumericAggregationValue.PAvg; import com.apple.foundationdb.record.planprotos.PNumericAggregationValue.PBitmapConstructAgg; @@ -64,6 +65,7 @@ import javax.annotation.Nonnull; import javax.annotation.Nullable; +import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.BitSet; import java.util.List; @@ -867,22 +869,90 @@ public Object finish() { @Nullable @Override - public PartialAggregationResult getPartialAggregationResult() { + public PartialAggregationResult getPartialAggregationResult(@Nonnull Message groupingKey, PlanSerializationContext serializationContext) { if (state == null) { return null; } + AccumulatorState.Builder builder = AccumulatorState.newBuilder().setPhysicalOperatorName(physicalOperator.name()); + switch (physicalOperator) { + case SUM_I: + case MAX_I: + case MIN_I: + case SUM_L: + case MAX_L: + case MIN_L: + case SUM_D: + case MAX_D: + case MIN_D: + case SUM_F: + case MAX_F: + case MIN_F: + builder.addState(String.valueOf(state)); + break; + case AVG_I: + case AVG_L: + case AVG_D: + case AVG_F: + Pair pair = (Pair) state; + builder.addState(String.valueOf(pair.getLeft())).addState(String.valueOf(pair.getRight())); + break; + case BITMAP_CONSTRUCT_AGG_I: + case BITMAP_CONSTRUCT_AGG_L: + builder.addState(new String(((BitSet) state).toByteArray(), StandardCharsets.UTF_8)); + break; + default: + break; + + } return PartialAggregationResult.newBuilder() - .setPhysicalOperatorName(physicalOperator.name()) - // (TODO) store state, maybe a string? + .setGroupKey(groupingKey.toByteString()) + .addAccumulatorStates(builder) .build(); } @Override - public void setInitialState(@Nullable PartialAggregationResult partialAggregationResult) { - if (partialAggregationResult != null) { - // check physical operator name are the same - // check this.state == null - this.state = partialAggregationResult.getState(); + public void setInitialState(@Nonnull List accumulatorStates) { + // check physical operator name are the same + // check this.state == null + switch (physicalOperator) { + case SUM_I: + case MAX_I: + case MIN_I: + state = Integer.parseInt(accumulatorStates.get(0).getState(0)); + break; + case SUM_L: + case MAX_L: + case MIN_L: + state = Long.parseLong(accumulatorStates.get(0).getState(0)); + break; + case SUM_D: + case MAX_D: + case MIN_D: + state = Double.parseDouble(accumulatorStates.get(0).getState(0)); + break; + case SUM_F: + case MAX_F: + case MIN_F: + state = Float.parseFloat(accumulatorStates.get(0).getState(0)); + break; + case AVG_I: + state = Pair.of(Integer.parseInt(accumulatorStates.get(0).getState(0)), Long.parseLong(accumulatorStates.get(0).getState(1))); + break; + case AVG_L: + state = Pair.of(Long.parseLong(accumulatorStates.get(0).getState(0)), Long.parseLong(accumulatorStates.get(0).getState(1))); + break; + case AVG_D: + state = Pair.of(Double.parseDouble(accumulatorStates.get(0).getState(0)), Long.parseLong(accumulatorStates.get(0).getState(1))); + break; + case AVG_F: + state = Pair.of(Float.parseFloat(accumulatorStates.get(0).getState(0)), Long.parseLong(accumulatorStates.get(0).getState(1))); + break; + case BITMAP_CONSTRUCT_AGG_I: + case BITMAP_CONSTRUCT_AGG_L: + state = BitSet.valueOf(accumulatorStates.get(0).getState(0).getBytes(StandardCharsets.UTF_8)); + break; + default: + break; } } } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/RecordConstructorValue.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/RecordConstructorValue.java index 74e5f82323..a2bf84fcfc 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/RecordConstructorValue.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/RecordConstructorValue.java @@ -27,6 +27,7 @@ import com.apple.foundationdb.record.PlanDeserializer; import com.apple.foundationdb.record.PlanHashable; import com.apple.foundationdb.record.PlanSerializationContext; +import com.apple.foundationdb.record.planprotos.AccumulatorState; import com.apple.foundationdb.record.planprotos.PRecordConstructorValue; import com.apple.foundationdb.record.planprotos.PValue; import com.apple.foundationdb.record.planprotos.PartialAggregationResult; @@ -50,6 +51,7 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Streams; +import com.google.protobuf.Any; import com.google.protobuf.Descriptors; import com.google.protobuf.DynamicMessage; import com.google.protobuf.Message; @@ -57,8 +59,10 @@ import javax.annotation.Nonnull; import javax.annotation.Nullable; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.LinkedList; import java.util.List; import java.util.Objects; import java.util.function.Supplier; @@ -378,12 +382,28 @@ private List buildAccumulators() { @Nullable @Override - public PartialAggregationResult getPartialAggregationResult() { - return null; + public PartialAggregationResult getPartialAggregationResult(Message groupingKey, PlanSerializationContext serializationContext) { + List accumulatorStates = new ArrayList<>(); + for (Accumulator accumulator: childAccumulators) { + // (TODO): check groupingKeyValues are the same here + if (accumulator.getPartialAggregationResult(groupingKey, serializationContext) != null) { + accumulatorStates.addAll(accumulator.getPartialAggregationResult(groupingKey, serializationContext).getAccumulatorStatesList()); + } + } + if (accumulatorStates.isEmpty()) { + return null; + } + return PartialAggregationResult.newBuilder() + .setGroupKey(groupingKey.toByteString()) + .addAllAccumulatorStates(accumulatorStates) + .build(); } @Override - public void setInitialState(@Nullable PartialAggregationResult partialAggregationResult) { + public void setInitialState(@Nonnull List accumulatorStates) { + for (int i = 0; i < accumulatorStates.size(); i++) { + childAccumulators.get(i).setInitialState(List.of(accumulatorStates.get(i))); + } } }; } diff --git a/fdb-record-layer-core/src/main/proto/continuation.proto b/fdb-record-layer-core/src/main/proto/continuation.proto index 17fdc48004..d5816ce3b1 100644 --- a/fdb-record-layer-core/src/main/proto/continuation.proto +++ b/fdb-record-layer-core/src/main/proto/continuation.proto @@ -46,7 +46,7 @@ message ContinuationProto { // The serialized package optional CompiledStatement compiled_statement = 5; optional Reason reason = 6; - optional com.apple.foundationdb.record.planprotos.PartialAggregationResult partial_aggregation_result = 7; + com.apple.foundationdb.record.planprotos.PartialAggregationResult partial_aggregation_results = 7; } message TypedQueryArgument { diff --git a/fdb-record-layer-core/src/main/proto/record_query_plan.proto b/fdb-record-layer-core/src/main/proto/record_query_plan.proto index a8323102f4..c75fc8ff67 100644 --- a/fdb-record-layer-core/src/main/proto/record_query_plan.proto +++ b/fdb-record-layer-core/src/main/proto/record_query_plan.proto @@ -1784,6 +1784,12 @@ message PRecursiveUnionQueryPlan { } message PartialAggregationResult { + optional bytes group_key = 1; + optional string group_key_alias = 2; + repeated AccumulatorState accumulator_states = 3; +} + +message AccumulatorState { optional string physical_operator_name = 1; - optional google.protobuf.Any state = 2; -} \ No newline at end of file + repeated string state = 2; // 2 for avg +} diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBStreamAggregationTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBStreamAggregationTest.java index 98fda6dba1..c44b271eea 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBStreamAggregationTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBStreamAggregationTest.java @@ -20,6 +20,7 @@ package com.apple.foundationdb.record.provider.foundationdb.query; +import com.apple.foundationdb.record.Bindings; import com.apple.foundationdb.record.ByteScanLimiterFactory; import com.apple.foundationdb.record.EvaluationContext; import com.apple.foundationdb.record.ExecuteProperties; @@ -28,7 +29,6 @@ import com.apple.foundationdb.record.RecordCursorContinuation; import com.apple.foundationdb.record.RecordCursorEndContinuation; import com.apple.foundationdb.record.RecordCursorResult; -import com.apple.foundationdb.record.RecordCursorStartContinuation; import com.apple.foundationdb.record.RecordMetaData; import com.apple.foundationdb.record.RecordScanLimiterFactory; import com.apple.foundationdb.record.TestRecords1Proto; @@ -48,6 +48,7 @@ import com.apple.foundationdb.record.query.plan.plans.RecordQueryScanPlan; import com.apple.foundationdb.record.query.plan.plans.RecordQueryStreamingAggregationPlan; import com.apple.foundationdb.record.query.plan.plans.RecordQueryTypeFilterPlan; +import com.apple.foundationdb.relational.continuation.ContinuationProto; import com.apple.test.Tags; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; @@ -328,15 +329,15 @@ void aggregateHitScanLimitReached() { // recordScanLimit = 5: scans 3 rows, and the 4th scan hits SCAN_LIMIT_REACHED // although the first group contains exactly 3 rows, we don't know we've finished the first group before we get to the 4th row, so nothing is returned, continuation is back to START RecordCursorContinuation continuation1 = executePlanWithRecordScanLimit(plan, 5, null, null); - Assertions.assertEquals(RecordCursorStartContinuation.START, continuation1); - // recordScanLimit = 6: scans 4 rows, and the 5th scan hits SCAN_LIMIT_REACHED, we know that we've finished the 1st group, aggregated result is returned - RecordCursorContinuation continuation2 = executePlanWithRecordScanLimit(plan, 6, continuation1.toBytes(), resultOf("0", 3)); - // continue with recordScanLimit = 5, scans 3 rows and hits SCAN_LIMIT_REACHED - // again, we don't know that we've finished the 2nd group, nothing is returned, continuation is back to where the scan starts - RecordCursorContinuation continuation3 = executePlanWithRecordScanLimit(plan, 5, continuation2.toBytes(), null); - Assertions.assertArrayEquals(continuation2.toBytes(), continuation3.toBytes()); - // finish the 2nd group, aggregated result is returned, exhausted the source - RecordCursorContinuation continuation4 = executePlanWithRecordScanLimit(plan, 6, continuation3.toBytes(), resultOf("1", 12)); + // start the next scan from 4th row, and scans the 4th row (recordScanLimit = 1), return the aggregated result of the first group + RecordCursorContinuation continuation2 = executePlanWithRecordScanLimit(plan, 1, continuation1.toBytes(), resultOf("0", 3)); + // start the next scan from 5th row, and scans the 5th row (recordScanLimit = 1), return nothing + RecordCursorContinuation continuation3 = executePlanWithRecordScanLimit(plan, 1, continuation2.toBytes(), null); + // start the next scan from 6th row, and scans the 6th row (recordScanLimit = 2), return 2nd group aggregated result + RecordCursorContinuation continuation4 = executePlanWithRecordScanLimit(plan, 1, continuation3.toBytes(), null); + // (TODO): return exhausted, but not result, probably when finish scan x row, needs to return (x-1) to avoid this from happening + RecordCursorContinuation continuation5 = executePlanWithRecordScanLimit(plan, 1, continuation4.toBytes(), resultOf("1", 12)); + Assertions.assertEquals(RecordCursorEndContinuation.END, continuation4); } } @@ -382,13 +383,20 @@ private RecordCursor executePlan(final RecordQueryPlan plan, final ExecuteProperties executeProperties = ExecuteProperties.SERIAL_EXECUTE; executeProperties = executeProperties.setReturnedRowLimit(rowLimit).setState(executeState); try { - return plan.executePlan(recordStore, EvaluationContext.forTypeRepository(typeRepository), continuation, executeProperties); + if (continuation == null) { + return plan.executePlan(recordStore, EvaluationContext.forTypeRepository(typeRepository), null, executeProperties); + } else { + ContinuationProto continuationProto = ContinuationProto.parseFrom(continuation); + System.out.println("partialAggregationResult:" + continuationProto.getPartialAggregationResults()); + return plan.executePlan(recordStore, EvaluationContext.forBindingsAndTypeRepositoryAndPartialAggregationResult(Bindings.EMPTY_BINDINGS, typeRepository, continuationProto.getPartialAggregationResults()), continuationProto.getExecutionState().toByteArray(), executeProperties); + } } catch (final Throwable t) { throw Assertions.fail(t); } } private RecordCursorContinuation executePlanWithRecordScanLimit(final RecordQueryPlan plan, final int recordScanLimit, byte[] continuation, @Nullable List expectedResult) { + System.out.println("executePlanWithRecordScanLimit called with recordScanLimit:" + recordScanLimit + " expectedResult:" + expectedResult); List queryResults = new LinkedList<>(); RecordCursor currentCursor = executePlan(plan, 0, recordScanLimit, continuation); RecordCursorResult currentCursorResult; @@ -396,10 +404,13 @@ private RecordCursorContinuation executePlanWithRecordScanLimit(final RecordQuer while (true) { currentCursorResult = currentCursor.getNext(); cursorContinuation = currentCursorResult.getContinuation(); + System.out.println("cursor hasNext:" + currentCursorResult.hasNext()); if (!currentCursorResult.hasNext()) { + System.out.println("cursor NoNextReason:" + currentCursorResult.getNoNextReason()); break; } queryResults.add(currentCursorResult.get()); + System.out.println("queryResults size:" + queryResults.size() + " new added:" + currentCursorResult.get().getMessage()); } if (expectedResult == null) { Assertions.assertTrue(queryResults.isEmpty()); From 0950385d08c8d06f0eb0c54baf20420150430bf0 Mon Sep 17 00:00:00 2001 From: Pengpeng Lu Date: Mon, 10 Mar 2025 14:57:43 -0700 Subject: [PATCH 06/14] save --- .../record/EvaluationContext.java | 9 +- .../record/RecordCursorContinuation.java | 6 - .../cursors/aggregate/AggregateCursor.java | 112 +++++++++++++----- .../cursors/aggregate/StreamGrouping.java | 23 ++-- .../foundationdb/KeyValueCursorBase.java | 29 +---- .../plan/cascades/values/Accumulator.java | 8 +- .../plan/cascades/values/CountValue.java | 12 +- .../values/NumericAggregationValue.java | 12 +- .../values/RecordConstructorValue.java | 13 +- .../src/main/proto/continuation.proto | 1 - .../src/main/proto/record_cursor.proto | 18 ++- .../src/main/proto/record_query_plan.proto | 11 -- .../query/FDBStreamAggregationTest.java | 10 +- 13 files changed, 142 insertions(+), 122 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/EvaluationContext.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/EvaluationContext.java index 4c9c385868..5acb954302 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/EvaluationContext.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/EvaluationContext.java @@ -25,7 +25,6 @@ import com.apple.foundationdb.record.planprotos.PValue; import com.apple.foundationdb.record.query.plan.cascades.CorrelationIdentifier; import com.apple.foundationdb.record.query.plan.cascades.typing.TypeRepository; -import com.apple.foundationdb.record.planprotos.PartialAggregationResult; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -48,7 +47,7 @@ public class EvaluationContext { private final TypeRepository typeRepository; @Nullable - private final PartialAggregationResult partialAggregationResultProto; + private final RecordCursorProto.PartialAggregationResult partialAggregationResultProto; public static final EvaluationContext EMPTY = new EvaluationContext(Bindings.EMPTY_BINDINGS, TypeRepository.EMPTY_SCHEMA); @@ -65,7 +64,7 @@ private EvaluationContext(@Nonnull Bindings bindings, @Nonnull TypeRepository ty this(bindings, typeRepository, null); } - private EvaluationContext(@Nonnull Bindings bindings, @Nonnull TypeRepository typeRepository, @Nullable PartialAggregationResult proto) { + private EvaluationContext(@Nonnull Bindings bindings, @Nonnull TypeRepository typeRepository, @Nullable RecordCursorProto.PartialAggregationResult proto) { this.bindings = bindings; this.typeRepository = typeRepository; this.partialAggregationResultProto = proto; @@ -100,7 +99,7 @@ public static EvaluationContext forTypeRepository(@Nonnull TypeRepository typeRe } @Nonnull - public static EvaluationContext forBindingsAndTypeRepositoryAndPartialAggregationResult(@Nonnull Bindings bindings, @Nonnull TypeRepository typeRepository, @Nullable PartialAggregationResult proto) { + public static EvaluationContext forBindingsAndTypeRepositoryAndPartialAggregationResult(@Nonnull Bindings bindings, @Nonnull TypeRepository typeRepository, @Nullable RecordCursorProto.PartialAggregationResult proto) { return new EvaluationContext(bindings, typeRepository, proto); } @@ -117,7 +116,7 @@ public static EvaluationContext forBinding(@Nonnull String bindingName, @Nullabl } @Nullable - public PartialAggregationResult getPartialAggregationResult() { + public RecordCursorProto.PartialAggregationResult getPartialAggregationResult() { return partialAggregationResultProto; } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/RecordCursorContinuation.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/RecordCursorContinuation.java index 45158451ec..5f10476d36 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/RecordCursorContinuation.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/RecordCursorContinuation.java @@ -21,7 +21,6 @@ package com.apple.foundationdb.record; import com.apple.foundationdb.annotation.API; -import com.apple.foundationdb.record.planprotos.PartialAggregationResult; import com.google.protobuf.ByteString; import javax.annotation.Nonnull; @@ -74,11 +73,6 @@ default ByteString toByteString() { return bytes == null ? ByteString.EMPTY : ByteString.copyFrom(bytes); } - @Nullable - default PartialAggregationResult getPartialAggregationResult() { - return null; - } - /** * Return whether this continuation is an "end continuation", i.e., represents that the iteration has reached * its end and would not produce more results even if restarted. If {@code isEnd()} returns {@code true}, then diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/AggregateCursor.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/AggregateCursor.java index 2777778da7..261548741e 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/AggregateCursor.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/AggregateCursor.java @@ -22,15 +22,17 @@ 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.planprotos.PartialAggregationResult; -import com.apple.foundationdb.record.provider.foundationdb.KeyValueCursorBase; import com.apple.foundationdb.record.query.plan.plans.QueryResult; +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; @@ -59,11 +61,8 @@ public class AggregateCursor implements RecordCursor previousValidResult; - // last row in last group, is null if the current group is the first group @Nullable - private RecordCursorResult lastInLastGroup; - @Nullable - private PartialAggregationResult partialAggregationResult; + private RecordCursorProto.PartialAggregationResult partialAggregationResult; @Nullable byte[] continuation; @@ -82,7 +81,7 @@ public AggregateCursor(@Nonnull RecordCursor inner, public CompletableFuture> onNext() { if (previousResult != null && !previousResult.hasNext()) { // we are done - return CompletableFuture.completedFuture(RecordCursorResult.withoutNextValue(previousResult.getContinuation(), + return CompletableFuture.completedFuture(RecordCursorResult.withoutNextValue(new AggregateCursorContinuation(previousResult.getContinuation()), previousResult.getNoNextReason())); } @@ -91,18 +90,16 @@ public CompletableFuture> onNext() { System.out.println("scan innerResult hasNext:" + innerResult.hasNext()); if (!innerResult.hasNext()) { System.out.println("cursor noNextReason:" + innerResult.getNoNextReason()); - if (!isNoRecords() || (isCreateDefaultOnEmpty && streamGrouping.isResultOnEmpty())) { + //if (!isNoRecords() || (isCreateDefaultOnEmpty && streamGrouping.isResultOnEmpty())) { // the method streamGrouping.finalizeGroup() computes previousCompleteResult and resets the accumulator partialAggregationResult = streamGrouping.finalizeGroup(); - } + //} return false; } else { final QueryResult queryResult = Objects.requireNonNull(innerResult.get()); System.out.println("inner result:" + innerResult.get().getMessage()); boolean groupBreak = streamGrouping.apply(queryResult); - if (groupBreak) { - lastInLastGroup = previousValidResult; - } else { + if (!groupBreak) { // previousValidResult is the last row before group break, it sets the continuation previousValidResult = innerResult; } @@ -113,7 +110,7 @@ public CompletableFuture> onNext() { 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 = previousValidResult == null ? ByteArrayContinuation.fromNullable(continuation) : previousValidResult.getContinuation(); + RecordCursorContinuation c = previousValidResult == null ? new AggregateCursorContinuation(continuation, false) : new AggregateCursorContinuation(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. @@ -143,27 +140,16 @@ public CompletableFuture> onNext() { // innerResult.hasNext() = false, might stop in the middle of a group if (Verify.verifyNotNull(previousResult).getNoNextReason() == NoNextReason.SOURCE_EXHAUSTED) { // exhausted - if (previousValidResult == null) { + if (previousValidResult == null && partialAggregationResult == null) { return RecordCursorResult.exhausted(); } else { - RecordCursorContinuation continuation = previousValidResult.getContinuation(); + RecordCursorContinuation c = previousValidResult == null ? new AggregateCursorContinuation(continuation, false) : new AggregateCursorContinuation(previousValidResult.getContinuation()); previousValidResult = previousResult; - return RecordCursorResult.withNextValue(QueryResult.ofComputed(streamGrouping.getCompletedGroupResult()), continuation); + return RecordCursorResult.withNextValue(QueryResult.ofComputed(streamGrouping.getCompletedGroupResult()), c); } } else { // stopped in the middle of a group - /* - RecordCursorContinuation currentContinuation; - // in the current scan, if current group is the first group, set the continuation to the start of the current scan - // otherwise set the continuation to the last row in the last group - if (lastInLastGroup == null) { - currentContinuation = continuation == null ? RecordCursorStartContinuation.START : ByteArrayContinuation.fromNullable(continuation); - } else { - currentContinuation = lastInLastGroup.getContinuation(); - } - - */ - RecordCursorContinuation currentContinuation = ((KeyValueCursorBase.Continuation) previousValidResult.getContinuation()).withPartialAggregationResult(partialAggregationResult); + RecordCursorContinuation currentContinuation = new AggregateCursorContinuation(previousValidResult.getContinuation(), partialAggregationResult); previousValidResult = previousResult; return RecordCursorResult.withoutNextValue(currentContinuation, Verify.verifyNotNull(previousResult).getNoNextReason()); } @@ -198,4 +184,72 @@ public boolean accept(@Nonnull RecordCursorVisitor visitor) { } return visitor.visitLeave(this); } + + private static class AggregateCursorContinuation implements RecordCursorContinuation { + @Nullable + private final ByteString innerContinuation; + + @Nullable + private final RecordCursorProto.PartialAggregationResult partialAggregationResult; + + @Nullable + private RecordCursorProto.AggregateCursorContinuation cachedProto; + + private final boolean isEnd; + + public AggregateCursorContinuation(@Nonnull RecordCursorContinuation other) { + this(other.toBytes(), other.isEnd()); + } + + public AggregateCursorContinuation(@Nonnull RecordCursorContinuation other, @Nullable RecordCursorProto.PartialAggregationResult partialAggregationResult) { + this.isEnd = other.isEnd(); + this.innerContinuation = other.toByteString(); + this.partialAggregationResult = partialAggregationResult; + } + + public AggregateCursorContinuation(@Nullable byte[] innerContinuation, boolean isEnd) { + this.isEnd = isEnd; + this.innerContinuation = innerContinuation == null ? null : ByteString.copyFrom(innerContinuation); + this.partialAggregationResult = null; + } + + @Nonnull + @Override + public ByteString toByteString() { + if (isEnd()) { + return ByteString.EMPTY; + } else { + return toProto().toByteString(); + } + } + + @Nullable + @Override + public byte[] toBytes() { + if (isEnd()) { + return null; + } + return toProto().toByteArray(); + } + + @Override + public boolean isEnd() { + return isEnd; + } + + @Nonnull + private RecordCursorProto.AggregateCursorContinuation toProto() { + if (cachedProto == null) { + RecordCursorProto.AggregateCursorContinuation.Builder cachedProtoBuilder = RecordCursorProto.AggregateCursorContinuation.newBuilder(); + if (partialAggregationResult != null) { + cachedProtoBuilder.setPartialAggregationResults(partialAggregationResult); + } + if (innerContinuation != null) { + cachedProtoBuilder.setContinuation(innerContinuation); + } + cachedProto = cachedProtoBuilder.build(); + } + return cachedProto; + } + } } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/StreamGrouping.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/StreamGrouping.java index 58be9d3122..f0ee0a26ac 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/StreamGrouping.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/StreamGrouping.java @@ -24,8 +24,8 @@ import com.apple.foundationdb.record.EvaluationContext; import com.apple.foundationdb.record.PlanHashable; import com.apple.foundationdb.record.PlanSerializationContext; +import com.apple.foundationdb.record.RecordCursorProto; import com.apple.foundationdb.record.RecordCursorResult; -import com.apple.foundationdb.record.planprotos.PartialAggregationResult; import com.apple.foundationdb.record.provider.foundationdb.FDBRecordStoreBase; import com.apple.foundationdb.record.query.plan.cascades.CorrelationIdentifier; import com.apple.foundationdb.record.query.plan.cascades.values.Accumulator; @@ -120,6 +120,11 @@ public StreamGrouping(@Nullable final Value groupingKeyValue, this.accumulator = aggregateValue.createAccumulator(context.getTypeRepository()); if (context.getPartialAggregationResult() != null) { this.accumulator.setInitialState(context.getPartialAggregationResult().getAccumulatorStatesList()); + try { + this.currentGroup = DynamicMessage.parseFrom(context.getTypeRepository().newMessageBuilder(groupingKeyValue.getResultType()).getDescriptorForType(), context.getPartialAggregationResult().getGroupKey().toByteArray()); + } catch (InvalidProtocolBufferException e) { + throw new RuntimeException(e); + } } this.store = store; this.context = context; @@ -174,32 +179,24 @@ public Object getCompletedGroupResult() { private boolean isGroupBreak(final Object currentGroup, final Object nextGroup) { if (currentGroup == null) { - if (context.getPartialAggregationResult() != null) { - try { - this.currentGroup = DynamicMessage.parseFrom(((Message) nextGroup).getDescriptorForType(), context.getPartialAggregationResult().getGroupKey()); - return (!this.currentGroup.equals(nextGroup)); - } catch (InvalidProtocolBufferException e) { - throw new RuntimeException(e); - } - } return false; } else { return (!currentGroup.equals(nextGroup)); } } - public PartialAggregationResult finalizeGroup() { + public RecordCursorProto.PartialAggregationResult finalizeGroup() { return finalizeGroup(null); } - private PartialAggregationResult finalizeGroup(Object nextGroup) { + private RecordCursorProto.PartialAggregationResult finalizeGroup(Object nextGroup) { final EvaluationContext nestedContext = context.childBuilder() .setBinding(groupingKeyAlias, currentGroup) .setBinding(aggregateAlias, accumulator.finish()) .build(context.getTypeRepository()); previousCompleteResult = completeResultValue.eval(store, nestedContext); - PartialAggregationResult result = currentGroup == null ? null : getPartialAggregationResult((Message) currentGroup); + RecordCursorProto.PartialAggregationResult result = currentGroup == null ? null : getPartialAggregationResult((Message) currentGroup); currentGroup = nextGroup; // "Reset" the accumulator by creating a fresh one. accumulator = aggregateValue.createAccumulator(context.getTypeRepository()); @@ -222,7 +219,7 @@ public boolean isResultOnEmpty() { } @Nullable - public PartialAggregationResult getPartialAggregationResult(@Nonnull Message groupingKey) { + public RecordCursorProto.PartialAggregationResult getPartialAggregationResult(@Nonnull Message groupingKey) { return accumulator.getPartialAggregationResult(groupingKey, serializationContext); } } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/KeyValueCursorBase.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/KeyValueCursorBase.java index 4521999453..57fce4f8ac 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/KeyValueCursorBase.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/KeyValueCursorBase.java @@ -39,8 +39,6 @@ import com.apple.foundationdb.record.cursors.AsyncIteratorCursor; import com.apple.foundationdb.record.cursors.BaseCursor; import com.apple.foundationdb.record.cursors.CursorLimitManager; -import com.apple.foundationdb.record.planprotos.PartialAggregationResult; -import com.apple.foundationdb.relational.continuation.ContinuationProto; import com.apple.foundationdb.subspace.Subspace; import com.apple.foundationdb.tuple.Tuple; import com.google.protobuf.ByteString; @@ -136,29 +134,18 @@ private RecordCursorContinuation continuationHelper() { return new Continuation(lastKey, prefixLength); } - public static class Continuation implements RecordCursorContinuation { + private static class Continuation implements RecordCursorContinuation { @Nullable private final byte[] lastKey; private final int prefixLength; - @Nullable - private final PartialAggregationResult partialAggregationResult; public Continuation(@Nullable final byte[] lastKey, final int prefixLength) { - // Note that doing this without a full copy is dangerous if the array is ever mutated. - // Currently, this never happens and the only thing that changes is which array lastKey points to. - // However, if logic in KeyValueCursor or KeyValue changes, this could break continuations. - // To resolve it, we could resort to doing a full copy here, although that's somewhat expensive. - this(lastKey, prefixLength, null); - } - - private Continuation(@Nullable final byte[] lastKey, final int prefixLength, @Nullable PartialAggregationResult partialAggregationResult) { // Note that doing this without a full copy is dangerous if the array is ever mutated. // Currently, this never happens and the only thing that changes is which array lastKey points to. // However, if logic in KeyValueCursor or KeyValue changes, this could break continuations. // To resolve it, we could resort to doing a full copy here, although that's somewhat expensive. this.lastKey = lastKey; this.prefixLength = prefixLength; - this.partialAggregationResult = partialAggregationResult; } @Override @@ -182,15 +169,7 @@ public byte[] toBytes() { if (lastKey == null) { return null; } - ContinuationProto proto = ContinuationProto.newBuilder() - .setExecutionState(ByteString.copyFrom(Arrays.copyOfRange(lastKey, prefixLength, lastKey.length))) - .setPartialAggregationResults(partialAggregationResult) - .build(); - return proto.toByteArray(); - } - - public Continuation withPartialAggregationResult(@Nullable PartialAggregationResult partialAggregationResult) { - return new Continuation(lastKey, prefixLength, partialAggregationResult); + return Arrays.copyOfRange(lastKey, prefixLength, lastKey.length); } } @@ -363,8 +342,8 @@ public T setHigh(@Nonnull byte[] highBytes, @Nonnull EndpointType highEndpoint) protected int calculatePrefixLength() { int prefixLength = subspace.pack().length; while ((prefixLength < lowBytes.length) && - (prefixLength < highBytes.length) && - (lowBytes[prefixLength] == highBytes[prefixLength])) { + (prefixLength < highBytes.length) && + (lowBytes[prefixLength] == highBytes[prefixLength])) { prefixLength++; } return prefixLength; diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/Accumulator.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/Accumulator.java index 5ba4db8e06..75bd871ce0 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/Accumulator.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/Accumulator.java @@ -21,9 +21,7 @@ package com.apple.foundationdb.record.query.plan.cascades.values; import com.apple.foundationdb.record.PlanSerializationContext; -import com.apple.foundationdb.record.RecordMetaDataProto; -import com.apple.foundationdb.record.planprotos.AccumulatorState; -import com.apple.foundationdb.record.planprotos.PartialAggregationResult; +import com.apple.foundationdb.record.RecordCursorProto; import com.google.protobuf.Message; import javax.annotation.Nonnull; @@ -39,7 +37,7 @@ public interface Accumulator { @Nullable Object finish(); @Nullable - PartialAggregationResult getPartialAggregationResult(Message groupingKey, PlanSerializationContext serializationContext); + RecordCursorProto.PartialAggregationResult getPartialAggregationResult(Message groupingKey, PlanSerializationContext serializationContext); - void setInitialState(@Nonnull List accumulatorStates); + void setInitialState(@Nonnull List accumulatorStates); } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/CountValue.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/CountValue.java index d40b69c24a..a019b0b395 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/CountValue.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/CountValue.java @@ -28,12 +28,11 @@ import com.apple.foundationdb.record.PlanHashable; import com.apple.foundationdb.record.PlanSerializationContext; import com.apple.foundationdb.record.RecordCoreException; +import com.apple.foundationdb.record.RecordCursorProto; import com.apple.foundationdb.record.metadata.IndexTypes; -import com.apple.foundationdb.record.planprotos.AccumulatorState; import com.apple.foundationdb.record.planprotos.PCountValue; import com.apple.foundationdb.record.planprotos.PCountValue.PPhysicalOperator; import com.apple.foundationdb.record.planprotos.PValue; -import com.apple.foundationdb.record.planprotos.PartialAggregationResult; import com.apple.foundationdb.record.provider.foundationdb.FDBRecordStoreBase; import com.apple.foundationdb.record.query.plan.cascades.AliasMap; import com.apple.foundationdb.record.query.plan.cascades.BuiltInFunction; @@ -47,7 +46,6 @@ import com.google.common.base.Verify; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; -import com.google.protobuf.Any; import com.google.protobuf.Message; import javax.annotation.Nonnull; @@ -351,20 +349,20 @@ public Object finish() { @Nullable @Override - public PartialAggregationResult getPartialAggregationResult(Message groupingKey, PlanSerializationContext serializationContext) { + public RecordCursorProto.PartialAggregationResult getPartialAggregationResult(Message groupingKey, PlanSerializationContext serializationContext) { if (state == null) { return null; } - return PartialAggregationResult.newBuilder() + return RecordCursorProto.PartialAggregationResult.newBuilder() .setGroupKey(groupingKey.toByteString()) - .addAccumulatorStates(AccumulatorState.newBuilder() + .addAccumulatorStates(RecordCursorProto.AccumulatorState.newBuilder() .setPhysicalOperatorName(physicalOperator.name()) .addState(String.valueOf(state))) .build(); } @Override - public void setInitialState(@Nonnull List accumulatorStates) { + public void setInitialState(@Nonnull List accumulatorStates) { // check physical operator name are the same // check this.state == null this.state = Long.parseLong(accumulatorStates.get(0).getState(0)); diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/NumericAggregationValue.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/NumericAggregationValue.java index 1aee645a12..987238936b 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/NumericAggregationValue.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/NumericAggregationValue.java @@ -28,8 +28,8 @@ import com.apple.foundationdb.record.PlanHashable; import com.apple.foundationdb.record.PlanSerializationContext; import com.apple.foundationdb.record.RecordCoreArgumentException; +import com.apple.foundationdb.record.RecordCursorProto; import com.apple.foundationdb.record.metadata.IndexTypes; -import com.apple.foundationdb.record.planprotos.AccumulatorState; import com.apple.foundationdb.record.planprotos.PNumericAggregationValue; import com.apple.foundationdb.record.planprotos.PNumericAggregationValue.PAvg; import com.apple.foundationdb.record.planprotos.PNumericAggregationValue.PBitmapConstructAgg; @@ -38,7 +38,6 @@ import com.apple.foundationdb.record.planprotos.PNumericAggregationValue.PPhysicalOperator; import com.apple.foundationdb.record.planprotos.PNumericAggregationValue.PSum; import com.apple.foundationdb.record.planprotos.PValue; -import com.apple.foundationdb.record.planprotos.PartialAggregationResult; import com.apple.foundationdb.record.provider.foundationdb.FDBRecordStoreBase; import com.apple.foundationdb.record.provider.foundationdb.indexes.BitmapValueIndexMaintainer; import com.apple.foundationdb.record.query.plan.cascades.AliasMap; @@ -60,7 +59,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; -import com.google.protobuf.Any; import com.google.protobuf.Message; import javax.annotation.Nonnull; @@ -869,11 +867,11 @@ public Object finish() { @Nullable @Override - public PartialAggregationResult getPartialAggregationResult(@Nonnull Message groupingKey, PlanSerializationContext serializationContext) { + public RecordCursorProto.PartialAggregationResult getPartialAggregationResult(@Nonnull Message groupingKey, PlanSerializationContext serializationContext) { if (state == null) { return null; } - AccumulatorState.Builder builder = AccumulatorState.newBuilder().setPhysicalOperatorName(physicalOperator.name()); + RecordCursorProto.AccumulatorState.Builder builder = RecordCursorProto.AccumulatorState.newBuilder().setPhysicalOperatorName(physicalOperator.name()); switch (physicalOperator) { case SUM_I: case MAX_I: @@ -904,14 +902,14 @@ public PartialAggregationResult getPartialAggregationResult(@Nonnull Message gro break; } - return PartialAggregationResult.newBuilder() + return RecordCursorProto.PartialAggregationResult.newBuilder() .setGroupKey(groupingKey.toByteString()) .addAccumulatorStates(builder) .build(); } @Override - public void setInitialState(@Nonnull List accumulatorStates) { + public void setInitialState(@Nonnull List accumulatorStates) { // check physical operator name are the same // check this.state == null switch (physicalOperator) { diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/RecordConstructorValue.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/RecordConstructorValue.java index a2bf84fcfc..0cac6ed33e 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/RecordConstructorValue.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/RecordConstructorValue.java @@ -27,10 +27,9 @@ import com.apple.foundationdb.record.PlanDeserializer; import com.apple.foundationdb.record.PlanHashable; import com.apple.foundationdb.record.PlanSerializationContext; -import com.apple.foundationdb.record.planprotos.AccumulatorState; +import com.apple.foundationdb.record.RecordCursorProto; import com.apple.foundationdb.record.planprotos.PRecordConstructorValue; import com.apple.foundationdb.record.planprotos.PValue; -import com.apple.foundationdb.record.planprotos.PartialAggregationResult; import com.apple.foundationdb.record.provider.foundationdb.FDBRecordStoreBase; import com.apple.foundationdb.record.provider.foundationdb.FDBRecordVersion; import com.apple.foundationdb.record.query.plan.cascades.AliasMap; @@ -51,7 +50,6 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Streams; -import com.google.protobuf.Any; import com.google.protobuf.Descriptors; import com.google.protobuf.DynamicMessage; import com.google.protobuf.Message; @@ -62,7 +60,6 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.LinkedList; import java.util.List; import java.util.Objects; import java.util.function.Supplier; @@ -382,8 +379,8 @@ private List buildAccumulators() { @Nullable @Override - public PartialAggregationResult getPartialAggregationResult(Message groupingKey, PlanSerializationContext serializationContext) { - List accumulatorStates = new ArrayList<>(); + public RecordCursorProto.PartialAggregationResult getPartialAggregationResult(Message groupingKey, PlanSerializationContext serializationContext) { + List accumulatorStates = new ArrayList<>(); for (Accumulator accumulator: childAccumulators) { // (TODO): check groupingKeyValues are the same here if (accumulator.getPartialAggregationResult(groupingKey, serializationContext) != null) { @@ -393,14 +390,14 @@ public PartialAggregationResult getPartialAggregationResult(Message groupingKey, if (accumulatorStates.isEmpty()) { return null; } - return PartialAggregationResult.newBuilder() + return RecordCursorProto.PartialAggregationResult.newBuilder() .setGroupKey(groupingKey.toByteString()) .addAllAccumulatorStates(accumulatorStates) .build(); } @Override - public void setInitialState(@Nonnull List accumulatorStates) { + public void setInitialState(@Nonnull List accumulatorStates) { for (int i = 0; i < accumulatorStates.size(); i++) { childAccumulators.get(i).setInitialState(List.of(accumulatorStates.get(i))); } diff --git a/fdb-record-layer-core/src/main/proto/continuation.proto b/fdb-record-layer-core/src/main/proto/continuation.proto index d5816ce3b1..4b5dfae0d3 100644 --- a/fdb-record-layer-core/src/main/proto/continuation.proto +++ b/fdb-record-layer-core/src/main/proto/continuation.proto @@ -46,7 +46,6 @@ message ContinuationProto { // The serialized package optional CompiledStatement compiled_statement = 5; optional Reason reason = 6; - com.apple.foundationdb.record.planprotos.PartialAggregationResult partial_aggregation_results = 7; } message TypedQueryArgument { diff --git a/fdb-record-layer-core/src/main/proto/record_cursor.proto b/fdb-record-layer-core/src/main/proto/record_cursor.proto index c1400c3c8a..4d122d92e9 100644 --- a/fdb-record-layer-core/src/main/proto/record_cursor.proto +++ b/fdb-record-layer-core/src/main/proto/record_cursor.proto @@ -131,4 +131,20 @@ message RecursiveCursorContinuation { optional bool isInitialState = 1; optional planprotos.PTempTable tempTable = 2; optional bytes activeStateContinuation = 3; -} \ No newline at end of file +} + +message AggregateCursorContinuation { + optional bytes continuation = 1; + optional PartialAggregationResult partial_aggregation_results = 2; +} + +message PartialAggregationResult { + optional bytes group_key = 1; + optional string group_key_alias = 2; + repeated AccumulatorState accumulator_states = 3; +} + +message AccumulatorState { + optional string physical_operator_name = 1; + repeated string state = 2; // 2 for avg +} diff --git a/fdb-record-layer-core/src/main/proto/record_query_plan.proto b/fdb-record-layer-core/src/main/proto/record_query_plan.proto index c75fc8ff67..f27a0f85e6 100644 --- a/fdb-record-layer-core/src/main/proto/record_query_plan.proto +++ b/fdb-record-layer-core/src/main/proto/record_query_plan.proto @@ -1782,14 +1782,3 @@ message PRecursiveUnionQueryPlan { optional string initialTempTableAlias = 3; optional string recursiveTempTableAlias = 4; } - -message PartialAggregationResult { - optional bytes group_key = 1; - optional string group_key_alias = 2; - repeated AccumulatorState accumulator_states = 3; -} - -message AccumulatorState { - optional string physical_operator_name = 1; - repeated string state = 2; // 2 for avg -} diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBStreamAggregationTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBStreamAggregationTest.java index c44b271eea..9f182f365b 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBStreamAggregationTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBStreamAggregationTest.java @@ -28,6 +28,7 @@ import com.apple.foundationdb.record.RecordCursor; import com.apple.foundationdb.record.RecordCursorContinuation; import com.apple.foundationdb.record.RecordCursorEndContinuation; +import com.apple.foundationdb.record.RecordCursorProto; import com.apple.foundationdb.record.RecordCursorResult; import com.apple.foundationdb.record.RecordMetaData; import com.apple.foundationdb.record.RecordScanLimiterFactory; @@ -48,7 +49,6 @@ import com.apple.foundationdb.record.query.plan.plans.RecordQueryScanPlan; import com.apple.foundationdb.record.query.plan.plans.RecordQueryStreamingAggregationPlan; import com.apple.foundationdb.record.query.plan.plans.RecordQueryTypeFilterPlan; -import com.apple.foundationdb.relational.continuation.ContinuationProto; import com.apple.test.Tags; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; @@ -338,7 +338,7 @@ void aggregateHitScanLimitReached() { // (TODO): return exhausted, but not result, probably when finish scan x row, needs to return (x-1) to avoid this from happening RecordCursorContinuation continuation5 = executePlanWithRecordScanLimit(plan, 1, continuation4.toBytes(), resultOf("1", 12)); - Assertions.assertEquals(RecordCursorEndContinuation.END, continuation4); + Assertions.assertEquals(RecordCursorEndContinuation.END, continuation5); } } @@ -386,9 +386,10 @@ private RecordCursor executePlan(final RecordQueryPlan plan, final if (continuation == null) { return plan.executePlan(recordStore, EvaluationContext.forTypeRepository(typeRepository), null, executeProperties); } else { - ContinuationProto continuationProto = ContinuationProto.parseFrom(continuation); + RecordCursorProto.AggregateCursorContinuation continuationProto = RecordCursorProto.AggregateCursorContinuation.parseFrom(continuation); System.out.println("partialAggregationResult:" + continuationProto.getPartialAggregationResults()); - return plan.executePlan(recordStore, EvaluationContext.forBindingsAndTypeRepositoryAndPartialAggregationResult(Bindings.EMPTY_BINDINGS, typeRepository, continuationProto.getPartialAggregationResults()), continuationProto.getExecutionState().toByteArray(), executeProperties); + System.out.println("plan class:" + plan.getClass()); + return plan.executePlan(recordStore, EvaluationContext.forBindingsAndTypeRepositoryAndPartialAggregationResult(Bindings.EMPTY_BINDINGS, typeRepository, continuationProto.getPartialAggregationResults()), continuationProto.getContinuation().toByteArray(), executeProperties); } } catch (final Throwable t) { throw Assertions.fail(t); @@ -434,6 +435,7 @@ private List executePlanWithRowLimit(final RecordQueryPlan plan, fi } queryResults.add(currentCursorResult.get()); } + System.out.println("getNoNextReason:" + currentCursorResult.getNoNextReason()); if (currentCursorResult.getNoNextReason() == RecordCursor.NoNextReason.SOURCE_EXHAUSTED) { break; } From b275ea67dfb15ee9301fa0adb8e48f07f986e4f8 Mon Sep 17 00:00:00 2001 From: Pengpeng Lu Date: Mon, 10 Mar 2025 16:46:58 -0700 Subject: [PATCH 07/14] pmd green --- .../record/EvaluationContext.java | 1 - .../cursors/aggregate/AggregateCursor.java | 25 ++++++++++--------- .../cursors/aggregate/StreamGrouping.java | 13 ++++------ .../plan/cascades/values/Accumulator.java | 3 +-- .../plan/cascades/values/CountValue.java | 2 +- .../values/NumericAggregationValue.java | 2 +- .../values/RecordConstructorValue.java | 6 ++--- .../query/FDBStreamAggregationTest.java | 13 ++++------ .../recordlayer/ContinuationImpl.java | 15 ----------- .../recordlayer/query/QueryPlan.java | 2 +- 10 files changed, 30 insertions(+), 52 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/EvaluationContext.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/EvaluationContext.java index 5acb954302..5bde91e1b5 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/EvaluationContext.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/EvaluationContext.java @@ -22,7 +22,6 @@ import com.apple.foundationdb.annotation.API; import com.apple.foundationdb.record.logging.LogMessageKeys; -import com.apple.foundationdb.record.planprotos.PValue; import com.apple.foundationdb.record.query.plan.cascades.CorrelationIdentifier; import com.apple.foundationdb.record.query.plan.cascades.typing.TypeRepository; diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/AggregateCursor.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/AggregateCursor.java index 261548741e..a7245b6d82 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/AggregateCursor.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/AggregateCursor.java @@ -22,17 +22,14 @@ import com.apple.foundationdb.annotation.API; import com.apple.foundationdb.async.AsyncUtil; -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.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; @@ -58,6 +55,9 @@ public class AggregateCursor implements RecordCursor previousResult; + @Nullable + // when previousResult = row x, lastResult = row (x-1); when previousResult = null, lastResult = null + private RecordCursorResult lastResult; // Previous non-empty record processed by this cursor @Nullable private RecordCursorResult previousValidResult; @@ -86,18 +86,17 @@ public CompletableFuture> onNext() { } return AsyncUtil.whileTrue(() -> inner.onNext().thenApply(innerResult -> { + lastResult = previousResult; previousResult = innerResult; - System.out.println("scan innerResult hasNext:" + innerResult.hasNext()); if (!innerResult.hasNext()) { - System.out.println("cursor noNextReason:" + innerResult.getNoNextReason()); - //if (!isNoRecords() || (isCreateDefaultOnEmpty && streamGrouping.isResultOnEmpty())) { - // the method streamGrouping.finalizeGroup() computes previousCompleteResult and resets the accumulator - partialAggregationResult = streamGrouping.finalizeGroup(); - //} + if (!isNoRecords() || (isCreateDefaultOnEmpty && streamGrouping.isResultOnEmpty())) { + // the method streamGrouping.finalizeGroup() computes previousCompleteResult and resets the accumulator + // (TODO) + partialAggregationResult = streamGrouping.finalizeGroup(); + } return false; } else { final QueryResult queryResult = Objects.requireNonNull(innerResult.get()); - System.out.println("inner result:" + innerResult.get().getMessage()); boolean groupBreak = streamGrouping.apply(queryResult); if (!groupBreak) { // previousValidResult is the last row before group break, it sets the continuation @@ -149,7 +148,7 @@ public CompletableFuture> onNext() { } } else { // stopped in the middle of a group - RecordCursorContinuation currentContinuation = new AggregateCursorContinuation(previousValidResult.getContinuation(), partialAggregationResult); + RecordCursorContinuation currentContinuation = new AggregateCursorContinuation(lastResult.getContinuation(), partialAggregationResult); previousValidResult = previousResult; return RecordCursorResult.withoutNextValue(currentContinuation, Verify.verifyNotNull(previousResult).getNoNextReason()); } @@ -157,8 +156,10 @@ public CompletableFuture> onNext() { }); } + + private boolean isNoRecords() { - return ((previousValidResult == null) && (!Verify.verifyNotNull(previousResult).hasNext())); + return ((previousValidResult == null) && (!Verify.verifyNotNull(previousResult).hasNext()) && (streamGrouping.getPartialAggregationResult() == null)); } @Override diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/StreamGrouping.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/StreamGrouping.java index f0ee0a26ac..cb813f227a 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/StreamGrouping.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/StreamGrouping.java @@ -22,8 +22,6 @@ import com.apple.foundationdb.record.Bindings; import com.apple.foundationdb.record.EvaluationContext; -import com.apple.foundationdb.record.PlanHashable; -import com.apple.foundationdb.record.PlanSerializationContext; import com.apple.foundationdb.record.RecordCursorProto; import com.apple.foundationdb.record.RecordCursorResult; import com.apple.foundationdb.record.provider.foundationdb.FDBRecordStoreBase; @@ -31,7 +29,6 @@ import com.apple.foundationdb.record.query.plan.cascades.values.Accumulator; import com.apple.foundationdb.record.query.plan.cascades.values.AggregateValue; import com.apple.foundationdb.record.query.plan.cascades.values.Value; -import com.apple.foundationdb.record.query.plan.serialization.DefaultPlanSerializationRegistry; import com.google.protobuf.DynamicMessage; import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.Message; @@ -91,10 +88,6 @@ public class StreamGrouping { @Nonnull private final Value completeResultValue; - @Nonnull - private static PlanSerializationContext serializationContext = new PlanSerializationContext(DefaultPlanSerializationRegistry.INSTANCE, - PlanHashable.CURRENT_FOR_CONTINUATION); - /** * Create a new group aggregator. * @@ -220,6 +213,10 @@ public boolean isResultOnEmpty() { @Nullable public RecordCursorProto.PartialAggregationResult getPartialAggregationResult(@Nonnull Message groupingKey) { - return accumulator.getPartialAggregationResult(groupingKey, serializationContext); + return accumulator.getPartialAggregationResult(groupingKey); + } + + public RecordCursorProto.PartialAggregationResult getPartialAggregationResult() { + return accumulator.getPartialAggregationResult((Message)currentGroup); } } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/Accumulator.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/Accumulator.java index 75bd871ce0..c9981e7f72 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/Accumulator.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/Accumulator.java @@ -20,7 +20,6 @@ package com.apple.foundationdb.record.query.plan.cascades.values; -import com.apple.foundationdb.record.PlanSerializationContext; import com.apple.foundationdb.record.RecordCursorProto; import com.google.protobuf.Message; @@ -37,7 +36,7 @@ public interface Accumulator { @Nullable Object finish(); @Nullable - RecordCursorProto.PartialAggregationResult getPartialAggregationResult(Message groupingKey, PlanSerializationContext serializationContext); + RecordCursorProto.PartialAggregationResult getPartialAggregationResult(Message groupingKey); void setInitialState(@Nonnull List accumulatorStates); } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/CountValue.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/CountValue.java index a019b0b395..6fc2ba6e0a 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/CountValue.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/CountValue.java @@ -349,7 +349,7 @@ public Object finish() { @Nullable @Override - public RecordCursorProto.PartialAggregationResult getPartialAggregationResult(Message groupingKey, PlanSerializationContext serializationContext) { + public RecordCursorProto.PartialAggregationResult getPartialAggregationResult(Message groupingKey) { if (state == null) { return null; } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/NumericAggregationValue.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/NumericAggregationValue.java index 987238936b..4548299e99 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/NumericAggregationValue.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/NumericAggregationValue.java @@ -867,7 +867,7 @@ public Object finish() { @Nullable @Override - public RecordCursorProto.PartialAggregationResult getPartialAggregationResult(@Nonnull Message groupingKey, PlanSerializationContext serializationContext) { + public RecordCursorProto.PartialAggregationResult getPartialAggregationResult(@Nonnull Message groupingKey) { if (state == null) { return null; } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/RecordConstructorValue.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/RecordConstructorValue.java index 0cac6ed33e..385d080b46 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/RecordConstructorValue.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/RecordConstructorValue.java @@ -379,12 +379,12 @@ private List buildAccumulators() { @Nullable @Override - public RecordCursorProto.PartialAggregationResult getPartialAggregationResult(Message groupingKey, PlanSerializationContext serializationContext) { + public RecordCursorProto.PartialAggregationResult getPartialAggregationResult(Message groupingKey) { List accumulatorStates = new ArrayList<>(); for (Accumulator accumulator: childAccumulators) { // (TODO): check groupingKeyValues are the same here - if (accumulator.getPartialAggregationResult(groupingKey, serializationContext) != null) { - accumulatorStates.addAll(accumulator.getPartialAggregationResult(groupingKey, serializationContext).getAccumulatorStatesList()); + if (accumulator.getPartialAggregationResult(groupingKey) != null) { + accumulatorStates.addAll(accumulator.getPartialAggregationResult(groupingKey).getAccumulatorStatesList()); } } if (accumulatorStates.isEmpty()) { diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBStreamAggregationTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBStreamAggregationTest.java index 9f182f365b..0d8b266c28 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBStreamAggregationTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBStreamAggregationTest.java @@ -387,9 +387,11 @@ private RecordCursor executePlan(final RecordQueryPlan plan, final return plan.executePlan(recordStore, EvaluationContext.forTypeRepository(typeRepository), null, executeProperties); } else { RecordCursorProto.AggregateCursorContinuation continuationProto = RecordCursorProto.AggregateCursorContinuation.parseFrom(continuation); - System.out.println("partialAggregationResult:" + continuationProto.getPartialAggregationResults()); - System.out.println("plan class:" + plan.getClass()); - return plan.executePlan(recordStore, EvaluationContext.forBindingsAndTypeRepositoryAndPartialAggregationResult(Bindings.EMPTY_BINDINGS, typeRepository, continuationProto.getPartialAggregationResults()), continuationProto.getContinuation().toByteArray(), executeProperties); + if (continuationProto.hasPartialAggregationResults()) { + return plan.executePlan(recordStore, EvaluationContext.forBindingsAndTypeRepositoryAndPartialAggregationResult(Bindings.EMPTY_BINDINGS, typeRepository, continuationProto.getPartialAggregationResults()), continuationProto.getContinuation().toByteArray(), executeProperties); + } else { + return plan.executePlan(recordStore, EvaluationContext.forTypeRepository(typeRepository), continuationProto.getContinuation().toByteArray(), executeProperties); + } } } catch (final Throwable t) { throw Assertions.fail(t); @@ -397,7 +399,6 @@ private RecordCursor executePlan(final RecordQueryPlan plan, final } private RecordCursorContinuation executePlanWithRecordScanLimit(final RecordQueryPlan plan, final int recordScanLimit, byte[] continuation, @Nullable List expectedResult) { - System.out.println("executePlanWithRecordScanLimit called with recordScanLimit:" + recordScanLimit + " expectedResult:" + expectedResult); List queryResults = new LinkedList<>(); RecordCursor currentCursor = executePlan(plan, 0, recordScanLimit, continuation); RecordCursorResult currentCursorResult; @@ -405,13 +406,10 @@ private RecordCursorContinuation executePlanWithRecordScanLimit(final RecordQuer while (true) { currentCursorResult = currentCursor.getNext(); cursorContinuation = currentCursorResult.getContinuation(); - System.out.println("cursor hasNext:" + currentCursorResult.hasNext()); if (!currentCursorResult.hasNext()) { - System.out.println("cursor NoNextReason:" + currentCursorResult.getNoNextReason()); break; } queryResults.add(currentCursorResult.get()); - System.out.println("queryResults size:" + queryResults.size() + " new added:" + currentCursorResult.get().getMessage()); } if (expectedResult == null) { Assertions.assertTrue(queryResults.isEmpty()); @@ -435,7 +433,6 @@ private List executePlanWithRowLimit(final RecordQueryPlan plan, fi } queryResults.add(currentCursorResult.get()); } - System.out.println("getNoNextReason:" + currentCursorResult.getNoNextReason()); if (currentCursorResult.getNoNextReason() == RecordCursor.NoNextReason.SOURCE_EXHAUSTED) { break; } diff --git a/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/ContinuationImpl.java b/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/ContinuationImpl.java index c0bc379969..7503d25d0b 100644 --- a/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/ContinuationImpl.java +++ b/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/ContinuationImpl.java @@ -23,7 +23,6 @@ import com.apple.foundationdb.annotation.API; import com.apple.foundationdb.record.RecordCursorContinuation; -import com.apple.foundationdb.record.planprotos.PartialAggregationResult; import com.apple.foundationdb.relational.api.Continuation; import com.apple.foundationdb.relational.api.exceptions.ErrorCode; import com.apple.foundationdb.relational.api.exceptions.RelationalException; @@ -82,10 +81,6 @@ public byte[] getExecutionState() { } } - public PartialAggregationResult getPartialAggregationResult() { - return proto.getPartialAggregationResult(); - } - @Override public Reason getReason() { if (proto.hasReason()) { @@ -181,16 +176,6 @@ public static Continuation fromRecordCursorContinuation(RecordCursorContinuation return cursorContinuation.isEnd() ? END : new ContinuationImpl(cursorContinuation.toBytes()); } - public static Continuation fromRecordCursorContinuationAndPartialAggregationResult(@Nonnull RecordCursorContinuation cursorContinuation, @Nullable PartialAggregationResult partialAggregationResultProto) { - if (cursorContinuation.isEnd()) { - return END; - } - ContinuationProto.Builder builder = ContinuationProto.newBuilder().setVersion(CURRENT_VERSION); - builder.setExecutionState(ByteString.copyFrom(Objects.requireNonNull(cursorContinuation.toBytes()))) - .setPartialAggregationResult(partialAggregationResultProto); - return new ContinuationImpl(builder.build()); - } - /** * Deserialize and parse a continuation. This would create a continuation from a previously serialized byte array. * @param bytes the serialized continuation diff --git a/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/QueryPlan.java b/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/QueryPlan.java index f00f4c096f..1e09fe533d 100644 --- a/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/QueryPlan.java +++ b/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/QueryPlan.java @@ -238,6 +238,7 @@ public RelationalResultSet executeInternal(@Nonnull final ExecutionContext execu final String schemaName = conn.getSchema(); try (RecordLayerSchema recordLayerSchema = conn.getRecordLayerDatabase().loadSchema(schemaName)) { final var evaluationContext = queryExecutionParameters.getEvaluationContext(); + final var typedEvaluationContext = EvaluationContext.forBindingsAndTypeRepository(evaluationContext.getBindings(), typeRepository); final ContinuationImpl parsedContinuation; try { parsedContinuation = ContinuationImpl.parseContinuation(queryExecutionParameters.getContinuation()); @@ -245,7 +246,6 @@ public RelationalResultSet executeInternal(@Nonnull final ExecutionContext execu executionContext.metricCollector.increment(RelationalMetric.RelationalCount.CONTINUATION_REJECTED); throw ExceptionUtil.toRelationalException(ipbe); } - final var typedEvaluationContext = EvaluationContext.forBindingsAndTypeRepositoryAndPartialAggregationResult(evaluationContext.getBindings(), typeRepository, parsedContinuation.getPartialAggregationResult()); if (queryExecutionParameters.isForExplain()) { return executeExplain(parsedContinuation); From c953df8cfff2886d82d2e17c52a7d70179ab8dfb Mon Sep 17 00:00:00 2001 From: Pengpeng Lu Date: Tue, 11 Mar 2025 15:37:35 -0700 Subject: [PATCH 08/14] add test for avg and bitmap --- .../query/FDBStreamAggregationTest.java | 69 ++++++++++++++++++- 1 file changed, 66 insertions(+), 3 deletions(-) diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBStreamAggregationTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBStreamAggregationTest.java index 0d8b266c28..6c36e3b59d 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBStreamAggregationTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBStreamAggregationTest.java @@ -52,6 +52,7 @@ import com.apple.test.Tags; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; +import com.google.protobuf.ByteString; import com.google.protobuf.Message; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; @@ -315,7 +316,7 @@ void aggregateNoRecordsNoGroupNoAggregate(final boolean useNestedResult, final i } @Test - void aggregateHitScanLimitReached() { + void partialAggregateSum() { try (final var context = openContext()) { openSimpleRecordStore(context, NO_HOOK); @@ -333,15 +334,77 @@ void aggregateHitScanLimitReached() { RecordCursorContinuation continuation2 = executePlanWithRecordScanLimit(plan, 1, continuation1.toBytes(), resultOf("0", 3)); // start the next scan from 5th row, and scans the 5th row (recordScanLimit = 1), return nothing RecordCursorContinuation continuation3 = executePlanWithRecordScanLimit(plan, 1, continuation2.toBytes(), null); - // start the next scan from 6th row, and scans the 6th row (recordScanLimit = 2), return 2nd group aggregated result + // start the next scan from 6th row, and scans the 6th row (recordScanLimit = 2), hit SCAN_LIMIT_REACHED, so return nothing RecordCursorContinuation continuation4 = executePlanWithRecordScanLimit(plan, 1, continuation3.toBytes(), null); - // (TODO): return exhausted, but not result, probably when finish scan x row, needs to return (x-1) to avoid this from happening + // return the aggregated result of the second group RecordCursorContinuation continuation5 = executePlanWithRecordScanLimit(plan, 1, continuation4.toBytes(), resultOf("1", 12)); Assertions.assertEquals(RecordCursorEndContinuation.END, continuation5); } } + @Test + void partialAggregateAvg() { + try (final var context = openContext()) { + openSimpleRecordStore(context, NO_HOOK); + + final var plan = + new AggregationPlanBuilder(recordStore.getRecordMetaData(), "MySimpleRecord") + .withAggregateValue("num_value_2", value -> new NumericAggregationValue.Avg(NumericAggregationValue.PhysicalOperator.AVG_I, value)) + .withGroupCriterion("str_value_indexed") + .build(false); + + // In the testing data, there are 2 groups, each group has 3 rows. + // recordScanLimit = 5: scans 3 rows, and the 4th scan hits SCAN_LIMIT_REACHED + // although the first group contains exactly 3 rows, we don't know we've finished the first group before we get to the 4th row, so nothing is returned, continuation is back to START + RecordCursorContinuation continuation1 = executePlanWithRecordScanLimit(plan, 5, null, null); + // start the next scan from 4th row, and scans the 4th row (recordScanLimit = 1), return the aggregated result of the first group + RecordCursorContinuation continuation2 = executePlanWithRecordScanLimit(plan, 1, continuation1.toBytes(), resultOf("0", 1.0)); + // start the next scan from 5th row, and scans the 5th row (recordScanLimit = 1), return nothing + RecordCursorContinuation continuation3 = executePlanWithRecordScanLimit(plan, 1, continuation2.toBytes(), null); + // start the next scan from 6th row, and scans the 6th row (recordScanLimit = 2), hit SCAN_LIMIT_REACHED, so return nothing + RecordCursorContinuation continuation4 = executePlanWithRecordScanLimit(plan, 1, continuation3.toBytes(), null); + // return the aggregated result of the second group + RecordCursorContinuation continuation5 = executePlanWithRecordScanLimit(plan, 1, continuation4.toBytes(), resultOf("1", 4.0)); + + Assertions.assertEquals(RecordCursorEndContinuation.END, continuation5); + } + } + + @Test + void partialAggregateBitmap() { + try (final var context = openContext()) { + openSimpleRecordStore(context, NO_HOOK); + + final var plan = + new AggregationPlanBuilder(recordStore.getRecordMetaData(), "MySimpleRecord") + .withAggregateValue("num_value_2", value -> new NumericAggregationValue.BitmapConstructAgg(NumericAggregationValue.PhysicalOperator.BITMAP_CONSTRUCT_AGG_I, value)) + .withGroupCriterion("str_value_indexed") + .build(false); + + // In the testing data, there are 2 groups, each group has 3 rows. + // recordScanLimit = 5: scans 3 rows, and the 4th scan hits SCAN_LIMIT_REACHED + // although the first group contains exactly 3 rows, we don't know we've finished the first group before we get to the 4th row, so nothing is returned, continuation is back to START + RecordCursorContinuation continuation1 = executePlanWithRecordScanLimit(plan, 5, null, null); + // start the next scan from 4th row, and scans the 4th row (recordScanLimit = 1), return the aggregated result of the first group + byte[] first = new byte[1250]; + // first[0] = b'00000111 + first[0] = 7; + RecordCursorContinuation continuation2 = executePlanWithRecordScanLimit(plan, 1, continuation1.toBytes(), resultOf("0", ByteString.copyFrom(first))); + // start the next scan from 5th row, and scans the 5th row (recordScanLimit = 1), return nothing + RecordCursorContinuation continuation3 = executePlanWithRecordScanLimit(plan, 1, continuation2.toBytes(), null); + // start the next scan from 6th row, and scans the 6th row (recordScanLimit = 2), hit SCAN_LIMIT_REACHED, so return nothing + RecordCursorContinuation continuation4 = executePlanWithRecordScanLimit(plan, 1, continuation3.toBytes(), null); + // return the aggregated result of the second group + byte[] second = new byte[1250]; + // second[0] = b'00111000 + second[0] = 56; + RecordCursorContinuation continuation5 = executePlanWithRecordScanLimit(plan, 1, continuation4.toBytes(), resultOf("1", ByteString.copyFrom(second))); + + Assertions.assertEquals(RecordCursorEndContinuation.END, continuation5); + } + } + private static Stream provideArguments() { // (boolean, rowLimit) // setting rowLimit = 0 is equivalent to no limit From df09e54fbd91336f529a24c69e1198cbbdad8fdb Mon Sep 17 00:00:00 2001 From: Pengpeng Lu Date: Thu, 13 Mar 2025 12:46:43 -0700 Subject: [PATCH 09/14] save test --- .../recordlayer/query/GroupByQueryTests.java | 44 +++++++++++++++++++ 1 file changed, 44 insertions(+) diff --git a/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/GroupByQueryTests.java b/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/GroupByQueryTests.java index 111e742b35..d243935b54 100644 --- a/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/GroupByQueryTests.java +++ b/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/GroupByQueryTests.java @@ -20,6 +20,8 @@ package com.apple.foundationdb.relational.recordlayer.query; +import com.apple.foundationdb.relational.api.Continuation; +import com.apple.foundationdb.relational.api.Options; import com.apple.foundationdb.relational.api.RelationalResultSet; import com.apple.foundationdb.relational.recordlayer.EmbeddedRelationalExtension; import com.apple.foundationdb.relational.recordlayer.Utils; @@ -33,6 +35,7 @@ import org.junit.jupiter.api.extension.RegisterExtension; import java.net.URI; +import java.util.Base64; import static com.apple.foundationdb.relational.recordlayer.query.QueryTestUtils.insertT1Record; @@ -46,6 +49,47 @@ public GroupByQueryTests() { Utils.enableCascadesDebugger(); } + @Test + void groupByWithScanLimit() throws Exception { + final String schemaTemplate = + "CREATE TABLE T1(pk bigint, a bigint, b bigint, c bigint, PRIMARY KEY(pk))" + + "CREATE INDEX idx1 as select a, b, c from t1 order by a, b, c"; + try (var ddl = Ddl.builder().database(URI.create("/TEST/QT")).relationalExtension(relationalExtension).schemaTemplate(schemaTemplate).build()) { + try (var conn = ddl.setSchemaAndGetConnection()) { + conn.setOption(Options.Name.EXECUTION_SCANNED_ROWS_LIMIT, 2); + try (var statement = conn.createStatement()) { + insertT1Record(statement, 2, 1, 1, 20); + insertT1Record(statement, 3, 1, 2, 5); + insertT1Record(statement, 4, 1, 2, 15); + insertT1Record(statement, 5, 1, 2, 5); + insertT1Record(statement, 6, 2, 1, 10); + insertT1Record(statement, 7, 2, 1, 40); + insertT1Record(statement, 8, 2, 1, 20); + insertT1Record(statement, 9, 2, 1, 90); + + String query = "SELECT a AS OK, b, MAX(c) FROM T1 GROUP BY a, b"; + Continuation continuation = null; + // scan 1st and 2nd rows + Assertions.assertTrue(statement.execute(query), "Did not return a result set from a select statement!"); + try (final RelationalResultSet resultSet = statement.getResultSet()) { + ResultSetAssert.assertThat(resultSet).hasNextRow() + .isRowExactly(1L, 1L, 20L) + .hasNoNextRow(); + continuation = resultSet.getContinuation(); + } + // scan 3rd and 4th rows + String postfix = " WITH CONTINUATION B64'" + Base64.getEncoder().encodeToString(continuation.serialize()) + "'"; + Assertions.assertTrue(statement.execute(query + postfix), "Did not return a result set from a select statement!"); + try (final RelationalResultSet resultSet = statement.getResultSet()) { + ResultSetAssert.assertThat(resultSet).hasNextRow() + .isRowExactly(1L, 2L, 15); + continuation = resultSet.getContinuation(); + } + } + } + } + } + @Test void groupByClauseWithPredicateWorks() throws Exception { final String schemaTemplate = From 195b68cd2b0f6dd62bc7952b1d1dd0b9aa93e6df Mon Sep 17 00:00:00 2001 From: Pengpeng Lu Date: Mon, 17 Mar 2025 15:13:05 -0700 Subject: [PATCH 10/14] green --- .../cursors/aggregate/AggregateCursor.java | 44 ++++++++++++++++--- .../cursors/aggregate/StreamGrouping.java | 9 ++-- .../values/NumericAggregationValue.java | 2 +- .../RecordQueryStreamingAggregationPlan.java | 10 +++-- .../query/FDBStreamAggregationTest.java | 15 +------ .../recordlayer/RecordLayerIterator.java | 1 + .../recordlayer/query/GroupByQueryTests.java | 34 ++++++++++++-- 7 files changed, 83 insertions(+), 32 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/AggregateCursor.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/AggregateCursor.java index a7245b6d82..658825bdc0 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/AggregateCursor.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/AggregateCursor.java @@ -90,9 +90,8 @@ public CompletableFuture> onNext() { previousResult = innerResult; if (!innerResult.hasNext()) { if (!isNoRecords() || (isCreateDefaultOnEmpty && streamGrouping.isResultOnEmpty())) { - // the method streamGrouping.finalizeGroup() computes previousCompleteResult and resets the accumulator - // (TODO) - partialAggregationResult = streamGrouping.finalizeGroup(); + // the method streamGrouping.finalizeGroup() computes previousCompleteResult and resets the accumulator + partialAggregationResult = streamGrouping.finalizeGroup(); } return false; } else { @@ -157,7 +156,6 @@ public CompletableFuture> onNext() { } - private boolean isNoRecords() { return ((previousValidResult == null) && (!Verify.verifyNotNull(previousResult).hasNext()) && (streamGrouping.getPartialAggregationResult() == null)); } @@ -186,7 +184,7 @@ public boolean accept(@Nonnull RecordCursorVisitor visitor) { return visitor.visitLeave(this); } - private static class AggregateCursorContinuation implements RecordCursorContinuation { + public static class AggregateCursorContinuation implements RecordCursorContinuation { @Nullable private final ByteString innerContinuation; @@ -208,10 +206,14 @@ public AggregateCursorContinuation(@Nonnull RecordCursorContinuation other, @Nul this.partialAggregationResult = partialAggregationResult; } - public AggregateCursorContinuation(@Nullable byte[] innerContinuation, boolean isEnd) { + public AggregateCursorContinuation(@Nullable byte[] innerContinuation, boolean isEnd, @Nullable RecordCursorProto.PartialAggregationResult partialAggregationResult) { this.isEnd = isEnd; this.innerContinuation = innerContinuation == null ? null : ByteString.copyFrom(innerContinuation); - this.partialAggregationResult = null; + this.partialAggregationResult = partialAggregationResult; + } + + public AggregateCursorContinuation(@Nullable byte[] innerContinuation, boolean isEnd) { + this(innerContinuation, isEnd, null); } @Nonnull @@ -238,6 +240,16 @@ public boolean isEnd() { return isEnd; } + @Nullable + public byte[] getInnerContinuation() { + return innerContinuation == null ? null : innerContinuation.toByteArray(); + } + + @Nullable + public RecordCursorProto.PartialAggregationResult getPartialAggregationResult() { + return partialAggregationResult; + } + @Nonnull private RecordCursorProto.AggregateCursorContinuation toProto() { if (cachedProto == null) { @@ -252,5 +264,23 @@ private RecordCursorProto.AggregateCursorContinuation toProto() { } return cachedProto; } + + public static AggregateCursorContinuation fromRawBytes(@Nullable byte[] rawBytes) { + if (rawBytes == null) { + return new AggregateCursorContinuation(null, true); + } + try { + RecordCursorProto.AggregateCursorContinuation continuationProto = RecordCursorProto.AggregateCursorContinuation.parseFrom(rawBytes); + if (!continuationProto.hasContinuation()) { + return new AggregateCursorContinuation(null, true); + } else if (continuationProto.hasPartialAggregationResults()) { + return new AggregateCursorContinuation(continuationProto.getContinuation().toByteArray(), false, continuationProto.getPartialAggregationResults()); + } else { + return new AggregateCursorContinuation(continuationProto.getContinuation().toByteArray(), false); + } + } catch (final Exception ex) { + throw new RuntimeException(ex); + } + } } } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/StreamGrouping.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/StreamGrouping.java index cb813f227a..689c96a0e0 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/StreamGrouping.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/aggregate/StreamGrouping.java @@ -107,14 +107,15 @@ public StreamGrouping(@Nullable final Value groupingKeyValue, @Nonnull final CorrelationIdentifier aggregateAlias, @Nonnull final FDBRecordStoreBase store, @Nonnull final EvaluationContext context, - @Nonnull final CorrelationIdentifier alias) { + @Nonnull final CorrelationIdentifier alias, + @Nullable final RecordCursorProto.PartialAggregationResult partialAggregationResult) { this.groupingKeyValue = groupingKeyValue; this.aggregateValue = aggregateValue; this.accumulator = aggregateValue.createAccumulator(context.getTypeRepository()); - if (context.getPartialAggregationResult() != null) { - this.accumulator.setInitialState(context.getPartialAggregationResult().getAccumulatorStatesList()); + if (partialAggregationResult != null) { + this.accumulator.setInitialState(partialAggregationResult.getAccumulatorStatesList()); try { - this.currentGroup = DynamicMessage.parseFrom(context.getTypeRepository().newMessageBuilder(groupingKeyValue.getResultType()).getDescriptorForType(), context.getPartialAggregationResult().getGroupKey().toByteArray()); + this.currentGroup = DynamicMessage.parseFrom(context.getTypeRepository().newMessageBuilder(groupingKeyValue.getResultType()).getDescriptorForType(), partialAggregationResult.getGroupKey().toByteArray()); } catch (InvalidProtocolBufferException e) { throw new RuntimeException(e); } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/NumericAggregationValue.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/NumericAggregationValue.java index 4548299e99..d5ad442f94 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/NumericAggregationValue.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/NumericAggregationValue.java @@ -953,5 +953,5 @@ public void setInitialState(@Nonnull List ac break; } } - } + } } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/plans/RecordQueryStreamingAggregationPlan.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/plans/RecordQueryStreamingAggregationPlan.java index 323be325ef..47dd499858 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/plans/RecordQueryStreamingAggregationPlan.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/plans/RecordQueryStreamingAggregationPlan.java @@ -137,7 +137,8 @@ public RecordCursor executePlan(@Nonnull FDBRec @Nonnull EvaluationContext context, @Nullable byte[] continuation, @Nonnull ExecuteProperties executeProperties) { - final var innerCursor = getInnerPlan().executePlan(store, context, continuation, executeProperties.clearSkipAndLimit()); + AggregateCursor.AggregateCursorContinuation aggregateCursorContinuation = AggregateCursor.AggregateCursorContinuation.fromRawBytes(continuation); + final var innerCursor = getInnerPlan().executePlan(store, context, aggregateCursorContinuation.getInnerContinuation(), executeProperties.clearSkipAndLimit()); final var streamGrouping = new StreamGrouping<>(groupingKeyValue, @@ -147,8 +148,11 @@ public RecordCursor executePlan(@Nonnull FDBRec aggregateAlias, (FDBRecordStoreBase)store, context, - inner.getAlias()); - return new AggregateCursor<>(innerCursor, streamGrouping, isCreateDefaultOnEmpty, continuation) + inner.getAlias(), + aggregateCursorContinuation.getPartialAggregationResult() + ); + + return new AggregateCursor<>(innerCursor, streamGrouping, isCreateDefaultOnEmpty, aggregateCursorContinuation.getInnerContinuation()) .skipThenLimit(executeProperties.getSkip(), executeProperties.getReturnedRowLimit()); } diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBStreamAggregationTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBStreamAggregationTest.java index 6c36e3b59d..9b7109aadd 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBStreamAggregationTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBStreamAggregationTest.java @@ -445,20 +445,7 @@ private RecordCursor executePlan(final RecordQueryPlan plan, final } ExecuteProperties executeProperties = ExecuteProperties.SERIAL_EXECUTE; executeProperties = executeProperties.setReturnedRowLimit(rowLimit).setState(executeState); - try { - if (continuation == null) { - return plan.executePlan(recordStore, EvaluationContext.forTypeRepository(typeRepository), null, executeProperties); - } else { - RecordCursorProto.AggregateCursorContinuation continuationProto = RecordCursorProto.AggregateCursorContinuation.parseFrom(continuation); - if (continuationProto.hasPartialAggregationResults()) { - return plan.executePlan(recordStore, EvaluationContext.forBindingsAndTypeRepositoryAndPartialAggregationResult(Bindings.EMPTY_BINDINGS, typeRepository, continuationProto.getPartialAggregationResults()), continuationProto.getContinuation().toByteArray(), executeProperties); - } else { - return plan.executePlan(recordStore, EvaluationContext.forTypeRepository(typeRepository), continuationProto.getContinuation().toByteArray(), executeProperties); - } - } - } catch (final Throwable t) { - throw Assertions.fail(t); - } + return plan.executePlan(recordStore, EvaluationContext.forTypeRepository(typeRepository), continuation, executeProperties); } private RecordCursorContinuation executePlanWithRecordScanLimit(final RecordQueryPlan plan, final int recordScanLimit, byte[] continuation, @Nullable List expectedResult) { diff --git a/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/RecordLayerIterator.java b/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/RecordLayerIterator.java index ed99e1130b..65deff5b79 100644 --- a/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/RecordLayerIterator.java +++ b/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/RecordLayerIterator.java @@ -83,6 +83,7 @@ public boolean hasNext() { private void fetchNextResult() { if (result != null) { + continuation = ContinuationImpl.fromRecordCursorContinuation(result.getContinuation()); return; } result = recordCursor.getNext(); diff --git a/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/GroupByQueryTests.java b/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/GroupByQueryTests.java index d243935b54..b6e4f5b06b 100644 --- a/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/GroupByQueryTests.java +++ b/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/GroupByQueryTests.java @@ -20,9 +20,11 @@ package com.apple.foundationdb.relational.recordlayer.query; +import com.apple.foundationdb.record.cursors.aggregate.AggregateCursor; import com.apple.foundationdb.relational.api.Continuation; import com.apple.foundationdb.relational.api.Options; import com.apple.foundationdb.relational.api.RelationalResultSet; +import com.apple.foundationdb.relational.recordlayer.ContinuationImpl; import com.apple.foundationdb.relational.recordlayer.EmbeddedRelationalExtension; import com.apple.foundationdb.relational.recordlayer.Utils; import com.apple.foundationdb.relational.utils.Ddl; @@ -69,7 +71,7 @@ void groupByWithScanLimit() throws Exception { String query = "SELECT a AS OK, b, MAX(c) FROM T1 GROUP BY a, b"; Continuation continuation = null; - // scan 1st and 2nd rows + // scan pk = 2 and pk = 3 and hit SCAN_LIMIT_REACHED Assertions.assertTrue(statement.execute(query), "Did not return a result set from a select statement!"); try (final RelationalResultSet resultSet = statement.getResultSet()) { ResultSetAssert.assertThat(resultSet).hasNextRow() @@ -77,14 +79,40 @@ void groupByWithScanLimit() throws Exception { .hasNoNextRow(); continuation = resultSet.getContinuation(); } - // scan 3rd and 4th rows + // scan pk = 5 and pk = 4 rows, hit SCAN_LIMIT_REACHED String postfix = " WITH CONTINUATION B64'" + Base64.getEncoder().encodeToString(continuation.serialize()) + "'"; Assertions.assertTrue(statement.execute(query + postfix), "Did not return a result set from a select statement!"); + try (final RelationalResultSet resultSet = statement.getResultSet()) { + ResultSetAssert.assertThat(resultSet) + .hasNoNextRow(); + continuation = resultSet.getContinuation(); + } + // scan pk = 6 and pk = 8 rows, hit SCAN_LIMIT_REACHED + postfix = " WITH CONTINUATION B64'" + Base64.getEncoder().encodeToString(continuation.serialize()) + "'"; + Assertions.assertTrue(statement.execute(query + postfix), "Did not return a result set from a select statement!"); + try (final RelationalResultSet resultSet = statement.getResultSet()) { + ResultSetAssert.assertThat(resultSet).hasNextRow() + .isRowExactly(1L, 2L, 15L) + .hasNoNextRow(); + continuation = resultSet.getContinuation(); + } + // scan pk = 7 and pk = 9 rows, hit SCAN_LIMIT_REACHED + postfix = " WITH CONTINUATION B64'" + Base64.getEncoder().encodeToString(continuation.serialize()) + "'"; + Assertions.assertTrue(statement.execute(query + postfix), "Did not return a result set from a select statement!"); + try (final RelationalResultSet resultSet = statement.getResultSet()) { + ResultSetAssert.assertThat(resultSet).hasNoNextRow(); + continuation = resultSet.getContinuation(); + } + // hit SOURCE_EXHAUSTED + postfix = " WITH CONTINUATION B64'" + Base64.getEncoder().encodeToString(continuation.serialize()) + "'"; + Assertions.assertTrue(statement.execute(query + postfix), "Did not return a result set from a select statement!"); try (final RelationalResultSet resultSet = statement.getResultSet()) { ResultSetAssert.assertThat(resultSet).hasNextRow() - .isRowExactly(1L, 2L, 15); + .isRowExactly(2L, 1L, 90L) + .hasNoNextRow(); continuation = resultSet.getContinuation(); } + Assertions.assertTrue(continuation.atEnd()); } } } From 97a3cdcc80a1b419d6c4a58bf59bc8c8327034e3 Mon Sep 17 00:00:00 2001 From: Pengpeng Lu Date: Mon, 17 Mar 2025 15:28:36 -0700 Subject: [PATCH 11/14] clean up --- .../foundationdb/record/EvaluationContext.java | 18 ------------------ .../query/plan/cascades/values/CountValue.java | 5 +++-- .../values/NumericAggregationValue.java | 6 ++++-- .../values/RecordConstructorValue.java | 1 - .../src/main/proto/record_cursor.proto | 3 +-- .../src/main/proto/record_query_plan.proto | 2 +- .../query/FDBStreamAggregationTest.java | 2 -- .../recordlayer/query/QueryPlan.java | 1 - .../src/main/proto/continuation.proto | 0 9 files changed, 9 insertions(+), 29 deletions(-) rename {fdb-record-layer-core => fdb-relational-core}/src/main/proto/continuation.proto (100%) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/EvaluationContext.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/EvaluationContext.java index 5bde91e1b5..71ef331cdc 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/EvaluationContext.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/EvaluationContext.java @@ -45,9 +45,6 @@ public class EvaluationContext { @Nonnull private final TypeRepository typeRepository; - @Nullable - private final RecordCursorProto.PartialAggregationResult partialAggregationResultProto; - public static final EvaluationContext EMPTY = new EvaluationContext(Bindings.EMPTY_BINDINGS, TypeRepository.EMPTY_SCHEMA); /** @@ -60,13 +57,8 @@ public static EvaluationContext empty() { } private EvaluationContext(@Nonnull Bindings bindings, @Nonnull TypeRepository typeRepository) { - this(bindings, typeRepository, null); - } - - private EvaluationContext(@Nonnull Bindings bindings, @Nonnull TypeRepository typeRepository, @Nullable RecordCursorProto.PartialAggregationResult proto) { this.bindings = bindings; this.typeRepository = typeRepository; - this.partialAggregationResultProto = proto; } /** @@ -97,11 +89,6 @@ public static EvaluationContext forTypeRepository(@Nonnull TypeRepository typeRe return new EvaluationContext(Bindings.EMPTY_BINDINGS, typeRepository); } - @Nonnull - public static EvaluationContext forBindingsAndTypeRepositoryAndPartialAggregationResult(@Nonnull Bindings bindings, @Nonnull TypeRepository typeRepository, @Nullable RecordCursorProto.PartialAggregationResult proto) { - return new EvaluationContext(bindings, typeRepository, proto); - } - /** * Create a new EvaluationContext with a single binding. * @@ -114,11 +101,6 @@ public static EvaluationContext forBinding(@Nonnull String bindingName, @Nullabl return new EvaluationContext(Bindings.newBuilder().set(bindingName, value).build(), TypeRepository.EMPTY_SCHEMA); } - @Nullable - public RecordCursorProto.PartialAggregationResult getPartialAggregationResult() { - return partialAggregationResultProto; - } - /** * Retrieve the mapping from parameter names to values associated with * this context. diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/CountValue.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/CountValue.java index 6fc2ba6e0a..0ed9e05418 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/CountValue.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/CountValue.java @@ -363,8 +363,9 @@ public RecordCursorProto.PartialAggregationResult getPartialAggregationResult(Me @Override public void setInitialState(@Nonnull List accumulatorStates) { - // check physical operator name are the same - // check this.state == null + Verify.verify(this.state == null); + Verify.verify(accumulatorStates.size() == 1); + Verify.verify(physicalOperator.name().equals(accumulatorStates.get(0).getPhysicalOperatorName())); this.state = Long.parseLong(accumulatorStates.get(0).getState(0)); } } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/NumericAggregationValue.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/NumericAggregationValue.java index d5ad442f94..174df7b9f9 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/NumericAggregationValue.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/NumericAggregationValue.java @@ -910,8 +910,10 @@ public RecordCursorProto.PartialAggregationResult getPartialAggregationResult(@N @Override public void setInitialState(@Nonnull List accumulatorStates) { - // check physical operator name are the same - // check this.state == null + Verify.verify(state == null); + Verify.verify(accumulatorStates.size() == 1); + Verify.verify(physicalOperator.name().equals(accumulatorStates.get(0).getPhysicalOperatorName())); + switch (physicalOperator) { case SUM_I: case MAX_I: diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/RecordConstructorValue.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/RecordConstructorValue.java index 385d080b46..bd38ebb18c 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/RecordConstructorValue.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/RecordConstructorValue.java @@ -382,7 +382,6 @@ private List buildAccumulators() { public RecordCursorProto.PartialAggregationResult getPartialAggregationResult(Message groupingKey) { List accumulatorStates = new ArrayList<>(); for (Accumulator accumulator: childAccumulators) { - // (TODO): check groupingKeyValues are the same here if (accumulator.getPartialAggregationResult(groupingKey) != null) { accumulatorStates.addAll(accumulator.getPartialAggregationResult(groupingKey).getAccumulatorStatesList()); } diff --git a/fdb-record-layer-core/src/main/proto/record_cursor.proto b/fdb-record-layer-core/src/main/proto/record_cursor.proto index 4d122d92e9..8f96f409ab 100644 --- a/fdb-record-layer-core/src/main/proto/record_cursor.proto +++ b/fdb-record-layer-core/src/main/proto/record_cursor.proto @@ -140,8 +140,7 @@ message AggregateCursorContinuation { message PartialAggregationResult { optional bytes group_key = 1; - optional string group_key_alias = 2; - repeated AccumulatorState accumulator_states = 3; + repeated AccumulatorState accumulator_states = 2; } message AccumulatorState { diff --git a/fdb-record-layer-core/src/main/proto/record_query_plan.proto b/fdb-record-layer-core/src/main/proto/record_query_plan.proto index f27a0f85e6..4e93153c64 100644 --- a/fdb-record-layer-core/src/main/proto/record_query_plan.proto +++ b/fdb-record-layer-core/src/main/proto/record_query_plan.proto @@ -1781,4 +1781,4 @@ message PRecursiveUnionQueryPlan { optional PPhysicalQuantifier recursiveStateQuantifier = 2; optional string initialTempTableAlias = 3; optional string recursiveTempTableAlias = 4; -} +} \ No newline at end of file diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBStreamAggregationTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBStreamAggregationTest.java index 9b7109aadd..dc36ea0e8f 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBStreamAggregationTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBStreamAggregationTest.java @@ -20,7 +20,6 @@ package com.apple.foundationdb.record.provider.foundationdb.query; -import com.apple.foundationdb.record.Bindings; import com.apple.foundationdb.record.ByteScanLimiterFactory; import com.apple.foundationdb.record.EvaluationContext; import com.apple.foundationdb.record.ExecuteProperties; @@ -28,7 +27,6 @@ import com.apple.foundationdb.record.RecordCursor; import com.apple.foundationdb.record.RecordCursorContinuation; import com.apple.foundationdb.record.RecordCursorEndContinuation; -import com.apple.foundationdb.record.RecordCursorProto; import com.apple.foundationdb.record.RecordCursorResult; import com.apple.foundationdb.record.RecordMetaData; import com.apple.foundationdb.record.RecordScanLimiterFactory; diff --git a/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/QueryPlan.java b/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/QueryPlan.java index 1e09fe533d..d196162add 100644 --- a/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/QueryPlan.java +++ b/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/QueryPlan.java @@ -246,7 +246,6 @@ public RelationalResultSet executeInternal(@Nonnull final ExecutionContext execu executionContext.metricCollector.increment(RelationalMetric.RelationalCount.CONTINUATION_REJECTED); throw ExceptionUtil.toRelationalException(ipbe); } - if (queryExecutionParameters.isForExplain()) { return executeExplain(parsedContinuation); } else { diff --git a/fdb-record-layer-core/src/main/proto/continuation.proto b/fdb-relational-core/src/main/proto/continuation.proto similarity index 100% rename from fdb-record-layer-core/src/main/proto/continuation.proto rename to fdb-relational-core/src/main/proto/continuation.proto From 4810d220844120168a75bed77851fd8aa430aec2 Mon Sep 17 00:00:00 2001 From: Pengpeng Lu Date: Mon, 17 Mar 2025 15:30:01 -0700 Subject: [PATCH 12/14] remove dent --- .../record/provider/foundationdb/KeyValueCursorBase.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/KeyValueCursorBase.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/KeyValueCursorBase.java index 57fce4f8ac..a389e7b4e5 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/KeyValueCursorBase.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/KeyValueCursorBase.java @@ -341,9 +341,7 @@ public T setHigh(@Nonnull byte[] highBytes, @Nonnull EndpointType highEndpoint) */ protected int calculatePrefixLength() { int prefixLength = subspace.pack().length; - while ((prefixLength < lowBytes.length) && - (prefixLength < highBytes.length) && - (lowBytes[prefixLength] == highBytes[prefixLength])) { + while ((prefixLength < lowBytes.length) && (prefixLength < highBytes.length) && (lowBytes[prefixLength] == highBytes[prefixLength])) { prefixLength++; } return prefixLength; From 0d2d050d614a6beef47c8b37ac1537e8acd56e67 Mon Sep 17 00:00:00 2001 From: Pengpeng Lu Date: Mon, 17 Mar 2025 16:55:58 -0700 Subject: [PATCH 13/14] revert format change --- .../record/provider/foundationdb/KeyValueCursorBase.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/KeyValueCursorBase.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/KeyValueCursorBase.java index a389e7b4e5..57fce4f8ac 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/KeyValueCursorBase.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/KeyValueCursorBase.java @@ -341,7 +341,9 @@ public T setHigh(@Nonnull byte[] highBytes, @Nonnull EndpointType highEndpoint) */ protected int calculatePrefixLength() { int prefixLength = subspace.pack().length; - while ((prefixLength < lowBytes.length) && (prefixLength < highBytes.length) && (lowBytes[prefixLength] == highBytes[prefixLength])) { + while ((prefixLength < lowBytes.length) && + (prefixLength < highBytes.length) && + (lowBytes[prefixLength] == highBytes[prefixLength])) { prefixLength++; } return prefixLength; From 9c4c50f37a3a6fe7cab422644d58037347cf3231 Mon Sep 17 00:00:00 2001 From: Pengpeng Lu Date: Mon, 17 Mar 2025 23:22:58 -0700 Subject: [PATCH 14/14] checkstyle --- .../relational/recordlayer/query/GroupByQueryTests.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/GroupByQueryTests.java b/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/GroupByQueryTests.java index b6e4f5b06b..6ea4815479 100644 --- a/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/GroupByQueryTests.java +++ b/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/GroupByQueryTests.java @@ -20,11 +20,9 @@ package com.apple.foundationdb.relational.recordlayer.query; -import com.apple.foundationdb.record.cursors.aggregate.AggregateCursor; import com.apple.foundationdb.relational.api.Continuation; import com.apple.foundationdb.relational.api.Options; import com.apple.foundationdb.relational.api.RelationalResultSet; -import com.apple.foundationdb.relational.recordlayer.ContinuationImpl; import com.apple.foundationdb.relational.recordlayer.EmbeddedRelationalExtension; import com.apple.foundationdb.relational.recordlayer.Utils; import com.apple.foundationdb.relational.utils.Ddl;