Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -116,10 +116,12 @@ public StreamGrouping(@Nullable final Value groupingKeyValue,
this.accumulator = aggregateValue.createAccumulatorWithInitialState(context.getTypeRepository(), null);
} else {
this.accumulator = aggregateValue.createAccumulatorWithInitialState(context.getTypeRepository(), partialAggregationResult.getAccumulatorStatesList());
try {
this.currentGroup = DynamicMessage.parseFrom(context.getTypeRepository().newMessageBuilder(groupingKeyValue.getResultType()).getDescriptorForType(), partialAggregationResult.getGroupKey().toByteArray());
} catch (InvalidProtocolBufferException e) {
throw new RuntimeException(e);
if (groupingKeyValue != null) {
try {
this.currentGroup = DynamicMessage.parseFrom(context.getTypeRepository().newMessageBuilder(groupingKeyValue.getResultType()).getDescriptorForType(), partialAggregationResult.getGroupKey().toByteArray());
} catch (InvalidProtocolBufferException e) {
throw new RuntimeException(e);
}
}
}
this.store = store;
Expand Down Expand Up @@ -212,16 +214,18 @@ private Object evalGroupingKey(@Nullable final Object currentObject) {

@Nullable
public RecordCursorProto.PartialAggregationResult getPartialAggregationResult() {
if (currentGroup == null) {
if (groupingKeyValue != null && currentGroup == null) {
return null;
}
List<RecordCursorProto.AccumulatorState> accumulatorStates = accumulator.getAccumulatorStates();
if (accumulatorStates.isEmpty()) {
return null;
}
return RecordCursorProto.PartialAggregationResult.newBuilder()
.setGroupKey(Objects.requireNonNull((Message)currentGroup).toByteString())
.addAllAccumulatorStates(accumulatorStates)
.build();
final RecordCursorProto.PartialAggregationResult.Builder builder = RecordCursorProto.PartialAggregationResult.newBuilder()
.addAllAccumulatorStates(accumulatorStates);
if (groupingKeyValue != null) {
builder.setGroupKey(Objects.requireNonNull((Message)currentGroup).toByteString());
}
return builder.build();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,62 @@
#
# aggregate-ungrouped-continuation.yamsql
#
# This source file is part of the FoundationDB open source project
#
# Copyright 2021-2024 Apple Inc. and the FoundationDB project authors
#
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
# You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.

# Regression test for aggregate queries without explicit GROUP BY
# that use continuations (maxRows). Previously, resuming from a
# continuation with no grouping key caused a NullPointerException
# in StreamGrouping when groupingKeyValue was null.

---
schema_template:
create table t1(id bigint, col1 bigint, col2 bigint, primary key(id))
create index mv1 as select sum(col2) from t1 group by col1
---
setup:
steps:
- query: INSERT INTO T1
VALUES (1, 10, 1),
(2, 10, 2),
(3, 10, 3),
(4, 20, 4),
(5, 20, 5),
(6, 20, 6)
---
test_block:
name: aggregate-ungrouped-continuation
tests:
-
# Aggregate without GROUP BY, no continuation needed (single result row)
- query: select sum(col2) from t1
- result: [{!l 21}]
-
# Aggregate without GROUP BY with maxRows=1 to force continuation.
# This triggered a NullPointerException before the fix because
# getPartialAggregationResult tried to serialize a null currentGroup.
- query: select sum(col2) from t1
- maxRows: 1
- result: [{!l 21}]
- result: []
-
# Aggregate with GROUP BY and maxRows=1 to verify continuations
# still work correctly with grouping keys present.
- query: select col1, sum(col2) from t1 group by col1
- maxRows: 1
- result: [{!l 10, !l 6}]
- result: [{!l 20, !l 15}]
- result: []
Loading