Skip to content

Conversation

@DHRUV6029
Copy link

@DHRUV6029 DHRUV6029 commented Dec 21, 2025

Description

Adds a new SQL aggregation function array_union_sum that combines multiple arrays by summing values at corresponding indices. This function is analogous to map_union_sum but operates on arrays instead of maps.

Example usage:

-- Basic usage                                          
SELECT array_union_sum(arr) FROM (                      
    VALUES (ARRAY[1, 2, 3]),                            
           (ARRAY[4, 5, 6]),                            
           (ARRAY[10, 20, 30])                          
) AS t(arr);                                            
-- Returns: [15, 27, 39]                                
                                                        
-- With GROUP BY                                        
SELECT category, array_union_sum(values)                
FROM metrics                                            
GROUP BY category;   

Implementation details:

  • ArrayUnionSumAggregation.java - Main aggregation function registration and input/combine/output methods
  • ArrayUnionSumResult.java - Result container with efficient union-sum logic using lazy evaluation
  • ArrayUnionSumState.java - State interface for accumulator
  • ArrayUnionSumStateFactory.java - Creates single and grouped states with type-specific adders
  • ArrayUnionSumStateSerializer.java - Handles serialization for distributed execution

Motivation and Context

Users often need to aggregate arrays element-wise, such as:

  • Combining metric arrays across time periods
  • Aggregating feature vectors
  • Summing histogram buckets stored as arrays

Currently, this requires complex workarounds using unnest, zip, and manual reconstruction. The array_union_sum function provides a clean, efficient solution similar to the existing map_union_sum function.

Impact

Public API changes:

  • Adds new SQL function: array_union_sum(array) -> array where T is a non-decimal numeric type

Behavior:

  • Result array length is the maximum length across all input arrays
  • Missing elements (shorter arrays) are treated as 0
  • NULL values within arrays are coalesced to 0
  • NULL arrays are skipped
  • Empty arrays are handled correctly

Supported types:

  • BIGINT, INTEGER, SMALLINT, TINYINT (exact numeric)
  • DOUBLE, REAL (floating point)

Performance:

  • Uses lazy evaluation to avoid copying the first input array
  • Memory-efficient state management with proper size tracking
  • Follows the same patterns as map_union_sum for consistency

Test Plan

  • Unit tests in TestArrayUnionSumResult.java (14 test cases):
    • Basic union sum with multiple arrays
    • NULL value handling (single NULL, both NULLs at same position)
    • Different length arrays
    • DOUBLE type with precision verification
    • REAL type with precision verification
    • NULL handling for DOUBLE and REAL types
    • State factory tests (single and grouped)
    • State serializer tests (serialize/deserialize round-trip)
    • Retained size tracking
    • Element type verification
  • Integration tests in AbstractTestQueries.java:
    • Basic aggregation queries
    • GROUP BY queries
    • Edge cases (empty arrays, NULL arrays)
  • Manual testing with Presto CLI against memory connector
  • All existing tests pass

Contributor checklist

Release Notes

== RELEASE NOTES ==

General

  • Add array_union_sum aggregation function that combines arrays by summing values at corresponding indices. Supports all non-decimal numeric types (BIGINT, INTEGER, DOUBLE, REAL). The result array length is the maximum of all input arrays, with missing elements treated as 0 and NULL values coalesced to 0.

@sourcery-ai
Copy link
Contributor

sourcery-ai bot commented Dec 21, 2025

Reviewer's Guide

Implements a new array_union_sum SQL aggregation function for non-decimal numeric arrays, including execution engine support, state management, serialization, registration in the built-in function set, tests, and documentation/release notes updates.

Sequence diagram for array_union_sum aggregation execution

sequenceDiagram
    actor User
    participant Parser as SqlParser
    participant Planner as QueryPlanner
    participant Engine as ExecutionEngine
    participant Acc as ArrayUnionSumAccumulator
    participant State as ArrayUnionSumState
    participant Result as ArrayUnionSumResult

    User->>Parser: Submit SELECT array_union_sum(arr) ...
    Parser->>Planner: ParsedQuery(plan with array_union_sum)
    Planner->>Engine: PhysicalPlan(using ArrayUnionSumAggregation)

    loop For each input page
        Engine->>Acc: addInput(page)
        loop For each row with array value
            Acc->>+State: input(elementType, state, arrayBlock)
            alt state.get() is null (first array)
                State-->>State: create ArrayUnionSumResult.create(elementType, adder, arrayBlock)
                State-->>Acc: updated state with SingleArrayBlock
            else subsequent arrays
                State->>Result: get()
                Result->>Result: unionSum(arrayBlock)
                Result-->>State: new AccumulatedValues
            end
        end
    end

    note over Engine,Acc: Partial aggregation results may be combined across threads/nodes

    Engine->>Acc: combine(partialState1, partialState2)
    Acc->>State: combine(state, otherState)
    State->>Result: get()
    Result->>Result: unionSum(otherState.get())
    Result-->>State: merged AccumulatedValues

    Engine->>Acc: evaluateFinal()
    Acc->>State: output(state, blockBuilder)
    alt state.get() is null
        State-->>Engine: appendNull()
    else non null
        State->>Result: get()
        Result->>Result: serialize(outBlockBuilder)
        Result-->>Engine: final array block
    end

    Engine-->>User: QueryResult(with aggregated array_union_sum column)
Loading

Class diagram for array_union_sum aggregation components

classDiagram
    class ArrayUnionSumAggregation {
        +String NAME
        +ArrayUnionSumAggregation()
        +String getDescription()
        +BuiltInAggregationFunctionImplementation specialize(BoundVariables boundVariables, int arity, FunctionAndTypeManager functionAndTypeManager)
        -static BuiltInAggregationFunctionImplementation generateAggregation(Type elementType, ArrayType outputType)
        -static List~ParameterMetadata~ createInputParameterMetadata(Type inputType)
        +static void input(Type elementType, ArrayUnionSumState state, Block arrayBlock)
        +static void combine(ArrayUnionSumState state, ArrayUnionSumState otherState)
        +static void output(ArrayUnionSumState state, BlockBuilder out)
        -static MethodHandle INPUT_FUNCTION
        -static MethodHandle COMBINE_FUNCTION
        -static MethodHandle OUTPUT_FUNCTION
    }

    class ArrayUnionSumResult {
        -Type elementType
        -Adder adder
        +ArrayUnionSumResult(Type elementType, Adder adder)
        +static ArrayUnionSumResult create(Type elementType, Adder adder, Block arrayBlock)
        +Type getElementType()
        +void serialize(BlockBuilder out)
        +ArrayUnionSumResult unionSum(ArrayUnionSumResult other)
        +ArrayUnionSumResult unionSum(Block arrayBlock)
        +long getRetainedSizeInBytes()
        +int size()
        +void appendValue(int i, BlockBuilder blockBuilder)
        +boolean isValueNull(int i)
        +Block getValueBlock()
        +int getValueBlockIndex(int i)
        +static void appendValue(Type elementType, Block block, int position, BlockBuilder blockBuilder)
    }

    class ArrayUnionSumResult_SingleArrayBlock {
        -Block arrayBlock
        +SingleArrayBlock(Type elementType, Adder adder, Block arrayBlock)
        +int size()
        +void appendValue(int i, BlockBuilder blockBuilder)
        +boolean isValueNull(int i)
        +long getRetainedSizeInBytes()
        +Block getValueBlock()
        +int getValueBlockIndex(int i)
    }

    class ArrayUnionSumResult_AccumulatedValues {
        -Block valueBlock
        +AccumulatedValues(Type elementType, Adder adder, Block valueBlock)
        +int size()
        +void appendValue(int i, BlockBuilder blockBuilder)
        +boolean isValueNull(int i)
        +long getRetainedSizeInBytes()
        +Block getValueBlock()
        +int getValueBlockIndex(int i)
    }

    class ArrayUnionSumState {
        <<interface>>
        +ArrayUnionSumResult get()
        +void set(ArrayUnionSumResult value)
        +void addMemoryUsage(long memory)
        +Type getElementType()
        +Adder getAdder()
    }

    class ArrayUnionSumStateFactory {
        +ArrayUnionSumStateFactory(Type elementType)
        +ArrayUnionSumState createSingleState()
        +Class~? extends ArrayUnionSumState~ getSingleStateClass()
        +ArrayUnionSumState createGroupedState()
        +Class~? extends ArrayUnionSumState~ getGroupedStateClass()
        -Type elementType
        -Adder adder
        -static Adder LONG_ADDER
        -static Adder DOUBLE_ADDER
        -static Adder FLOAT_ADDER
        -static Adder getAdder(Type type)
    }

    class ArrayUnionSumStateFactory_GroupedState {
        -Type elementType
        -Adder adder
        -ObjectBigArray~ArrayUnionSumResult~ results
        -long size
        +GroupedState(Type elementType, Adder adder)
        +void ensureCapacity(long size)
        +ArrayUnionSumResult get()
        +void set(ArrayUnionSumResult value)
        +void addMemoryUsage(long memory)
        +Type getElementType()
        +long getEstimatedSize()
        +Adder getAdder()
    }

    class ArrayUnionSumStateFactory_SingleState {
        -Type elementType
        -Adder adder
        -ArrayUnionSumResult result
        +SingleState(Type elementType, Adder adder)
        +ArrayUnionSumResult get()
        +void set(ArrayUnionSumResult value)
        +void addMemoryUsage(long memory)
        +Type getElementType()
        +long getEstimatedSize()
        +Adder getAdder()
    }

    class ArrayUnionSumStateSerializer {
        -ArrayType arrayType
        +ArrayUnionSumStateSerializer(ArrayType arrayType)
        +Type getSerializedType()
        +void serialize(ArrayUnionSumState state, BlockBuilder out)
        +void deserialize(Block block, int index, ArrayUnionSumState state)
    }

    class Adder {
        <<interface>>
        +void writeSum(Type type, Block block1, int position1, Block block2, int position2, BlockBuilder blockBuilder)
    }

    class SqlAggregationFunction
    class AccumulatorStateFactory
    class AccumulatorStateSerializer
    class AccumulatorState
    class AbstractGroupedAccumulatorState
    class ArrayType
    class Type
    class Block
    class BlockBuilder

    ArrayUnionSumAggregation --> ArrayUnionSumState : uses
    ArrayUnionSumAggregation --> ArrayUnionSumStateFactory : creates
    ArrayUnionSumAggregation --> ArrayUnionSumStateSerializer : uses
    ArrayUnionSumAggregation --> ArrayUnionSumResult : uses

    ArrayUnionSumAggregation --|> SqlAggregationFunction

    ArrayUnionSumResult <|-- ArrayUnionSumResult_SingleArrayBlock
    ArrayUnionSumResult <|-- ArrayUnionSumResult_AccumulatedValues
    ArrayUnionSumResult --> Adder : uses
    ArrayUnionSumResult --> Type : elementType
    ArrayUnionSumResult --> Block : holds

    ArrayUnionSumState <|.. ArrayUnionSumStateFactory_GroupedState
    ArrayUnionSumState <|.. ArrayUnionSumStateFactory_SingleState

    ArrayUnionSumStateFactory --|> AccumulatorStateFactory
    ArrayUnionSumStateFactory --> ArrayUnionSumStateFactory_GroupedState : creates
    ArrayUnionSumStateFactory --> ArrayUnionSumStateFactory_SingleState : creates
    ArrayUnionSumStateFactory --> Adder : configures

    ArrayUnionSumStateFactory_GroupedState --|> AbstractGroupedAccumulatorState
    ArrayUnionSumStateFactory_GroupedState --> ArrayUnionSumResult : stores

    ArrayUnionSumStateFactory_SingleState --> ArrayUnionSumResult : stores

    ArrayUnionSumStateSerializer --|> AccumulatorStateSerializer
    ArrayUnionSumStateSerializer --> ArrayUnionSumResult : reconstructs
    ArrayUnionSumStateSerializer --> ArrayType : uses

    ArrayUnionSumState --|> AccumulatorState

    ArrayType --> Type
    Block --> Type
    BlockBuilder --> Type
Loading

File-Level Changes

Change Details Files
Introduce array_union_sum aggregation, including execution logic, state management, and function wiring.
  • Add ArrayUnionSumAggregation implementing SqlAggregationFunction with non-decimal numeric type parameter T and array input/output, wiring input/combine/output methods via generated accumulators.
  • Implement ArrayUnionSumResult as a lazy, array-backed aggregation result that can union-sum either a Block or another ArrayUnionSumResult, padding shorter arrays with zeros and treating nulls as zero while tracking retained size.
  • Create ArrayUnionSumState, ArrayUnionSumStateFactory, and ArrayUnionSumStateSerializer to manage single/grouped aggregation state, choose the proper numeric Adder (long/double/real), track memory, and support distributed serialization/deserialization.
presto-main-base/src/main/java/com/facebook/presto/operator/aggregation/ArrayUnionSumAggregation.java
presto-main-base/src/main/java/com/facebook/presto/operator/aggregation/ArrayUnionSumResult.java
presto-main-base/src/main/java/com/facebook/presto/operator/aggregation/state/ArrayUnionSumState.java
presto-main-base/src/main/java/com/facebook/presto/operator/aggregation/state/ArrayUnionSumStateFactory.java
presto-main-base/src/main/java/com/facebook/presto/operator/aggregation/state/ArrayUnionSumStateSerializer.java
Register the new aggregation in the function namespace so it is available as a built-in SQL function.
  • Add ARRAY_UNION_SUM constant to the built-in aggregation set and include it in the getBuiltInFunctions registration chain alongside existing map aggregations.
presto-main-base/src/main/java/com/facebook/presto/metadata/BuiltInTypeAndFunctionNamespaceManager.java
Add unit tests for the aggregation result and state, and integration tests for SQL behavior, type checking, and overflow handling.
  • Add TestArrayUnionSumResult to cover union-sum behavior for BIGINT/DOUBLE/REAL, null handling, varying lengths, retained size, state factories, serializer round-trip, and element type accessors.
  • Extend AbstractTestQueries with testArrayUnionSum, testInvalidArrayUnionSum, and testArrayUnionSumOverflow to validate query-level behavior, supported/unsupported types, grouping semantics, null and empty arrays, and overflow errors.
presto-main-base/src/test/java/com/facebook/presto/operator/aggregation/TestArrayUnionSumResult.java
presto-tests/src/main/java/com/facebook/presto/tests/AbstractTestQueries.java
Document the new aggregation and add a release-note entry.
  • Update aggregate function documentation to describe array_union_sum’s semantics, supported types, and examples.
  • Add release-0.296 notes mentioning the new array_union_sum aggregation and its behavior for non-decimal numeric arrays.
presto-docs/src/main/sphinx/functions/aggregate.rst
presto-docs/src/main/sphinx/release/release-0.296.rst

Tips and commands

Interacting with Sourcery

  • Trigger a new review: Comment @sourcery-ai review on the pull request.
  • Continue discussions: Reply directly to Sourcery's review comments.
  • Generate a GitHub issue from a review comment: Ask Sourcery to create an
    issue from a review comment by replying to it. You can also reply to a
    review comment with @sourcery-ai issue to create an issue from it.
  • Generate a pull request title: Write @sourcery-ai anywhere in the pull
    request title to generate a title at any time. You can also comment
    @sourcery-ai title on the pull request to (re-)generate the title at any time.
  • Generate a pull request summary: Write @sourcery-ai summary anywhere in
    the pull request body to generate a PR summary at any time exactly where you
    want it. You can also comment @sourcery-ai summary on the pull request to
    (re-)generate the summary at any time.
  • Generate reviewer's guide: Comment @sourcery-ai guide on the pull
    request to (re-)generate the reviewer's guide at any time.
  • Resolve all Sourcery comments: Comment @sourcery-ai resolve on the
    pull request to resolve all Sourcery comments. Useful if you've already
    addressed all the comments and don't want to see them anymore.
  • Dismiss all Sourcery reviews: Comment @sourcery-ai dismiss on the pull
    request to dismiss all existing Sourcery reviews. Especially useful if you
    want to start fresh with a new review - don't forget to comment
    @sourcery-ai review to trigger a new review!

Customizing Your Experience

Access your dashboard to:

  • Enable or disable review features such as the Sourcery-generated pull request
    summary, the reviewer's guide, and others.
  • Change the review language.
  • Add, remove or edit custom review instructions.
  • Adjust other review settings.

Getting Help

Adds a new SQL aggregation function that combines arrays by summing
values at corresponding indices.

Features:
- Result array length is the maximum of all input arrays
- Missing elements treated as 0
- Null values coalesced to 0
- Supports BIGINT, INTEGER, SMALLINT, TINYINT, DOUBLE, and REAL

Files:
- ArrayUnionSumAggregation.java - Main aggregation logic
- ArrayUnionSumResult.java - Result container with union-sum logic
- ArrayUnionSumState*.java - State management for aggregation
- TestArrayUnionSumResult.java - Unit tests (14 test cases)
- Updated docs and integration tests
Copy link
Contributor

@sourcery-ai sourcery-ai bot left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hey - I've found 4 issues, and left some high level feedback:

  • In ArrayUnionSumAggregation.combine, calling state.get().unionSum(otherState.get()) when otherState.get() is null will throw; add an early return when otherState.get() == null to mirror typical combine semantics and avoid NPEs.
  • The input function currently treats every row as having a non-null array block; if the contract is that NULL arrays should be skipped (as described in the PR), add an explicit check to ignore null input values rather than creating an ArrayUnionSumResult for them.
Prompt for AI Agents
Please address the comments from this code review:

## Overall Comments
- In `ArrayUnionSumAggregation.combine`, calling `state.get().unionSum(otherState.get())` when `otherState.get()` is null will throw; add an early return when `otherState.get() == null` to mirror typical combine semantics and avoid NPEs.
- The `input` function currently treats every row as having a non-null array block; if the contract is that NULL arrays should be skipped (as described in the PR), add an explicit check to ignore null input values rather than creating an `ArrayUnionSumResult` for them.

## Individual Comments

### Comment 1
<location> `presto-main-base/src/main/java/com/facebook/presto/operator/aggregation/ArrayUnionSumAggregation.java:130-135` </location>
<code_context>
+            arrayUnionSumResult = ArrayUnionSumResult.create(elementType, state.getAdder(), arrayBlock);
+            state.set(arrayUnionSumResult);
+        }
+        else {
+            startSize = arrayUnionSumResult.getRetainedSizeInBytes();
+            state.set(state.get().unionSum(arrayBlock));
+        }
+
+        state.addMemoryUsage(arrayUnionSumResult.getRetainedSizeInBytes() - startSize);
+    }
+
</code_context>

<issue_to_address>
**issue (bug_risk):** Memory accounting in input() uses the old result size instead of the new one

In the `else` branch, `arrayUnionSumResult` still points to the pre-union result. After `state.set(state.get().unionSum(arrayBlock));`, the new size is only available via `state.get()`, so using `arrayUnionSumResult.getRetainedSizeInBytes()` keeps using the old size and the delta is always 0.

You could fix this by capturing the current state and its size, creating the new result, then updating memory based on the new result, e.g.:

```java
else {
    ArrayUnionSumResult current = state.get();
    startSize = current.getRetainedSizeInBytes();
    ArrayUnionSumResult newResult = current.unionSum(arrayBlock);
    state.set(newResult);
    state.addMemoryUsage(newResult.getRetainedSizeInBytes() - startSize);
}
```

or by recomputing the size from `state.get()` after the `set`.
</issue_to_address>

### Comment 2
<location> `presto-main-base/src/main/java/com/facebook/presto/operator/aggregation/ArrayUnionSumAggregation.java:140-142` </location>
<code_context>
+
+    public static void combine(ArrayUnionSumState state, ArrayUnionSumState otherState)
+    {
+        if (state.get() == null) {
+            state.set(otherState.get());
+            return;
+        }
+
</code_context>

<issue_to_address>
**issue (bug_risk):** combine() does not update memory usage when the target state is initially null

When `state.get() == null`, you assign `otherState.get()` and return without updating memory usage, so combining into an empty state under-reports the memory of the incoming result. You should add the retained size of `otherState.get()` in this branch (as you do in the non-null case) to keep memory accounting consistent.
</issue_to_address>

### Comment 3
<location> `presto-tests/src/main/java/com/facebook/presto/tests/AbstractTestQueries.java:6339` </location>
<code_context>
+                ImmutableList.of(),
+                parseTypeSignature("array<T>"),
+                ImmutableList.of(parseTypeSignature("array<T>")));
+    }
+
+    @Override
</code_context>

<issue_to_address>
**suggestion (testing):** Consider adding query tests for NULL arrays and empty arrays to cover documented behavior

`testArrayUnionSum` currently only covers arrays with elements and element-level NULLs. Please also add tests that:

- Mix `CAST(NULL AS array<bigint>)` with non-null arrays and assert that NULL-array rows are skipped, producing the same result as aggregating only non-null arrays.
- Mix `CAST(array[] AS array<bigint>)` with non-empty arrays and assert that empty arrays don’t affect the result, and that queries with only empty arrays succeed without errors.

This will exercise the documented semantics for NULL and empty arrays.
</issue_to_address>

### Comment 4
<location> `presto-main-base/src/test/java/com/facebook/presto/operator/aggregation/TestArrayUnionSumResult.java:40` </location>
<code_context>
+    private static final ArrayType ARRAY_DOUBLE = new ArrayType(DOUBLE);
+    private static final ArrayType ARRAY_REAL = new ArrayType(REAL);
+
+    @Test
+    public void testBasicUnionSum()
+    {
</code_context>

<issue_to_address>
**suggestion (testing):** Add a unit test for handling empty input arrays in ArrayUnionSumResult

Current tests don’t cover the case where `positionCount = 0`. Please add a test that builds an empty `Block` (BIGINT, and optionally other types), constructs an `ArrayUnionSumResult` from it, asserts `size() == 0`, verifies serialization produces an array block with 0 elements, and (optionally) unions this empty result/block with a non-empty one to confirm the non-empty result is preserved. This guards against implementations that assume `size() > 0` and validates empty-result serialization behavior.

Suggested implementation:

```java
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertNotNull;
import static org.testng.Assert.assertNull;
import static org.testng.Assert.assertTrue;

```

```java
public class TestArrayUnionSumResult
{
    private static final ArrayType ARRAY_BIGINT = new ArrayType(BIGINT);
    private static final ArrayType ARRAY_DOUBLE = new ArrayType(DOUBLE);
    private static final ArrayType ARRAY_REAL = new ArrayType(REAL);

    @Test
    public void testEmptyInputArrays()
    {
        // Build an empty BIGINT array block
        BlockBuilder arrayBlockBuilder = ARRAY_BIGINT.createBlockBuilder(null, 0);
        Block emptyArrayBlock = arrayBlockBuilder.build();

        // Create an ArrayUnionSumState and update it with the empty block
        ArrayUnionSumState state = new ArrayUnionSumStateFactory(BIGINT).createSingleState();
        state.setResult(new ArrayUnionSumResult(ARRAY_BIGINT, BIGINT, emptyArrayBlock));

        ArrayUnionSumResult result = state.getResult();

        // Verify empty result
        assertNotNull(result);
        assertEquals(result.size(), 0);

        // Verify serialization produces an empty array block
        Block serialized = result.serialize();
        assertEquals(serialized.getPositionCount(), 0);

        // Build a non-empty BIGINT array block containing [1, 2]
        BlockBuilder nonEmptyArrayBlockBuilder = ARRAY_BIGINT.createBlockBuilder(null, 1);
        BlockBuilder elementBlockBuilder = BIGINT.createBlockBuilder(null, 2);
        BIGINT.writeLong(elementBlockBuilder, 1L);
        BIGINT.writeLong(elementBlockBuilder, 2L);
        nonEmptyArrayBlockBuilder.writeObject(elementBlockBuilder.build()).closeEntry();
        Block nonEmptyArrayBlock = nonEmptyArrayBlockBuilder.build();

        ArrayUnionSumState nonEmptyState = new ArrayUnionSumStateFactory(BIGINT).createSingleState();
        nonEmptyState.setResult(new ArrayUnionSumResult(ARRAY_BIGINT, BIGINT, nonEmptyArrayBlock));
        ArrayUnionSumResult nonEmptyResult = nonEmptyState.getResult();

        // Union empty with non-empty and verify non-empty is preserved
        ArrayUnionSumResult unionResult = result.union(nonEmptyResult);
        assertEquals(unionResult.size(), nonEmptyResult.size());
        Block unionSerialized = unionResult.serialize();
        assertEquals(unionSerialized.getPositionCount(), nonEmptyArrayBlock.getPositionCount());
    }

    @Test
    public void testBasicUnionSum()
    {
        ArrayUnionSumState state = new ArrayUnionSumStateFactory(BIGINT).createSingleState();

        // Create array [1, 2, 3]

```

The above test assumes the following APIs exist and are accessible in this test:

1. `Block` and `BlockBuilder` are already imported (typically from `com.facebook.presto.spi.block`).
2. `ArrayUnionSumResult` has:
   - A constructor `ArrayUnionSumResult(ArrayType arrayType, Type elementType, Block arrayBlock)`.
   - Methods `int size()`, `Block serialize()`, and `ArrayUnionSumResult union(ArrayUnionSumResult other)`.
3. `ArrayUnionSumState` has `setResult(ArrayUnionSumResult)` and `getResult()` methods.
4. `ArrayUnionSumStateFactory` has a constructor `ArrayUnionSumStateFactory(Type elementType)` and a `createSingleState()` method.

If the actual APIs differ slightly, adjust the constructor and method calls in `testEmptyInputArrays()` to match the real `ArrayUnionSumResult` / state APIs while preserving the test semantics:
- Construct a state/result from an empty input array.
- Assert `size() == 0`.
- Assert the serialized block has `positionCount == 0`.
- Union the empty result with a non-empty one and assert the non-empty union behavior.
</issue_to_address>

Sourcery is free for open source - if you like our reviews please consider sharing them ✨
Help me be more useful! Please click 👍 or 👎 on each comment and I'll use the feedback to improve your reviews.

Comment on lines +130 to +135
else {
startSize = arrayUnionSumResult.getRetainedSizeInBytes();
state.set(state.get().unionSum(arrayBlock));
}

state.addMemoryUsage(arrayUnionSumResult.getRetainedSizeInBytes() - startSize);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

issue (bug_risk): Memory accounting in input() uses the old result size instead of the new one

In the else branch, arrayUnionSumResult still points to the pre-union result. After state.set(state.get().unionSum(arrayBlock));, the new size is only available via state.get(), so using arrayUnionSumResult.getRetainedSizeInBytes() keeps using the old size and the delta is always 0.

You could fix this by capturing the current state and its size, creating the new result, then updating memory based on the new result, e.g.:

else {
    ArrayUnionSumResult current = state.get();
    startSize = current.getRetainedSizeInBytes();
    ArrayUnionSumResult newResult = current.unionSum(arrayBlock);
    state.set(newResult);
    state.addMemoryUsage(newResult.getRetainedSizeInBytes() - startSize);
}

or by recomputing the size from state.get() after the set.

Comment on lines +140 to +142
if (state.get() == null) {
state.set(otherState.get());
return;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

issue (bug_risk): combine() does not update memory usage when the target state is initially null

When state.get() == null, you assign otherState.get() and return without updating memory usage, so combining into an empty state under-reports the memory of the incoming result. You should add the retained size of otherState.get() in this branch (as you do in the non-null case) to keep memory accounting consistent.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

1 participant