From b4cfa0ec2d1c6ca1b9ce6035970b27dda8ae97be Mon Sep 17 00:00:00 2001 From: "Charles P. Wright" Date: Wed, 10 Dec 2025 14:34:31 -0500 Subject: [PATCH 01/24] reaggregate support for groupBy. --- .../table/impl/by/AggregationProcessor.java | 24 +- .../impl/by/GroupByReaggreagateOperator.java | 493 ++++++++++++++++++ .../impl/hierarchical/RollupTableImpl.java | 1 + .../aggregate/AggregateColumnSource.java | 2 + .../aggregate/BaseAggregateColumnSource.java | 5 + .../BaseAggregateSlicedColumnSource.java | 5 + .../aggregate/RangeAggregateColumnSource.java | 5 + .../engine/table/impl/TestRollupTable.java | 107 +++- 8 files changed, 634 insertions(+), 8 deletions(-) create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggreagateOperator.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index 28d9d96f8cd..a4cbde5b73c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -1015,12 +1015,6 @@ default void visit(AggSpecFreeze freeze) { rollupUnsupported("Freeze"); } - @Override - @FinalDefault - default void visit(@NotNull final AggSpecGroup group) { - rollupUnsupported("Group"); - } - @Override @FinalDefault default void visit(@NotNull final AggSpecFormula formula) { @@ -1108,6 +1102,13 @@ public void visit(@NotNull final Partition partition) { addNoInputOperator(partitionOperator); } + @Override + public void visit(AggSpecGroup group) { + unsupportedForBlinkTables("Group for rollup"); + addNoInputOperator(new GroupByChunkedOperator(table, true, EXPOSED_GROUP_ROW_SETS.name(), + MatchPair.fromPairs(resultPairs))); + } + // ------------------------------------------------------------------------------------------------------------- // AggSpec.Visitor // ------------------------------------------------------------------------------------------------------------- @@ -1265,6 +1266,17 @@ public void visit(@NotNull final Partition partition) { addNoInputOperator(partitionOperator); } + @Override + public void visit(AggSpecGroup group) { + final ColumnSource groupRowSet = table.getColumnSource(EXPOSED_GROUP_ROW_SETS.name()); + final MatchPair[] pairs = new MatchPair[resultPairs.size()]; + for (int ii = 0; ii < resultPairs.size(); ++ii) { + pairs[ii] = new MatchPair(resultPairs.get(ii).output().name(), resultPairs.get(ii).output().name()); + } + addOperator(new GroupByReaggreagateOperator(table, true, EXPOSED_GROUP_ROW_SETS.name(), pairs), groupRowSet, + EXPOSED_GROUP_ROW_SETS.name()); + } + // ------------------------------------------------------------------------------------------------------------- // AggSpec.Visitor // ------------------------------------------------------------------------------------------------------------- diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggreagateOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggreagateOperator.java new file mode 100644 index 00000000000..164d0eb35a2 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggreagateOperator.java @@ -0,0 +1,493 @@ +// +// Copyright (c) 2016-2025 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.by; + +import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.*; +import io.deephaven.chunk.attributes.ChunkLengths; +import io.deephaven.chunk.attributes.ChunkPositions; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.liveness.LivenessReferent; +import io.deephaven.engine.rowset.*; +import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.table.*; +import io.deephaven.engine.table.impl.MatchPair; +import io.deephaven.engine.table.impl.QueryTable; +import io.deephaven.engine.table.impl.sources.ObjectArraySource; +import io.deephaven.engine.table.impl.sources.aggregate.AggregateColumnSource; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.util.Arrays; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.function.UnaryOperator; + +import static io.deephaven.engine.table.impl.sources.ArrayBackedColumnSource.BLOCK_SIZE; + +/** + * An {@link IterativeChunkedAggregationOperator} used in the implementation of {@link Table#groupBy}, + * {@link io.deephaven.api.agg.spec.AggSpecGroup}, and {@link io.deephaven.api.agg.Aggregation#AggGroup(String...)}. + */ +public final class GroupByReaggreagateOperator implements IterativeChunkedAggregationOperator { + + private final QueryTable inputTable; + private final boolean registeredWithHelper; + private final String exposeRowSetsAs; + + private final boolean live; + private final ObjectArraySource rowSets; + private final ObjectArraySource addedBuilders; + private final ObjectArraySource removedBuilders; + + private final String[] inputColumnNames; + + + private final Map> inputSources; + + private final Map> inputAggregatedColumns; + private final Map> resultAggregatedColumns; + private final ModifiedColumnSet aggregationInputsModifiedColumnSet; + + private RowSetBuilderRandom stepDestinationsModified; + + private boolean stepValuesModified; + private boolean initialized; + + public GroupByReaggreagateOperator( + @NotNull final QueryTable inputTable, + final boolean registeredWithHelper, + @Nullable final String exposeRowSetsAs, + @NotNull final MatchPair... aggregatedColumnPairs) { + this.inputTable = inputTable; + this.registeredWithHelper = registeredWithHelper; + this.exposeRowSetsAs = exposeRowSetsAs; + + live = inputTable.isRefreshing(); + rowSets = new ObjectArraySource<>(WritableRowSet.class); + addedBuilders = new ObjectArraySource<>(Object.class); + + inputSources = new LinkedHashMap<>(aggregatedColumnPairs.length); + + inputAggregatedColumns = new LinkedHashMap<>(aggregatedColumnPairs.length); + resultAggregatedColumns = new LinkedHashMap<>(aggregatedColumnPairs.length); + Arrays.stream(aggregatedColumnPairs).forEach(pair -> { + final ColumnSource source = inputTable.getColumnSource(pair.rightColumn()); + final ColumnSource realSource; + if (source instanceof AggregateColumnSource) { + final AggregateColumnSource aggregateSource = (AggregateColumnSource) source; + realSource = aggregateSource.getAggregatedSource(); + inputSources.put(pair.rightColumn(), aggregateSource); + } else { + throw new IllegalStateException("Expect to reaggregate AggregateColumnSources for a group operation."); + } + final AggregateColumnSource aggregateColumnSource = AggregateColumnSource.make(realSource, rowSets); + inputAggregatedColumns.put(pair.rightColumn(), aggregateColumnSource); + resultAggregatedColumns.put(pair.leftColumn(), aggregateColumnSource); + }); + + if (exposeRowSetsAs != null && resultAggregatedColumns.containsKey(exposeRowSetsAs)) { + throw new IllegalArgumentException(String.format( + "Exposing group RowSets as %s, but this conflicts with a requested grouped output column name", + exposeRowSetsAs)); + } + inputColumnNames = MatchPair.getRightColumns(aggregatedColumnPairs); + if (live) { + final String[] allInputs = Arrays.copyOf(inputColumnNames, inputColumnNames.length + 1); + allInputs[allInputs.length - 1] = exposeRowSetsAs; + aggregationInputsModifiedColumnSet = inputTable.newModifiedColumnSet(allInputs); + removedBuilders = new ObjectArraySource<>(Object.class); + } else { + aggregationInputsModifiedColumnSet = null; + removedBuilders = null; + } + initialized = false; + } + + @Override + public void addChunk(final BucketedContext bucketedContext, final Chunk values, + final LongChunk inputRowKeys, + @NotNull final IntChunk destinations, + @NotNull final IntChunk startPositions, + @NotNull final IntChunk length, + @NotNull final WritableBooleanChunk stateModified) { + for (int ii = 0; ii < startPositions.size(); ++ii) { + final int startPosition = startPositions.get(ii); + final int runLength = length.get(ii); + final long destination = destinations.get(startPosition); + addChunk(values.asObjectChunk(), startPosition, runLength, destination); + } + stateModified.fillWithValue(0, startPositions.size(), true); + } + + @Override + public void removeChunk(final BucketedContext bucketedContext, final Chunk values, + final LongChunk inputRowKeys, + @NotNull final IntChunk destinations, @NotNull final IntChunk startPositions, + @NotNull final IntChunk length, @NotNull final WritableBooleanChunk stateModified) { + for (int ii = 0; ii < startPositions.size(); ++ii) { + final int startPosition = startPositions.get(ii); + final int runLength = length.get(ii); + final long destination = destinations.get(startPosition); + removeChunk(values.asObjectChunk(), startPosition, runLength, destination); + } + stateModified.fillWithValue(0, startPositions.size(), true); + } + + @Override + public void modifyChunk(final BucketedContext bucketedContext, + final Chunk previousValues, + final Chunk newValues, + final LongChunk postShiftRowKeys, + @NotNull final IntChunk destinations, + @NotNull final IntChunk startPositions, + @NotNull final IntChunk length, @NotNull final WritableBooleanChunk stateModified) { + for (int ii = 0; ii < startPositions.size(); ++ii) { + final int startPosition = startPositions.get(ii); + final int runLength = length.get(ii); + final long destination = destinations.get(startPosition); + modifyChunk(previousValues.asObjectChunk(), newValues.asObjectChunk(), startPosition, runLength, + destination); + } + stateModified.fillWithValue(0, startPositions.size(), true); + } + + @Override + public void shiftChunk(final BucketedContext bucketedContext, final Chunk previousValues, + final Chunk newValues, + @NotNull final LongChunk preShiftRowKeys, + @NotNull final LongChunk postShiftRowKeys, + @NotNull final IntChunk destinations, @NotNull final IntChunk startPositions, + @NotNull final IntChunk length, @NotNull final WritableBooleanChunk stateModified) { + throw new IllegalStateException("GroupByReaggreagateOperator should never be called with shiftChunk"); + } + + @Override + public boolean addChunk(final SingletonContext singletonContext, final int chunkSize, + final Chunk values, + @NotNull final LongChunk inputRowKeys, + final long destination) { + addChunk(values.asObjectChunk(), 0, chunkSize, destination); + return true; + } + + @Override + public boolean removeChunk(final SingletonContext singletonContext, final int chunkSize, + final Chunk values, + @NotNull final LongChunk inputRowKeys, final long destination) { + removeChunk(values.asObjectChunk(), 0, chunkSize, destination); + return true; + } + + @Override + public boolean modifyChunk(final SingletonContext singletonContext, final int chunkSize, + final Chunk previousValues, final Chunk newValues, + final LongChunk postShiftRowKeys, + final long destination) { + modifyChunk(previousValues.asObjectChunk(), newValues.asObjectChunk(), 0, chunkSize, destination); + return true; + } + + @Override + public boolean shiftChunk(final SingletonContext singletonContext, final Chunk previousValues, + final Chunk newValues, + @NotNull final LongChunk preShiftRowKeys, + @NotNull final LongChunk postShiftRowKeys, + final long destination) { + // we don't need to deal with these yet + throw new IllegalStateException(); + } + + private void addChunk(@NotNull final ObjectChunk rowSets, final int start, + final int length, + final long destination) { + if (length == 0) { + return; + } + accumulateToBuilderRandom(addedBuilders, rowSets, start, length, destination, false); + if (stepDestinationsModified != null) { + stepDestinationsModified.addKey(destination); + } + } + + private void removeChunk(@NotNull final ObjectChunk rowSets, final int start, + final int length, + final long destination) { + if (length == 0) { + return; + } + accumulateToBuilderRandom(removedBuilders, rowSets, start, length, destination, false); + stepDestinationsModified.addKey(destination); + } + + private void modifyChunk(ObjectChunk previousValues, + ObjectChunk newValues, + int start, + int length, + long destination) { + if (length == 0) { + return; + } + + accumulateToBuilderRandom(removedBuilders, previousValues, start, length, destination, true); + accumulateToBuilderRandom(addedBuilders, newValues, start, length, destination, false); + + stepDestinationsModified.addKey(destination); + } + + private static void accumulateToBuilderRandom(@NotNull final ObjectArraySource builderColumn, + @NotNull final ObjectChunk rowSetsToAdd, + final int start, final int length, final long destination, + final boolean previous) { + RowSetBuilderRandom builder = (RowSetBuilderRandom) builderColumn.getUnsafe(destination); + if (builder == null) { + builderColumn.set(destination, builder = RowSetFactory.builderRandom()); + } + // add the keys to the stored builder + for (int ii = 0; ii < length; ++ii) { + RowSet rowSet = rowSetsToAdd.get(start + ii); + if (previous) { + builder.addRowSet(rowSet.trackingCast().prev()); + } else { + builder.addRowSet(rowSet); + } + } + } + + private static WritableRowSet extractAndClearBuilderRandom( + @NotNull final WritableObjectChunk builderChunk, + final int offset) { + final RowSetBuilderRandom builder = builderChunk.get(offset); + if (builder != null) { + final WritableRowSet rowSet = builder.build(); + builderChunk.set(offset, null); + return rowSet; + } + return null; + } + + private static WritableRowSet nullToEmpty(@Nullable final WritableRowSet rowSet) { + return rowSet == null ? RowSetFactory.empty() : rowSet; + } + + @Override + public void ensureCapacity(final long tableSize) { + rowSets.ensureCapacity(tableSize); + addedBuilders.ensureCapacity(tableSize); + if (live) { + removedBuilders.ensureCapacity(tableSize); + } + } + + @Override + public Map> getResultColumns() { + if (exposeRowSetsAs != null) { + final Map> allResultColumns = + new LinkedHashMap<>(resultAggregatedColumns.size() + 1); + allResultColumns.put(exposeRowSetsAs, rowSets); + allResultColumns.putAll(resultAggregatedColumns); + return allResultColumns; + } + return resultAggregatedColumns; + } + + @Override + public void startTrackingPrevValues() { + // NB: We don't need previous tracking on the rowSets ColumnSource, even if it's exposed. It's in destination + // space, and we never move anything. Nothing should be asking for previous values if they didn't exist + // previously. + // NB: These are usually (always, as of now) instances of AggregateColumnSource, meaning + // startTrackingPrevValues() is a no-op. + resultAggregatedColumns.values().forEach(ColumnSource::startTrackingPrevValues); + } + + @Override + public UnaryOperator initializeRefreshing( + @NotNull final QueryTable resultTable, + @NotNull final LivenessReferent aggregationUpdateListener) { + initializeNewRowSetPreviousValues(resultTable.getRowSet()); + return registeredWithHelper + ? new InputToResultModifiedColumnSetFactory(resultTable, + resultAggregatedColumns.keySet().toArray(String[]::new)) + : null; + } + + private class InputToResultModifiedColumnSetFactory implements UnaryOperator { + + private final ModifiedColumnSet updateModifiedColumnSet; + private final ModifiedColumnSet allResultColumns; + private final ModifiedColumnSet.Transformer aggregatedColumnsTransformer; + + private InputToResultModifiedColumnSetFactory( + @NotNull final QueryTable resultTable, + @NotNull final String[] resultAggregatedColumnNames) { + updateModifiedColumnSet = new ModifiedColumnSet(resultTable.getModifiedColumnSetForUpdates()); + + if (exposeRowSetsAs != null) { + // resultAggregatedColumnNames may be empty (e.g. when the row set is the only result column) + allResultColumns = resultTable.newModifiedColumnSet(exposeRowSetsAs); + allResultColumns.setAll(resultAggregatedColumnNames); + } else { + allResultColumns = resultTable.newModifiedColumnSet(resultAggregatedColumnNames); + } + + final String[] allInputs = Arrays.copyOf(inputColumnNames, inputColumnNames.length + 1); + allInputs[allInputs.length - 1] = exposeRowSetsAs; + final ModifiedColumnSet[] affectedColumns = new ModifiedColumnSet[allInputs.length]; + for (int ci = 0; ci < inputColumnNames.length; ++ci) { + affectedColumns[ci] = resultTable.newModifiedColumnSet(resultAggregatedColumnNames[ci]); + } + affectedColumns[allInputs.length - 1] = resultTable.newModifiedColumnSet(allInputs); + + aggregatedColumnsTransformer = inputTable.newModifiedColumnSetTransformer(allInputs, affectedColumns); + } + + @Override + public ModifiedColumnSet apply(@NotNull final ModifiedColumnSet upstreamModifiedColumnSet) { + aggregatedColumnsTransformer.clearAndTransform(upstreamModifiedColumnSet, updateModifiedColumnSet); + return updateModifiedColumnSet; + } + } + + @Override + public void resetForStep(@NotNull final TableUpdate upstream, final int startingDestinationsCount) { + stepValuesModified = upstream.modified().isNonempty() && upstream.modifiedColumnSet().nonempty() + && upstream.modifiedColumnSet().containsAny(aggregationInputsModifiedColumnSet); + stepDestinationsModified = new BitmapRandomBuilder(startingDestinationsCount); + } + + @Override + public void propagateInitialState(@NotNull final QueryTable resultTable, int startingDestinationsCount) { + Assert.neqTrue(initialized, "initialized"); + + // use the builders to create the initial rowsets + try (final RowSet initialDestinations = RowSetFactory.flat(startingDestinationsCount); + final ResettableWritableObjectChunk rowSetResettableChunk = + ResettableWritableObjectChunk.makeResettableChunk(); + final ResettableWritableObjectChunk addedBuildersResettableChunk = + ResettableWritableObjectChunk.makeResettableChunk(); + final RowSequence.Iterator destinationsIterator = + initialDestinations.getRowSequenceIterator()) { + + final WritableObjectChunk rowSetBackingChunk = + rowSetResettableChunk.asWritableObjectChunk(); + final WritableObjectChunk addedBuildersBackingChunk = + addedBuildersResettableChunk.asWritableObjectChunk(); + + while (destinationsIterator.hasMore()) { + final long firstSliceDestination = destinationsIterator.peekNextKey(); + final long firstBackingChunkDestination = + rowSets.resetWritableChunkToBackingStore(rowSetResettableChunk, firstSliceDestination); + addedBuilders.resetWritableChunkToBackingStore(addedBuildersResettableChunk, firstSliceDestination); + + final long lastBackingChunkDestination = + firstBackingChunkDestination + rowSetBackingChunk.size() - 1; + final RowSequence initialDestinationsSlice = + destinationsIterator.getNextRowSequenceThrough(lastBackingChunkDestination); + + initialDestinationsSlice.forAllRowKeys((final long destination) -> { + final int backingChunkOffset = + Math.toIntExact(destination - firstBackingChunkDestination); + final WritableRowSet addRowSet = nullToEmpty( + extractAndClearBuilderRandom(addedBuildersBackingChunk, backingChunkOffset)); + rowSetBackingChunk.set(backingChunkOffset, live ? addRowSet.toTracking() : addRowSet); + }); + } + } + initialized = true; + } + + @Override + public void propagateUpdates(@NotNull final TableUpdate downstream, @NotNull final RowSet newDestinations) { + // get the rowset for the updated items + try (final WritableRowSet stepDestinations = stepDestinationsModified.build()) { + // add the new destinations so a rowset will get created if it doesn't exist + stepDestinations.insert(newDestinations); + + if (stepDestinations.isEmpty()) { + return; + } + + // use the builders to modify the rowsets + try (final ResettableWritableObjectChunk rowSetResettableChunk = + ResettableWritableObjectChunk.makeResettableChunk(); + final ResettableWritableObjectChunk addedBuildersResettableChunk = + ResettableWritableObjectChunk.makeResettableChunk(); + final ResettableWritableObjectChunk removedBuildersResettableChunk = + ResettableWritableObjectChunk.makeResettableChunk(); + final RowSequence.Iterator destinationsIterator = + stepDestinations.getRowSequenceIterator()) { + + final WritableObjectChunk rowSetBackingChunk = + rowSetResettableChunk.asWritableObjectChunk(); + final WritableObjectChunk addedBuildersBackingChunk = + addedBuildersResettableChunk.asWritableObjectChunk(); + final WritableObjectChunk removedBuildersBackingChunk = + removedBuildersResettableChunk.asWritableObjectChunk(); + + while (destinationsIterator.hasMore()) { + final long firstSliceDestination = destinationsIterator.peekNextKey(); + final long firstBackingChunkDestination = + rowSets.resetWritableChunkToBackingStore(rowSetResettableChunk, firstSliceDestination); + addedBuilders.resetWritableChunkToBackingStore(addedBuildersResettableChunk, + firstSliceDestination); + removedBuilders.resetWritableChunkToBackingStore(removedBuildersResettableChunk, + firstSliceDestination); + + final long lastBackingChunkDestination = + firstBackingChunkDestination + rowSetBackingChunk.size() - 1; + final RowSequence initialDestinationsSlice = + destinationsIterator.getNextRowSequenceThrough(lastBackingChunkDestination); + + initialDestinationsSlice.forAllRowKeys((final long destination) -> { + final int backingChunkOffset = + Math.toIntExact(destination - firstBackingChunkDestination); + final WritableRowSet workingRowSet = rowSetBackingChunk.get(backingChunkOffset); + if (workingRowSet == null) { + // use the addRowSet as the new rowset + final WritableRowSet addRowSet = nullToEmpty( + extractAndClearBuilderRandom(addedBuildersBackingChunk, backingChunkOffset)); + rowSetBackingChunk.set(backingChunkOffset, live ? addRowSet.toTracking() : addRowSet); + } else { + try (final WritableRowSet addRowSet = + nullToEmpty(extractAndClearBuilderRandom(addedBuildersBackingChunk, + backingChunkOffset)); + final WritableRowSet removeRowSet = + nullToEmpty(extractAndClearBuilderRandom(removedBuildersBackingChunk, + backingChunkOffset))) { + workingRowSet.remove(removeRowSet); + workingRowSet.insert(addRowSet); + } + } + }); + } + } + stepDestinationsModified = null; + } + initializeNewRowSetPreviousValues(newDestinations); + } + + private void initializeNewRowSetPreviousValues(@NotNull final RowSequence newDestinations) { + if (newDestinations.isEmpty()) { + return; + } + try (final ChunkSource.GetContext rowSetsGetContext = rowSets.makeGetContext(BLOCK_SIZE); + final RowSequence.Iterator newDestinationsIterator = newDestinations.getRowSequenceIterator()) { + while (newDestinationsIterator.hasMore()) { + final long nextDestination = newDestinationsIterator.peekNextKey(); + final long nextBlockEnd = (nextDestination / BLOCK_SIZE) * BLOCK_SIZE + BLOCK_SIZE - 1; + // This RowSequence slice should be exactly aligned to a slice of a single data block in rowsets (since + // it is an ArrayBackedColumnSource), allowing getChunk to skip a copy. + final RowSequence newDestinationsSlice = + newDestinationsIterator.getNextRowSequenceThrough(nextBlockEnd); + final ObjectChunk rowSetsChunk = + rowSets.getChunk(rowSetsGetContext, newDestinationsSlice).asObjectChunk(); + final int rowSetsChunkSize = rowSetsChunk.size(); + for (int ii = 0; ii < rowSetsChunkSize; ++ii) { + rowSetsChunk.get(ii).initializePreviousValue(); + } + } + } + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/hierarchical/RollupTableImpl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/hierarchical/RollupTableImpl.java index b96631dabe3..2339699f505 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/hierarchical/RollupTableImpl.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/hierarchical/RollupTableImpl.java @@ -27,6 +27,7 @@ import io.deephaven.engine.table.impl.select.WhereFilter; import io.deephaven.engine.table.impl.sources.NullValueColumnSource; import io.deephaven.engine.table.impl.util.RowRedirection; +import io.deephaven.engine.util.PrintListener; import io.deephaven.util.annotations.InternalUseOnly; import io.deephaven.util.type.TypeUtils; import org.apache.commons.lang3.mutable.MutableObject; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/aggregate/AggregateColumnSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/aggregate/AggregateColumnSource.java index 46b75652af1..50052ee268b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/aggregate/AggregateColumnSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/aggregate/AggregateColumnSource.java @@ -22,6 +22,8 @@ public interface AggregateColumnSource, UngroupedColumnSource ungrouped(); + ColumnSource getAggregatedSource(); + static , DATA_TYPE> AggregateColumnSource make( @NotNull final ColumnSource aggregatedSource, @NotNull final ColumnSource groupRowSetSource) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/aggregate/BaseAggregateColumnSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/aggregate/BaseAggregateColumnSource.java index 92208f64a0e..4da56a40bff 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/aggregate/BaseAggregateColumnSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/aggregate/BaseAggregateColumnSource.java @@ -290,4 +290,9 @@ public boolean isStateless() { public boolean isImmutable() { return aggregatedSource.isImmutable() && groupRowSetSource.isImmutable(); } + + @Override + public ColumnSource getAggregatedSource() { + return aggregatedSource; + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/aggregate/BaseAggregateSlicedColumnSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/aggregate/BaseAggregateSlicedColumnSource.java index cac0f4b742f..9f19bbd8d13 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/aggregate/BaseAggregateSlicedColumnSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/aggregate/BaseAggregateSlicedColumnSource.java @@ -381,4 +381,9 @@ public boolean isImmutable() { && (startSource == null || startSource.isImmutable()) && (endSource == null || endSource.isImmutable()); } + + @Override + public ColumnSource getAggregatedSource() { + return aggregatedSource; + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/aggregate/RangeAggregateColumnSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/aggregate/RangeAggregateColumnSource.java index 66525b669de..aea9ea46d31 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/aggregate/RangeAggregateColumnSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/aggregate/RangeAggregateColumnSource.java @@ -346,4 +346,9 @@ public boolean isImmutable() { && startPositionsInclusive.isImmutable() && endPositionsExclusive.isImmutable(); } + + @Override + public ColumnSource getAggregatedSource() { + return aggregated; + } } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java index ebdfd3ecb31..02c909cd323 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java @@ -8,10 +8,13 @@ import io.deephaven.api.agg.Aggregation; import io.deephaven.engine.context.QueryScope; import io.deephaven.engine.rowset.RowSetFactory; +import io.deephaven.engine.rowset.RowSetShiftData; +import io.deephaven.engine.table.ModifiedColumnSet; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.hierarchical.HierarchicalTable; import io.deephaven.engine.table.hierarchical.RollupTable; import io.deephaven.engine.testutil.ColumnInfo; +import io.deephaven.engine.testutil.ControlledUpdateGraph; import io.deephaven.engine.testutil.EvalNuggetInterface; import io.deephaven.engine.testutil.TstUtils; import io.deephaven.engine.testutil.generator.IntGenerator; @@ -22,6 +25,7 @@ import io.deephaven.test.types.OutOfBandTest; import io.deephaven.vector.IntVector; import io.deephaven.vector.IntVectorDirect; +import org.jspecify.annotations.NonNull; import org.junit.Assert; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -62,7 +66,8 @@ public class TestRollupTable extends RefreshingTableTestCase { AggUnique("unique=intCol"), AggVar("var=intCol"), AggWAvg("intCol", "wavg=intCol"), - AggWSum("intCol", "wsum=intCol")); + AggWSum("intCol", "wsum=intCol"), + AggGroup("grp=intCol")); // Companion list of columns to compare between rollup root and the zero-key equivalent private final String[] columnsToCompare = new String[] { @@ -83,7 +88,8 @@ public class TestRollupTable extends RefreshingTableTestCase { "unique", "var", "wavg", - "wsum" + "wsum", + "grp" }; /** @@ -339,4 +345,101 @@ public void testVectorKeyColumn() { snapshot); freeSnapshotTableChunks(snapshot); } + + @Test + public void testRollupGroupStatic() { + final Table source = TableTools.newTable( + stringCol("Key1", "Alpha", "Bravo", "Alpha", "Charlie", "Charlie", "Bravo", "Bravo"), + stringCol("Key2", "Delta", "Delta", "Echo", "Echo", "Echo", "Echo", "Echo"), + intCol("Sentinel", 1, 2, 3, 4, 5, 6, 7)); + + final RollupTable rollup1 = + source.rollup(List.of(AggGroup("Sentinel"), AggSum("Sum=Sentinel")), "Key1", "Key2"); + + final String[] arrayWithNull = new String[1]; + final Table keyTable = newTable( + intCol(rollup1.getRowDepthColumn().name(), 0), + stringCol("Key1", arrayWithNull), + stringCol("Key2", arrayWithNull), + byteCol("Action", HierarchicalTable.KEY_TABLE_ACTION_EXPAND_ALL)); + + final HierarchicalTable.SnapshotState ss1 = rollup1.makeSnapshotState(); + final Table snapshot = + snapshotToTable(rollup1, ss1, keyTable, ColumnName.of("Action"), null, RowSetFactory.flat(30)); + TableTools.showWithRowSet(snapshot); + + final Table expected = initialExpectedGrouped(rollup1); + assertTableEquals(expected, snapshot.dropColumns("__EXPOSED_GROUP_ROW_SETS__")); + freeSnapshotTableChunks(snapshot); + } + + private static Table initialExpectedGrouped(RollupTable rollup1) { + return TableTools.newTable(intCol(rollup1.getRowDepthColumn().name(), 1, 2, 3, 3, 2, 3, 3, 2, 3), + booleanCol(rollup1.getRowExpandedColumn().name(), true, true, null, null, true, null, null, + true, null), + col("Key1", null, "Alpha", "Alpha", "Alpha", "Bravo", "Bravo", "Bravo", "Charlie", "Charlie"), + col("Key2", null, null, "Delta", "Echo", null, "Delta", "Echo", null, "Echo"), + col("Sentinel", iv(1, 2, 3, 4, 5, 6, 7), iv(1, 3), iv(1), iv(3), iv(2, 6, 7), iv(2), iv(6, 7), + iv(4, 5), iv(4, 5))) + .update("Sum=sum(Sentinel)"); + } + + private static Table secondExpectedGrouped(RollupTable rollup1) { + return TableTools.newTable(intCol(rollup1.getRowDepthColumn().name(), 1, 2, 3, 3, 2, 3, 3, 2, 3), + booleanCol(rollup1.getRowExpandedColumn().name(), true, true, null, null, true, null, null, + true, null), + col("Key1", null, "Alpha", "Alpha", "Alpha", "Bravo", "Bravo", "Bravo", "Charlie", "Charlie"), + col("Key2", null, null, "Delta", "Echo", null, "Delta", "Echo", null, "Echo"), + col("Sentinel", iv(1, 2, 3, 4, 5, 7, 8, 9), iv(1, 3, 8), iv(1), iv(3, 8), iv(2, 7), iv(2), iv(7), + iv(4, 5, 9), iv(4, 5, 9))) + .update("Sum=sum(Sentinel)"); + } + + private static @NonNull IntVector iv(final int... ints) { + return new IntVectorDirect(ints); + } + + @Test + public void testRollupGroupIncremental() { + final QueryTable source = TstUtils.testRefreshingTable( + stringCol("Key1", "Alpha", "Bravo", "Alpha", "Charlie", "Charlie", "Bravo", "Bravo"), + stringCol("Key2", "Delta", "Delta", "Echo", "Echo", "Echo", "Echo", "Echo"), + intCol("Sentinel", 1, 2, 3, 4, 5, 6, 7)); + + final RollupTable rollup1 = + source.rollup(List.of(AggGroup("Sentinel"), AggSum("Sum=Sentinel")), "Key1", "Key2"); + + final String[] arrayWithNull = new String[1]; + final Table keyTable = newTable( + intCol(rollup1.getRowDepthColumn().name(), 0), + stringCol("Key1", arrayWithNull), + stringCol("Key2", arrayWithNull), + byteCol("Action", HierarchicalTable.KEY_TABLE_ACTION_EXPAND_ALL)); + + final HierarchicalTable.SnapshotState ss1 = rollup1.makeSnapshotState(); + final Table snapshot = + snapshotToTable(rollup1, ss1, keyTable, ColumnName.of("Action"), null, RowSetFactory.flat(30)); + TableTools.showWithRowSet(snapshot); + + final Table expected = initialExpectedGrouped(rollup1); + assertTableEquals(expected, snapshot.dropColumns("__EXPOSED_GROUP_ROW_SETS__")); + freeSnapshotTableChunks(snapshot); + + final ControlledUpdateGraph cug = source.getUpdateGraph().cast(); + cug.runWithinUnitTestCycle(() -> { + addToTable(source, i(10, 11), stringCol("Key1", "Alpha", "Charlie"), stringCol("Key2", "Echo", "Echo"), + intCol("Sentinel", 8, 9)); + removeRows(source, i(5)); + source.notifyListeners( + new TableUpdateImpl(i(10, 11), i(5), i(), RowSetShiftData.EMPTY, ModifiedColumnSet.EMPTY)); + }); + + final Table snapshot2 = + snapshotToTable(rollup1, ss1, keyTable, ColumnName.of("Action"), null, RowSetFactory.flat(30)); + TableTools.showWithRowSet(snapshot2); + Table expected2 = secondExpectedGrouped(rollup1); + TableTools.showWithRowSet(expected2); + assertTableEquals(expected2, snapshot2.dropColumns("__EXPOSED_GROUP_ROW_SETS__")); + freeSnapshotTableChunks(snapshot2); + } } From 453aab1439d36e3e892ad9cd5e59808ed76c797b Mon Sep 17 00:00:00 2001 From: "Charles P. Wright" Date: Wed, 10 Dec 2025 15:04:32 -0500 Subject: [PATCH 02/24] copilot review --- .../table/impl/by/AggregationProcessor.java | 2 +- ...r.java => GroupByReaggregateOperator.java} | 84 ++++++++----------- .../impl/hierarchical/RollupTableImpl.java | 1 - .../aggregate/AggregateColumnSource.java | 5 ++ 4 files changed, 39 insertions(+), 53 deletions(-) rename engine/table/src/main/java/io/deephaven/engine/table/impl/by/{GroupByReaggreagateOperator.java => GroupByReaggregateOperator.java} (87%) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index a4cbde5b73c..69112d76250 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -1273,7 +1273,7 @@ public void visit(AggSpecGroup group) { for (int ii = 0; ii < resultPairs.size(); ++ii) { pairs[ii] = new MatchPair(resultPairs.get(ii).output().name(), resultPairs.get(ii).output().name()); } - addOperator(new GroupByReaggreagateOperator(table, true, EXPOSED_GROUP_ROW_SETS.name(), pairs), groupRowSet, + addOperator(new GroupByReaggregateOperator(table, true, EXPOSED_GROUP_ROW_SETS.name(), pairs), groupRowSet, EXPOSED_GROUP_ROW_SETS.name()); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggreagateOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java similarity index 87% rename from engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggreagateOperator.java rename to engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java index 164d0eb35a2..db9cb248a3d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggreagateOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java @@ -27,10 +27,23 @@ import static io.deephaven.engine.table.impl.sources.ArrayBackedColumnSource.BLOCK_SIZE; /** - * An {@link IterativeChunkedAggregationOperator} used in the implementation of {@link Table#groupBy}, - * {@link io.deephaven.api.agg.spec.AggSpecGroup}, and {@link io.deephaven.api.agg.Aggregation#AggGroup(String...)}. + * An {@link IterativeChunkedAggregationOperator} used to re-aggregate the results of an + * {@link io.deephaven.api.agg.Aggregation#AggGroup(String...) AggGroup} as part of a rollup. + * + *

+ * The operator is fundamentally different than the {@link GroupByChunkedOperator}. Rather than examining row keys, it + * listens to the rollup's base (or intermediate) level and reads the exposed RowSet column. The relevant RowSets are + * added to a random builder for each state while processing an update (or initialization). At the end of the update + * cycle, it builds the rowsets and updates an internal ObjectArraySource of RowSets. + *

+ * + *

+ * The resulting column sources are once again {@link AggregateColumnSource}, which reuse the wrapped aggregated column + * source from the source table (thus each level of the rollup uses the original table's sources as the input to the + * AggregateColumnSources -- not the immediately prior level). + *

*/ -public final class GroupByReaggreagateOperator implements IterativeChunkedAggregationOperator { +public final class GroupByReaggregateOperator implements IterativeChunkedAggregationOperator { private final QueryTable inputTable; private final boolean registeredWithHelper; @@ -43,19 +56,13 @@ public final class GroupByReaggreagateOperator implements IterativeChunkedAggreg private final String[] inputColumnNames; - - private final Map> inputSources; - - private final Map> inputAggregatedColumns; private final Map> resultAggregatedColumns; - private final ModifiedColumnSet aggregationInputsModifiedColumnSet; private RowSetBuilderRandom stepDestinationsModified; - private boolean stepValuesModified; private boolean initialized; - public GroupByReaggreagateOperator( + public GroupByReaggregateOperator( @NotNull final QueryTable inputTable, final boolean registeredWithHelper, @Nullable final String exposeRowSetsAs, @@ -64,44 +71,33 @@ public GroupByReaggreagateOperator( this.registeredWithHelper = registeredWithHelper; this.exposeRowSetsAs = exposeRowSetsAs; + if (exposeRowSetsAs == null) { + throw new IllegalArgumentException("Must exposing group RowSets for rollup."); + } + live = inputTable.isRefreshing(); rowSets = new ObjectArraySource<>(WritableRowSet.class); addedBuilders = new ObjectArraySource<>(Object.class); - inputSources = new LinkedHashMap<>(aggregatedColumnPairs.length); - - inputAggregatedColumns = new LinkedHashMap<>(aggregatedColumnPairs.length); resultAggregatedColumns = new LinkedHashMap<>(aggregatedColumnPairs.length); Arrays.stream(aggregatedColumnPairs).forEach(pair -> { final ColumnSource source = inputTable.getColumnSource(pair.rightColumn()); - final ColumnSource realSource; - if (source instanceof AggregateColumnSource) { - final AggregateColumnSource aggregateSource = (AggregateColumnSource) source; - realSource = aggregateSource.getAggregatedSource(); - inputSources.put(pair.rightColumn(), aggregateSource); - } else { + if (!(source instanceof AggregateColumnSource)) { throw new IllegalStateException("Expect to reaggregate AggregateColumnSources for a group operation."); } + @SuppressWarnings("rawtypes") + final ColumnSource realSource = ((AggregateColumnSource) source).getAggregatedSource(); final AggregateColumnSource aggregateColumnSource = AggregateColumnSource.make(realSource, rowSets); - inputAggregatedColumns.put(pair.rightColumn(), aggregateColumnSource); resultAggregatedColumns.put(pair.leftColumn(), aggregateColumnSource); }); - if (exposeRowSetsAs != null && resultAggregatedColumns.containsKey(exposeRowSetsAs)) { + if (resultAggregatedColumns.containsKey(exposeRowSetsAs)) { throw new IllegalArgumentException(String.format( "Exposing group RowSets as %s, but this conflicts with a requested grouped output column name", exposeRowSetsAs)); } inputColumnNames = MatchPair.getRightColumns(aggregatedColumnPairs); - if (live) { - final String[] allInputs = Arrays.copyOf(inputColumnNames, inputColumnNames.length + 1); - allInputs[allInputs.length - 1] = exposeRowSetsAs; - aggregationInputsModifiedColumnSet = inputTable.newModifiedColumnSet(allInputs); - removedBuilders = new ObjectArraySource<>(Object.class); - } else { - aggregationInputsModifiedColumnSet = null; - removedBuilders = null; - } + removedBuilders = live ? new ObjectArraySource<>(Object.class) : null; initialized = false; } @@ -160,7 +156,7 @@ public void shiftChunk(final BucketedContext bucketedContext, final Chunk postShiftRowKeys, @NotNull final IntChunk destinations, @NotNull final IntChunk startPositions, @NotNull final IntChunk length, @NotNull final WritableBooleanChunk stateModified) { - throw new IllegalStateException("GroupByReaggreagateOperator should never be called with shiftChunk"); + throw new IllegalStateException("GroupByReaggregateOperator should never be called with shiftChunk"); } @Override @@ -221,7 +217,7 @@ private void removeChunk(@NotNull final ObjectChunk ro stepDestinationsModified.addKey(destination); } - private void modifyChunk(ObjectChunk previousValues, + private void modifyChunk(ObjectChunk previousValues, ObjectChunk newValues, int start, int length, @@ -282,14 +278,11 @@ public void ensureCapacity(final long tableSize) { @Override public Map> getResultColumns() { - if (exposeRowSetsAs != null) { - final Map> allResultColumns = - new LinkedHashMap<>(resultAggregatedColumns.size() + 1); - allResultColumns.put(exposeRowSetsAs, rowSets); - allResultColumns.putAll(resultAggregatedColumns); - return allResultColumns; - } - return resultAggregatedColumns; + final Map> allResultColumns = + new LinkedHashMap<>(resultAggregatedColumns.size() + 1); + allResultColumns.put(exposeRowSetsAs, rowSets); + allResultColumns.putAll(resultAggregatedColumns); + return allResultColumns; } @Override @@ -316,7 +309,6 @@ public UnaryOperator initializeRefreshing( private class InputToResultModifiedColumnSetFactory implements UnaryOperator { private final ModifiedColumnSet updateModifiedColumnSet; - private final ModifiedColumnSet allResultColumns; private final ModifiedColumnSet.Transformer aggregatedColumnsTransformer; private InputToResultModifiedColumnSetFactory( @@ -324,14 +316,6 @@ private InputToResultModifiedColumnSetFactory( @NotNull final String[] resultAggregatedColumnNames) { updateModifiedColumnSet = new ModifiedColumnSet(resultTable.getModifiedColumnSetForUpdates()); - if (exposeRowSetsAs != null) { - // resultAggregatedColumnNames may be empty (e.g. when the row set is the only result column) - allResultColumns = resultTable.newModifiedColumnSet(exposeRowSetsAs); - allResultColumns.setAll(resultAggregatedColumnNames); - } else { - allResultColumns = resultTable.newModifiedColumnSet(resultAggregatedColumnNames); - } - final String[] allInputs = Arrays.copyOf(inputColumnNames, inputColumnNames.length + 1); allInputs[allInputs.length - 1] = exposeRowSetsAs; final ModifiedColumnSet[] affectedColumns = new ModifiedColumnSet[allInputs.length]; @@ -352,8 +336,6 @@ public ModifiedColumnSet apply(@NotNull final ModifiedColumnSet upstreamModified @Override public void resetForStep(@NotNull final TableUpdate upstream, final int startingDestinationsCount) { - stepValuesModified = upstream.modified().isNonempty() && upstream.modifiedColumnSet().nonempty() - && upstream.modifiedColumnSet().containsAny(aggregationInputsModifiedColumnSet); stepDestinationsModified = new BitmapRandomBuilder(startingDestinationsCount); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/hierarchical/RollupTableImpl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/hierarchical/RollupTableImpl.java index 2339699f505..b96631dabe3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/hierarchical/RollupTableImpl.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/hierarchical/RollupTableImpl.java @@ -27,7 +27,6 @@ import io.deephaven.engine.table.impl.select.WhereFilter; import io.deephaven.engine.table.impl.sources.NullValueColumnSource; import io.deephaven.engine.table.impl.util.RowRedirection; -import io.deephaven.engine.util.PrintListener; import io.deephaven.util.annotations.InternalUseOnly; import io.deephaven.util.type.TypeUtils; import org.apache.commons.lang3.mutable.MutableObject; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/aggregate/AggregateColumnSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/aggregate/AggregateColumnSource.java index 50052ee268b..9266160d718 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/aggregate/AggregateColumnSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/aggregate/AggregateColumnSource.java @@ -22,6 +22,11 @@ public interface AggregateColumnSource, UngroupedColumnSource ungrouped(); + /** + * Get the underlying source that is aggregated by this ColumnSource. + * + * @return the underlying source that is aggregated by this ColumnSource + */ ColumnSource getAggregatedSource(); static , DATA_TYPE> AggregateColumnSource make( From 4959126ea044195e33d80840ccfa5f378feac81c Mon Sep 17 00:00:00 2001 From: "Charles P. Wright" Date: Wed, 10 Dec 2025 15:25:08 -0500 Subject: [PATCH 03/24] Update engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com> --- .../engine/table/impl/by/GroupByReaggregateOperator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java index db9cb248a3d..059fe95b516 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java @@ -72,7 +72,7 @@ public GroupByReaggregateOperator( this.exposeRowSetsAs = exposeRowSetsAs; if (exposeRowSetsAs == null) { - throw new IllegalArgumentException("Must exposing group RowSets for rollup."); + throw new IllegalArgumentException("Must expose group RowSets for rollup."); } live = inputTable.isRefreshing(); From fae01cbbfc98162a16403ba8870f785cc3fb2e8c Mon Sep 17 00:00:00 2001 From: "Charles P. Wright" Date: Wed, 10 Dec 2025 15:25:31 -0500 Subject: [PATCH 04/24] Update engine/table/src/main/java/io/deephaven/engine/table/impl/sources/aggregate/AggregateColumnSource.java Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com> --- .../table/impl/sources/aggregate/AggregateColumnSource.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/aggregate/AggregateColumnSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/aggregate/AggregateColumnSource.java index 9266160d718..7f40094a90c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/aggregate/AggregateColumnSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/aggregate/AggregateColumnSource.java @@ -24,7 +24,7 @@ public interface AggregateColumnSource, /** * Get the underlying source that is aggregated by this ColumnSource. - * + * * @return the underlying source that is aggregated by this ColumnSource */ ColumnSource getAggregatedSource(); From 2ca4ae42b3396c910fa4b7f52395474857079a26 Mon Sep 17 00:00:00 2001 From: "Charles P. Wright" Date: Thu, 11 Dec 2025 08:11:00 -0500 Subject: [PATCH 05/24] formula stuff --- .../table/impl/by/AggregationProcessor.java | 192 +++++++++++++++--- .../by/FormulaMultiColumnChunkedOperator.java | 19 +- .../table/impl/by/GroupByChunkedOperator.java | 11 +- .../engine/table/impl/by/GroupByOperator.java | 23 +++ .../impl/by/GroupByReaggregateOperator.java | 16 +- .../impl/hierarchical/RollupTableImpl.java | 10 +- .../engine/table/impl/TestAggBy.java | 21 ++ .../engine/table/impl/TestRollupTable.java | 34 ++++ 8 files changed, 277 insertions(+), 49 deletions(-) create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByOperator.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index 69112d76250..939fba13bdc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -33,11 +33,8 @@ import io.deephaven.base.verify.Assert; import io.deephaven.chunk.ChunkType; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ChunkSource; -import io.deephaven.engine.table.ColumnDefinition; -import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.*; -import io.deephaven.engine.table.Table; import io.deephaven.engine.table.impl.by.rollup.NullColumns; import io.deephaven.engine.table.impl.by.rollup.RollupAggregation; import io.deephaven.engine.table.impl.by.rollup.RollupAggregationOutputs; @@ -93,10 +90,12 @@ import io.deephaven.engine.table.impl.by.ssmpercentile.SsmChunkedPercentileOperator; import io.deephaven.engine.table.impl.select.SelectColumn; import io.deephaven.engine.table.impl.select.WhereFilter; +import io.deephaven.engine.table.impl.sources.IntegerSingleValueSource; import io.deephaven.engine.table.impl.sources.ReinterpretUtils; import io.deephaven.engine.table.impl.ssms.SegmentedSortedMultiSet; import io.deephaven.engine.table.impl.util.freezeby.FreezeByCountOperator; import io.deephaven.engine.table.impl.util.freezeby.FreezeByOperator; +import io.deephaven.qst.type.IntType; import io.deephaven.time.DateTimeUtils; import io.deephaven.util.annotations.FinalDefault; import io.deephaven.util.type.ArrayTypeUtils; @@ -206,7 +205,8 @@ public static AggregationContextFactory forRollupBase( public static AggregationContextFactory forRollupReaggregated( @NotNull final Collection aggregations, @NotNull final Collection> nullColumns, - @NotNull final ColumnName rollupColumn) { + @NotNull final ColumnName rollupColumn, + @NotNull final Table source) { if (aggregations.stream().anyMatch(agg -> agg instanceof Partition)) { rollupUnsupported("Partition"); } @@ -214,7 +214,7 @@ public static AggregationContextFactory forRollupReaggregated( reaggregations.add(RollupAggregation.nullColumns(nullColumns)); reaggregations.addAll(aggregations); reaggregations.add(Partition.of(rollupColumn)); - return new AggregationProcessor(reaggregations, Type.ROLLUP_REAGGREGATED); + return new WithSource(reaggregations, Type.ROLLUP_REAGGREGATED, source); } /** @@ -263,6 +263,8 @@ public static AggregationContextFactory forSelectDistinct() { public static final ColumnName EXPOSED_GROUP_ROW_SETS = ColumnName.of("__EXPOSED_GROUP_ROW_SETS__"); + public static final ColumnName ROLLUP_FORMULA_DEPTH = ColumnName.of("__FORMULA_DEPTH__"); + /** * Create a trivial {@link AggregationContextFactory} to {@link Aggregation#AggGroup(String...) group} the input * table and expose the group {@link io.deephaven.engine.rowset.RowSet row sets} as {@link #EXPOSED_GROUP_ROW_SETS}. @@ -803,16 +805,7 @@ public void visit(@NotNull final Formula formula) { // Get or create a column definition map composed of vectors of the original column types (or scalars when // part of the key columns). final Set groupByColumnSet = Set.of(groupByColumnNames); - if (vectorColumnDefinitions == null) { - vectorColumnDefinitions = table.getDefinition().getColumnStream().collect(Collectors.toMap( - ColumnDefinition::getName, - (final ColumnDefinition cd) -> groupByColumnSet.contains(cd.getName()) - ? cd - : ColumnDefinition.fromGenericType( - cd.getName(), - VectorFactory.forElementType(cd.getDataType()).vectorType(), - cd.getDataType()))); - } + maybeInitializeVectorColumns(groupByColumnSet, table.getDefinition(), Map.of()); // Get the input column names from the formula and provide them to the groupBy operator final String[] allInputColumns = @@ -823,20 +816,13 @@ public void visit(@NotNull final Formula formula) { final String[] inputKeyColumns = partitioned.get(true).toArray(String[]::new); final String[] inputNonKeyColumns = partitioned.get(false).toArray(String[]::new); - if (!selectColumn.getColumnArrays().isEmpty()) { - throw new IllegalArgumentException("AggFormula does not support column arrays (" - + selectColumn.getColumnArrays() + ")"); - } - if (selectColumn.hasVirtualRowVariables()) { - throw new IllegalArgumentException("AggFormula does not support virtual row variables"); - } + validateSelectColumnForFormula(selectColumn); // TODO: re-use shared groupBy operators (https://github.com/deephaven/deephaven-core/issues/6363) - final GroupByChunkedOperator groupByChunkedOperator = new GroupByChunkedOperator(table, false, null, - Arrays.stream(inputNonKeyColumns).map(col -> MatchPair.of(Pair.parse(col))) - .toArray(MatchPair[]::new)); + final GroupByChunkedOperator groupByChunkedOperator = + makeGroupByOperatorForFormula(inputNonKeyColumns, table, null); final FormulaMultiColumnChunkedOperator op = new FormulaMultiColumnChunkedOperator(table, - groupByChunkedOperator, true, selectColumn, inputKeyColumns); + groupByChunkedOperator, true, selectColumn, inputKeyColumns, null); addNoInputOperator(op); } @@ -972,6 +958,45 @@ public void visit(@NotNull final AggSpecVar var) { } } + private static void validateSelectColumnForFormula(SelectColumn selectColumn) { + if (!selectColumn.getColumnArrays().isEmpty()) { + throw new IllegalArgumentException("AggFormula does not support column arrays (" + + selectColumn.getColumnArrays() + ")"); + } + if (selectColumn.hasVirtualRowVariables()) { + throw new IllegalArgumentException("AggFormula does not support virtual row variables"); + } + } + + private void maybeInitializeVectorColumns(Set groupByColumnSet, final TableDefinition definition, + Map> extraColumns) { + if (vectorColumnDefinitions != null) { + return; + } + vectorColumnDefinitions = new LinkedHashMap<>(); + definition.getColumnStream().forEach(cd -> { + ColumnDefinition resultDefinition; + if (groupByColumnSet.contains(cd.getName())) { + resultDefinition = cd; + } else { + resultDefinition = ColumnDefinition.fromGenericType( + cd.getName(), + VectorFactory.forElementType(cd.getDataType()).vectorType(), + cd.getDataType()); + } + vectorColumnDefinitions.put(cd.getName(), resultDefinition); + }); + vectorColumnDefinitions.putAll(extraColumns); + } + + + private @NotNull GroupByChunkedOperator makeGroupByOperatorForFormula(String[] inputNonKeyColumns, + final QueryTable table, final String exposedRowsets) { + return new GroupByChunkedOperator(table, false, exposedRowsets, + Arrays.stream(inputNonKeyColumns).map(col -> MatchPair.of(Pair.parse(col))) + .toArray(MatchPair[]::new)); + } + // ----------------------------------------------------------------------------------------------------------------- // Rollup Unsupported Operations // ----------------------------------------------------------------------------------------------------------------- @@ -994,12 +1019,6 @@ default void visit(@NotNull final LastRowKey lastRowKey) { rollupUnsupported("LastRowKey"); } - @Override - @FinalDefault - default void visit(@NotNull final Formula formula) { - rollupUnsupported("Formula"); - } - // ------------------------------------------------------------------------------------------------------------- // AggSpec.Visitor for unsupported column aggregation specs // ------------------------------------------------------------------------------------------------------------- @@ -1060,6 +1079,7 @@ private static void rollupUnsupported(@NotNull final String operationName, final */ private final class RollupBaseConverter extends Converter implements RollupAggregation.Visitor, UnsupportedRollupAggregations { + private final QueryCompilerRequestProcessor.BatchProcessor compilationProcessor; private int nextColumnIdentifier = 0; @@ -1068,6 +1088,14 @@ private RollupBaseConverter( final boolean requireStateChangeRecorder, @NotNull final String... groupByColumnNames) { super(table, requireStateChangeRecorder, groupByColumnNames); + this.compilationProcessor = QueryCompilerRequestProcessor.batch(); + } + + @Override + AggregationContext build() { + final AggregationContext resultContext = super.build(); + compilationProcessor.compile(); + return resultContext; } // ------------------------------------------------------------------------------------------------------------- @@ -1109,6 +1137,40 @@ public void visit(AggSpecGroup group) { MatchPair.fromPairs(resultPairs))); } + @Override + public void visit(Formula formula) { + final SelectColumn selectColumn = SelectColumn.of(formula.selectable()); + + // Get or create a column definition map composed of vectors of the original column types (or scalars when + // part of the key columns). + final Set groupByColumnSet = Set.of(groupByColumnNames); + // For the base of a rollup, we use the original table definition, but tack on a rollup depth column + maybeInitializeVectorColumns(groupByColumnSet, table.getDefinition(), Map.of(ROLLUP_FORMULA_DEPTH.name(), + ColumnDefinition.of(ROLLUP_FORMULA_DEPTH.name(), IntType.of()))); + + // Get the input column names from the formula and provide them to the groupBy operator + final String[] allInputColumns = + selectColumn.initDef(vectorColumnDefinitions, compilationProcessor).toArray(String[]::new); + + final Map> partitioned = Arrays.stream(allInputColumns) + .collect(Collectors.partitioningBy( + o -> groupByColumnSet.contains(o) || ROLLUP_FORMULA_DEPTH.name().equals(o))); + final String[] inputKeyColumns = partitioned.get(true).toArray(String[]::new); + final String[] inputNonKeyColumns = partitioned.get(false).toArray(String[]::new); + + validateSelectColumnForFormula(selectColumn); + // TODO: re-use shared groupBy operators (https://github.com/deephaven/deephaven-core/issues/6363) + final GroupByChunkedOperator groupByChunkedOperator = + makeGroupByOperatorForFormula(inputNonKeyColumns, table, EXPOSED_GROUP_ROW_SETS.name()); + + final IntegerSingleValueSource depthSource = new IntegerSingleValueSource(); + depthSource.set(groupByColumnNames.length); + + final FormulaMultiColumnChunkedOperator op = new FormulaMultiColumnChunkedOperator(table, + groupByChunkedOperator, true, selectColumn, inputKeyColumns, depthSource); + addNoInputOperator(op); + } + // ------------------------------------------------------------------------------------------------------------- // AggSpec.Visitor // ------------------------------------------------------------------------------------------------------------- @@ -1216,6 +1278,7 @@ IterativeChunkedAggregationOperator apply( private final class RollupReaggregatedConverter extends Converter implements RollupAggregation.Visitor, UnsupportedRollupAggregations { + private final QueryCompilerRequestProcessor.BatchProcessor compilationProcessor; private int nextColumnIdentifier = 0; private RollupReaggregatedConverter( @@ -1223,6 +1286,14 @@ private RollupReaggregatedConverter( final boolean requireStateChangeRecorder, @NotNull final String... groupByColumnNames) { super(table, requireStateChangeRecorder, groupByColumnNames); + this.compilationProcessor = QueryCompilerRequestProcessor.batch(); + } + + @Override + AggregationContext build() { + final AggregationContext resultContext = super.build(); + compilationProcessor.compile(); + return resultContext; } // ------------------------------------------------------------------------------------------------------------- @@ -1277,6 +1348,51 @@ public void visit(AggSpecGroup group) { EXPOSED_GROUP_ROW_SETS.name()); } + @Override + public void visit(Formula formula) { + final SelectColumn selectColumn = SelectColumn.of(formula.selectable()); + + // Get or create a column definition map composed of vectors of the original column types (or scalars when + // part of the key columns). + final Set groupByColumnSet = Set.of(groupByColumnNames); + + // for a reaggregated formula, we can't use the input definition as is; we want to use the definition from + // the source table; but tack on our rollup depth column + AggregationProcessor thisProcessor = AggregationProcessor.this; + final TableDefinition sourceDefinition = ((WithSource) thisProcessor).source.getDefinition(); + maybeInitializeVectorColumns(groupByColumnSet, sourceDefinition, Map.of(ROLLUP_FORMULA_DEPTH.name(), + ColumnDefinition.of(ROLLUP_FORMULA_DEPTH.name(), IntType.of()))); + + // Get the input column names from the formula and provide them to the groupBy operator + final String[] allInputColumns = + selectColumn.initDef(vectorColumnDefinitions, compilationProcessor).toArray(String[]::new); + + final Map> partitioned = Arrays.stream(allInputColumns) + .collect(Collectors.partitioningBy( + o -> groupByColumnSet.contains(o) || ROLLUP_FORMULA_DEPTH.name().equals(o))); + final String[] inputKeyColumns = partitioned.get(true).toArray(String[]::new); + final String[] inputNonKeyColumns = partitioned.get(false).toArray(String[]::new); + + validateSelectColumnForFormula(selectColumn); + // TODO: re-use shared groupBy operators (https://github.com/deephaven/deephaven-core/issues/6363) + // final GroupByChunkedOperator groupByChunkedOperator = makeGroupByOperatorForFormula(inputNonKeyColumns, + // table); + + final ColumnSource groupRowSet = table.getColumnSource(EXPOSED_GROUP_ROW_SETS.name()); + + final MatchPair[] groupPairs = Arrays.stream(inputNonKeyColumns).map(col -> MatchPair.of(Pair.parse(col))) + .toArray(MatchPair[]::new); + GroupByReaggregateOperator groupByOperator = + new GroupByReaggregateOperator(table, false, EXPOSED_GROUP_ROW_SETS.name(), groupPairs); + + final IntegerSingleValueSource depthSource = new IntegerSingleValueSource(); + depthSource.set(groupByColumnNames.length); + + final FormulaMultiColumnChunkedOperator op = new FormulaMultiColumnChunkedOperator(table, groupByOperator, + true, selectColumn, inputKeyColumns, depthSource); + addOperator(op, groupRowSet, EXPOSED_GROUP_ROW_SETS.name()); + } + // ------------------------------------------------------------------------------------------------------------- // AggSpec.Visitor // ------------------------------------------------------------------------------------------------------------- @@ -2171,4 +2287,14 @@ public static AggregationRowLookup getRowLookup(@NotNull final Table aggregation Assert.neqNull(value, "aggregation result row lookup"); return (AggregationRowLookup) value; } + + private static class WithSource extends AggregationProcessor { + private final @NotNull Table source; + + private WithSource(@NotNull Collection aggregations, @NotNull Type type, + @NotNull Table source) { + super(aggregations, type); + this.source = source; + } + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaMultiColumnChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaMultiColumnChunkedOperator.java index 2bf75160fdf..6bb66faf6d0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaMultiColumnChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaMultiColumnChunkedOperator.java @@ -19,6 +19,7 @@ import io.deephaven.engine.table.impl.sources.ArrayBackedColumnSource; import io.deephaven.util.SafeCloseable; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.util.Arrays; import java.util.HashMap; @@ -34,11 +35,13 @@ class FormulaMultiColumnChunkedOperator implements IterativeChunkedAggregationOp private final QueryTable inputTable; - private final GroupByChunkedOperator groupBy; + private final GroupByOperator groupBy; private final boolean delegateToBy; private final SelectColumn selectColumn; private final WritableColumnSource resultColumn; private final String[] inputKeyColumns; + @Nullable + private final ColumnSource formulaDepthSource; private ChunkSource formulaDataSource; @@ -63,15 +66,17 @@ class FormulaMultiColumnChunkedOperator implements IterativeChunkedAggregationOp */ FormulaMultiColumnChunkedOperator( @NotNull final QueryTable inputTable, - @NotNull final GroupByChunkedOperator groupBy, + @NotNull final GroupByOperator groupBy, final boolean delegateToBy, @NotNull final SelectColumn selectColumn, - @NotNull final String[] inputKeyColumns) { + @NotNull final String[] inputKeyColumns, + @Nullable final ColumnSource formulaDepthSource) { this.inputTable = inputTable; this.groupBy = groupBy; this.delegateToBy = delegateToBy; this.selectColumn = selectColumn; this.inputKeyColumns = inputKeyColumns; + this.formulaDepthSource = formulaDepthSource; resultColumn = ArrayBackedColumnSource.getMemoryColumnSource( 0, selectColumn.getReturnedType(), selectColumn.getReturnedComponentType()); @@ -199,7 +204,7 @@ public boolean modifyRowKeys(final SingletonContext context, @Override public boolean requiresRowKeys() { - return delegateToBy; + return delegateToBy && groupBy.requiresRowKeys(); } @Override @@ -222,13 +227,14 @@ public void propagateInitialState(@NotNull final QueryTable resultTable, int sta } final Map> sourceColumns; - if (inputKeyColumns.length == 0) { + if (inputKeyColumns.length == 0 && formulaDepthSource == null) { // noinspection unchecked sourceColumns = (Map>) groupBy.getInputResultColumns(); } else { final Map> columnSourceMap = resultTable.getColumnSourceMap(); sourceColumns = new HashMap<>(groupBy.getInputResultColumns()); Arrays.stream(inputKeyColumns).forEach(col -> sourceColumns.put(col, columnSourceMap.get(col))); + sourceColumns.put(AggregationProcessor.ROLLUP_FORMULA_DEPTH.name(), formulaDepthSource); } selectColumn.initInputs(resultTable.getRowSet(), sourceColumns); formulaDataSource = selectColumn.getDataView(); @@ -263,8 +269,7 @@ public UnaryOperator initializeRefreshing(@NotNull final Quer final String[] inputColumnNames = selectColumn.getColumns().toArray(String[]::new); final ModifiedColumnSet inputMCS = inputTable.newModifiedColumnSet(inputColumnNames); return inputToResultModifiedColumnSetFactory = input -> { - if (groupBy.getSomeKeyHasAddsOrRemoves() || - (groupBy.getSomeKeyHasModifies() && input.containsAny(inputMCS))) { + if (groupBy.hasModifications(input.containsAny(inputMCS))) { return resultMCS; } return ModifiedColumnSet.EMPTY; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java index f5d28a99731..4dd8e6ddfa3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java @@ -33,7 +33,7 @@ * An {@link IterativeChunkedAggregationOperator} used in the implementation of {@link Table#groupBy}, * {@link io.deephaven.api.agg.spec.AggSpecGroup}, and {@link io.deephaven.api.agg.Aggregation#AggGroup(String...)}. */ -public final class GroupByChunkedOperator implements IterativeChunkedAggregationOperator { +public final class GroupByChunkedOperator implements GroupByOperator { private final QueryTable inputTable; private final boolean registeredWithHelper; @@ -392,9 +392,7 @@ public void ensureCapacity(final long tableSize) { return resultAggregatedColumns; } - /** - * Get a map from input column names to the corresponding output {@link ColumnSource}. - */ + @Override public Map> getInputResultColumns() { return inputAggregatedColumns; } @@ -635,4 +633,9 @@ boolean getSomeKeyHasAddsOrRemoves() { boolean getSomeKeyHasModifies() { return someKeyHasModifies; } + + @Override + public boolean hasModifications(boolean columnsModified) { + return getSomeKeyHasAddsOrRemoves() || (getSomeKeyHasModifies() && columnsModified); + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByOperator.java new file mode 100644 index 00000000000..3a0e688c7a3 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByOperator.java @@ -0,0 +1,23 @@ +// +// Copyright (c) 2016-2025 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.by; + +import io.deephaven.engine.table.ColumnSource; + +import java.util.Map; + +public interface GroupByOperator extends IterativeChunkedAggregationOperator { + /** + * Get a map from input column names to the corresponding output {@link ColumnSource}. + */ + Map> getInputResultColumns(); + + /** + * Given that there have been modified input columns, should we propagate changes? + * + * @param columnsModified have any of the input columns been modified (as per the MCS)? + * @return true if we have modified our output (e.g., because of additions or modifications) + */ + boolean hasModifications(final boolean columnsModified); +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java index 059fe95b516..475ef7698f0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java @@ -43,7 +43,7 @@ * AggregateColumnSources -- not the immediately prior level). *

*/ -public final class GroupByReaggregateOperator implements IterativeChunkedAggregationOperator { +public final class GroupByReaggregateOperator implements GroupByOperator { private final QueryTable inputTable; private final boolean registeredWithHelper; @@ -56,6 +56,7 @@ public final class GroupByReaggregateOperator implements IterativeChunkedAggrega private final String[] inputColumnNames; + private final Map> inputAggregatedColumns; private final Map> resultAggregatedColumns; private RowSetBuilderRandom stepDestinationsModified; @@ -79,6 +80,7 @@ public GroupByReaggregateOperator( rowSets = new ObjectArraySource<>(WritableRowSet.class); addedBuilders = new ObjectArraySource<>(Object.class); + inputAggregatedColumns = new LinkedHashMap<>(aggregatedColumnPairs.length); resultAggregatedColumns = new LinkedHashMap<>(aggregatedColumnPairs.length); Arrays.stream(aggregatedColumnPairs).forEach(pair -> { final ColumnSource source = inputTable.getColumnSource(pair.rightColumn()); @@ -89,6 +91,7 @@ public GroupByReaggregateOperator( final ColumnSource realSource = ((AggregateColumnSource) source).getAggregatedSource(); final AggregateColumnSource aggregateColumnSource = AggregateColumnSource.make(realSource, rowSets); resultAggregatedColumns.put(pair.leftColumn(), aggregateColumnSource); + inputAggregatedColumns.put(pair.rightColumn(), aggregateColumnSource); }); if (resultAggregatedColumns.containsKey(exposeRowSetsAs)) { @@ -306,6 +309,17 @@ public UnaryOperator initializeRefreshing( : null; } + @Override + public Map> getInputResultColumns() { + return inputAggregatedColumns; + } + + @Override + public boolean hasModifications(boolean columnsModified) { + /* TODO: FIX THIS. */ + return true; + } + private class InputToResultModifiedColumnSetFactory implements UnaryOperator { private final ModifiedColumnSet updateModifiedColumnSet; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/hierarchical/RollupTableImpl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/hierarchical/RollupTableImpl.java index b96631dabe3..0223cb01ded 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/hierarchical/RollupTableImpl.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/hierarchical/RollupTableImpl.java @@ -272,7 +272,7 @@ public RollupTable withFilter(@NotNull final Filter filter) { final AggregationRowLookup[] levelRowLookups = makeLevelRowLookupsArray(numLevels, filteredBaseLevelRowLookup); final ColumnSource[] levelNodeTableSources = makeLevelNodeTableSourcesArray( numLevels, filteredBaseLevel.getColumnSource(ROLLUP_COLUMN.name(), Table.class)); - rollupFromBase(levelTables, levelRowLookups, levelNodeTableSources, aggregations, groupByColumns); + rollupFromBase(levelTables, levelRowLookups, levelNodeTableSources, aggregations, groupByColumns, source); final WhereFilter[] newFilters; if (rollupKeyFilters == null) { @@ -589,7 +589,7 @@ public static RollupTable makeRollup( final AggregationRowLookup[] levelRowLookups = makeLevelRowLookupsArray(numLevels, getRowLookup(baseLevel)); final ColumnSource
[] levelNodeTableSources = makeLevelNodeTableSourcesArray( numLevels, baseLevel.getColumnSource(ROLLUP_COLUMN.name(), Table.class)); - rollupFromBase(levelTables, levelRowLookups, levelNodeTableSources, aggregations, groupByColumns); + rollupFromBase(levelTables, levelRowLookups, levelNodeTableSources, aggregations, groupByColumns, source); return new RollupTableImpl( attributes, @@ -670,13 +670,15 @@ private static ColumnSource
[] makeLevelNodeTableSourcesArray( * already filled * @param aggregations The aggregations * @param groupByColumns The group-by columns + * @param source */ private static void rollupFromBase( @NotNull final QueryTable[] levelTables, @NotNull final AggregationRowLookup[] levelRowLookups, @NotNull final ColumnSource
[] levelNodeTableSources, @NotNull final Collection aggregations, - @NotNull final Collection groupByColumns) { + @NotNull final Collection groupByColumns, + @NotNull final QueryTable source) { final Deque columnsToReaggregateBy = new ArrayDeque<>(groupByColumns); final Deque nullColumnNames = new ArrayDeque<>(groupByColumns.size()); int lastLevelIndex = levelTables.length - 1; @@ -688,7 +690,7 @@ private static void rollupFromBase( nullColumnNames.stream().map(lastLevelDefinition::getColumn).collect(Collectors.toList()); lastLevel = lastLevel.aggNoMemo( - AggregationProcessor.forRollupReaggregated(aggregations, nullColumns, ROLLUP_COLUMN), + AggregationProcessor.forRollupReaggregated(aggregations, nullColumns, ROLLUP_COLUMN, source), false, null, new ArrayList<>(columnsToReaggregateBy)); --lastLevelIndex; levelTables[lastLevelIndex] = lastLevel; diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestAggBy.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestAggBy.java index 6cd7f5db7ea..dcea413a1ce 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestAggBy.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestAggBy.java @@ -62,6 +62,27 @@ public void setUp() throws Exception { super.setUp(); } + @Test + public void testDoubleFormula() { + ColumnHolder aHolder = col("A", 0, 0, 1, 1, 0, 0, 1, 1, 0, 0); + ColumnHolder bHolder = col("B", 1, 2, 3, 4, 5, 6, 7, 8, 9, 10); + ColumnHolder cHolder = col("C", 1, 1, 1, 1, 1, 1, 1, 1, 1, 1); + Table table = TableTools.newTable(aHolder, bHolder, cHolder); + show(table); + assertEquals(10, table.size()); + assertEquals(2, table.groupBy("A").size()); + + Table minMax = table.aggBy( + List.of( + AggFormula("f_const=6.0 + 3"), + AggFormula("f_max=max(B)"), + AggFormula("f_sum_two_col=sum(B) + sum(C)")), + "A"); + show(minMax); + + assertEquals(2, minMax.size()); + } + @Test public void testBy() { ColumnHolder aHolder = col("A", 0, 0, 1, 1, 0, 0, 1, 1, 0, 0); diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java index 02c909cd323..0b9fc17eaf4 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java @@ -373,6 +373,40 @@ public void testRollupGroupStatic() { freeSnapshotTableChunks(snapshot); } + @Test + public void testRollupFormulaStatic() { + final Table source = TableTools.newTable( + stringCol("Key1", "Alpha", "Bravo", "Alpha", "Charlie", "Charlie", "Bravo", "Bravo"), + stringCol("Key2", "Delta", "Delta", "Echo", "Echo", "Echo", "Echo", "Echo"), + intCol("Sentinel", 1, 2, 3, 4, 5, 6, 7)); + TableTools.show(source); + + final RollupTable rollup1 = + source.rollup( + List.of(AggGroup("Sentinel"), AggSum("Sum=Sentinel"), + AggFormula("FSum", "__FORMULA_DEPTH__ == 0 ? max(Sentinel) : 1 + sum(Sentinel)")), + "Key1", "Key2"); + + final String[] arrayWithNull = new String[1]; + final Table keyTable = newTable( + intCol(rollup1.getRowDepthColumn().name(), 0), + stringCol("Key1", arrayWithNull), + stringCol("Key2", arrayWithNull), + byteCol("Action", HierarchicalTable.KEY_TABLE_ACTION_EXPAND_ALL)); + + final HierarchicalTable.SnapshotState ss1 = rollup1.makeSnapshotState(); + final Table snapshot = + snapshotToTable(rollup1, ss1, keyTable, ColumnName.of("Action"), null, RowSetFactory.flat(30)); + TableTools.showWithRowSet(snapshot); + + TableTools.show(snapshot.view(rollup1.getRowDepthColumn().name(), rollup1.getRowExpandedColumn().name(), "Key1", + "Key2", "Sentinel", "Sum", "FSum")); + + final Table expected = initialExpectedGrouped(rollup1).update("FSum=ii == 0 ? 7 : 1 + Sum"); + assertTableEquals(expected, snapshot.dropColumns("__EXPOSED_GROUP_ROW_SETS__")); + freeSnapshotTableChunks(snapshot); + } + private static Table initialExpectedGrouped(RollupTable rollup1) { return TableTools.newTable(intCol(rollup1.getRowDepthColumn().name(), 1, 2, 3, 3, 2, 3, 3, 2, 3), booleanCol(rollup1.getRowExpandedColumn().name(), true, true, null, null, true, null, null, From 52c0fb816fbee44e39859882121bb93fe24b156c Mon Sep 17 00:00:00 2001 From: "Charles P. Wright" Date: Fri, 12 Dec 2025 09:48:02 -0500 Subject: [PATCH 06/24] reagg formula examples --- .../table/impl/by/AggregationProcessor.java | 47 +++++++++++--- .../impl/by/GroupByReaggregateOperator.java | 8 +-- .../engine/table/impl/TestRollupTable.java | 61 +++++++++++++++++++ .../api/agg/AggregationDescriptions.java | 2 +- .../java/io/deephaven/api/agg/Formula.java | 11 +++- 5 files changed, 111 insertions(+), 18 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index 939fba13bdc..55762d02520 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -800,6 +800,7 @@ public void visit(@NotNull final Partition partition) { @Override public void visit(@NotNull final Formula formula) { + validateFormulaIsNotReaggregating(formula); final SelectColumn selectColumn = SelectColumn.of(formula.selectable()); // Get or create a column definition map composed of vectors of the original column types (or scalars when @@ -968,6 +969,12 @@ private static void validateSelectColumnForFormula(SelectColumn selectColumn) { } } + private static void validateFormulaIsNotReaggregating(Formula formula) { + if (formula.reaggregateAggregatedValues()) { + throw new IllegalArgumentException("AggFormula does not support reaggregating except in a rollup."); + } + } + private void maybeInitializeVectorColumns(Set groupByColumnSet, final TableDefinition definition, Map> extraColumns) { if (vectorColumnDefinitions != null) { @@ -1160,8 +1167,15 @@ public void visit(Formula formula) { validateSelectColumnForFormula(selectColumn); // TODO: re-use shared groupBy operators (https://github.com/deephaven/deephaven-core/issues/6363) - final GroupByChunkedOperator groupByChunkedOperator = - makeGroupByOperatorForFormula(inputNonKeyColumns, table, EXPOSED_GROUP_ROW_SETS.name()); + + final GroupByChunkedOperator groupByChunkedOperator; + + if (formula.reaggregateAggregatedValues()) { + groupByChunkedOperator = makeGroupByOperatorForFormula(inputNonKeyColumns, table, null); + } else { + groupByChunkedOperator = makeGroupByOperatorForFormula(inputNonKeyColumns, table, EXPOSED_GROUP_ROW_SETS.name()); + // TODO: the next level somehow needs access to the raw column sources so that we can continue the aggregation + } final IntegerSingleValueSource depthSource = new IntegerSingleValueSource(); depthSource.set(groupByColumnNames.length); @@ -1378,19 +1392,32 @@ public void visit(Formula formula) { // final GroupByChunkedOperator groupByChunkedOperator = makeGroupByOperatorForFormula(inputNonKeyColumns, // table); - final ColumnSource groupRowSet = table.getColumnSource(EXPOSED_GROUP_ROW_SETS.name()); - - final MatchPair[] groupPairs = Arrays.stream(inputNonKeyColumns).map(col -> MatchPair.of(Pair.parse(col))) + final MatchPair[] groupPairs = Arrays.stream(inputNonKeyColumns).map(col -> { + // reagg uses the output name + final Pair parse = Pair.parse(col); + return MatchPair.of(Pair.of(parse.output(), parse.output())); + }) .toArray(MatchPair[]::new); - GroupByReaggregateOperator groupByOperator = - new GroupByReaggregateOperator(table, false, EXPOSED_GROUP_ROW_SETS.name(), groupPairs); + + GroupByOperator groupByOperator; final IntegerSingleValueSource depthSource = new IntegerSingleValueSource(); depthSource.set(groupByColumnNames.length); - final FormulaMultiColumnChunkedOperator op = new FormulaMultiColumnChunkedOperator(table, groupByOperator, - true, selectColumn, inputKeyColumns, depthSource); - addOperator(op, groupRowSet, EXPOSED_GROUP_ROW_SETS.name()); + if (formula.reaggregateAggregatedValues()) { + groupByOperator = new GroupByChunkedOperator(table, false, null, groupPairs); + + final FormulaMultiColumnChunkedOperator op = new FormulaMultiColumnChunkedOperator(table, groupByOperator, + true, selectColumn, inputKeyColumns, depthSource); + + addOperator(op, null, inputNonKeyColumns); + } else { + final ColumnSource groupRowSet = table.getColumnSource(EXPOSED_GROUP_ROW_SETS.name()); + groupByOperator = new GroupByReaggregateOperator(table, false, EXPOSED_GROUP_ROW_SETS.name(), groupPairs); + final FormulaMultiColumnChunkedOperator op = new FormulaMultiColumnChunkedOperator(table, groupByOperator, + true, selectColumn, inputKeyColumns, depthSource); + addOperator(op, groupRowSet, EXPOSED_GROUP_ROW_SETS.name()); + } } // ------------------------------------------------------------------------------------------------------------- diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java index 475ef7698f0..995c1216bdd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java @@ -56,7 +56,6 @@ public final class GroupByReaggregateOperator implements GroupByOperator { private final String[] inputColumnNames; - private final Map> inputAggregatedColumns; private final Map> resultAggregatedColumns; private RowSetBuilderRandom stepDestinationsModified; @@ -80,10 +79,10 @@ public GroupByReaggregateOperator( rowSets = new ObjectArraySource<>(WritableRowSet.class); addedBuilders = new ObjectArraySource<>(Object.class); - inputAggregatedColumns = new LinkedHashMap<>(aggregatedColumnPairs.length); resultAggregatedColumns = new LinkedHashMap<>(aggregatedColumnPairs.length); Arrays.stream(aggregatedColumnPairs).forEach(pair -> { - final ColumnSource source = inputTable.getColumnSource(pair.rightColumn()); + // we are reaggregationg so have to use the left column for everything + final ColumnSource source = inputTable.getColumnSource(pair.leftColumn()); if (!(source instanceof AggregateColumnSource)) { throw new IllegalStateException("Expect to reaggregate AggregateColumnSources for a group operation."); } @@ -91,7 +90,6 @@ public GroupByReaggregateOperator( final ColumnSource realSource = ((AggregateColumnSource) source).getAggregatedSource(); final AggregateColumnSource aggregateColumnSource = AggregateColumnSource.make(realSource, rowSets); resultAggregatedColumns.put(pair.leftColumn(), aggregateColumnSource); - inputAggregatedColumns.put(pair.rightColumn(), aggregateColumnSource); }); if (resultAggregatedColumns.containsKey(exposeRowSetsAs)) { @@ -311,7 +309,7 @@ public UnaryOperator initializeRefreshing( @Override public Map> getInputResultColumns() { - return inputAggregatedColumns; + return resultAggregatedColumns; } @Override diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java index 0b9fc17eaf4..3bd888b7153 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java @@ -407,6 +407,67 @@ public void testRollupFormulaStatic() { freeSnapshotTableChunks(snapshot); } + @Test + public void testRollupFormulaStatic2() { + final Table source = TableTools.newTable( + stringCol("Account", "acct1", "acct1", "acct2", "acct2"), + stringCol("Sym", "leg1", "leg2", "leg1", "leg2"), + intCol("qty", 100, 100, 200, 200), + doubleCol("Dollars", 1000, -500, 2000, -1000)); + TableTools.show(source); + + final RollupTable rollup1 = + source.updateView("qty=(long)qty").rollup(List.of(AggFormula("qty", "__FORMULA_DEPTH__ > 0 ? first(qty) : sum(qty)").asReggregating(), AggSum("Dollars")), + "Account", "Sym"); + + final String[] arrayWithNull = new String[1]; + final Table keyTable = newTable( + intCol(rollup1.getRowDepthColumn().name(), 0), + stringCol("Account", arrayWithNull), + stringCol("Sym", arrayWithNull), + byteCol("Action", HierarchicalTable.KEY_TABLE_ACTION_EXPAND_ALL)); + + final HierarchicalTable.SnapshotState ss1 = rollup1.makeSnapshotState(); + final Table snapshot = + snapshotToTable(rollup1, ss1, keyTable, ColumnName.of("Action"), null, RowSetFactory.flat(30)); + TableTools.showWithRowSet(snapshot); + +// final Table expected = initialExpectedGrouped(rollup1).update("FSum=ii == 0 ? 7 : 1 + Sum"); +// assertTableEquals(expected, snapshot); + freeSnapshotTableChunks(snapshot); + } + + @Test + public void testRollupFormulaStatic3() { + final Table source = TableTools.newTable( + stringCol("Account", "Aardvark", "Aardvark", "Aardvark", "Aardvark", "Badger", "Badger", "Badger", "Cobra", "Cobra", "Cobra", "Cobra"), + stringCol("Sym", "Apple", "Banana", "Apple", "Apple", "Carrot", "Carrot", "Carrot", "Apple", "Apple", "Apple", "Dragonfruit"), + longCol("qty", 500, 100, 500, 200, 300, 300, 200, 100, 200, 300, 1500)); + TableTools.show(source); + + final RollupTable rollup1 = + source.rollup(List.of(AggFormula("qty", "__FORMULA_DEPTH__ == 2 ? min(1000, sum(qty)) : sum(qty)").asReggregating(), AggSum("sqty=qty")), + "Account", "Sym"); + + final RollupTable rollup2= rollup1.withNodeOperations(rollup1.makeNodeOperationsRecorder(RollupTable.NodeType.Aggregated).updateView("SumDiff=sqty-qty")); + + final String[] arrayWithNull = new String[1]; + final Table keyTable = newTable( + intCol(rollup1.getRowDepthColumn().name(), 0), + stringCol("Account", arrayWithNull), + stringCol("Sym", arrayWithNull), + byteCol("Action", HierarchicalTable.KEY_TABLE_ACTION_EXPAND_ALL)); + + final HierarchicalTable.SnapshotState ss1 = rollup2.makeSnapshotState(); + final Table snapshot = + snapshotToTable(rollup2, ss1, keyTable, ColumnName.of("Action"), null, RowSetFactory.flat(30)); + TableTools.showWithRowSet(snapshot); + +// final Table expected = initialExpectedGrouped(rollup1).update("FSum=ii == 0 ? 7 : 1 + Sum"); +// assertTableEquals(expected, snapshot); + freeSnapshotTableChunks(snapshot); + } + private static Table initialExpectedGrouped(RollupTable rollup1) { return TableTools.newTable(intCol(rollup1.getRowDepthColumn().name(), 1, 2, 3, 3, 2, 3, 3, 2, 3), booleanCol(rollup1.getRowExpandedColumn().name(), true, true, null, null, true, null, null, diff --git a/table-api/src/main/java/io/deephaven/api/agg/AggregationDescriptions.java b/table-api/src/main/java/io/deephaven/api/agg/AggregationDescriptions.java index 1b268984c96..6acd8f597d1 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/AggregationDescriptions.java +++ b/table-api/src/main/java/io/deephaven/api/agg/AggregationDescriptions.java @@ -88,6 +88,6 @@ public void visit(Partition partition) { @Override public void visit(Formula formula) { - out.put(formula.column().name(), "from formula `" + Strings.of(formula.expression()) + "`"); + out.put(formula.column().name(), "from formula `" + Strings.of(formula.expression()) + "`" + (formula.reaggregateAggregatedValues() ? " (reaggregates values)" : "")); } } diff --git a/table-api/src/main/java/io/deephaven/api/agg/Formula.java b/table-api/src/main/java/io/deephaven/api/agg/Formula.java index 4b78717a58a..dfc550677be 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/Formula.java +++ b/table-api/src/main/java/io/deephaven/api/agg/Formula.java @@ -33,12 +33,19 @@ public static Formula of(ColumnName name, String formula) { } public static Formula of(Selectable selectable) { - return ImmutableFormula.of(selectable); + return ImmutableFormula.of(selectable, false); + } + + public Formula asReggregating() { + return ImmutableFormula.of(selectable(), true); } @Parameter public abstract Selectable selectable(); + @Parameter + public abstract boolean reaggregateAggregatedValues(); + public ColumnName column() { return selectable().newColumn(); } @@ -59,4 +66,4 @@ public final V walk(V visitor) { visitor.visit(this); return visitor; } -} +} \ No newline at end of file From 946ca370a4a241ffe8b39108c678e4448459fb93 Mon Sep 17 00:00:00 2001 From: "Charles P. Wright" Date: Mon, 29 Dec 2025 12:13:42 -0500 Subject: [PATCH 07/24] handle things without the group operator present. --- .../table/impl/by/AggregationProcessor.java | 157 +++++++++++++++--- .../by/FormulaMultiColumnChunkedOperator.java | 16 +- .../table/impl/by/GroupByChunkedOperator.java | 27 ++- .../engine/table/impl/by/RollupConstants.java | 6 + .../engine/table/impl/TestRollupTable.java | 31 ++-- .../api/agg/AggregationDescriptions.java | 3 +- .../java/io/deephaven/api/agg/Formula.java | 2 +- 7 files changed, 198 insertions(+), 44 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index 55762d02520..ac064ca7523 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -823,7 +823,7 @@ public void visit(@NotNull final Formula formula) { makeGroupByOperatorForFormula(inputNonKeyColumns, table, null); final FormulaMultiColumnChunkedOperator op = new FormulaMultiColumnChunkedOperator(table, - groupByChunkedOperator, true, selectColumn, inputKeyColumns, null); + groupByChunkedOperator, true, selectColumn, inputKeyColumns, null, null); addNoInputOperator(op); } @@ -866,7 +866,7 @@ public void visit(@NotNull final AggSpecFirst first) { public void visit(@NotNull final AggSpecFormula formula) { unsupportedForBlinkTables("Formula"); // TODO: re-use shared groupBy operators (https://github.com/deephaven/deephaven-core/issues/6363) - final GroupByChunkedOperator groupByChunkedOperator = new GroupByChunkedOperator(table, false, null, + final GroupByChunkedOperator groupByChunkedOperator = new GroupByChunkedOperator(table, false, null, null, resultPairs.stream().map(pair -> MatchPair.of((Pair) pair.input())).toArray(MatchPair[]::new)); final FormulaChunkedOperator formulaChunkedOperator = new FormulaChunkedOperator(groupByChunkedOperator, true, formula.formula(), formula.paramToken(), compilationProcessor, @@ -882,7 +882,7 @@ public void visit(AggSpecFreeze freeze) { @Override public void visit(@NotNull final AggSpecGroup group) { unsupportedForBlinkTables("Group"); - addNoInputOperator(new GroupByChunkedOperator(table, true, null, MatchPair.fromPairs(resultPairs))); + addNoInputOperator(new GroupByChunkedOperator(table, true, null, null, MatchPair.fromPairs(resultPairs))); } @Override @@ -999,9 +999,24 @@ private void maybeInitializeVectorColumns(Set groupByColumnSet, final Ta private @NotNull GroupByChunkedOperator makeGroupByOperatorForFormula(String[] inputNonKeyColumns, final QueryTable table, final String exposedRowsets) { - return new GroupByChunkedOperator(table, false, exposedRowsets, - Arrays.stream(inputNonKeyColumns).map(col -> MatchPair.of(Pair.parse(col))) - .toArray(MatchPair[]::new)); + final MatchPair[] pairs; + final List hiddenResults; + final boolean register; + if (exposedRowsets == null) { + hiddenResults = null; + register = false; + pairs = Arrays.stream(inputNonKeyColumns).map(col -> MatchPair.of(Pair.parse(col))) + .toArray(MatchPair[]::new); + } else { + hiddenResults = null;// Arrays.stream(pairs).map(mp -> mp.output().name()).collect(Collectors.toList()); + register = true; + pairs = Arrays + .stream(inputNonKeyColumns).map(col -> MatchPair.of( + Pair + .of(ColumnName.of(col), ColumnName.of(col + ROLLUP_GRP_COLUMN_ID + ROLLUP_COLUMN_SUFFIX)))) + .toArray(MatchPair[]::new); + } + return new GroupByChunkedOperator(table, register, exposedRowsets, hiddenResults, pairs); } // ----------------------------------------------------------------------------------------------------------------- @@ -1141,9 +1156,22 @@ public void visit(@NotNull final Partition partition) { public void visit(AggSpecGroup group) { unsupportedForBlinkTables("Group for rollup"); addNoInputOperator(new GroupByChunkedOperator(table, true, EXPOSED_GROUP_ROW_SETS.name(), + null, MatchPair.fromPairs(resultPairs))); } + /** + * @return the index of an existing group by operator, or -1 if it no operator was found + */ + private int existingGroupByOperatorIndex() { + for (int ii = 0; ii < operators.size(); ++ii) { + if (operators.get(ii) instanceof GroupByChunkedOperator) { + return ii; + } + } + return -1; + } + @Override public void visit(Formula formula) { final SelectColumn selectColumn = SelectColumn.of(formula.selectable()); @@ -1169,19 +1197,84 @@ public void visit(Formula formula) { // TODO: re-use shared groupBy operators (https://github.com/deephaven/deephaven-core/issues/6363) final GroupByChunkedOperator groupByChunkedOperator; + final int existingGroupByOperatorIndex = existingGroupByOperatorIndex(); + + final boolean delegate; if (formula.reaggregateAggregatedValues()) { - groupByChunkedOperator = makeGroupByOperatorForFormula(inputNonKeyColumns, table, null); + if (existingGroupByOperatorIndex >= 0) { + final GroupByChunkedOperator existing = + (GroupByChunkedOperator) operators.get(existingGroupByOperatorIndex); + MatchPair[] existingPairs = existing.getAggregatedColumnPairs(); + for (final String searchColumn : inputNonKeyColumns) { + int searchIdx = 0; + while (searchIdx < existingPairs.length) { + if (existingPairs[searchIdx].left().name().equals(searchColumn) + && existingPairs[searchIdx].right().name().equals(searchColumn)) { + break; + } + searchIdx++; + } + if (searchIdx == existingPairs.length) { + throw new IllegalStateException( + "Existing groupBy operator does not have the required input column " + + searchColumn); + } + } + // TODO: we have one, let's replace it if it is insufficient + groupByChunkedOperator = existing; + delegate = false; + } else { + // We we are reaggregating, we do not expose the rowsets, because the next level creates a + // completely + // fresh operator + groupByChunkedOperator = makeGroupByOperatorForFormula(inputNonKeyColumns, table, null); + // the operator is not added, so there is delegation + delegate = true; + } } else { - groupByChunkedOperator = makeGroupByOperatorForFormula(inputNonKeyColumns, table, EXPOSED_GROUP_ROW_SETS.name()); - // TODO: the next level somehow needs access to the raw column sources so that we can continue the aggregation + if (existingGroupByOperatorIndex >= 0) { + final GroupByChunkedOperator existing = + (GroupByChunkedOperator) operators.get(existingGroupByOperatorIndex); + if (!EXPOSED_GROUP_ROW_SETS.name().equals(existing.getExposedRowSetsAs())) { + throw new IllegalStateException("Existing groupBy operator does not expose rowsets"); + } + + MatchPair[] existingPairs = existing.getAggregatedColumnPairs(); + for (int ii = 0; ii < inputNonKeyColumns.length; ++ii) { + final String searchColumn = inputNonKeyColumns[ii]; + int searchIdx = 0; + while (searchIdx < existingPairs.length) { + if (existingPairs[searchIdx].left().name().equals(searchColumn) + && existingPairs[searchIdx].right().name().equals(searchColumn)) { + break; + } + searchIdx++; + } + if (searchIdx == existingPairs.length) { + throw new IllegalStateException( + "Existing groupBy operator does not have the required input column " + + searchColumn); + } + } + // TODO: we have one, let's replace it if it is insufficient + groupByChunkedOperator = existing; + delegate = false; + } else { + // When we do not reaggregate, the next level needs access to our exposed group row sets + groupByChunkedOperator = + makeGroupByOperatorForFormula(inputNonKeyColumns, table, EXPOSED_GROUP_ROW_SETS.name()); + addNoInputOperator(groupByChunkedOperator); + // we added the operator, so we cannot delegate + delegate = false; + } } final IntegerSingleValueSource depthSource = new IntegerSingleValueSource(); depthSource.set(groupByColumnNames.length); final FormulaMultiColumnChunkedOperator op = new FormulaMultiColumnChunkedOperator(table, - groupByChunkedOperator, true, selectColumn, inputKeyColumns, depthSource); + groupByChunkedOperator, delegate, selectColumn, inputKeyColumns, null, depthSource); addNoInputOperator(op); } @@ -1392,12 +1485,21 @@ public void visit(Formula formula) { // final GroupByChunkedOperator groupByChunkedOperator = makeGroupByOperatorForFormula(inputNonKeyColumns, // table); - final MatchPair[] groupPairs = Arrays.stream(inputNonKeyColumns).map(col -> { - // reagg uses the output name - final Pair parse = Pair.parse(col); - return MatchPair.of(Pair.of(parse.output(), parse.output())); - }) - .toArray(MatchPair[]::new); + final Map renames = new HashMap<>(); + final MatchPair [] groupPairs = new MatchPair[inputNonKeyColumns.length]; + + for (int ii = 0; ii < inputNonKeyColumns.length; ++ii) { + final String mangledColumn = inputNonKeyColumns[ii] + ROLLUP_GRP_COLUMN_ID + ROLLUP_COLUMN_SUFFIX; + if (table.hasColumns(mangledColumn)) { + groupPairs[ii] = new MatchPair(mangledColumn, mangledColumn); + renames.put(mangledColumn, inputNonKeyColumns[ii]); + } else { + // reagg uses the output name + groupPairs[ii] = new MatchPair(mangledColumn, inputNonKeyColumns[ii]); + // we are not changing the input column name, so don't need the rename + renames.put(inputNonKeyColumns[ii], inputNonKeyColumns[ii]); + } + } GroupByOperator groupByOperator; @@ -1405,17 +1507,24 @@ public void visit(Formula formula) { depthSource.set(groupByColumnNames.length); if (formula.reaggregateAggregatedValues()) { - groupByOperator = new GroupByChunkedOperator(table, false, null, groupPairs); + // everything gets hidden + final List hiddenPairs = + Arrays.stream(groupPairs).map(mp -> mp.left().name()).collect(Collectors.toList()); + groupByOperator = new GroupByChunkedOperator(table, false, null, hiddenPairs, groupPairs); - final FormulaMultiColumnChunkedOperator op = new FormulaMultiColumnChunkedOperator(table, groupByOperator, - true, selectColumn, inputKeyColumns, depthSource); + final FormulaMultiColumnChunkedOperator op = + new FormulaMultiColumnChunkedOperator(table, groupByOperator, + true, selectColumn, inputKeyColumns, null, depthSource); addOperator(op, null, inputNonKeyColumns); } else { final ColumnSource groupRowSet = table.getColumnSource(EXPOSED_GROUP_ROW_SETS.name()); - groupByOperator = new GroupByReaggregateOperator(table, false, EXPOSED_GROUP_ROW_SETS.name(), groupPairs); - final FormulaMultiColumnChunkedOperator op = new FormulaMultiColumnChunkedOperator(table, groupByOperator, - true, selectColumn, inputKeyColumns, depthSource); + groupByOperator = + new GroupByReaggregateOperator(table, true, EXPOSED_GROUP_ROW_SETS.name(), groupPairs); + addOperator(groupByOperator, groupRowSet, EXPOSED_GROUP_ROW_SETS.name()); + final FormulaMultiColumnChunkedOperator op = + new FormulaMultiColumnChunkedOperator(table, groupByOperator, + false, selectColumn, inputKeyColumns, renames, depthSource); addOperator(op, groupRowSet, EXPOSED_GROUP_ROW_SETS.name()); } } @@ -1738,7 +1847,7 @@ private static AggregationContext makeExposedGroupRowSetAggregationContext( // noinspection unchecked return new AggregationContext( new IterativeChunkedAggregationOperator[] { - new GroupByChunkedOperator(inputQueryTable, true, EXPOSED_GROUP_ROW_SETS.name()), + new GroupByChunkedOperator(inputQueryTable, true, EXPOSED_GROUP_ROW_SETS.name(), null), new CountAggregationOperator(null) }, new String[][] {ArrayTypeUtils.EMPTY_STRING_ARRAY, ArrayTypeUtils.EMPTY_STRING_ARRAY}, @@ -1748,7 +1857,7 @@ private static AggregationContext makeExposedGroupRowSetAggregationContext( // noinspection unchecked return new AggregationContext( new IterativeChunkedAggregationOperator[] { - new GroupByChunkedOperator(inputQueryTable, true, EXPOSED_GROUP_ROW_SETS.name()) + new GroupByChunkedOperator(inputQueryTable, true, EXPOSED_GROUP_ROW_SETS.name(), null) }, new String[][] {ArrayTypeUtils.EMPTY_STRING_ARRAY}, new ChunkSource.WithPrev[] {null}, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaMultiColumnChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaMultiColumnChunkedOperator.java index 6bb66faf6d0..475828fb5bf 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaMultiColumnChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaMultiColumnChunkedOperator.java @@ -42,6 +42,8 @@ class FormulaMultiColumnChunkedOperator implements IterativeChunkedAggregationOp private final String[] inputKeyColumns; @Nullable private final ColumnSource formulaDepthSource; + @Nullable + private final Map renames; private ChunkSource formulaDataSource; @@ -58,11 +60,12 @@ class FormulaMultiColumnChunkedOperator implements IterativeChunkedAggregationOp /** * Construct an operator for applying a formula to slot-vectors over an aggregated table.. * - * @param groupBy The {@link GroupByChunkedOperator} to use for tracking indices + * @param groupBy The {@link GroupByChunkedOperator} to use for tracking indices * @param delegateToBy Whether this operator is responsible for passing methods through to {@code groupBy}. Should - * be false if {@code groupBy} is updated by the helper, or if this is not the first operator sharing - * {@code groupBy}. + * be false if {@code groupBy} is updated by the helper, or if this is not the first operator sharing + * {@code groupBy}. * @param selectColumn The formula column that will produce the results + * @param renames a map from input names in the groupBy operator (i.e. mangled names) to input column names in the formula */ FormulaMultiColumnChunkedOperator( @NotNull final QueryTable inputTable, @@ -70,12 +73,14 @@ class FormulaMultiColumnChunkedOperator implements IterativeChunkedAggregationOp final boolean delegateToBy, @NotNull final SelectColumn selectColumn, @NotNull final String[] inputKeyColumns, + @Nullable Map renames, @Nullable final ColumnSource formulaDepthSource) { this.inputTable = inputTable; this.groupBy = groupBy; this.delegateToBy = delegateToBy; this.selectColumn = selectColumn; this.inputKeyColumns = inputKeyColumns; + this.renames = renames; this.formulaDepthSource = formulaDepthSource; resultColumn = ArrayBackedColumnSource.getMemoryColumnSource( @@ -227,12 +232,13 @@ public void propagateInitialState(@NotNull final QueryTable resultTable, int sta } final Map> sourceColumns; - if (inputKeyColumns.length == 0 && formulaDepthSource == null) { + if (inputKeyColumns.length == 0 && formulaDepthSource == null && renames == null) { // noinspection unchecked sourceColumns = (Map>) groupBy.getInputResultColumns(); } else { final Map> columnSourceMap = resultTable.getColumnSourceMap(); - sourceColumns = new HashMap<>(groupBy.getInputResultColumns()); + sourceColumns = new HashMap<>(groupBy.getInputResultColumns().size() + 1); + groupBy.getInputResultColumns().forEach((k, v) -> sourceColumns.put(renames == null ? k : renames.get(k), v)); Arrays.stream(inputKeyColumns).forEach(col -> sourceColumns.put(col, columnSourceMap.get(col))); sourceColumns.put(AggregationProcessor.ROLLUP_FORMULA_DEPTH.name(), formulaDepthSource); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java index 4dd8e6ddfa3..b541f26472a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java @@ -24,6 +24,7 @@ import java.util.Arrays; import java.util.LinkedHashMap; +import java.util.List; import java.util.Map; import java.util.function.UnaryOperator; @@ -56,14 +57,28 @@ public final class GroupByChunkedOperator implements GroupByOperator { private boolean someKeyHasModifies; private boolean initialized; + private MatchPair[] aggregatedColumnPairs; + + /** + * + * @param inputTable the table we are aggregating + * @param registeredWithHelper true if we are registered with the helper (meaning we independently produce result + * columns), false otherwise. For a normal AggGroup this is true; for a group-by that is only part of an + * AggFormula this is false. + * @param exposeRowSetsAs the name of the column to expose the rowsets for each group as + * @param hiddenResults a list (possibly empty) of columns that are not exposed to the helper + * @param aggregatedColumnPairs the list of input and output columns for this operation + */ public GroupByChunkedOperator( @NotNull final QueryTable inputTable, final boolean registeredWithHelper, @Nullable final String exposeRowSetsAs, + @Nullable final List hiddenResults, @NotNull final MatchPair... aggregatedColumnPairs) { this.inputTable = inputTable; this.registeredWithHelper = registeredWithHelper; this.exposeRowSetsAs = exposeRowSetsAs; + this.aggregatedColumnPairs = aggregatedColumnPairs; live = inputTable.isRefreshing(); rowSets = new ObjectArraySource<>(WritableRowSet.class); @@ -75,7 +90,9 @@ public GroupByChunkedOperator( final AggregateColumnSource aggregateColumnSource = AggregateColumnSource.make(inputTable.getColumnSource(pair.rightColumn()), rowSets); inputAggregatedColumns.put(pair.rightColumn(), aggregateColumnSource); - resultAggregatedColumns.put(pair.leftColumn(), aggregateColumnSource); + if (hiddenResults == null || !hiddenResults.contains(pair.leftColumn())) { + resultAggregatedColumns.put(pair.leftColumn(), aggregateColumnSource); + } }); if (exposeRowSetsAs != null && resultAggregatedColumns.containsKey(exposeRowSetsAs)) { @@ -638,4 +655,12 @@ boolean getSomeKeyHasModifies() { public boolean hasModifications(boolean columnsModified) { return getSomeKeyHasAddsOrRemoves() || (getSomeKeyHasModifies() && columnsModified); } + + public String getExposedRowSetsAs() { + return exposeRowSetsAs; + } + + public MatchPair[] getAggregatedColumnPairs() { + return aggregatedColumnPairs; + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/RollupConstants.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/RollupConstants.java index 7d34a87adbb..0c2d3aa7bea 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/RollupConstants.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/RollupConstants.java @@ -67,4 +67,10 @@ private RollupConstants() {} * infinity count columns used in rollup aggregations. */ static final String ROLLUP_NI_COUNT_COLUMN_ID = "_NIC_"; + + /** + * Middle column name component (between source column name and {@link #ROLLUP_COLUMN_SUFFIX suffix}) for group + * columns used in support of rollup formulas. + */ + static final String ROLLUP_GRP_COLUMN_ID = "_GRP_"; } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java index 3bd888b7153..183d30f245b 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java @@ -383,7 +383,7 @@ public void testRollupFormulaStatic() { final RollupTable rollup1 = source.rollup( - List.of(AggGroup("Sentinel"), AggSum("Sum=Sentinel"), + List.of(AggSum("Sum=Sentinel"), AggFormula("FSum", "__FORMULA_DEPTH__ == 0 ? max(Sentinel) : 1 + sum(Sentinel)")), "Key1", "Key2"); @@ -400,9 +400,9 @@ public void testRollupFormulaStatic() { TableTools.showWithRowSet(snapshot); TableTools.show(snapshot.view(rollup1.getRowDepthColumn().name(), rollup1.getRowExpandedColumn().name(), "Key1", - "Key2", "Sentinel", "Sum", "FSum")); + "Key2", "Sum", "FSum")); - final Table expected = initialExpectedGrouped(rollup1).update("FSum=ii == 0 ? 7 : 1 + Sum"); + final Table expected = initialExpectedGrouped(rollup1).dropColumns("Sentinel").update("FSum=ii == 0 ? 7 : 1 + Sum"); assertTableEquals(expected, snapshot.dropColumns("__EXPOSED_GROUP_ROW_SETS__")); freeSnapshotTableChunks(snapshot); } @@ -417,7 +417,9 @@ public void testRollupFormulaStatic2() { TableTools.show(source); final RollupTable rollup1 = - source.updateView("qty=(long)qty").rollup(List.of(AggFormula("qty", "__FORMULA_DEPTH__ > 0 ? first(qty) : sum(qty)").asReggregating(), AggSum("Dollars")), + source.updateView("qty=(long)qty").rollup( + List.of(AggFormula("qty", "__FORMULA_DEPTH__ > 0 ? first(qty) : sum(qty)").asReggregating(), + AggSum("Dollars")), "Account", "Sym"); final String[] arrayWithNull = new String[1]; @@ -432,24 +434,29 @@ public void testRollupFormulaStatic2() { snapshotToTable(rollup1, ss1, keyTable, ColumnName.of("Action"), null, RowSetFactory.flat(30)); TableTools.showWithRowSet(snapshot); -// final Table expected = initialExpectedGrouped(rollup1).update("FSum=ii == 0 ? 7 : 1 + Sum"); -// assertTableEquals(expected, snapshot); + // final Table expected = initialExpectedGrouped(rollup1).update("FSum=ii == 0 ? 7 : 1 + Sum"); + // assertTableEquals(expected, snapshot); freeSnapshotTableChunks(snapshot); } @Test public void testRollupFormulaStatic3() { final Table source = TableTools.newTable( - stringCol("Account", "Aardvark", "Aardvark", "Aardvark", "Aardvark", "Badger", "Badger", "Badger", "Cobra", "Cobra", "Cobra", "Cobra"), - stringCol("Sym", "Apple", "Banana", "Apple", "Apple", "Carrot", "Carrot", "Carrot", "Apple", "Apple", "Apple", "Dragonfruit"), + stringCol("Account", "Aardvark", "Aardvark", "Aardvark", "Aardvark", "Badger", "Badger", "Badger", + "Cobra", "Cobra", "Cobra", "Cobra"), + stringCol("Sym", "Apple", "Banana", "Apple", "Apple", "Carrot", "Carrot", "Carrot", "Apple", "Apple", + "Apple", "Dragonfruit"), longCol("qty", 500, 100, 500, 200, 300, 300, 200, 100, 200, 300, 1500)); TableTools.show(source); final RollupTable rollup1 = - source.rollup(List.of(AggFormula("qty", "__FORMULA_DEPTH__ == 2 ? min(1000, sum(qty)) : sum(qty)").asReggregating(), AggSum("sqty=qty")), + source.rollup( + List.of(AggFormula("qty", "__FORMULA_DEPTH__ == 2 ? min(1000, sum(qty)) : sum(qty)") + .asReggregating(), AggSum("sqty=qty")), "Account", "Sym"); - final RollupTable rollup2= rollup1.withNodeOperations(rollup1.makeNodeOperationsRecorder(RollupTable.NodeType.Aggregated).updateView("SumDiff=sqty-qty")); + final RollupTable rollup2 = rollup1.withNodeOperations( + rollup1.makeNodeOperationsRecorder(RollupTable.NodeType.Aggregated).updateView("SumDiff=sqty-qty")); final String[] arrayWithNull = new String[1]; final Table keyTable = newTable( @@ -463,8 +470,8 @@ public void testRollupFormulaStatic3() { snapshotToTable(rollup2, ss1, keyTable, ColumnName.of("Action"), null, RowSetFactory.flat(30)); TableTools.showWithRowSet(snapshot); -// final Table expected = initialExpectedGrouped(rollup1).update("FSum=ii == 0 ? 7 : 1 + Sum"); -// assertTableEquals(expected, snapshot); + // final Table expected = initialExpectedGrouped(rollup1).update("FSum=ii == 0 ? 7 : 1 + Sum"); + // assertTableEquals(expected, snapshot); freeSnapshotTableChunks(snapshot); } diff --git a/table-api/src/main/java/io/deephaven/api/agg/AggregationDescriptions.java b/table-api/src/main/java/io/deephaven/api/agg/AggregationDescriptions.java index 6acd8f597d1..61df885f0d7 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/AggregationDescriptions.java +++ b/table-api/src/main/java/io/deephaven/api/agg/AggregationDescriptions.java @@ -88,6 +88,7 @@ public void visit(Partition partition) { @Override public void visit(Formula formula) { - out.put(formula.column().name(), "from formula `" + Strings.of(formula.expression()) + "`" + (formula.reaggregateAggregatedValues() ? " (reaggregates values)" : "")); + out.put(formula.column().name(), "from formula `" + Strings.of(formula.expression()) + "`" + + (formula.reaggregateAggregatedValues() ? " (reaggregates values)" : "")); } } diff --git a/table-api/src/main/java/io/deephaven/api/agg/Formula.java b/table-api/src/main/java/io/deephaven/api/agg/Formula.java index dfc550677be..28c7dc15471 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/Formula.java +++ b/table-api/src/main/java/io/deephaven/api/agg/Formula.java @@ -66,4 +66,4 @@ public final V walk(V visitor) { visitor.visit(this); return visitor; } -} \ No newline at end of file +} From b1a3f4bf7e5333e407f7475b35b838becfabd7c9 Mon Sep 17 00:00:00 2001 From: "Charles P. Wright" Date: Mon, 29 Dec 2025 16:37:24 -0500 Subject: [PATCH 08/24] Reuse group by for normal aggs. --- .../table/impl/by/AggregationProcessor.java | 111 ++++++++++++--- .../table/impl/by/FormulaChunkedOperator.java | 9 +- .../by/FormulaMultiColumnChunkedOperator.java | 21 ++- .../table/impl/by/GroupByChunkedOperator.java | 79 ++++++++++- .../engine/table/impl/TestRollupTable.java | 3 +- .../impl/by/AggregationProcessorTest.java | 129 ++++++++++-------- 6 files changed, 258 insertions(+), 94 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index ac064ca7523..ce26aabbb8a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -734,6 +734,71 @@ final void addCountWhereOperator(@NotNull CountWhere countWhere) { addOperator(new CountWhereOperator(countWhere.column().name(), whereFilters, recorders, filterRecorders), null, inputColumnNames); } + + /** + * @return the index of an existing group by operator, or -1 if it no operator was found + */ + int existingGroupByOperatorIndex() { + for (int ii = 0; ii < operators.size(); ++ii) { + if (operators.get(ii) instanceof GroupByChunkedOperator) { + return ii; + } + } + return -1; + } + + GroupByChunkedOperator ensureGroupingOperator(final QueryTable table, + final int existingOperatorIndex, + final String exposeRowSetAs, + final MatchPair[] matchPairs) { + boolean recreate = false; + final GroupByChunkedOperator existing = (GroupByChunkedOperator) operators.get(existingOperatorIndex); + if (exposeRowSetAs != null) { + if (existing.getExposedRowSetsAs() == null) { + recreate = true; + } else { + if (!existing.getExposedRowSetsAs().equals(exposeRowSetAs)) { + throw new UnsupportedOperationException( + "AggGroupBy cannot have inconsistent exposed row redirections names: " + + existing.getExposedRowSetsAs() + " != " + exposeRowSetAs); + } + } + } + final List newPairs = new ArrayList<>(Arrays.asList(existing.getAggregatedColumnPairs())); + List existingHidden = existing.getHiddenResults(); + final List hiddenResults = new ArrayList<>(existingHidden == null ? List.of() : existingHidden); + for (MatchPair matchPair : matchPairs) { + final String input = matchPair.input().name(); + if (Arrays.stream(existing.getAggregatedColumnPairs()).noneMatch(p -> p.input().name().equals(input))) { + newPairs.add(matchPair); + hiddenResults.add(matchPair.output().name()); + recreate = true; + } + } + if (!recreate) { + // we're totally satisfied with the existing operator for use with a secondary operator that pulls an + // output from it to the desired name + return existing; + } + + final String newExposeRowsetName = exposeRowSetAs == null ? existing.getExposedRowSetsAs() : exposeRowSetAs; + final MatchPair[] newMatchPairArray = newPairs.toArray(MatchPair[]::new); + final GroupByChunkedOperator newOperator = + new GroupByChunkedOperator(table, true, newExposeRowsetName, hiddenResults, newMatchPairArray); + + // any formula operators that used the old group by operator must be updated + for (IterativeChunkedAggregationOperator operator : operators) { + if (operator instanceof FormulaMultiColumnChunkedOperator) { + ((FormulaMultiColumnChunkedOperator) operator).updateGroupBy(newOperator, false); + } + else if (operator instanceof FormulaChunkedOperator) { + ((FormulaChunkedOperator) operator).updateGroupBy(newOperator, false); + } + } + + operators.set(existingOperatorIndex, newOperator); + return newOperator; + } } // ----------------------------------------------------------------------------------------------------------------- @@ -818,12 +883,20 @@ public void visit(@NotNull final Formula formula) { final String[] inputNonKeyColumns = partitioned.get(false).toArray(String[]::new); validateSelectColumnForFormula(selectColumn); - // TODO: re-use shared groupBy operators (https://github.com/deephaven/deephaven-core/issues/6363) - final GroupByChunkedOperator groupByChunkedOperator = - makeGroupByOperatorForFormula(inputNonKeyColumns, table, null); + final GroupByChunkedOperator groupByChunkedOperator; + final int existingGroupByOperatorIndex = existingGroupByOperatorIndex(); + if (existingGroupByOperatorIndex >= 0) { + // if we have an existing group by operator, then use it (or update it to reflect our input columns) + final MatchPair[] matchPairs = + Arrays.stream(inputNonKeyColumns).map(cn -> new MatchPair(cn, cn)).toArray(MatchPair[]::new); + groupByChunkedOperator = ensureGroupingOperator(table, existingGroupByOperatorIndex, null, matchPairs); + } else { + groupByChunkedOperator = makeGroupByOperatorForFormula(inputNonKeyColumns, table, null); + } final FormulaMultiColumnChunkedOperator op = new FormulaMultiColumnChunkedOperator(table, - groupByChunkedOperator, true, selectColumn, inputKeyColumns, null, null); + groupByChunkedOperator, existingGroupByOperatorIndex < 0, selectColumn, inputKeyColumns, null, + null); addNoInputOperator(op); } @@ -882,7 +955,18 @@ public void visit(AggSpecFreeze freeze) { @Override public void visit(@NotNull final AggSpecGroup group) { unsupportedForBlinkTables("Group"); - addNoInputOperator(new GroupByChunkedOperator(table, true, null, null, MatchPair.fromPairs(resultPairs))); + + // TODO: re-use shared groupBy operators (https://github.com/deephaven/deephaven-core/issues/6363) + final int existingOperator = existingGroupByOperatorIndex(); + if (existingOperator >= 0) { + // TODO: we must ensure the input columns are all properly represented but hidden + GroupByChunkedOperator existing = + ensureGroupingOperator(table, existingOperator, null, MatchPair.fromPairs(resultPairs)); + addNoInputOperator(existing.resultExtractor(resultPairs)); + } else { + addNoInputOperator( + new GroupByChunkedOperator(table, true, null, null, MatchPair.fromPairs(resultPairs))); + } } @Override @@ -1013,7 +1097,8 @@ private void maybeInitializeVectorColumns(Set groupByColumnSet, final Ta pairs = Arrays .stream(inputNonKeyColumns).map(col -> MatchPair.of( Pair - .of(ColumnName.of(col), ColumnName.of(col + ROLLUP_GRP_COLUMN_ID + ROLLUP_COLUMN_SUFFIX)))) + .of(ColumnName.of(col), + ColumnName.of(col + ROLLUP_GRP_COLUMN_ID + ROLLUP_COLUMN_SUFFIX)))) .toArray(MatchPair[]::new); } return new GroupByChunkedOperator(table, register, exposedRowsets, hiddenResults, pairs); @@ -1160,18 +1245,6 @@ public void visit(AggSpecGroup group) { MatchPair.fromPairs(resultPairs))); } - /** - * @return the index of an existing group by operator, or -1 if it no operator was found - */ - private int existingGroupByOperatorIndex() { - for (int ii = 0; ii < operators.size(); ++ii) { - if (operators.get(ii) instanceof GroupByChunkedOperator) { - return ii; - } - } - return -1; - } - @Override public void visit(Formula formula) { final SelectColumn selectColumn = SelectColumn.of(formula.selectable()); @@ -1486,7 +1559,7 @@ public void visit(Formula formula) { // table); final Map renames = new HashMap<>(); - final MatchPair [] groupPairs = new MatchPair[inputNonKeyColumns.length]; + final MatchPair[] groupPairs = new MatchPair[inputNonKeyColumns.length]; for (int ii = 0; ii < inputNonKeyColumns.length; ++ii) { final String mangledColumn = inputNonKeyColumns[ii] + ROLLUP_GRP_COLUMN_ID + ROLLUP_COLUMN_SUFFIX; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaChunkedOperator.java index 176a14a28f6..b2b9464d81d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaChunkedOperator.java @@ -39,8 +39,8 @@ */ class FormulaChunkedOperator implements IterativeChunkedAggregationOperator { - private final GroupByChunkedOperator groupBy; - private final boolean delegateToBy; + private GroupByChunkedOperator groupBy; + private boolean delegateToBy; private final String[] inputColumnNames; private final String[] resultColumnNames; @@ -494,4 +494,9 @@ private boolean[] makeObjectOrModifiedColumnsMask(@NotNull final ModifiedColumnS } return columnsMask; } + + public void updateGroupBy(GroupByChunkedOperator groupBy, boolean delegateToBy) { + this.groupBy = groupBy; + this.delegateToBy = delegateToBy; + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaMultiColumnChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaMultiColumnChunkedOperator.java index 475828fb5bf..269355d9e83 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaMultiColumnChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaMultiColumnChunkedOperator.java @@ -35,8 +35,8 @@ class FormulaMultiColumnChunkedOperator implements IterativeChunkedAggregationOp private final QueryTable inputTable; - private final GroupByOperator groupBy; - private final boolean delegateToBy; + private GroupByOperator groupBy; + private boolean delegateToBy; private final SelectColumn selectColumn; private final WritableColumnSource resultColumn; private final String[] inputKeyColumns; @@ -60,12 +60,13 @@ class FormulaMultiColumnChunkedOperator implements IterativeChunkedAggregationOp /** * Construct an operator for applying a formula to slot-vectors over an aggregated table.. * - * @param groupBy The {@link GroupByChunkedOperator} to use for tracking indices + * @param groupBy The {@link GroupByChunkedOperator} to use for tracking indices * @param delegateToBy Whether this operator is responsible for passing methods through to {@code groupBy}. Should - * be false if {@code groupBy} is updated by the helper, or if this is not the first operator sharing - * {@code groupBy}. + * be false if {@code groupBy} is updated by the helper, or if this is not the first operator sharing + * {@code groupBy}. * @param selectColumn The formula column that will produce the results - * @param renames a map from input names in the groupBy operator (i.e. mangled names) to input column names in the formula + * @param renames a map from input names in the groupBy operator (i.e. mangled names) to input column names in the + * formula */ FormulaMultiColumnChunkedOperator( @NotNull final QueryTable inputTable, @@ -238,7 +239,8 @@ public void propagateInitialState(@NotNull final QueryTable resultTable, int sta } else { final Map> columnSourceMap = resultTable.getColumnSourceMap(); sourceColumns = new HashMap<>(groupBy.getInputResultColumns().size() + 1); - groupBy.getInputResultColumns().forEach((k, v) -> sourceColumns.put(renames == null ? k : renames.get(k), v)); + groupBy.getInputResultColumns() + .forEach((k, v) -> sourceColumns.put(renames == null ? k : renames.get(k), v)); Arrays.stream(inputKeyColumns).forEach(col -> sourceColumns.put(col, columnSourceMap.get(col))); sourceColumns.put(AggregationProcessor.ROLLUP_FORMULA_DEPTH.name(), formulaDepthSource); } @@ -419,4 +421,9 @@ public void close() { private static long calculateContainingBlockLastKey(final long firstKey) { return (firstKey / BLOCK_SIZE) * BLOCK_SIZE + BLOCK_SIZE - 1; } + + public void updateGroupBy(GroupByChunkedOperator groupBy, boolean delegateToBy) { + this.groupBy = groupBy; + this.delegateToBy = delegateToBy; + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java index b541f26472a..18eb400b28e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java @@ -3,6 +3,7 @@ // package io.deephaven.engine.table.impl.by; +import io.deephaven.api.Pair; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.attributes.ChunkLengths; import io.deephaven.chunk.attributes.ChunkPositions; @@ -22,10 +23,7 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.util.Arrays; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; +import java.util.*; import java.util.function.UnaryOperator; import static io.deephaven.engine.table.impl.sources.ArrayBackedColumnSource.BLOCK_SIZE; @@ -47,6 +45,7 @@ public final class GroupByChunkedOperator implements GroupByOperator { private final String[] inputColumnNames; private final Map> inputAggregatedColumns; + private final String[] inputColumnNamesForResults; private final Map> resultAggregatedColumns; private final ModifiedColumnSet aggregationInputsModifiedColumnSet; @@ -58,6 +57,7 @@ public final class GroupByChunkedOperator implements GroupByOperator { private boolean initialized; private MatchPair[] aggregatedColumnPairs; + private List hiddenResults; /** * @@ -78,6 +78,7 @@ public GroupByChunkedOperator( this.inputTable = inputTable; this.registeredWithHelper = registeredWithHelper; this.exposeRowSetsAs = exposeRowSetsAs; + this.hiddenResults = hiddenResults; this.aggregatedColumnPairs = aggregatedColumnPairs; live = inputTable.isRefreshing(); @@ -86,14 +87,17 @@ public GroupByChunkedOperator( inputAggregatedColumns = new LinkedHashMap<>(aggregatedColumnPairs.length); resultAggregatedColumns = new LinkedHashMap<>(aggregatedColumnPairs.length); + final List inputResultNameList = new ArrayList<>(aggregatedColumnPairs.length); Arrays.stream(aggregatedColumnPairs).forEach(pair -> { final AggregateColumnSource aggregateColumnSource = AggregateColumnSource.make(inputTable.getColumnSource(pair.rightColumn()), rowSets); inputAggregatedColumns.put(pair.rightColumn(), aggregateColumnSource); - if (hiddenResults == null || !hiddenResults.contains(pair.leftColumn())) { - resultAggregatedColumns.put(pair.leftColumn(), aggregateColumnSource); + if (hiddenResults == null || !hiddenResults.contains(pair.output().name())) { + resultAggregatedColumns.put(pair.output().name(), aggregateColumnSource); + inputResultNameList.add(pair.input().name()); } }); + inputColumnNamesForResults = inputResultNameList.toArray(String[]::new); if (exposeRowSetsAs != null && resultAggregatedColumns.containsKey(exposeRowSetsAs)) { throw new IllegalArgumentException(String.format( @@ -467,7 +471,7 @@ private InputToResultModifiedColumnSetFactory( allResultColumns = resultTable.newModifiedColumnSet(resultAggregatedColumnNames); } aggregatedColumnsTransformer = inputTable.newModifiedColumnSetTransformer( - inputColumnNames, + inputColumnNamesForResults, Arrays.stream(resultAggregatedColumnNames).map(resultTable::newModifiedColumnSet) .toArray(ModifiedColumnSet[]::new)); } @@ -663,4 +667,65 @@ public String getExposedRowSetsAs() { public MatchPair[] getAggregatedColumnPairs() { return aggregatedColumnPairs; } + + public List getHiddenResults() { + return hiddenResults; + } + + private class ResultExtractor implements IterativeChunkedAggregationOperator { + final Map> resultColumns; + + private ResultExtractor(Map> resultColumns) { + this.resultColumns = resultColumns; + } + + @Override + public Map> getResultColumns() { + return resultColumns; + } + + @Override + public void addChunk(BucketedContext context, Chunk values, + LongChunk inputRowKeys, IntChunk destinations, + IntChunk startPositions, IntChunk length, + WritableBooleanChunk stateModified) {} + + @Override + public void removeChunk(BucketedContext context, Chunk values, + LongChunk inputRowKeys, IntChunk destinations, + IntChunk startPositions, IntChunk length, + WritableBooleanChunk stateModified) {} + + @Override + public boolean addChunk(SingletonContext context, int chunkSize, Chunk values, + LongChunk inputRowKeys, long destination) { + return false; + } + + @Override + public boolean removeChunk(SingletonContext context, int chunkSize, Chunk values, + LongChunk inputRowKeys, long destination) { + return false; + } + + @Override + public void ensureCapacity(long tableSize) {} + + @Override + public void startTrackingPrevValues() {} + + @Override + public UnaryOperator initializeRefreshing(@NotNull QueryTable resultTable, @NotNull LivenessReferent aggregationUpdateListener) { + return new InputToResultModifiedColumnSetFactory(resultTable, resultColumns.keySet().toArray(String[]::new)); + } + } + + @NotNull + public IterativeChunkedAggregationOperator resultExtractor(List resultPairs) { + final Map> resultColumns = new LinkedHashMap<>(resultPairs.size()); + for (final Pair pair : resultPairs) { + resultColumns.put(pair.output().name(), inputAggregatedColumns.get(pair.input().name())); + } + return new ResultExtractor(resultColumns); + } } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java index 183d30f245b..0565ed3a82a 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java @@ -402,7 +402,8 @@ public void testRollupFormulaStatic() { TableTools.show(snapshot.view(rollup1.getRowDepthColumn().name(), rollup1.getRowExpandedColumn().name(), "Key1", "Key2", "Sum", "FSum")); - final Table expected = initialExpectedGrouped(rollup1).dropColumns("Sentinel").update("FSum=ii == 0 ? 7 : 1 + Sum"); + final Table expected = + initialExpectedGrouped(rollup1).dropColumns("Sentinel").update("FSum=ii == 0 ? 7 : 1 + Sum"); assertTableEquals(expected, snapshot.dropColumns("__EXPOSED_GROUP_ROW_SETS__")); freeSnapshotTableChunks(snapshot); } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/by/AggregationProcessorTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/by/AggregationProcessorTest.java index c996fe69542..469b18bab5a 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/by/AggregationProcessorTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/by/AggregationProcessorTest.java @@ -3,82 +3,30 @@ // package io.deephaven.engine.table.impl.by; -import io.deephaven.api.ColumnName; -import io.deephaven.api.Selectable; import io.deephaven.api.agg.Aggregation; -import io.deephaven.api.agg.Count; -import io.deephaven.api.agg.spec.AggSpec; -import io.deephaven.base.FileUtils; import io.deephaven.chunk.util.pools.ChunkPoolReleaseTracking; -import io.deephaven.engine.context.ExecutionContext; -import io.deephaven.engine.context.QueryScope; -import io.deephaven.engine.liveness.LivenessScopeStack; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.rowset.RowSetFactory; -import io.deephaven.engine.rowset.RowSetShiftData; -import io.deephaven.engine.rowset.TrackingWritableRowSet; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.*; import io.deephaven.engine.table.impl.by.ssmminmax.SsmChunkedMinMaxOperator; -import io.deephaven.engine.table.impl.indexer.DataIndexer; import io.deephaven.engine.table.impl.perf.UpdatePerformanceTracker; -import io.deephaven.engine.table.impl.select.IncrementalReleaseFilter; -import io.deephaven.engine.table.impl.select.SelectColumn; -import io.deephaven.engine.table.impl.select.SelectColumnFactory; -import io.deephaven.engine.table.impl.select.SourceColumn; +import io.deephaven.engine.table.impl.sources.IntegerSingleValueSource; +import io.deephaven.engine.table.impl.sources.LongSingleValueSource; import io.deephaven.engine.table.impl.sources.NullValueColumnSource; -import io.deephaven.engine.table.impl.sources.UnionRedirection; -import io.deephaven.engine.table.impl.util.ColumnHolder; -import io.deephaven.engine.table.vectors.ColumnVectors; import io.deephaven.engine.testutil.*; -import io.deephaven.engine.testutil.QueryTableTestBase.TableComparator; import io.deephaven.engine.testutil.generator.*; import io.deephaven.engine.testutil.junit4.EngineCleanup; -import io.deephaven.engine.testutil.sources.TestColumnSource; -import io.deephaven.engine.testutil.testcase.RefreshingTableTestCase; -import io.deephaven.engine.util.TableDiff; import io.deephaven.engine.util.TableTools; -import io.deephaven.engine.util.systemicmarking.SystemicObjectTracker; -import io.deephaven.parquet.table.ParquetInstructions; -import io.deephaven.parquet.table.ParquetTools; -import io.deephaven.parquet.table.layout.ParquetKeyValuePartitionedLayout; -import io.deephaven.test.types.OutOfBandTest; -import io.deephaven.time.DateTimeUtils; import io.deephaven.util.QueryConstants; -import io.deephaven.util.SafeCloseable; -import io.deephaven.util.mutable.MutableInt; -import io.deephaven.vector.IntVector; -import io.deephaven.vector.ObjectVector; -import junit.framework.ComparisonFailure; -import junit.framework.TestCase; -import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; +import io.deephaven.vector.*; import org.junit.*; -import org.junit.experimental.categories.Category; - -import java.io.File; -import java.io.IOException; -import java.lang.reflect.Array; -import java.math.BigDecimal; -import java.math.BigInteger; -import java.nio.file.Files; + import java.time.Instant; import java.util.*; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.function.Consumer; -import java.util.function.Supplier; -import java.util.stream.Collectors; -import java.util.stream.IntStream; -import java.util.stream.Stream; import static io.deephaven.api.agg.Aggregation.*; -import static io.deephaven.api.agg.spec.AggSpec.percentile; import static io.deephaven.engine.testutil.TstUtils.*; -import static io.deephaven.engine.util.TableTools.*; -import static io.deephaven.parquet.base.ParquetUtils.PARQUET_FILE_EXTENSION; -import static io.deephaven.util.QueryConstants.*; -import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.Assert.*; +import static io.deephaven.engine.util.TableTools.col; +import static io.deephaven.engine.util.TableTools.longCol; import static org.junit.Assert.assertEquals; public class AggregationProcessorTest { @@ -160,4 +108,69 @@ public void testMinMaxSecondaryTypesBoolean() { .filter(o -> o.getClass().getCanonicalName().contains(".SecondaryOperator")).count()); } + + @Test + public void testGroupReuse() { + // We should only need a single group by operator; but we want to make sure our output order is correct + final Map> csmap = new LinkedHashMap<>(); + csmap.put("Timestamp", NullValueColumnSource.getInstance(Instant.class, null)); + csmap.put("LongValue", NullValueColumnSource.getInstance(long.class, null)); + csmap.put("IntValue", NullValueColumnSource.getInstance(int.class, null)); + + final QueryTable input = new QueryTable(i(0).toTracking(), csmap); + input.setRefreshing(true); + + final List aggs = + Arrays.asList(AggGroup("Timestamp"), AggGroup("LV=LongValue", "IV=IntValue"), AggGroup("TS=Timestamp")); + final Table agged = input.aggBy(aggs); + + assertEquals(ObjectVector.class, agged.getColumnSource("Timestamp").getType()); + assertEquals(Instant.class, agged.getColumnSource("Timestamp").getComponentType()); + assertEquals(LongVector.class, agged.getColumnSource("LV").getType()); + assertEquals(IntVector.class, agged.getColumnSource("IV").getType()); + assertEquals(ObjectVector.class, agged.getColumnSource("TS").getType()); + assertEquals(Instant.class, agged.getColumnSource("TS").getComponentType()); + + final ObjectVector tsVec = new ObjectVectorDirect<>(new Instant[] {null}); + final LongVector longVec = new LongVectorDirect(QueryConstants.NULL_LONG); + final IntVector intVec = new IntVectorDirect(QueryConstants.NULL_INT); + final Table expected = + TableTools.newTable(col("Timestamp", tsVec), col("LV", longVec), col("IV", intVec), col("TS", tsVec)); + assertTableEquals(expected, agged); + + // this part of the test just verifies that we have the secondary operators we expect + final AggregationContext ac = AggregationProcessor.forAggregation(aggs).makeAggregationContext(input, false); + Arrays.stream(ac.operators).forEach(o -> System.out.println(o.getClass().getCanonicalName())); + assertEquals(3, ac.operators.length); + assertEquals(1, Arrays.stream(ac.operators).filter(o -> o instanceof GroupByChunkedOperator).count()); + assertEquals(2, Arrays.stream(ac.operators) + .filter(o -> o.getClass().getCanonicalName().contains("ResultExtractor")).count()); + } + + @Test + public void testFormulaGroupReuse() { + final Map> csmap = new LinkedHashMap<>(); + csmap.put("LongValue", new LongSingleValueSource()); + csmap.put("IntValue", new IntegerSingleValueSource()); + + ((LongSingleValueSource)csmap.get("LongValue")).set(10L); + ((IntegerSingleValueSource)csmap.get("IntValue")).set(20); + + final QueryTable input = new QueryTable(i(0).toTracking(), csmap); + input.setRefreshing(true); + + final List aggs = + Arrays.asList(AggGroup("LongValue"), AggFormula("LS=sum(LongValue)"), AggFormula("IS=sum(IntValue)")); + final Table agged = input.aggBy(aggs); + + final LongVectorDirect lvd = new LongVectorDirect(10L); + assertTableEquals(TableTools.newTable(col("LongValue", (LongVector)lvd), longCol("LS", 10L), longCol("IS", 20L)), agged); + + // this part of the test just verifies that we have the secondary operators we expect + final AggregationContext ac = AggregationProcessor.forAggregation(aggs).makeAggregationContext(input, false); + Arrays.stream(ac.operators).forEach(o -> System.out.println(o.getClass().getCanonicalName())); + assertEquals(3, ac.operators.length); + assertEquals(1, Arrays.stream(ac.operators).filter(o -> o instanceof GroupByChunkedOperator).count()); + assertEquals(2, Arrays.stream(ac.operators).filter(o -> o instanceof FormulaMultiColumnChunkedOperator).count()); + } } From eb10ff507cb74c20d9093796296d70c9c4eed95f Mon Sep 17 00:00:00 2001 From: "Charles P. Wright" Date: Mon, 29 Dec 2025 16:37:46 -0500 Subject: [PATCH 09/24] spotless --- .../engine/table/impl/by/AggregationProcessor.java | 9 ++++----- .../engine/table/impl/by/GroupByChunkedOperator.java | 6 ++++-- .../engine/table/impl/by/AggregationProcessorTest.java | 10 ++++++---- 3 files changed, 14 insertions(+), 11 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index ce26aabbb8a..60aa2cebbe5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -748,9 +748,9 @@ int existingGroupByOperatorIndex() { } GroupByChunkedOperator ensureGroupingOperator(final QueryTable table, - final int existingOperatorIndex, - final String exposeRowSetAs, - final MatchPair[] matchPairs) { + final int existingOperatorIndex, + final String exposeRowSetAs, + final MatchPair[] matchPairs) { boolean recreate = false; final GroupByChunkedOperator existing = (GroupByChunkedOperator) operators.get(existingOperatorIndex); if (exposeRowSetAs != null) { @@ -790,8 +790,7 @@ GroupByChunkedOperator ensureGroupingOperator(final QueryTable table, for (IterativeChunkedAggregationOperator operator : operators) { if (operator instanceof FormulaMultiColumnChunkedOperator) { ((FormulaMultiColumnChunkedOperator) operator).updateGroupBy(newOperator, false); - } - else if (operator instanceof FormulaChunkedOperator) { + } else if (operator instanceof FormulaChunkedOperator) { ((FormulaChunkedOperator) operator).updateGroupBy(newOperator, false); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java index 18eb400b28e..a006fe3f99c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java @@ -715,8 +715,10 @@ public void ensureCapacity(long tableSize) {} public void startTrackingPrevValues() {} @Override - public UnaryOperator initializeRefreshing(@NotNull QueryTable resultTable, @NotNull LivenessReferent aggregationUpdateListener) { - return new InputToResultModifiedColumnSetFactory(resultTable, resultColumns.keySet().toArray(String[]::new)); + public UnaryOperator initializeRefreshing(@NotNull QueryTable resultTable, + @NotNull LivenessReferent aggregationUpdateListener) { + return new InputToResultModifiedColumnSetFactory(resultTable, + resultColumns.keySet().toArray(String[]::new)); } } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/by/AggregationProcessorTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/by/AggregationProcessorTest.java index 469b18bab5a..473cf9d76ce 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/by/AggregationProcessorTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/by/AggregationProcessorTest.java @@ -153,8 +153,8 @@ public void testFormulaGroupReuse() { csmap.put("LongValue", new LongSingleValueSource()); csmap.put("IntValue", new IntegerSingleValueSource()); - ((LongSingleValueSource)csmap.get("LongValue")).set(10L); - ((IntegerSingleValueSource)csmap.get("IntValue")).set(20); + ((LongSingleValueSource) csmap.get("LongValue")).set(10L); + ((IntegerSingleValueSource) csmap.get("IntValue")).set(20); final QueryTable input = new QueryTable(i(0).toTracking(), csmap); input.setRefreshing(true); @@ -164,13 +164,15 @@ public void testFormulaGroupReuse() { final Table agged = input.aggBy(aggs); final LongVectorDirect lvd = new LongVectorDirect(10L); - assertTableEquals(TableTools.newTable(col("LongValue", (LongVector)lvd), longCol("LS", 10L), longCol("IS", 20L)), agged); + assertTableEquals( + TableTools.newTable(col("LongValue", (LongVector) lvd), longCol("LS", 10L), longCol("IS", 20L)), agged); // this part of the test just verifies that we have the secondary operators we expect final AggregationContext ac = AggregationProcessor.forAggregation(aggs).makeAggregationContext(input, false); Arrays.stream(ac.operators).forEach(o -> System.out.println(o.getClass().getCanonicalName())); assertEquals(3, ac.operators.length); assertEquals(1, Arrays.stream(ac.operators).filter(o -> o instanceof GroupByChunkedOperator).count()); - assertEquals(2, Arrays.stream(ac.operators).filter(o -> o instanceof FormulaMultiColumnChunkedOperator).count()); + assertEquals(2, + Arrays.stream(ac.operators).filter(o -> o instanceof FormulaMultiColumnChunkedOperator).count()); } } From 9b9e4ad66a0058d5df87ba39ea4c5c580462f092 Mon Sep 17 00:00:00 2001 From: "Charles P. Wright" Date: Mon, 29 Dec 2025 18:29:04 -0500 Subject: [PATCH 10/24] test fix --- py/server/tests/test_table.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/py/server/tests/test_table.py b/py/server/tests/test_table.py index 329d6bc707e..e8f35c9d04b 100644 --- a/py/server/tests/test_table.py +++ b/py/server/tests/test_table.py @@ -93,9 +93,9 @@ def setUp(self): abs_sum(["aggAbsSum=var"]), var(["aggVar=var"]), weighted_avg("var", ["weights"]), + group(["aggGroup=var"]), ] self.aggs_not_for_rollup = [ - group(["aggGroup=var"]), partition("aggPartition"), median(["aggMed=var"]), pct(0.20, ["aggPct=var"]), From c71c46c7e19e9e59316dac0f2764202d821e39e0 Mon Sep 17 00:00:00 2001 From: "Charles P. Wright" Date: Tue, 30 Dec 2025 11:50:22 -0500 Subject: [PATCH 11/24] more todos for reuse. --- .../table/impl/by/AggregationProcessor.java | 23 ++++++++++++++----- 1 file changed, 17 insertions(+), 6 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index 60aa2cebbe5..041b2e802d1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -937,7 +937,8 @@ public void visit(@NotNull final AggSpecFirst first) { @Override public void visit(@NotNull final AggSpecFormula formula) { unsupportedForBlinkTables("Formula"); - // TODO: re-use shared groupBy operators (https://github.com/deephaven/deephaven-core/issues/6363) + // Note: we do not attempt to reuse the groupBy operator for the deprecated "each" formula, we only reuse + // them for the new-style multi-column formula operators final GroupByChunkedOperator groupByChunkedOperator = new GroupByChunkedOperator(table, false, null, null, resultPairs.stream().map(pair -> MatchPair.of((Pair) pair.input())).toArray(MatchPair[]::new)); final FormulaChunkedOperator formulaChunkedOperator = new FormulaChunkedOperator(groupByChunkedOperator, @@ -955,10 +956,9 @@ public void visit(AggSpecFreeze freeze) { public void visit(@NotNull final AggSpecGroup group) { unsupportedForBlinkTables("Group"); - // TODO: re-use shared groupBy operators (https://github.com/deephaven/deephaven-core/issues/6363) final int existingOperator = existingGroupByOperatorIndex(); if (existingOperator >= 0) { - // TODO: we must ensure the input columns are all properly represented but hidden + // Reuse the operator, adding a result extractor for the new result pairs GroupByChunkedOperator existing = ensureGroupingOperator(table, existingOperator, null, MatchPair.fromPairs(resultPairs)); addNoInputOperator(existing.resultExtractor(resultPairs)); @@ -1239,9 +1239,17 @@ public void visit(@NotNull final Partition partition) { @Override public void visit(AggSpecGroup group) { unsupportedForBlinkTables("Group for rollup"); - addNoInputOperator(new GroupByChunkedOperator(table, true, EXPOSED_GROUP_ROW_SETS.name(), - null, - MatchPair.fromPairs(resultPairs))); + + final int indexOfExistingOperator = existingGroupByOperatorIndex(); + if (indexOfExistingOperator >= 0) { + // share the existing operator for groupBy in a rollup base + final GroupByChunkedOperator existing = ensureGroupingOperator(table, indexOfExistingOperator, EXPOSED_GROUP_ROW_SETS.name(), MatchPair.fromPairs(resultPairs)); + addNoInputOperator(existing.resultExtractor(resultPairs)); + } else { + addNoInputOperator(new GroupByChunkedOperator(table, true, EXPOSED_GROUP_ROW_SETS.name(), + null, + MatchPair.fromPairs(resultPairs))); + } } @Override @@ -1266,6 +1274,8 @@ public void visit(Formula formula) { final String[] inputNonKeyColumns = partitioned.get(false).toArray(String[]::new); validateSelectColumnForFormula(selectColumn); + + // TODO: rollup base formula share groupBy // TODO: re-use shared groupBy operators (https://github.com/deephaven/deephaven-core/issues/6363) final GroupByChunkedOperator groupByChunkedOperator; @@ -1523,6 +1533,7 @@ public void visit(AggSpecGroup group) { for (int ii = 0; ii < resultPairs.size(); ++ii) { pairs[ii] = new MatchPair(resultPairs.get(ii).output().name(), resultPairs.get(ii).output().name()); } + // TODO: share the existing group by operator for a rollup reaggregation addOperator(new GroupByReaggregateOperator(table, true, EXPOSED_GROUP_ROW_SETS.name(), pairs), groupRowSet, EXPOSED_GROUP_ROW_SETS.name()); } From da011ea7392c1b4d543a39a7c27daf90842a8ad3 Mon Sep 17 00:00:00 2001 From: "Charles P. Wright" Date: Wed, 31 Dec 2025 07:57:56 -0500 Subject: [PATCH 12/24] Reaggregate groupby reuse. --- .../table/impl/by/AggregationProcessor.java | 150 +++++++++++------- .../by/FormulaMultiColumnChunkedOperator.java | 2 +- .../table/impl/by/GroupByChunkedOperator.java | 18 ++- .../impl/by/GroupByReaggregateOperator.java | 105 +++++++++++- .../engine/table/impl/TestRollupTable.java | 47 ++++++ 5 files changed, 248 insertions(+), 74 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index 041b2e802d1..ef74ff7bd6c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -747,7 +747,19 @@ int existingGroupByOperatorIndex() { return -1; } - GroupByChunkedOperator ensureGroupingOperator(final QueryTable table, + /** + * @return the index of an existing group by reaggregation operator, or -1 if it no operator was found + */ + int existingGroupByReggregateIndex() { + for (int ii = 0; ii < operators.size(); ++ii) { + if (operators.get(ii) instanceof GroupByReaggregateOperator) { + return ii; + } + } + return -1; + } + + GroupByChunkedOperator ensureGroupByOperator(final QueryTable table, final int existingOperatorIndex, final String exposeRowSetAs, final MatchPair[] matchPairs) { @@ -798,6 +810,59 @@ GroupByChunkedOperator ensureGroupingOperator(final QueryTable table, operators.set(existingOperatorIndex, newOperator); return newOperator; } + + GroupByReaggregateOperator ensureGroupByReaggregateOperator(final QueryTable table, + final int existingOperatorIndex, + final String exposeRowSetAs, + final MatchPair[] matchPairs) { + boolean recreate = false; + final GroupByReaggregateOperator existing = + (GroupByReaggregateOperator) operators.get(existingOperatorIndex); + if (exposeRowSetAs != null) { + if (existing.getExposedRowSetsAs() == null) { + recreate = true; + } else { + if (!existing.getExposedRowSetsAs().equals(exposeRowSetAs)) { + throw new UnsupportedOperationException( + "AggGroupBy cannot have inconsistent exposed row redirections names: " + + existing.getExposedRowSetsAs() + " != " + exposeRowSetAs); + } + } + } + final List newPairs = new ArrayList<>(Arrays.asList(existing.getAggregatedColumnPairs())); + List existingHidden = existing.getHiddenResults(); + final List hiddenResults = new ArrayList<>(existingHidden == null ? List.of() : existingHidden); + for (MatchPair matchPair : matchPairs) { + final String input = matchPair.input().name(); + if (Arrays.stream(existing.getAggregatedColumnPairs()).noneMatch(p -> p.input().name().equals(input))) { + newPairs.add(matchPair); + hiddenResults.add(matchPair.output().name()); + recreate = true; + } + } + if (!recreate) { + // we're totally satisfied with the existing operator for use with a secondary operator that pulls an + // output from it to the desired name + return existing; + } + + final String newExposeRowsetName = exposeRowSetAs == null ? existing.getExposedRowSetsAs() : exposeRowSetAs; + final MatchPair[] newMatchPairArray = newPairs.toArray(MatchPair[]::new); + final GroupByReaggregateOperator newOperator = + new GroupByReaggregateOperator(table, true, newExposeRowsetName, hiddenResults, newMatchPairArray); + + // any formula operators that used the old group by operator must be updated + for (IterativeChunkedAggregationOperator operator : operators) { + // Only FormulaMultiColumn operators need to be adjusted, a FormulaChunkedOperator cannot participate + // in a rollup. + if (operator instanceof FormulaMultiColumnChunkedOperator) { + ((FormulaMultiColumnChunkedOperator) operator).updateGroupBy(newOperator, false); + } + } + + operators.set(existingOperatorIndex, newOperator); + return newOperator; + } } // ----------------------------------------------------------------------------------------------------------------- @@ -888,7 +953,7 @@ public void visit(@NotNull final Formula formula) { // if we have an existing group by operator, then use it (or update it to reflect our input columns) final MatchPair[] matchPairs = Arrays.stream(inputNonKeyColumns).map(cn -> new MatchPair(cn, cn)).toArray(MatchPair[]::new); - groupByChunkedOperator = ensureGroupingOperator(table, existingGroupByOperatorIndex, null, matchPairs); + groupByChunkedOperator = ensureGroupByOperator(table, existingGroupByOperatorIndex, null, matchPairs); } else { groupByChunkedOperator = makeGroupByOperatorForFormula(inputNonKeyColumns, table, null); } @@ -960,7 +1025,7 @@ public void visit(@NotNull final AggSpecGroup group) { if (existingOperator >= 0) { // Reuse the operator, adding a result extractor for the new result pairs GroupByChunkedOperator existing = - ensureGroupingOperator(table, existingOperator, null, MatchPair.fromPairs(resultPairs)); + ensureGroupByOperator(table, existingOperator, null, MatchPair.fromPairs(resultPairs)); addNoInputOperator(existing.resultExtractor(resultPairs)); } else { addNoInputOperator( @@ -1243,7 +1308,8 @@ public void visit(AggSpecGroup group) { final int indexOfExistingOperator = existingGroupByOperatorIndex(); if (indexOfExistingOperator >= 0) { // share the existing operator for groupBy in a rollup base - final GroupByChunkedOperator existing = ensureGroupingOperator(table, indexOfExistingOperator, EXPOSED_GROUP_ROW_SETS.name(), MatchPair.fromPairs(resultPairs)); + final GroupByChunkedOperator existing = ensureGroupByOperator(table, indexOfExistingOperator, + EXPOSED_GROUP_ROW_SETS.name(), MatchPair.fromPairs(resultPairs)); addNoInputOperator(existing.resultExtractor(resultPairs)); } else { addNoInputOperator(new GroupByChunkedOperator(table, true, EXPOSED_GROUP_ROW_SETS.name(), @@ -1275,72 +1341,26 @@ public void visit(Formula formula) { validateSelectColumnForFormula(selectColumn); - // TODO: rollup base formula share groupBy - // TODO: re-use shared groupBy operators (https://github.com/deephaven/deephaven-core/issues/6363) - final GroupByChunkedOperator groupByChunkedOperator; - final int existingGroupByOperatorIndex = existingGroupByOperatorIndex(); - final boolean delegate; + final int existingGroupByOperatorIndex = existingGroupByOperatorIndex(); if (formula.reaggregateAggregatedValues()) { if (existingGroupByOperatorIndex >= 0) { - final GroupByChunkedOperator existing = - (GroupByChunkedOperator) operators.get(existingGroupByOperatorIndex); - MatchPair[] existingPairs = existing.getAggregatedColumnPairs(); - for (final String searchColumn : inputNonKeyColumns) { - int searchIdx = 0; - while (searchIdx < existingPairs.length) { - if (existingPairs[searchIdx].left().name().equals(searchColumn) - && existingPairs[searchIdx].right().name().equals(searchColumn)) { - break; - } - searchIdx++; - } - if (searchIdx == existingPairs.length) { - throw new IllegalStateException( - "Existing groupBy operator does not have the required input column " - + searchColumn); - } - } - // TODO: we have one, let's replace it if it is insufficient - groupByChunkedOperator = existing; + groupByChunkedOperator = ensureGroupByOperator(table, existingGroupByOperatorIndex, null, + MatchPair.fromPairs(Pair.from(inputNonKeyColumns))); delegate = false; } else { // We we are reaggregating, we do not expose the rowsets, because the next level creates a - // completely - // fresh operator + // completely fresh operator groupByChunkedOperator = makeGroupByOperatorForFormula(inputNonKeyColumns, table, null); // the operator is not added, so there is delegation delegate = true; } } else { if (existingGroupByOperatorIndex >= 0) { - final GroupByChunkedOperator existing = - (GroupByChunkedOperator) operators.get(existingGroupByOperatorIndex); - if (!EXPOSED_GROUP_ROW_SETS.name().equals(existing.getExposedRowSetsAs())) { - throw new IllegalStateException("Existing groupBy operator does not expose rowsets"); - } - - MatchPair[] existingPairs = existing.getAggregatedColumnPairs(); - for (int ii = 0; ii < inputNonKeyColumns.length; ++ii) { - final String searchColumn = inputNonKeyColumns[ii]; - int searchIdx = 0; - while (searchIdx < existingPairs.length) { - if (existingPairs[searchIdx].left().name().equals(searchColumn) - && existingPairs[searchIdx].right().name().equals(searchColumn)) { - break; - } - searchIdx++; - } - if (searchIdx == existingPairs.length) { - throw new IllegalStateException( - "Existing groupBy operator does not have the required input column " - + searchColumn); - } - } - // TODO: we have one, let's replace it if it is insufficient - groupByChunkedOperator = existing; + groupByChunkedOperator = ensureGroupByOperator(table, existingGroupByOperatorIndex, + EXPOSED_GROUP_ROW_SETS.name(), MatchPair.fromPairs(Pair.from(inputNonKeyColumns))); delegate = false; } else { // When we do not reaggregate, the next level needs access to our exposed group row sets @@ -1533,9 +1553,17 @@ public void visit(AggSpecGroup group) { for (int ii = 0; ii < resultPairs.size(); ++ii) { pairs[ii] = new MatchPair(resultPairs.get(ii).output().name(), resultPairs.get(ii).output().name()); } - // TODO: share the existing group by operator for a rollup reaggregation - addOperator(new GroupByReaggregateOperator(table, true, EXPOSED_GROUP_ROW_SETS.name(), pairs), groupRowSet, - EXPOSED_GROUP_ROW_SETS.name()); + final int existingGroupByOperatorIndex = existingGroupByReggregateIndex(); + if (existingGroupByOperatorIndex >= 0) { + final GroupByReaggregateOperator existing = ensureGroupByReaggregateOperator(table, + existingGroupByOperatorIndex, EXPOSED_GROUP_ROW_SETS.name(), pairs); + // TODO: test the result extractor to sharing + addNoInputOperator(existing.resultExtractor(resultPairs)); + } else { + addOperator(new GroupByReaggregateOperator(table, true, EXPOSED_GROUP_ROW_SETS.name(), null, pairs), + groupRowSet, + EXPOSED_GROUP_ROW_SETS.name()); + } } @Override @@ -1564,7 +1592,7 @@ public void visit(Formula formula) { final String[] inputNonKeyColumns = partitioned.get(false).toArray(String[]::new); validateSelectColumnForFormula(selectColumn); - // TODO: re-use shared groupBy operators (https://github.com/deephaven/deephaven-core/issues/6363) + // TODO: re-use shared groupBy operators for a rollup formula reaggregation // final GroupByChunkedOperator groupByChunkedOperator = makeGroupByOperatorForFormula(inputNonKeyColumns, // table); @@ -1603,7 +1631,7 @@ public void visit(Formula formula) { } else { final ColumnSource groupRowSet = table.getColumnSource(EXPOSED_GROUP_ROW_SETS.name()); groupByOperator = - new GroupByReaggregateOperator(table, true, EXPOSED_GROUP_ROW_SETS.name(), groupPairs); + new GroupByReaggregateOperator(table, true, EXPOSED_GROUP_ROW_SETS.name(), null, groupPairs); addOperator(groupByOperator, groupRowSet, EXPOSED_GROUP_ROW_SETS.name()); final FormulaMultiColumnChunkedOperator op = new FormulaMultiColumnChunkedOperator(table, groupByOperator, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaMultiColumnChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaMultiColumnChunkedOperator.java index 269355d9e83..e327d3794dc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaMultiColumnChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaMultiColumnChunkedOperator.java @@ -422,7 +422,7 @@ private static long calculateContainingBlockLastKey(final long firstKey) { return (firstKey / BLOCK_SIZE) * BLOCK_SIZE + BLOCK_SIZE - 1; } - public void updateGroupBy(GroupByChunkedOperator groupBy, boolean delegateToBy) { + public void updateGroupBy(GroupByOperator groupBy, boolean delegateToBy) { this.groupBy = groupBy; this.delegateToBy = delegateToBy; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java index a006fe3f99c..4baf3745c51 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java @@ -435,6 +435,7 @@ public UnaryOperator initializeRefreshing( initializeNewRowSetPreviousValues(resultTable.getRowSet()); return registeredWithHelper ? new InputToResultModifiedColumnSetFactory(resultTable, + inputColumnNamesForResults, resultAggregatedColumns.keySet().toArray(String[]::new)) : null; } @@ -449,7 +450,7 @@ public UnaryOperator initializeRefreshing( UnaryOperator makeInputToResultModifiedColumnSetFactory( @NotNull final QueryTable resultTable, @NotNull final String[] resultColumnNames) { - return new InputToResultModifiedColumnSetFactory(resultTable, resultColumnNames); + return new InputToResultModifiedColumnSetFactory(resultTable, inputColumnNamesForResults, resultColumnNames); } private class InputToResultModifiedColumnSetFactory implements UnaryOperator { @@ -460,6 +461,7 @@ private class InputToResultModifiedColumnSetFactory implements UnaryOperator getHiddenResults() { private class ResultExtractor implements IterativeChunkedAggregationOperator { final Map> resultColumns; + final String[] inputColumnNames; - private ResultExtractor(Map> resultColumns) { + private ResultExtractor(Map> resultColumns, String[] inputColumnNames) { this.resultColumns = resultColumns; + this.inputColumnNames = inputColumnNames; } @Override @@ -718,16 +722,20 @@ public void startTrackingPrevValues() {} public UnaryOperator initializeRefreshing(@NotNull QueryTable resultTable, @NotNull LivenessReferent aggregationUpdateListener) { return new InputToResultModifiedColumnSetFactory(resultTable, + inputColumnNames, resultColumns.keySet().toArray(String[]::new)); } } @NotNull public IterativeChunkedAggregationOperator resultExtractor(List resultPairs) { + final List inputColumnNamesList = new ArrayList<>(resultPairs.size()); final Map> resultColumns = new LinkedHashMap<>(resultPairs.size()); for (final Pair pair : resultPairs) { - resultColumns.put(pair.output().name(), inputAggregatedColumns.get(pair.input().name())); + final String inputName = pair.input().name(); + inputColumnNamesList.add(inputName); + resultColumns.put(pair.output().name(), inputAggregatedColumns.get(inputName)); } - return new ResultExtractor(resultColumns); + return new ResultExtractor(resultColumns, inputColumnNamesList.toArray(String[]::new)); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java index 995c1216bdd..c7dd98fe32d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java @@ -3,6 +3,7 @@ // package io.deephaven.engine.table.impl.by; +import io.deephaven.api.Pair; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.ChunkLengths; @@ -19,9 +20,7 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.util.Arrays; -import java.util.LinkedHashMap; -import java.util.Map; +import java.util.*; import java.util.function.UnaryOperator; import static io.deephaven.engine.table.impl.sources.ArrayBackedColumnSource.BLOCK_SIZE; @@ -48,14 +47,17 @@ public final class GroupByReaggregateOperator implements GroupByOperator { private final QueryTable inputTable; private final boolean registeredWithHelper; private final String exposeRowSetsAs; + private final MatchPair[] aggregatedColumnPairs; + private final List hiddenResults; private final boolean live; private final ObjectArraySource rowSets; private final ObjectArraySource addedBuilders; private final ObjectArraySource removedBuilders; - private final String[] inputColumnNames; + private final String[] inputColumnNamesForResults; + private final Map> inputAggregatedColumns; private final Map> resultAggregatedColumns; private RowSetBuilderRandom stepDestinationsModified; @@ -66,10 +68,13 @@ public GroupByReaggregateOperator( @NotNull final QueryTable inputTable, final boolean registeredWithHelper, @Nullable final String exposeRowSetsAs, + @Nullable List hiddenResults, @NotNull final MatchPair... aggregatedColumnPairs) { this.inputTable = inputTable; this.registeredWithHelper = registeredWithHelper; this.exposeRowSetsAs = exposeRowSetsAs; + this.hiddenResults = hiddenResults; + this.aggregatedColumnPairs = aggregatedColumnPairs; if (exposeRowSetsAs == null) { throw new IllegalArgumentException("Must expose group RowSets for rollup."); @@ -79,7 +84,9 @@ public GroupByReaggregateOperator( rowSets = new ObjectArraySource<>(WritableRowSet.class); addedBuilders = new ObjectArraySource<>(Object.class); + inputAggregatedColumns = new LinkedHashMap<>(aggregatedColumnPairs.length); resultAggregatedColumns = new LinkedHashMap<>(aggregatedColumnPairs.length); + final List inputColumnNamesForResultsList = new ArrayList<>(); Arrays.stream(aggregatedColumnPairs).forEach(pair -> { // we are reaggregationg so have to use the left column for everything final ColumnSource source = inputTable.getColumnSource(pair.leftColumn()); @@ -89,7 +96,11 @@ public GroupByReaggregateOperator( @SuppressWarnings("rawtypes") final ColumnSource realSource = ((AggregateColumnSource) source).getAggregatedSource(); final AggregateColumnSource aggregateColumnSource = AggregateColumnSource.make(realSource, rowSets); - resultAggregatedColumns.put(pair.leftColumn(), aggregateColumnSource); + if (hiddenResults == null || !hiddenResults.contains(pair.output().name())) { + resultAggregatedColumns.put(pair.output().name(), aggregateColumnSource); + inputColumnNamesForResultsList.add(pair.input().name()); + } + inputAggregatedColumns.put(pair.input().name(), aggregateColumnSource); }); if (resultAggregatedColumns.containsKey(exposeRowSetsAs)) { @@ -97,7 +108,7 @@ public GroupByReaggregateOperator( "Exposing group RowSets as %s, but this conflicts with a requested grouped output column name", exposeRowSetsAs)); } - inputColumnNames = MatchPair.getRightColumns(aggregatedColumnPairs); + inputColumnNamesForResults = inputColumnNamesForResultsList.toArray(String[]::new); removedBuilders = live ? new ObjectArraySource<>(Object.class) : null; initialized = false; } @@ -293,7 +304,7 @@ public void startTrackingPrevValues() { // previously. // NB: These are usually (always, as of now) instances of AggregateColumnSource, meaning // startTrackingPrevValues() is a no-op. - resultAggregatedColumns.values().forEach(ColumnSource::startTrackingPrevValues); + inputAggregatedColumns.values().forEach(ColumnSource::startTrackingPrevValues); } @Override @@ -303,6 +314,7 @@ public UnaryOperator initializeRefreshing( initializeNewRowSetPreviousValues(resultTable.getRowSet()); return registeredWithHelper ? new InputToResultModifiedColumnSetFactory(resultTable, + inputColumnNamesForResults, resultAggregatedColumns.keySet().toArray(String[]::new)) : null; } @@ -325,6 +337,7 @@ private class InputToResultModifiedColumnSetFactory implements UnaryOperator getHiddenResults() { + return hiddenResults; + } + + private class ResultExtractor implements IterativeChunkedAggregationOperator { + final Map> resultColumns; + final String[] inputColumnNames; + + private ResultExtractor(Map> resultColumns, String[] inputColumnNames) { + this.resultColumns = resultColumns; + this.inputColumnNames = inputColumnNames; + } + + @Override + public Map> getResultColumns() { + return resultColumns; + } + + @Override + public void addChunk(BucketedContext context, Chunk values, + LongChunk inputRowKeys, IntChunk destinations, + IntChunk startPositions, IntChunk length, + WritableBooleanChunk stateModified) {} + + @Override + public void removeChunk(BucketedContext context, Chunk values, + LongChunk inputRowKeys, IntChunk destinations, + IntChunk startPositions, IntChunk length, + WritableBooleanChunk stateModified) {} + + @Override + public boolean addChunk(SingletonContext context, int chunkSize, Chunk values, + LongChunk inputRowKeys, long destination) { + return false; + } + + @Override + public boolean removeChunk(SingletonContext context, int chunkSize, Chunk values, + LongChunk inputRowKeys, long destination) { + return false; + } + + @Override + public void ensureCapacity(long tableSize) {} + + @Override + public void startTrackingPrevValues() {} + + @Override + public UnaryOperator initializeRefreshing(@NotNull QueryTable resultTable, + @NotNull LivenessReferent aggregationUpdateListener) { + return new InputToResultModifiedColumnSetFactory(resultTable, + inputColumnNames, + resultColumns.keySet().toArray(String[]::new)); + } + } + + @NotNull + public IterativeChunkedAggregationOperator resultExtractor(List resultPairs) { + final List inputColumnNamesList = new ArrayList<>(resultPairs.size()); + final Map> resultColumns = new LinkedHashMap<>(resultPairs.size()); + for (final Pair pair : resultPairs) { + final String inputName = pair.input().name(); + inputColumnNamesList.add(inputName); + resultColumns.put(pair.output().name(), inputAggregatedColumns.get(inputName)); + } + return new ResultExtractor(resultColumns, inputColumnNamesList.toArray(String[]::new)); + } } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java index 0565ed3a82a..95386513367 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java @@ -545,4 +545,51 @@ public void testRollupGroupIncremental() { assertTableEquals(expected2, snapshot2.dropColumns("__EXPOSED_GROUP_ROW_SETS__")); freeSnapshotTableChunks(snapshot2); } + + @Test + public void testReusedGrouping() { + final QueryTable source = TstUtils.testRefreshingTable( + stringCol("Key1", "Alpha", "Bravo", "Alpha", "Charlie", "Charlie", "Bravo", "Bravo"), + stringCol("Key2", "Delta", "Delta", "Echo", "Echo", "Echo", "Echo", "Echo"), + intCol("Sentinel", 1, 2, 3, 4, 5, 6, 7)); + + final RollupTable rollup1 = + source.rollup(List.of(AggGroup("Sentinel"), AggSum("Sum=Sentinel"), AggGroup("S2=Sentinel")), "Key1", + "Key2"); + + final String[] arrayWithNull = new String[1]; + final Table keyTable = newTable( + intCol(rollup1.getRowDepthColumn().name(), 0), + stringCol("Key1", arrayWithNull), + stringCol("Key2", arrayWithNull), + byteCol("Action", HierarchicalTable.KEY_TABLE_ACTION_EXPAND_ALL)); + + final HierarchicalTable.SnapshotState ss1 = rollup1.makeSnapshotState(); + final Table snapshot = + snapshotToTable(rollup1, ss1, keyTable, ColumnName.of("Action"), null, RowSetFactory.flat(30)); + TableTools.showWithRowSet(snapshot); + + final Table expected = initialExpectedGrouped(rollup1).update("S2=Sentinel"); + assertTableEquals(expected, snapshot.dropColumns("__EXPOSED_GROUP_ROW_SETS__")); + freeSnapshotTableChunks(snapshot); + + final ControlledUpdateGraph cug = source.getUpdateGraph().cast(); + cug.runWithinUnitTestCycle(() -> { + addToTable(source, i(10, 11), stringCol("Key1", "Alpha", "Charlie"), stringCol("Key2", "Echo", "Echo"), + intCol("Sentinel", 8, 9)); + removeRows(source, i(5)); + source.notifyListeners( + new TableUpdateImpl(i(10, 11), i(5), i(), RowSetShiftData.EMPTY, ModifiedColumnSet.EMPTY)); + }); + + final Table snapshot2 = + snapshotToTable(rollup1, ss1, keyTable, ColumnName.of("Action"), null, RowSetFactory.flat(30)); + TableTools.showWithRowSet(snapshot2); + Table expected2 = secondExpectedGrouped(rollup1).update("S2=Sentinel"); + TableTools.showWithRowSet(expected2); + assertTableEquals(expected2, snapshot2.dropColumns("__EXPOSED_GROUP_ROW_SETS__")); + freeSnapshotTableChunks(snapshot2); + + // TODO: modify only one column, validate that we get results that we expect without excess modifications + } } From 801caa4c52372bfb40d5ea9fa35eb13306a0ff02 Mon Sep 17 00:00:00 2001 From: "Charles P. Wright" Date: Wed, 31 Dec 2025 08:41:08 -0500 Subject: [PATCH 13/24] some coverage of operator reuse. --- .../table/impl/by/AggregationProcessor.java | 37 +++++++++++------ .../engine/table/impl/TestRollupTable.java | 41 ++++++++++++++----- 2 files changed, 55 insertions(+), 23 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index ef74ff7bd6c..763f829887b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -1557,7 +1557,6 @@ public void visit(AggSpecGroup group) { if (existingGroupByOperatorIndex >= 0) { final GroupByReaggregateOperator existing = ensureGroupByReaggregateOperator(table, existingGroupByOperatorIndex, EXPOSED_GROUP_ROW_SETS.name(), pairs); - // TODO: test the result extractor to sharing addNoInputOperator(existing.resultExtractor(resultPairs)); } else { addOperator(new GroupByReaggregateOperator(table, true, EXPOSED_GROUP_ROW_SETS.name(), null, pairs), @@ -1592,9 +1591,6 @@ public void visit(Formula formula) { final String[] inputNonKeyColumns = partitioned.get(false).toArray(String[]::new); validateSelectColumnForFormula(selectColumn); - // TODO: re-use shared groupBy operators for a rollup formula reaggregation - // final GroupByChunkedOperator groupByChunkedOperator = makeGroupByOperatorForFormula(inputNonKeyColumns, - // table); final Map renames = new HashMap<>(); final MatchPair[] groupPairs = new MatchPair[inputNonKeyColumns.length]; @@ -1612,17 +1608,22 @@ public void visit(Formula formula) { } } - GroupByOperator groupByOperator; - final IntegerSingleValueSource depthSource = new IntegerSingleValueSource(); depthSource.set(groupByColumnNames.length); if (formula.reaggregateAggregatedValues()) { - // everything gets hidden - final List hiddenPairs = - Arrays.stream(groupPairs).map(mp -> mp.left().name()).collect(Collectors.toList()); - groupByOperator = new GroupByChunkedOperator(table, false, null, hiddenPairs, groupPairs); + GroupByChunkedOperator groupByOperator; + + final int existingIndex = existingGroupByOperatorIndex(); + if (existingIndex >= 0) { + groupByOperator = ensureGroupByOperator(table, existingIndex, null, groupPairs); + } else { + final List hiddenPairs = + Arrays.stream(groupPairs).map(mp -> mp.left().name()).collect(Collectors.toList()); + groupByOperator = new GroupByChunkedOperator(table, false, null, hiddenPairs, groupPairs); + } + // everything gets hidden final FormulaMultiColumnChunkedOperator op = new FormulaMultiColumnChunkedOperator(table, groupByOperator, true, selectColumn, inputKeyColumns, null, depthSource); @@ -1630,9 +1631,19 @@ public void visit(Formula formula) { addOperator(op, null, inputNonKeyColumns); } else { final ColumnSource groupRowSet = table.getColumnSource(EXPOSED_GROUP_ROW_SETS.name()); - groupByOperator = - new GroupByReaggregateOperator(table, true, EXPOSED_GROUP_ROW_SETS.name(), null, groupPairs); - addOperator(groupByOperator, groupRowSet, EXPOSED_GROUP_ROW_SETS.name()); + GroupByReaggregateOperator groupByOperator; + + final int existingIndex = existingGroupByReggregateIndex(); + if (existingIndex >= 0) { + groupByOperator = ensureGroupByReaggregateOperator(table, existingIndex, + EXPOSED_GROUP_ROW_SETS.name(), groupPairs); + } else { + groupByOperator = + new GroupByReaggregateOperator(table, true, EXPOSED_GROUP_ROW_SETS.name(), null, + groupPairs); + addOperator(groupByOperator, groupRowSet, EXPOSED_GROUP_ROW_SETS.name()); + } + final FormulaMultiColumnChunkedOperator op = new FormulaMultiColumnChunkedOperator(table, groupByOperator, false, selectColumn, inputKeyColumns, renames, depthSource); diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java index 95386513367..5d719a33e58 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java @@ -30,10 +30,7 @@ import org.junit.Test; import org.junit.experimental.categories.Category; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.Random; +import java.util.*; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; @@ -375,16 +372,27 @@ public void testRollupGroupStatic() { @Test public void testRollupFormulaStatic() { + testRollupFormulaStatic(true); + testRollupFormulaStatic(false); + } + + private void testRollupFormulaStatic(boolean withGroup) { final Table source = TableTools.newTable( stringCol("Key1", "Alpha", "Bravo", "Alpha", "Charlie", "Charlie", "Bravo", "Bravo"), stringCol("Key2", "Delta", "Delta", "Echo", "Echo", "Echo", "Echo", "Echo"), intCol("Sentinel", 1, 2, 3, 4, 5, 6, 7)); TableTools.show(source); + final List aggList = new ArrayList<>(); + if (withGroup) { + aggList.add(AggGroup("Sentinel")); + } + aggList.add(AggSum("Sum=Sentinel")); + aggList.add(AggFormula("FSum", "__FORMULA_DEPTH__ == 0 ? max(Sentinel) : 1 + sum(Sentinel)")); + final RollupTable rollup1 = source.rollup( - List.of(AggSum("Sum=Sentinel"), - AggFormula("FSum", "__FORMULA_DEPTH__ == 0 ? max(Sentinel) : 1 + sum(Sentinel)")), + aggList, "Key1", "Key2"); final String[] arrayWithNull = new String[1]; @@ -402,8 +410,9 @@ public void testRollupFormulaStatic() { TableTools.show(snapshot.view(rollup1.getRowDepthColumn().name(), rollup1.getRowExpandedColumn().name(), "Key1", "Key2", "Sum", "FSum")); - final Table expected = - initialExpectedGrouped(rollup1).dropColumns("Sentinel").update("FSum=ii == 0 ? 7 : 1 + Sum"); + final Table expectedBase = initialExpectedGrouped(rollup1); + final Table expectedSentinel = withGroup ? expectedBase : expectedBase.dropColumns("Sentinel"); + final Table expected = expectedSentinel.update("FSum=ii == 0 ? 7 : 1 + Sum"); assertTableEquals(expected, snapshot.dropColumns("__EXPOSED_GROUP_ROW_SETS__")); freeSnapshotTableChunks(snapshot); } @@ -442,6 +451,11 @@ public void testRollupFormulaStatic2() { @Test public void testRollupFormulaStatic3() { + testRollupFormulaStatic3(true); + testRollupFormulaStatic3(false); + } + + private void testRollupFormulaStatic3(boolean hasGroup) { final Table source = TableTools.newTable( stringCol("Account", "Aardvark", "Aardvark", "Aardvark", "Aardvark", "Badger", "Badger", "Badger", "Cobra", "Cobra", "Cobra", "Cobra"), @@ -450,10 +464,17 @@ public void testRollupFormulaStatic3() { longCol("qty", 500, 100, 500, 200, 300, 300, 200, 100, 200, 300, 1500)); TableTools.show(source); + final List aggList = new ArrayList<>(); + + if (hasGroup) { + aggList.add(AggGroup("gqty=qty")); + } + aggList.add(AggFormula("qty", "__FORMULA_DEPTH__ == 2 ? min(1000, sum(qty)) : sum(qty)").asReggregating()); + aggList.add(AggSum("sqty=qty")); + final RollupTable rollup1 = source.rollup( - List.of(AggFormula("qty", "__FORMULA_DEPTH__ == 2 ? min(1000, sum(qty)) : sum(qty)") - .asReggregating(), AggSum("sqty=qty")), + aggList, "Account", "Sym"); final RollupTable rollup2 = rollup1.withNodeOperations( From bbe40427c69852a4725c975bea2bb53846f3dda8 Mon Sep 17 00:00:00 2001 From: "Charles P. Wright" Date: Wed, 31 Dec 2025 09:17:30 -0500 Subject: [PATCH 14/24] Update engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByOperator.java Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com> --- .../io/deephaven/engine/table/impl/by/GroupByOperator.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByOperator.java index 3a0e688c7a3..df900929f68 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByOperator.java @@ -14,10 +14,10 @@ public interface GroupByOperator extends IterativeChunkedAggregationOperator { Map> getInputResultColumns(); /** - * Given that there have been modified input columns, should we propagate changes? - * + * Determine whether to propagate changes when input columns have been modified. + * * @param columnsModified have any of the input columns been modified (as per the MCS)? - * @return true if we have modified our output (e.g., because of additions or modifications) + * @return true if we have modified our output (e.g., because of additions or modifications). */ boolean hasModifications(final boolean columnsModified); } From 41739fd341517a511eb652f12b4fe9e296e44d1b Mon Sep 17 00:00:00 2001 From: "Charles P. Wright" Date: Wed, 31 Dec 2025 09:19:05 -0500 Subject: [PATCH 15/24] Update engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com> --- .../engine/table/impl/by/GroupByReaggregateOperator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java index c7dd98fe32d..8e6b877cd93 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java @@ -88,7 +88,7 @@ public GroupByReaggregateOperator( resultAggregatedColumns = new LinkedHashMap<>(aggregatedColumnPairs.length); final List inputColumnNamesForResultsList = new ArrayList<>(); Arrays.stream(aggregatedColumnPairs).forEach(pair -> { - // we are reaggregationg so have to use the left column for everything + // we are reaggregating so have to use the left column for everything final ColumnSource source = inputTable.getColumnSource(pair.leftColumn()); if (!(source instanceof AggregateColumnSource)) { throw new IllegalStateException("Expect to reaggregate AggregateColumnSources for a group operation."); From be21002bf51c9ab4a2982fc9ed61983312664e03 Mon Sep 17 00:00:00 2001 From: "Charles P. Wright" Date: Wed, 31 Dec 2025 10:58:27 -0500 Subject: [PATCH 16/24] demonstrate broken modifications in unit test. --- .../table/impl/by/AggregationProcessor.java | 5 +- .../table/impl/by/GroupByChunkedOperator.java | 3 +- .../impl/by/GroupByReaggregateOperator.java | 11 +- .../engine/table/impl/TestAggGroup.java | 110 ++++++++++++++++++ .../engine/table/impl/TestRollupTable.java | 5 +- .../java/io/deephaven/api/agg/Formula.java | 2 +- 6 files changed, 124 insertions(+), 12 deletions(-) create mode 100644 engine/table/src/test/java/io/deephaven/engine/table/impl/TestAggGroup.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index 763f829887b..a5fcb409eb3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -1148,15 +1148,12 @@ private void maybeInitializeVectorColumns(Set groupByColumnSet, final Ta private @NotNull GroupByChunkedOperator makeGroupByOperatorForFormula(String[] inputNonKeyColumns, final QueryTable table, final String exposedRowsets) { final MatchPair[] pairs; - final List hiddenResults; final boolean register; if (exposedRowsets == null) { - hiddenResults = null; register = false; pairs = Arrays.stream(inputNonKeyColumns).map(col -> MatchPair.of(Pair.parse(col))) .toArray(MatchPair[]::new); } else { - hiddenResults = null;// Arrays.stream(pairs).map(mp -> mp.output().name()).collect(Collectors.toList()); register = true; pairs = Arrays .stream(inputNonKeyColumns).map(col -> MatchPair.of( @@ -1165,7 +1162,7 @@ private void maybeInitializeVectorColumns(Set groupByColumnSet, final Ta ColumnName.of(col + ROLLUP_GRP_COLUMN_ID + ROLLUP_COLUMN_SUFFIX)))) .toArray(MatchPair[]::new); } - return new GroupByChunkedOperator(table, register, exposedRowsets, hiddenResults, pairs); + return new GroupByChunkedOperator(table, register, exposedRowsets, null, pairs); } // ----------------------------------------------------------------------------------------------------------------- diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java index 4baf3745c51..744817e743c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java @@ -43,7 +43,6 @@ public final class GroupByChunkedOperator implements GroupByOperator { private final ObjectArraySource addedBuilders; private final ObjectArraySource removedBuilders; - private final String[] inputColumnNames; private final Map> inputAggregatedColumns; private final String[] inputColumnNamesForResults; private final Map> resultAggregatedColumns; @@ -104,7 +103,7 @@ public GroupByChunkedOperator( "Exposing group RowSets as %s, but this conflicts with a requested grouped output column name", exposeRowSetsAs)); } - inputColumnNames = MatchPair.getRightColumns(aggregatedColumnPairs); + final String[] inputColumnNames = MatchPair.getRightColumns(aggregatedColumnPairs); if (live) { aggregationInputsModifiedColumnSet = inputTable.newModifiedColumnSet(inputColumnNames); removedBuilders = new ObjectArraySource<>(Object.class); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java index 8e6b877cd93..e1db514ce22 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java @@ -61,6 +61,7 @@ public final class GroupByReaggregateOperator implements GroupByOperator { private final Map> resultAggregatedColumns; private RowSetBuilderRandom stepDestinationsModified; + private boolean rowsetsModified = false; private boolean initialized; @@ -326,8 +327,7 @@ public UnaryOperator initializeRefreshing( @Override public boolean hasModifications(boolean columnsModified) { - /* TODO: FIX THIS. */ - return true; + return columnsModified || rowsetsModified; } private class InputToResultModifiedColumnSetFactory implements UnaryOperator { @@ -362,6 +362,7 @@ public ModifiedColumnSet apply(@NotNull final ModifiedColumnSet upstreamModified @Override public void resetForStep(@NotNull final TableUpdate upstream, final int startingDestinationsCount) { stepDestinationsModified = new BitmapRandomBuilder(startingDestinationsCount); + rowsetsModified = false; } @Override @@ -455,6 +456,9 @@ public void propagateUpdates(@NotNull final TableUpdate downstream, @NotNull fin // use the addRowSet as the new rowset final WritableRowSet addRowSet = nullToEmpty( extractAndClearBuilderRandom(addedBuildersBackingChunk, backingChunkOffset)); + if (!addRowSet.isEmpty()) { + rowsetsModified = true; + } rowSetBackingChunk.set(backingChunkOffset, live ? addRowSet.toTracking() : addRowSet); } else { try (final WritableRowSet addRowSet = @@ -465,6 +469,9 @@ public void propagateUpdates(@NotNull final TableUpdate downstream, @NotNull fin backingChunkOffset))) { workingRowSet.remove(removeRowSet); workingRowSet.insert(addRowSet); + if (!addRowSet.isEmpty() || !removeRowSet.isEmpty()) { + rowsetsModified = true; + } } } }); diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestAggGroup.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestAggGroup.java new file mode 100644 index 00000000000..6cba8562538 --- /dev/null +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestAggGroup.java @@ -0,0 +1,110 @@ +// +// Copyright (c) 2016-2025 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl; + +import io.deephaven.api.ColumnName; +import io.deephaven.api.agg.Aggregation; +import io.deephaven.engine.context.ExecutionContext; +import io.deephaven.engine.rowset.RowSetShiftData; +import io.deephaven.engine.table.ColumnDefinition; +import io.deephaven.engine.table.ModifiedColumnSet; +import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.impl.by.AggregationProcessor; +import io.deephaven.engine.testutil.ControlledUpdateGraph; +import io.deephaven.engine.testutil.TstUtils; +import io.deephaven.engine.testutil.testcase.RefreshingTableTestCase; +import io.deephaven.engine.util.TableTools; +import io.deephaven.test.types.OutOfBandTest; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import java.util.*; + +import static io.deephaven.api.agg.Aggregation.*; +import static io.deephaven.engine.table.impl.by.RollupConstants.ROLLUP_COLUMN_SUFFIX; +import static io.deephaven.engine.testutil.TstUtils.assertTableEquals; +import static io.deephaven.engine.testutil.TstUtils.i; +import static io.deephaven.engine.util.TableTools.*; + +@Category(OutOfBandTest.class) +public class TestAggGroup extends RefreshingTableTestCase { + @Test + public void testGroupModifications() { + final QueryTable source = TstUtils.testRefreshingTable( + stringCol("Key1", "Alpha", "Bravo", "Alpha", "Charlie", "Charlie", "Bravo", "Bravo"), + stringCol("Key2", "Delta", "Delta", "Echo", "Echo", "Echo", "Echo", "Echo"), + intCol("Sentinel", 1, 2, 3, 4, 5, 6, 7), + intCol("Sentinel2", 101, 102, 103, 104, 105, 106, 107)); + + final List aggs = + List.of(AggGroup("Sentinel"), AggSum("Sum=Sentinel"), AggGroup("Sentinel2"), AggSum("Sum2=Sentinel2")); + + final QueryTable normal = source.aggNoMemo(AggregationProcessor.forAggregation(aggs), false, null, + List.of(ColumnName.of("Key1"))); + final ColumnName rollupColumn = ColumnName.of(ROLLUP_COLUMN_SUFFIX); + final QueryTable base = source.aggNoMemo(AggregationProcessor.forRollupBase(aggs, false, rollupColumn), false, + null, List.of(ColumnName.of("Key1"), ColumnName.of("Key2"))); + final QueryTable reaggregated = base.aggNoMemo(AggregationProcessor.forRollupReaggregated(aggs, + List.of(ColumnDefinition.ofString("Key2")), rollupColumn, source), false, null, + List.of(ColumnName.of("Key1"))); + + TableTools.show(normal); + TableTools.show(base); + TableTools.show(reaggregated); + + doCheck(normal, base, reaggregated); + + final SimpleListener normalListener = new SimpleListener(normal); + normal.addUpdateListener(normalListener); + final SimpleListener baseListener = new SimpleListener(base); + base.addUpdateListener(baseListener); + final SimpleListener reaggListener = new SimpleListener(reaggregated); + reaggregated.addUpdateListener(reaggListener); + + final ControlledUpdateGraph cug = ExecutionContext.getContext().getUpdateGraph().cast(); + // modify the value of a Sentinel; check the updates + cug.runWithinUnitTestCycle(() -> { + TstUtils.addToTable(source, i(0), stringCol("Key1", "Alpha"), stringCol("Key2", "Delta"), + intCol("Sentinel", 8), intCol("Sentinel2", 101)); + final ModifiedColumnSet mcs = source.getModifiedColumnSetForUpdates(); + mcs.clear(); + mcs.setAll("Sentinel"); + source.notifyListeners(new TableUpdateImpl(i(), i(), i(0), RowSetShiftData.EMPTY, mcs)); + }); + + TableTools.show(normal); + TableTools.show(base); + TableTools.show(reaggregated); + + // make sure the aggregation is still consistent + doCheck(normal, base, reaggregated); + + // we should have gotten an update from each of our listeners + checkModified(normalListener, normal, "Sentinel", "Sentinel2"); + checkModified(baseListener, base, "Sentinel", "Sentinel2"); + checkModified(reaggListener, reaggregated, "Sentinel", "Sentinel2"); + } + + private static void checkModified(SimpleListener listener, QueryTable table, final String modColumn, + final String noModColumn) { + System.out.println("update = " + listener.update); + assertEquals(1, listener.count); + assertTrue(listener.update.added().isEmpty()); + assertTrue(listener.update.removed().isEmpty()); + assertEquals(1, listener.update.modified().size()); + assertTrue(listener.update.modifiedColumnSet().containsAll(table.newModifiedColumnSet(modColumn))); + assertFalse(listener.update.modifiedColumnSet().containsAny(table.newModifiedColumnSet(noModColumn))); + } + + private static void doCheck(Table normal, QueryTable base, QueryTable reaggregated) { + assertEquals(0, normal.update("CheckSum=sum(Sentinel)", "CheckSum2=sum(Sentinel2)") + .where("Sum != CheckSum || Sum2 != CheckSum2").size()); + assertEquals(0, base.update("CheckSum=sum(Sentinel)", "CheckSum2=sum(Sentinel2)") + .where("Sum != CheckSum || Sum2 != CheckSum2").size()); + assertEquals(0, reaggregated.update("CheckSum=sum(Sentinel)", "CheckSum2=sum(Sentinel2)") + .where("Sum != CheckSum || Sum2 != CheckSum2").size()); + assertTableEquals(normal.view("Key1", "Sentinel", "Sum", "Sentinel2", "Sum2"), + reaggregated.view("Key1", "Sentinel", "Sum", "Sentinel2", "Sum2")); + } +} diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java index 5d719a33e58..4b31d916d2a 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java @@ -428,7 +428,7 @@ public void testRollupFormulaStatic2() { final RollupTable rollup1 = source.updateView("qty=(long)qty").rollup( - List.of(AggFormula("qty", "__FORMULA_DEPTH__ > 0 ? first(qty) : sum(qty)").asReggregating(), + List.of(AggFormula("qty", "__FORMULA_DEPTH__ > 0 ? first(qty) : sum(qty)").asReaggregating(), AggSum("Dollars")), "Account", "Sym"); @@ -469,7 +469,7 @@ private void testRollupFormulaStatic3(boolean hasGroup) { if (hasGroup) { aggList.add(AggGroup("gqty=qty")); } - aggList.add(AggFormula("qty", "__FORMULA_DEPTH__ == 2 ? min(1000, sum(qty)) : sum(qty)").asReggregating()); + aggList.add(AggFormula("qty", "__FORMULA_DEPTH__ == 2 ? min(1000, sum(qty)) : sum(qty)").asReaggregating()); aggList.add(AggSum("sqty=qty")); final RollupTable rollup1 = @@ -610,7 +610,6 @@ public void testReusedGrouping() { TableTools.showWithRowSet(expected2); assertTableEquals(expected2, snapshot2.dropColumns("__EXPOSED_GROUP_ROW_SETS__")); freeSnapshotTableChunks(snapshot2); - // TODO: modify only one column, validate that we get results that we expect without excess modifications } } diff --git a/table-api/src/main/java/io/deephaven/api/agg/Formula.java b/table-api/src/main/java/io/deephaven/api/agg/Formula.java index 28c7dc15471..682f39799ab 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/Formula.java +++ b/table-api/src/main/java/io/deephaven/api/agg/Formula.java @@ -36,7 +36,7 @@ public static Formula of(Selectable selectable) { return ImmutableFormula.of(selectable, false); } - public Formula asReggregating() { + public Formula asReaggregating() { return ImmutableFormula.of(selectable(), true); } From 307a803705b1bb22ea908a532fcfd6902fa58d8a Mon Sep 17 00:00:00 2001 From: "Charles P. Wright" Date: Wed, 31 Dec 2025 11:24:07 -0500 Subject: [PATCH 17/24] fill in tests. --- .../engine/table/impl/TestRollupTable.java | 43 ++++++++++++++++--- 1 file changed, 36 insertions(+), 7 deletions(-) diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java index 4b31d916d2a..b4aa1db0fea 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java @@ -13,6 +13,7 @@ import io.deephaven.engine.table.Table; import io.deephaven.engine.table.hierarchical.HierarchicalTable; import io.deephaven.engine.table.hierarchical.RollupTable; +import io.deephaven.engine.table.impl.util.ColumnHolder; import io.deephaven.engine.testutil.ColumnInfo; import io.deephaven.engine.testutil.ControlledUpdateGraph; import io.deephaven.engine.testutil.EvalNuggetInterface; @@ -25,6 +26,8 @@ import io.deephaven.test.types.OutOfBandTest; import io.deephaven.vector.IntVector; import io.deephaven.vector.IntVectorDirect; +import io.deephaven.vector.LongVector; +import io.deephaven.vector.LongVectorDirect; import org.jspecify.annotations.NonNull; import org.junit.Assert; import org.junit.Test; @@ -424,7 +427,6 @@ public void testRollupFormulaStatic2() { stringCol("Sym", "leg1", "leg2", "leg1", "leg2"), intCol("qty", 100, 100, 200, 200), doubleCol("Dollars", 1000, -500, 2000, -1000)); - TableTools.show(source); final RollupTable rollup1 = source.updateView("qty=(long)qty").rollup( @@ -442,10 +444,15 @@ public void testRollupFormulaStatic2() { final HierarchicalTable.SnapshotState ss1 = rollup1.makeSnapshotState(); final Table snapshot = snapshotToTable(rollup1, ss1, keyTable, ColumnName.of("Action"), null, RowSetFactory.flat(30)); - TableTools.showWithRowSet(snapshot); - // final Table expected = initialExpectedGrouped(rollup1).update("FSum=ii == 0 ? 7 : 1 + Sum"); - // assertTableEquals(expected, snapshot); + final Table expected = TableTools.newTable(intCol(rollup1.getRowDepthColumn().name(), 1, 2, 3, 3, 2, 3, 3), + booleanCol(rollup1.getRowExpandedColumn().name(), true, true, null, null, true, null, null), + col("Account", null, "acct1", "acct1", "acct1", "acct2", "acct2", "acct2"), + col("Sym", null, null, "leg1", "leg2", null, "leg1", "leg2"), + longCol("qty", 300, 100, 100, 100, 200, 200, 200), + doubleCol("Dollars", 1500, 500, 1000, -500, 1000, 2000, -1000)); + + assertTableEquals(expected, snapshot); freeSnapshotTableChunks(snapshot); } @@ -492,8 +499,27 @@ private void testRollupFormulaStatic3(boolean hasGroup) { snapshotToTable(rollup2, ss1, keyTable, ColumnName.of("Action"), null, RowSetFactory.flat(30)); TableTools.showWithRowSet(snapshot); - // final Table expected = initialExpectedGrouped(rollup1).update("FSum=ii == 0 ? 7 : 1 + Sum"); - // assertTableEquals(expected, snapshot); + final List> columnHolders = new ArrayList<>(); + columnHolders.add(intCol(rollup1.getRowDepthColumn().name(), 1, 2, 3, 3, 2, 3, 2, 3, 3)); + columnHolders.add(booleanCol(rollup1.getRowExpandedColumn().name(), true, true, null, null, true, null, true, + null, null)); + columnHolders.add(stringCol("Account", null, "Aardvark", "Aardvark", "Aardvark", "Badger", "Badger", "Cobra", + "Cobra", "Cobra")); + columnHolders + .add(stringCol("Sym", null, null, "Apple", "Banana", null, "Carrot", null, "Apple", "Dragonfruit")); + columnHolders.add(col("gqty", lv(500, 100, 500, 200, 300, 300, 200, 100, 200, 300, 1500), + /* aardvark */ lv(500, 100, 500, 200), lv(500, 500, 200), lv(100), /* badger */lv(300, 300, 200), + lv(300, 300, 200), /* cobra */ lv(100, 200, 300, 1500), lv(100, 200, 300), lv(1500))); + columnHolders.add(longCol("qty", 3500, /* aardvark */ 1100, 1000, 100, /* badger */800, 800, /* cobra */ 1600, + 600, 1000)); + final Table expected = TableTools.newTable(columnHolders.toArray(ColumnHolder[]::new)) + .update("sqty = sum(gqty)", "SumDiff=sqty-qty"); + + TableTools.show(expected); + + assertTableEquals(hasGroup ? expected : expected.dropColumns("gqty"), + hasGroup ? snapshot.dropColumns("__EXPOSED_GROUP_ROW_SETS__") : snapshot); + freeSnapshotTableChunks(snapshot); } @@ -523,6 +549,10 @@ private static Table secondExpectedGrouped(RollupTable rollup1) { return new IntVectorDirect(ints); } + private static @NonNull LongVector lv(final long... ints) { + return new LongVectorDirect(ints); + } + @Test public void testRollupGroupIncremental() { final QueryTable source = TstUtils.testRefreshingTable( @@ -610,6 +640,5 @@ public void testReusedGrouping() { TableTools.showWithRowSet(expected2); assertTableEquals(expected2, snapshot2.dropColumns("__EXPOSED_GROUP_ROW_SETS__")); freeSnapshotTableChunks(snapshot2); - // TODO: modify only one column, validate that we get results that we expect without excess modifications } } From c42360e660520887e08789ac62f85651143229ce Mon Sep 17 00:00:00 2001 From: "Charles P. Wright" Date: Mon, 12 Jan 2026 11:17:59 -0500 Subject: [PATCH 18/24] Modification fix. --- .../table/impl/by/AggregationContext.java | 8 ++++--- .../BaseBlinkFirstOrLastChunkedOperator.java | 3 ++- ...BlinkSortedFirstOrLastChunkedOperator.java | 5 +++-- ...BlinkSortedFirstOrLastChunkedOperator.java | 5 +++-- .../by/ChunkedOperatorAggregationHelper.java | 7 ++++--- ...BlinkSortedFirstOrLastChunkedOperator.java | 5 +++-- ...BlinkSortedFirstOrLastChunkedOperator.java | 5 +++-- .../table/impl/by/FormulaChunkedOperator.java | 3 ++- .../by/FormulaMultiColumnChunkedOperator.java | 3 ++- .../table/impl/by/GroupByChunkedOperator.java | 3 ++- .../impl/by/GroupByReaggregateOperator.java | 7 ++++++- ...BlinkSortedFirstOrLastChunkedOperator.java | 5 +++-- .../IterativeChunkedAggregationOperator.java | 8 +++++-- ...BlinkSortedFirstOrLastChunkedOperator.java | 5 +++-- ...BlinkSortedFirstOrLastChunkedOperator.java | 5 +++-- .../impl/by/PartitionByChunkedOperator.java | 3 ++- ...BlinkSortedFirstOrLastChunkedOperator.java | 5 +++-- .../impl/by/TDigestPercentileOperator.java | 3 ++- .../engine/table/impl/TestAggGroup.java | 21 ++++++++++++++++--- .../table/stats/ObjectChunkedStats.java | 17 +++++---------- 20 files changed, 80 insertions(+), 46 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationContext.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationContext.java index fd9ee394974..02f5958f67a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationContext.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationContext.java @@ -270,10 +270,12 @@ UnaryOperator[] initializeRefreshing(@NotNull final QueryTabl * * @param upstream The upstream {@link TableUpdateImpl} * @param startingDestinationsCount The number of used destinations at the beginning of this step + * @param modifiedOperators an array of booleans, parallel to operators, indicating which operators were modified */ - void resetOperatorsForStep(@NotNull final TableUpdate upstream, final int startingDestinationsCount) { - for (final IterativeChunkedAggregationOperator operator : operators) { - operator.resetForStep(upstream, startingDestinationsCount); + void resetOperatorsForStep(@NotNull final TableUpdate upstream, final int startingDestinationsCount, + final boolean[] modifiedOperators) { + for (int ii = 0; ii < operators.length; ii++) { + modifiedOperators[ii] = operators[ii].resetForStep(upstream, startingDestinationsCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/BaseBlinkFirstOrLastChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/BaseBlinkFirstOrLastChunkedOperator.java index 9b5cee1f077..0e505e3c901 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/BaseBlinkFirstOrLastChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/BaseBlinkFirstOrLastChunkedOperator.java @@ -98,11 +98,12 @@ public final boolean requiresRowKeys() { @Override @OverridingMethodsMustInvokeSuper - public void resetForStep(@NotNull final TableUpdate upstream, final int startingDestinationsCount) { + public boolean resetForStep(@NotNull final TableUpdate upstream, final int startingDestinationsCount) { if ((redirections = cachedRedirections.get()) == null) { cachedRedirections = new SoftReference<>(redirections = new LongArraySource()); ensureCapacity(startingDestinationsCount); } + return false; } // ----------------------------------------------------------------------------------------------------------------- diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ByteBlinkSortedFirstOrLastChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ByteBlinkSortedFirstOrLastChunkedOperator.java index d4520da86f6..1bdd13a7bde 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ByteBlinkSortedFirstOrLastChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ByteBlinkSortedFirstOrLastChunkedOperator.java @@ -57,11 +57,12 @@ public void ensureCapacity(final long tableSize) { } @Override - public void resetForStep(@NotNull final TableUpdate upstream, final int startingDestinationsCount) { - super.resetForStep(upstream, startingDestinationsCount); + public boolean resetForStep(@NotNull final TableUpdate upstream, final int startingDestinationsCount) { + final boolean modified = super.resetForStep(upstream, startingDestinationsCount); if (isCombo) { changedDestinationsBuilder = RowSetFactory.builderRandom(); } + return modified; } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/CharBlinkSortedFirstOrLastChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/CharBlinkSortedFirstOrLastChunkedOperator.java index 092f753192a..13268ffa445 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/CharBlinkSortedFirstOrLastChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/CharBlinkSortedFirstOrLastChunkedOperator.java @@ -53,11 +53,12 @@ public void ensureCapacity(final long tableSize) { } @Override - public void resetForStep(@NotNull final TableUpdate upstream, final int startingDestinationsCount) { - super.resetForStep(upstream, startingDestinationsCount); + public boolean resetForStep(@NotNull final TableUpdate upstream, final int startingDestinationsCount) { + final boolean modified = super.resetForStep(upstream, startingDestinationsCount); if (isCombo) { changedDestinationsBuilder = RowSetFactory.builderRandom(); } + return modified; } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ChunkedOperatorAggregationHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ChunkedOperatorAggregationHelper.java index 234b3be7ef9..b876e1347ea 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ChunkedOperatorAggregationHelper.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ChunkedOperatorAggregationHelper.java @@ -537,7 +537,7 @@ private TableUpdate computeDownstreamIndicesAndCopyKeys( @NotNull final ModifiedColumnSet resultModifiedColumnSet, @NotNull final UnaryOperator[] resultModifiedColumnSetFactories) { final int firstStateToAdd = outputPosition.get(); - ac.resetOperatorsForStep(upstream, firstStateToAdd); + ac.resetOperatorsForStep(upstream, firstStateToAdd, modifiedOperators); if (upstream.removed().isNonempty()) { doRemoves(upstream.removed()); @@ -2085,7 +2085,9 @@ public void onUpdate(@NotNull final TableUpdate upstream) { } private void processNoKeyUpdate(@NotNull final TableUpdate upstream) { - ac.resetOperatorsForStep(upstream, 1); + final boolean[] modifiedOperators = new boolean[ac.size()]; + + ac.resetOperatorsForStep(upstream, 1, modifiedOperators); final ModifiedColumnSet upstreamModifiedColumnSet = upstream.modified().isEmpty() ? ModifiedColumnSet.EMPTY @@ -2099,7 +2101,6 @@ private void processNoKeyUpdate(@NotNull final TableUpdate upstream) { ac.initializeSingletonContexts(opContexts, upstream, od.operatorsWithModifiedInputColumns); - final boolean[] modifiedOperators = new boolean[ac.size()]; // remove all the removals if (upstream.removed().isNonempty()) { doNoKeyRemoval(upstream.removed(), ac, opContexts, allColumns, modifiedOperators); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/DoubleBlinkSortedFirstOrLastChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/DoubleBlinkSortedFirstOrLastChunkedOperator.java index 14822045a1b..bff986e9440 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/DoubleBlinkSortedFirstOrLastChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/DoubleBlinkSortedFirstOrLastChunkedOperator.java @@ -57,11 +57,12 @@ public void ensureCapacity(final long tableSize) { } @Override - public void resetForStep(@NotNull final TableUpdate upstream, final int startingDestinationsCount) { - super.resetForStep(upstream, startingDestinationsCount); + public boolean resetForStep(@NotNull final TableUpdate upstream, final int startingDestinationsCount) { + final boolean modified = super.resetForStep(upstream, startingDestinationsCount); if (isCombo) { changedDestinationsBuilder = RowSetFactory.builderRandom(); } + return modified; } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FloatBlinkSortedFirstOrLastChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FloatBlinkSortedFirstOrLastChunkedOperator.java index e5a8bc53e42..7bb38240ca2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FloatBlinkSortedFirstOrLastChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FloatBlinkSortedFirstOrLastChunkedOperator.java @@ -57,11 +57,12 @@ public void ensureCapacity(final long tableSize) { } @Override - public void resetForStep(@NotNull final TableUpdate upstream, final int startingDestinationsCount) { - super.resetForStep(upstream, startingDestinationsCount); + public boolean resetForStep(@NotNull final TableUpdate upstream, final int startingDestinationsCount) { + final boolean modified = super.resetForStep(upstream, startingDestinationsCount); if (isCombo) { changedDestinationsBuilder = RowSetFactory.builderRandom(); } + return modified; } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaChunkedOperator.java index b2b9464d81d..fe9bb782d03 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaChunkedOperator.java @@ -301,12 +301,13 @@ public UnaryOperator initializeRefreshing(@NotNull final Quer } @Override - public void resetForStep(@NotNull final TableUpdate upstream, final int startingDestinationsCount) { + public boolean resetForStep(@NotNull final TableUpdate upstream, final int startingDestinationsCount) { if (delegateToBy) { groupBy.resetForStep(upstream, startingDestinationsCount); } updateUpstreamModifiedColumnSet = upstream.modified().isEmpty() ? ModifiedColumnSet.EMPTY : upstream.modifiedColumnSet(); + return false; } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaMultiColumnChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaMultiColumnChunkedOperator.java index e327d3794dc..a1543c88b69 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaMultiColumnChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaMultiColumnChunkedOperator.java @@ -285,12 +285,13 @@ public UnaryOperator initializeRefreshing(@NotNull final Quer } @Override - public void resetForStep(@NotNull final TableUpdate upstream, final int startingDestinationsCount) { + public boolean resetForStep(@NotNull final TableUpdate upstream, final int startingDestinationsCount) { if (delegateToBy) { groupBy.resetForStep(upstream, startingDestinationsCount); } updateUpstreamModifiedColumnSet = upstream.modified().isEmpty() ? ModifiedColumnSet.EMPTY : upstream.modifiedColumnSet(); + return false; } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java index 744817e743c..c26a9d5c206 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java @@ -491,12 +491,13 @@ public ModifiedColumnSet apply(@NotNull final ModifiedColumnSet upstreamModified } @Override - public void resetForStep(@NotNull final TableUpdate upstream, final int startingDestinationsCount) { + public boolean resetForStep(@NotNull final TableUpdate upstream, final int startingDestinationsCount) { stepValuesModified = upstream.modified().isNonempty() && upstream.modifiedColumnSet().nonempty() && upstream.modifiedColumnSet().containsAny(aggregationInputsModifiedColumnSet); someKeyHasAddsOrRemoves = false; someKeyHasModifies = false; stepDestinationsModified = new BitmapRandomBuilder(startingDestinationsCount); + return false; } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java index e1db514ce22..940bad9092e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java @@ -56,6 +56,7 @@ public final class GroupByReaggregateOperator implements GroupByOperator { private final ObjectArraySource removedBuilders; private final String[] inputColumnNamesForResults; + private final ModifiedColumnSet inputAggregatedColumnsModifiedColumnSet; private final Map> inputAggregatedColumns; private final Map> resultAggregatedColumns; @@ -104,6 +105,9 @@ public GroupByReaggregateOperator( inputAggregatedColumns.put(pair.input().name(), aggregateColumnSource); }); + inputAggregatedColumnsModifiedColumnSet = + inputTable.newModifiedColumnSet(inputAggregatedColumns.keySet().toArray(String[]::new)); + if (resultAggregatedColumns.containsKey(exposeRowSetsAs)) { throw new IllegalArgumentException(String.format( "Exposing group RowSets as %s, but this conflicts with a requested grouped output column name", @@ -360,9 +364,10 @@ public ModifiedColumnSet apply(@NotNull final ModifiedColumnSet upstreamModified } @Override - public void resetForStep(@NotNull final TableUpdate upstream, final int startingDestinationsCount) { + public boolean resetForStep(@NotNull final TableUpdate upstream, final int startingDestinationsCount) { stepDestinationsModified = new BitmapRandomBuilder(startingDestinationsCount); rowsetsModified = false; + return upstream.modifiedColumnSet().containsAny(inputAggregatedColumnsModifiedColumnSet); } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IntBlinkSortedFirstOrLastChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IntBlinkSortedFirstOrLastChunkedOperator.java index b1c748aafaa..55b80c3724b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IntBlinkSortedFirstOrLastChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IntBlinkSortedFirstOrLastChunkedOperator.java @@ -57,11 +57,12 @@ public void ensureCapacity(final long tableSize) { } @Override - public void resetForStep(@NotNull final TableUpdate upstream, final int startingDestinationsCount) { - super.resetForStep(upstream, startingDestinationsCount); + public boolean resetForStep(@NotNull final TableUpdate upstream, final int startingDestinationsCount) { + final boolean modified = super.resetForStep(upstream, startingDestinationsCount); if (isCombo) { changedDestinationsBuilder = RowSetFactory.builderRandom(); } + return modified; } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IterativeChunkedAggregationOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IterativeChunkedAggregationOperator.java index 40881244dab..9630178e9a1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IterativeChunkedAggregationOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IterativeChunkedAggregationOperator.java @@ -301,11 +301,15 @@ default UnaryOperator initializeRefreshing(@NotNull final Que /** * Reset any per-step internal state. Note that the arguments to this method should not be mutated in any way. - * + * * @param upstream The upstream ShiftAwareListener.Update * @param startingDestinationsCount The number of used destinations at the beginning of this step + * @return true if this operator must generate modifications on this cycle; typically an operator returns false and + * depends on the actual add/remove/modify/shift calls to determine modifications */ - default void resetForStep(@NotNull TableUpdate upstream, int startingDestinationsCount) {} + default boolean resetForStep(@NotNull TableUpdate upstream, int startingDestinationsCount) { + return false; + } /** * Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/LongBlinkSortedFirstOrLastChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/LongBlinkSortedFirstOrLastChunkedOperator.java index 1e4fb6f0503..2c13e8eeda7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/LongBlinkSortedFirstOrLastChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/LongBlinkSortedFirstOrLastChunkedOperator.java @@ -57,11 +57,12 @@ public void ensureCapacity(final long tableSize) { } @Override - public void resetForStep(@NotNull final TableUpdate upstream, final int startingDestinationsCount) { - super.resetForStep(upstream, startingDestinationsCount); + public boolean resetForStep(@NotNull final TableUpdate upstream, final int startingDestinationsCount) { + final boolean modified = super.resetForStep(upstream, startingDestinationsCount); if (isCombo) { changedDestinationsBuilder = RowSetFactory.builderRandom(); } + return modified; } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ObjectBlinkSortedFirstOrLastChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ObjectBlinkSortedFirstOrLastChunkedOperator.java index 96ce7fc5e0e..708c5d9c8e8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ObjectBlinkSortedFirstOrLastChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ObjectBlinkSortedFirstOrLastChunkedOperator.java @@ -57,11 +57,12 @@ public void ensureCapacity(final long tableSize) { } @Override - public void resetForStep(@NotNull final TableUpdate upstream, final int startingDestinationsCount) { - super.resetForStep(upstream, startingDestinationsCount); + public boolean resetForStep(@NotNull final TableUpdate upstream, final int startingDestinationsCount) { + final boolean modified = super.resetForStep(upstream, startingDestinationsCount); if (isCombo) { changedDestinationsBuilder = RowSetFactory.builderRandom(); } + return modified; } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/PartitionByChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/PartitionByChunkedOperator.java index 28f34bfde8a..2166bd527f7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/PartitionByChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/PartitionByChunkedOperator.java @@ -618,7 +618,7 @@ private void linkTableReferences(@NotNull final Table subTable) { } @Override - public void resetForStep(@NotNull final TableUpdate upstream, int startingDestinationsCount) { + public boolean resetForStep(@NotNull final TableUpdate upstream, int startingDestinationsCount) { stepUpdatedDestinations = RowSetFactory.empty(); final boolean upstreamModified = upstream.modified().isNonempty() && upstream.modifiedColumnSet().nonempty(); if (upstreamModified) { @@ -628,6 +628,7 @@ public void resetForStep(@NotNull final TableUpdate upstream, int startingDestin } else { stepValuesModified = false; } + return false; } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ShortBlinkSortedFirstOrLastChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ShortBlinkSortedFirstOrLastChunkedOperator.java index 775904c204f..171ea170c1d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ShortBlinkSortedFirstOrLastChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ShortBlinkSortedFirstOrLastChunkedOperator.java @@ -57,11 +57,12 @@ public void ensureCapacity(final long tableSize) { } @Override - public void resetForStep(@NotNull final TableUpdate upstream, final int startingDestinationsCount) { - super.resetForStep(upstream, startingDestinationsCount); + public boolean resetForStep(@NotNull final TableUpdate upstream, final int startingDestinationsCount) { + final boolean modified = super.resetForStep(upstream, startingDestinationsCount); if (isCombo) { changedDestinationsBuilder = RowSetFactory.builderRandom(); } + return modified; } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/TDigestPercentileOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/TDigestPercentileOperator.java index f9cace78d97..f91e1fa6022 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/TDigestPercentileOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/TDigestPercentileOperator.java @@ -202,8 +202,9 @@ private void updateDestination(final long destination) { } @Override - public void resetForStep(@NotNull final TableUpdate upstream, final int startingDestinationsCount) { + public boolean resetForStep(@NotNull final TableUpdate upstream, final int startingDestinationsCount) { modifiedThisStep = false; + return false; } @Override diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestAggGroup.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestAggGroup.java index 6cba8562538..9f830ad623f 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestAggGroup.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestAggGroup.java @@ -42,18 +42,25 @@ public void testGroupModifications() { final QueryTable normal = source.aggNoMemo(AggregationProcessor.forAggregation(aggs), false, null, List.of(ColumnName.of("Key1"))); + final QueryTable normalNoKey = + source.aggNoMemo(AggregationProcessor.forAggregation(aggs), false, null, List.of()); final ColumnName rollupColumn = ColumnName.of(ROLLUP_COLUMN_SUFFIX); final QueryTable base = source.aggNoMemo(AggregationProcessor.forRollupBase(aggs, false, rollupColumn), false, null, List.of(ColumnName.of("Key1"), ColumnName.of("Key2"))); final QueryTable reaggregated = base.aggNoMemo(AggregationProcessor.forRollupReaggregated(aggs, List.of(ColumnDefinition.ofString("Key2")), rollupColumn, source), false, null, List.of(ColumnName.of("Key1"))); + final QueryTable reaggregated2 = reaggregated.aggNoMemo(AggregationProcessor.forRollupReaggregated(aggs, + List.of(ColumnDefinition.ofString("Key1"), ColumnDefinition.ofString("Key2")), rollupColumn, source), + false, null, + List.of()); TableTools.show(normal); TableTools.show(base); TableTools.show(reaggregated); + TableTools.show(reaggregated2); - doCheck(normal, base, reaggregated); + doCheck(normal, base, reaggregated, normalNoKey, reaggregated2); final SimpleListener normalListener = new SimpleListener(normal); normal.addUpdateListener(normalListener); @@ -61,6 +68,8 @@ public void testGroupModifications() { base.addUpdateListener(baseListener); final SimpleListener reaggListener = new SimpleListener(reaggregated); reaggregated.addUpdateListener(reaggListener); + final SimpleListener reaggListener2 = new SimpleListener(reaggregated2); + reaggregated2.addUpdateListener(reaggListener2); final ControlledUpdateGraph cug = ExecutionContext.getContext().getUpdateGraph().cast(); // modify the value of a Sentinel; check the updates @@ -78,12 +87,13 @@ public void testGroupModifications() { TableTools.show(reaggregated); // make sure the aggregation is still consistent - doCheck(normal, base, reaggregated); + doCheck(normal, base, reaggregated, normalNoKey, reaggregated2); // we should have gotten an update from each of our listeners checkModified(normalListener, normal, "Sentinel", "Sentinel2"); checkModified(baseListener, base, "Sentinel", "Sentinel2"); checkModified(reaggListener, reaggregated, "Sentinel", "Sentinel2"); + checkModified(reaggListener2, reaggregated2, "Sentinel", "Sentinel2"); } private static void checkModified(SimpleListener listener, QueryTable table, final String modColumn, @@ -97,7 +107,8 @@ private static void checkModified(SimpleListener listener, QueryTable table, fin assertFalse(listener.update.modifiedColumnSet().containsAny(table.newModifiedColumnSet(noModColumn))); } - private static void doCheck(Table normal, QueryTable base, QueryTable reaggregated) { + private static void doCheck(Table normal, QueryTable base, QueryTable reaggregated, QueryTable normalNoKey, + QueryTable reaggregated2) { assertEquals(0, normal.update("CheckSum=sum(Sentinel)", "CheckSum2=sum(Sentinel2)") .where("Sum != CheckSum || Sum2 != CheckSum2").size()); assertEquals(0, base.update("CheckSum=sum(Sentinel)", "CheckSum2=sum(Sentinel2)") @@ -106,5 +117,9 @@ private static void doCheck(Table normal, QueryTable base, QueryTable reaggregat .where("Sum != CheckSum || Sum2 != CheckSum2").size()); assertTableEquals(normal.view("Key1", "Sentinel", "Sum", "Sentinel2", "Sum2"), reaggregated.view("Key1", "Sentinel", "Sum", "Sentinel2", "Sum2")); + + assertTableEquals(normalNoKey.view("Sentinel", "Sum", "Sentinel2", "Sum2"), + reaggregated2.view("Sentinel", "Sum", "Sentinel2", "Sum2")); + } } diff --git a/server/src/main/java/io/deephaven/server/table/stats/ObjectChunkedStats.java b/server/src/main/java/io/deephaven/server/table/stats/ObjectChunkedStats.java index 22f890532d2..7eef40b8b14 100644 --- a/server/src/main/java/io/deephaven/server/table/stats/ObjectChunkedStats.java +++ b/server/src/main/java/io/deephaven/server/table/stats/ObjectChunkedStats.java @@ -19,9 +19,9 @@ import io.deephaven.engine.table.iterators.ObjectColumnIterator; import io.deephaven.engine.table.iterators.ChunkedObjectColumnIterator; import io.deephaven.engine.util.TableTools; -import io.deephaven.util.type.ArrayTypeUtils; import java.util.ArrayList; +import java.util.Comparator; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -83,17 +83,10 @@ public Table processChunks(final RowSet rowSet, final ColumnSource columnSour } List> sorted = new ArrayList<>(countValues.size()); - if (columnSource.getType().isArray()) { - countValues.forEachEntry((o, c) -> { - sorted.add(Map.entry(ArrayTypeUtils.toString(o), c)); - return true; - }); - } else { - countValues.forEachEntry((o, c) -> { - sorted.add(Map.entry(Objects.toString(o), c)); - return true; - }); - } + countValues.forEachEntry((o, c) -> { + sorted.add(Map.entry(Objects.toString(o), c)); + return true; + }); sorted.sort(Map.Entry.comparingByValue().reversed()); int resultCount = Math.min(maxUniqueToDisplay, sorted.size()); From 4b2c73c9a088ec36a947ce635f3d70427b9acc4d Mon Sep 17 00:00:00 2001 From: "Charles P. Wright" Date: Mon, 12 Jan 2026 11:42:23 -0500 Subject: [PATCH 19/24] copilot speling concerns. --- .../engine/table/impl/by/AggregationProcessor.java | 10 +++++----- .../engine/table/impl/by/GroupByChunkedOperator.java | 5 ++++- 2 files changed, 9 insertions(+), 6 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index a5fcb409eb3..c1572d89f60 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -736,7 +736,7 @@ final void addCountWhereOperator(@NotNull CountWhere countWhere) { } /** - * @return the index of an existing group by operator, or -1 if it no operator was found + * @return the index of an existing group by operator, or -1 if no operator was found */ int existingGroupByOperatorIndex() { for (int ii = 0; ii < operators.size(); ++ii) { @@ -748,9 +748,9 @@ int existingGroupByOperatorIndex() { } /** - * @return the index of an existing group by reaggregation operator, or -1 if it no operator was found + * @return the index of an existing group by reaggregation operator, or -1 if no operator was found */ - int existingGroupByReggregateIndex() { + int existingGroupByReaggregateIndex() { for (int ii = 0; ii < operators.size(); ++ii) { if (operators.get(ii) instanceof GroupByReaggregateOperator) { return ii; @@ -1550,7 +1550,7 @@ public void visit(AggSpecGroup group) { for (int ii = 0; ii < resultPairs.size(); ++ii) { pairs[ii] = new MatchPair(resultPairs.get(ii).output().name(), resultPairs.get(ii).output().name()); } - final int existingGroupByOperatorIndex = existingGroupByReggregateIndex(); + final int existingGroupByOperatorIndex = existingGroupByReaggregateIndex(); if (existingGroupByOperatorIndex >= 0) { final GroupByReaggregateOperator existing = ensureGroupByReaggregateOperator(table, existingGroupByOperatorIndex, EXPOSED_GROUP_ROW_SETS.name(), pairs); @@ -1630,7 +1630,7 @@ public void visit(Formula formula) { final ColumnSource groupRowSet = table.getColumnSource(EXPOSED_GROUP_ROW_SETS.name()); GroupByReaggregateOperator groupByOperator; - final int existingIndex = existingGroupByReggregateIndex(); + final int existingIndex = existingGroupByReaggregateIndex(); if (existingIndex >= 0) { groupByOperator = ensureGroupByReaggregateOperator(table, existingIndex, EXPOSED_GROUP_ROW_SETS.name(), groupPairs); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java index c26a9d5c206..a15610c4fe6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java @@ -59,13 +59,16 @@ public final class GroupByChunkedOperator implements GroupByOperator { private List hiddenResults; /** + * Create a GroupedByChunkedOperator, which produces a column of {@link io.deephaven.vector.Vector vectors} for each + * of the input columns. * * @param inputTable the table we are aggregating * @param registeredWithHelper true if we are registered with the helper (meaning we independently produce result * columns), false otherwise. For a normal AggGroup this is true; for a group-by that is only part of an * AggFormula this is false. * @param exposeRowSetsAs the name of the column to expose the rowsets for each group as - * @param hiddenResults a list (possibly empty) of columns that are not exposed to the helper + * @param hiddenResults a list (possibly empty) of columns that are not exposed to the helper; or null if all + * columns should be exposed * @param aggregatedColumnPairs the list of input and output columns for this operation */ public GroupByChunkedOperator( From 5d8e6ef4e3841d4c434b97a9a4e4287d93a5de37 Mon Sep 17 00:00:00 2001 From: "Charles P. Wright" Date: Mon, 12 Jan 2026 17:16:59 -0500 Subject: [PATCH 20/24] initial rollup aggformula docs. --- .../table-operations/create/rollup.md | 60 +++++++++++++++++++ .../group-and-aggregate/AggFormula.md | 9 ++- .../74a7db0b5abe5682f828a576c69e2222.json | 1 - .../a79ff17b43af37762a7fc93060ed494e.json | 1 + .../ae837734776de985db98db1a878d070e.json | 1 + .../db3562b0dc23b199a91d0762dad1ac03.json | 1 - .../e2ecbb33eca1c30622522fbd85188e95.json | 1 + .../74a7db0b5abe5682f828a576c69e2222.json | 1 - .../db3562b0dc23b199a91d0762dad1ac03.json | 1 - .../impl/hierarchical/RollupTableImpl.java | 3 +- 10 files changed, 71 insertions(+), 8 deletions(-) delete mode 100644 docs/groovy/snapshots/74a7db0b5abe5682f828a576c69e2222.json create mode 100644 docs/groovy/snapshots/a79ff17b43af37762a7fc93060ed494e.json create mode 100644 docs/groovy/snapshots/ae837734776de985db98db1a878d070e.json delete mode 100644 docs/groovy/snapshots/db3562b0dc23b199a91d0762dad1ac03.json create mode 100644 docs/groovy/snapshots/e2ecbb33eca1c30622522fbd85188e95.json delete mode 100644 docs/python/snapshots/74a7db0b5abe5682f828a576c69e2222.json delete mode 100644 docs/python/snapshots/db3562b0dc23b199a91d0762dad1ac03.json diff --git a/docs/groovy/reference/table-operations/create/rollup.md b/docs/groovy/reference/table-operations/create/rollup.md index 87966bbbb89..c91aa5ec93c 100644 --- a/docs/groovy/reference/table-operations/create/rollup.md +++ b/docs/groovy/reference/table-operations/create/rollup.md @@ -28,6 +28,8 @@ The following aggregations are supported: - [`AggCountDistinct`](../group-and-aggregate/AggCountDistinct.md) - [`AggCountWhere`](../group-and-aggregate/AggCountWhere.md) - [`AggFirst`](../group-and-aggregate/AggFirst.md) +- [`AggFormula`](../group-and-aggregate/AggFormula.md) +- [`AggGroup`](../group-and-aggregate/AggGroup.md) - [`AggLast`](../group-and-aggregate/AggLast.md) - [`AggMax`](../group-and-aggregate/AggMax.md) - [`AggMin`](../group-and-aggregate/AggMin.md) @@ -124,6 +126,64 @@ result = source.rollup(aggList, false, "N", "M") ![The above `result` rollup table](../../../assets/how-to/rollup-table-realtime.gif) +## Formula Aggregations in Rollups + +When a rollups includes a formula aggregation, care should be taken with the function being applied. On each tick, the formula is evaluated for each changed row in the output table. Because the aggregated rows include many source rows, the input vectors to a formula aggregation can be very large (at the root level, they are the entire source table). If the formula is not efficient with large input vectors, the performance of the rollup can be poor. + +By default, the formula aggregation operates on a group of all the values as they appeared in the source table. In this example, the `Value` column contains the same vector that is used as input to the formula: + +```groovy +source = newTable( + stringCol("Key", "Alpha", "Alpha", "Alpha", "Bravo", "Bravo", "Charlie", "Charlie"), + intCol("Value", 10, 10, 10, 20, 20, 30, 30)) +simpleSum = source.rollup(List.of(AggGroup("Value"), AggFormula("Sum = sum(Value)")), "Key") +``` + +To calculate the sum for the root row, every row in the source table is read. The Deephaven engine provides no mechanism to provide detailed update information for a vector. Thus, every time the table ticks, the formula is completely re-evaluated. + +### Formula Reaggregation + +Formula reaggregation can be used to limit the size of input vectors while evaluating changes to a rollup. Each level of the rollup must have the same constituent types and names, which can make formulating your query more complicated. + +```groovy +source = newTable( +stringCol("Key", "Alpha", "Alpha", "Alpha", "Bravo", "Bravo", "Charlie", "Charlie"), +intCol("Value", 10, 10, 10, 20, 20, 30, 30)) +reaggregatedSum = source.updateView("Sum=(long)Value").rollup(List.of(AggFormula("Sum = sum(Sum)").asReaggregating()), "Key") +``` + +The results of `reaggregatedSum` are identical to `simpleSum`; but they are evaluated differently. In simpleSum, the source table is read twice: once to calculate the lowest-level sums, and a second time to calculate the top-level sum. In the reaggregated example, the source table is read once to calculate the lowest-level sums; and then the intermediate sums for each `Key` are read to calculate the top-level sum. If a row was added to `source` with the key `Delta`; then `simpleSum` would read that row, calculate a new sum for `Delta` and the top-level would read all eight rows of the table. The `reaggregatedSum` would similarly calculate the new sum for `Delta`, but the top-level would only read the intermediate sums for `Alpha`, `Bravo`, `Charlie`, and `Delta` instead of all eight source rows. As the number of states and the size of the input tables increase, the performance impact of evaluating a formula over all rows the table increases. + +In the previous example, the `Sum` column evaluated the [`sum(IntVector)`](https://docs.deephaven.io/core/javadoc/io/deephaven/function/Numeric.html#sum(io.deephaven.vector.IntVector)) function at every level of the rollup and produced a `long`. If the original table with an `int` column was used, then the lowest-level rollup would provide an `IntVector` as input to the `sum` and the next-level would provide `LongVector`. Similarly, the source table had a column named `Value`; whereas the aggregation produces a result named `Sum`. To address both these issues, before passing `source` to rollup, we called `updateView` to cast the `Value` column to `long` as `Sum`. If we ran the same example without the cast: + +```groovy syntax +source = newTable( +stringCol("Key", "Alpha", "Alpha", "Alpha", "Bravo", "Bravo", "Charlie", "Charlie"), +intCol("Value", 10, 10, 10, 20, 20, 30, 30)) +reaggregatedSum = source.rollup(List.of(AggFormula("Value = sum(Value)").asReaggregating()), "Key") +``` + +We instead get an Exception message indicating that the formula cannot be applied properly: + +```text +java.lang.ClassCastException: class io.deephaven.engine.table.vectors.LongVectorColumnWrapper cannot be cast to class io.deephaven.vector.IntVector (io.deephaven.engine.table.vectors.LongVectorColumnWrapper and io.deephaven.vector.IntVector are in unnamed module of loader 'app') +``` + +### Formula Depth + +Formula aggregations may include the constant `__FORMULA_DEPTH__` column, which is the depth of the formula aggregation in the rollup tree. The root node of the rollup has a depth of 0, the next level is 1, and so on. This can be used to implement distinct aggregations at each level of the rollup. For example: + +```groovy +source = newTable(stringCol("Key", "Alpha", "Alpha", "Alpha", "Bravo", "Bravo", "Charlie", "Charlie"), intCol("Value", 10, 10, 10, 20, 20, 30, 30)) +firstThenSum = source.rollup(List.of(AggFormula("Result = __FORMULA_DEPTH__ == 0 ? sum(Value) : first(Value)")), "Key") +``` + +In this case, for each value of `Key`, the aggregation returns the first value. For the root level, the aggregation returns the sum of all values. + +### Examples + +#### Capped Sum + ## Related documentation - [`AggAvg`](../group-and-aggregate/AggAvg.md) diff --git a/docs/groovy/reference/table-operations/group-and-aggregate/AggFormula.md b/docs/groovy/reference/table-operations/group-and-aggregate/AggFormula.md index d82a335f7bf..bec64775953 100644 --- a/docs/groovy/reference/table-operations/group-and-aggregate/AggFormula.md +++ b/docs/groovy/reference/table-operations/group-and-aggregate/AggFormula.md @@ -7,6 +7,7 @@ title: AggFormula ## Syntax ``` +AggFormula(formula) AggFormula(formula, paramToken, columnNames...) ``` @@ -21,19 +22,21 @@ The user-defined formula to apply to each group. This formula can contain: - Mathematical operations such as `*`, `+`, `/`, etc. - [User-defined closures](../../../how-to-guides/groovy-closures.md) -If `paramToken` is not `null`, the formula can only be applied to one column at a time, and it is applied to the specified `paramToken`. If `paramToken` is `null`, the formula is applied to any column or literal value present in the formula. The use of `paramToken` is deprecated. +The formula is typically specified as `OutputColumn = Expression` (when `paramToken` is `null`). The formula is applied to any column or literal value present in the formula. For example, `Out = KeyColumn * max(ValueColumn)`, produces an output column with the name `Out` and uses `KeyColumn` and `ValueColumn` as inputs. + +If `paramToken` is not `null`, the formula can only be applied to one column at a time, and it is applied to the specified `paramToken`. In this case, the formula does not supply an output column, but rather it is derived from the `columnNames` parameter. The use of `paramToken` is deprecated. Key column(s) can be used as input to the formula. When this happens, key values are treated as scalars. -The parameter name within the formula. If `paramToken` is `each`, then `formula` must contain `each`. For example, `max(each)`, `min(each)`, etc. Use of this parameter is deprecated. +The parameter name within the formula. If `paramToken` is `each`, then `formula` must contain `each`. For example, `max(each)`, `min(each)`, etc. Use of this parameter is deprecated. A non-null value is not permitted in [rollups](../create/rollup.md). -The source column(s) for the calculations. +The source column(s) for the calculations. The source column names are only used when `paramToken` is not `null`, and are thus similarly deprecated. - `"X"` applies the formula to each value in the `X` column for each group. - `"Y = X"` applies the formula to each value in the `X` column for each group and renames it to `Y`. diff --git a/docs/groovy/snapshots/74a7db0b5abe5682f828a576c69e2222.json b/docs/groovy/snapshots/74a7db0b5abe5682f828a576c69e2222.json deleted file mode 100644 index 25793bcdd4b..00000000000 --- a/docs/groovy/snapshots/74a7db0b5abe5682f828a576c69e2222.json +++ /dev/null @@ -1 +0,0 @@ -{"file":"how-to-guides/excel/excel-client.md","objects":{"static_table":{"type":"Table","data":{"columns":[{"name":"X","type":"int"}],"rows":[[{"value":"5"}],[{"value":"2"}],[{"value":"9"}],[{"value":"8"}],[{"value":"1"}],[{"value":"7"}],[{"value":"4"}],[{"value":"8"}],[{"value":"6"}],[{"value":"9"}]]}}}} \ No newline at end of file diff --git a/docs/groovy/snapshots/a79ff17b43af37762a7fc93060ed494e.json b/docs/groovy/snapshots/a79ff17b43af37762a7fc93060ed494e.json new file mode 100644 index 00000000000..4abb4083908 --- /dev/null +++ b/docs/groovy/snapshots/a79ff17b43af37762a7fc93060ed494e.json @@ -0,0 +1 @@ +{"file":"reference/table-operations/create/rollup.md","objects":{"source":{"type":"Table","data":{"columns":[{"name":"Key","type":"java.lang.String"},{"name":"Value","type":"int"}],"rows":[[{"value":"Alpha"},{"value":"10"}],[{"value":"Alpha"},{"value":"10"}],[{"value":"Alpha"},{"value":"10"}],[{"value":"Bravo"},{"value":"20"}],[{"value":"Bravo"},{"value":"20"}],[{"value":"Charlie"},{"value":"30"}],[{"value":"Charlie"},{"value":"30"}]]}},"simpleSum":{"type":"HierarchicalTable","data":{"columns":[{"name":"Key","type":"java.lang.String"},{"name":"Value","type":"io.deephaven.vector.IntVector"},{"name":"Sum","type":"long"}],"rows":[[{"value":""},{"value":"10,10,10,20,20,30,30"},{"value":"130"}],[{"value":"Alpha"},{"value":"10,10,10"},{"value":"30"}],[{"value":"Bravo"},{"value":"20,20"},{"value":"40"}],[{"value":"Charlie"},{"value":"30,30"},{"value":"60"}]],"rowDepths":[1,2,2,2]}}}} \ No newline at end of file diff --git a/docs/groovy/snapshots/ae837734776de985db98db1a878d070e.json b/docs/groovy/snapshots/ae837734776de985db98db1a878d070e.json new file mode 100644 index 00000000000..7a63af13972 --- /dev/null +++ b/docs/groovy/snapshots/ae837734776de985db98db1a878d070e.json @@ -0,0 +1 @@ +{"file":"reference/table-operations/create/rollup.md","objects":{"source":{"type":"Table","data":{"columns":[{"name":"Key","type":"java.lang.String"},{"name":"Value","type":"int"}],"rows":[[{"value":"Alpha"},{"value":"10"}],[{"value":"Alpha"},{"value":"10"}],[{"value":"Alpha"},{"value":"10"}],[{"value":"Bravo"},{"value":"20"}],[{"value":"Bravo"},{"value":"20"}],[{"value":"Charlie"},{"value":"30"}],[{"value":"Charlie"},{"value":"30"}]]}},"reaggregatedSum":{"type":"HierarchicalTable","data":{"columns":[{"name":"Key","type":"java.lang.String"},{"name":"Sum","type":"long"}],"rows":[[{"value":""},{"value":"130"}],[{"value":"Alpha"},{"value":"30"}],[{"value":"Bravo"},{"value":"40"}],[{"value":"Charlie"},{"value":"60"}]],"rowDepths":[1,2,2,2]}}}} \ No newline at end of file diff --git a/docs/groovy/snapshots/db3562b0dc23b199a91d0762dad1ac03.json b/docs/groovy/snapshots/db3562b0dc23b199a91d0762dad1ac03.json deleted file mode 100644 index d22df14b1c1..00000000000 --- a/docs/groovy/snapshots/db3562b0dc23b199a91d0762dad1ac03.json +++ /dev/null @@ -1 +0,0 @@ -{"file":"how-to-guides/excel/excel-client.md","objects":{"crypto_table":{"type":"Table","data":{"columns":[{"name":"Timestamp","type":"java.time.Instant"},{"name":"Exchange","type":"java.lang.String"},{"name":"Price","type":"double"},{"name":"Size","type":"double"}],"rows":[]}}}} \ No newline at end of file diff --git a/docs/groovy/snapshots/e2ecbb33eca1c30622522fbd85188e95.json b/docs/groovy/snapshots/e2ecbb33eca1c30622522fbd85188e95.json new file mode 100644 index 00000000000..ec80137999d --- /dev/null +++ b/docs/groovy/snapshots/e2ecbb33eca1c30622522fbd85188e95.json @@ -0,0 +1 @@ +{"file":"reference/table-operations/create/rollup.md","objects":{"source":{"type":"Table","data":{"columns":[{"name":"Key","type":"java.lang.String"},{"name":"Value","type":"int"}],"rows":[[{"value":"Alpha"},{"value":"10"}],[{"value":"Alpha"},{"value":"10"}],[{"value":"Alpha"},{"value":"10"}],[{"value":"Bravo"},{"value":"20"}],[{"value":"Bravo"},{"value":"20"}],[{"value":"Charlie"},{"value":"30"}],[{"value":"Charlie"},{"value":"30"}]]}},"firstThenSum":{"type":"HierarchicalTable","data":{"columns":[{"name":"Key","type":"java.lang.String"},{"name":"Result","type":"long"}],"rows":[[{"value":""},{"value":"130"}],[{"value":"Alpha"},{"value":"10"}],[{"value":"Bravo"},{"value":"20"}],[{"value":"Charlie"},{"value":"30"}]],"rowDepths":[1,2,2,2]}}}} \ No newline at end of file diff --git a/docs/python/snapshots/74a7db0b5abe5682f828a576c69e2222.json b/docs/python/snapshots/74a7db0b5abe5682f828a576c69e2222.json deleted file mode 100644 index 33cb2460f88..00000000000 --- a/docs/python/snapshots/74a7db0b5abe5682f828a576c69e2222.json +++ /dev/null @@ -1 +0,0 @@ -{"file":"core/docs/how-to-guides/excel/excel-add-in.md","objects":{"static_table":{"type":"Table","data":{"columns":[{"name":"X","type":"int"}],"rows":[[{"value":"4"}],[{"value":"7"}],[{"value":"7"}],[{"value":"6"}],[{"value":"7"}],[{"value":"9"}],[{"value":"1"}],[{"value":"2"}],[{"value":"8"}],[{"value":"4"}]]}}}} \ No newline at end of file diff --git a/docs/python/snapshots/db3562b0dc23b199a91d0762dad1ac03.json b/docs/python/snapshots/db3562b0dc23b199a91d0762dad1ac03.json deleted file mode 100644 index f0726a5616d..00000000000 --- a/docs/python/snapshots/db3562b0dc23b199a91d0762dad1ac03.json +++ /dev/null @@ -1 +0,0 @@ -{"file":"core/docs/how-to-guides/excel/excel-client.md","objects":{"crypto_table":{"type":"Table","data":{"columns":[{"name":"Timestamp","type":"java.time.Instant"},{"name":"Exchange","type":"java.lang.String"},{"name":"Price","type":"double"},{"name":"Size","type":"double"}],"rows":[]}}}} \ No newline at end of file diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/hierarchical/RollupTableImpl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/hierarchical/RollupTableImpl.java index 0223cb01ded..d3986d77ef6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/hierarchical/RollupTableImpl.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/hierarchical/RollupTableImpl.java @@ -42,6 +42,7 @@ import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; import static io.deephaven.engine.table.impl.AbsoluteSortColumnConventions.*; import static io.deephaven.engine.table.impl.BaseTable.shouldCopyAttribute; +import static io.deephaven.engine.table.impl.by.AggregationProcessor.EXPOSED_GROUP_ROW_SETS; import static io.deephaven.engine.table.impl.by.AggregationProcessor.getRowLookup; import static io.deephaven.engine.table.impl.by.AggregationRowLookup.DEFAULT_UNKNOWN_ROW; import static io.deephaven.engine.table.impl.by.AggregationRowLookup.EMPTY_KEY; @@ -702,7 +703,7 @@ private static void rollupFromBase( private static Stream> filterRollupInternalColumns( @NotNull final Stream> columnDefinitions) { - return columnDefinitions.filter(cd -> !cd.getName().endsWith(ROLLUP_COLUMN_SUFFIX)); + return columnDefinitions.filter(cd -> !cd.getName().endsWith(ROLLUP_COLUMN_SUFFIX) && !cd.getName().equals(EXPOSED_GROUP_ROW_SETS.name())); } @Override From 169c3008ddf9929c1bd6763c20ecd8d9f3075fea Mon Sep 17 00:00:00 2001 From: "Charles P. Wright" Date: Tue, 13 Jan 2026 08:59:42 -0500 Subject: [PATCH 21/24] Capped sum doc, fix tests now that we hide the exposed group column. --- .../table-operations/create/rollup.md | 32 +++++++++++++------ ... => 147d1e4ef58979040021c4e15859a139.json} | 0 .../df5a083869378962413e5e1ec88361d6.json | 1 + ... => f1ee5c94013279dcc75428cd1e365ea4.json} | 2 +- .../f2208a297db36f58594c7ecc25d3713f.json | 1 + .../impl/hierarchical/RollupTableImpl.java | 3 +- .../engine/table/impl/TestRollupTable.java | 15 ++++----- 7 files changed, 35 insertions(+), 19 deletions(-) rename docs/groovy/snapshots/{ae837734776de985db98db1a878d070e.json => 147d1e4ef58979040021c4e15859a139.json} (100%) create mode 100644 docs/groovy/snapshots/df5a083869378962413e5e1ec88361d6.json rename docs/groovy/snapshots/{e2ecbb33eca1c30622522fbd85188e95.json => f1ee5c94013279dcc75428cd1e365ea4.json} (68%) create mode 100644 docs/groovy/snapshots/f2208a297db36f58594c7ecc25d3713f.json diff --git a/docs/groovy/reference/table-operations/create/rollup.md b/docs/groovy/reference/table-operations/create/rollup.md index c91aa5ec93c..56779ad74f6 100644 --- a/docs/groovy/reference/table-operations/create/rollup.md +++ b/docs/groovy/reference/table-operations/create/rollup.md @@ -147,8 +147,8 @@ Formula reaggregation can be used to limit the size of input vectors while evalu ```groovy source = newTable( -stringCol("Key", "Alpha", "Alpha", "Alpha", "Bravo", "Bravo", "Charlie", "Charlie"), -intCol("Value", 10, 10, 10, 20, 20, 30, 30)) + stringCol("Key", "Alpha", "Alpha", "Alpha", "Bravo", "Bravo", "Charlie", "Charlie"), + intCol("Value", 10, 10, 10, 20, 20, 30, 30)) reaggregatedSum = source.updateView("Sum=(long)Value").rollup(List.of(AggFormula("Sum = sum(Sum)").asReaggregating()), "Key") ``` @@ -158,8 +158,8 @@ In the previous example, the `Sum` column evaluated the [`sum(IntVector)`](https ```groovy syntax source = newTable( -stringCol("Key", "Alpha", "Alpha", "Alpha", "Bravo", "Bravo", "Charlie", "Charlie"), -intCol("Value", 10, 10, 10, 20, 20, 30, 30)) + stringCol("Key", "Alpha", "Alpha", "Alpha", "Bravo", "Bravo", "Charlie", "Charlie"), + intCol("Value", 10, 10, 10, 20, 20, 30, 30)) reaggregatedSum = source.rollup(List.of(AggFormula("Value = sum(Value)").asReaggregating()), "Key") ``` @@ -174,15 +174,29 @@ java.lang.ClassCastException: class io.deephaven.engine.table.vectors.LongVector Formula aggregations may include the constant `__FORMULA_DEPTH__` column, which is the depth of the formula aggregation in the rollup tree. The root node of the rollup has a depth of 0, the next level is 1, and so on. This can be used to implement distinct aggregations at each level of the rollup. For example: ```groovy -source = newTable(stringCol("Key", "Alpha", "Alpha", "Alpha", "Bravo", "Bravo", "Charlie", "Charlie"), intCol("Value", 10, 10, 10, 20, 20, 30, 30)) -firstThenSum = source.rollup(List.of(AggFormula("Result = __FORMULA_DEPTH__ == 0 ? sum(Value) : first(Value)")), "Key") +source = newTable( + stringCol("Key", "Alpha", "Alpha", "Alpha", "Bravo", "Bravo", "Charlie", "Charlie"), + intCol("Value", 10, 10, 10, 20, 20, 30, 30)) +firstThenSum = source.rollup(List.of(AggFormula("Value = __FORMULA_DEPTH__ == 0 ? sum(Value) : first(Value)")), "Key") ``` -In this case, for each value of `Key`, the aggregation returns the first value. For the root level, the aggregation returns the sum of all values. +In this case, for each value of `Key`, the aggregation returns the first value. For the root level, the aggregation returns the sum of all values. When combined with a reaggregating formula, even more interesting semantics are possible. For example, rather than summing all of the values; we can sum the values from the prior level: -### Examples +```groovy +source = newTable( + stringCol("Key", "Alpha", "Alpha", "Alpha", "Bravo", "Bravo", "Charlie", "Charlie"), + intCol("Value", 10, 10, 10, 20, 20, 30, 30)) +firstThenSum = source.updateView("Value=(long)Value").rollup(List.of(AggFormula("Value = __FORMULA_DEPTH__ == 0 ? sum(Value) : first(Value)").asReaggregating()), "Key") +``` + +Another simple example of reaggration is a capped sum. In this example, the sums below the root level are capped at 40: -#### Capped Sum +```groovy +source = newTable( + stringCol("Key", "Alpha", "Alpha", "Alpha", "Bravo", "Bravo", "Charlie", "Charlie"), + intCol("Value", 10, 20, 15, 20, 15, 25, 35)) +cappedSum = source.updateView("Value=(long)Value").rollup(List.of(AggFormula("Value = __FORMULA_DEPTH__ == 0 ? sum(Value) : min(sum(Value), 40)").asReaggregating()), "Key") +``` ## Related documentation diff --git a/docs/groovy/snapshots/ae837734776de985db98db1a878d070e.json b/docs/groovy/snapshots/147d1e4ef58979040021c4e15859a139.json similarity index 100% rename from docs/groovy/snapshots/ae837734776de985db98db1a878d070e.json rename to docs/groovy/snapshots/147d1e4ef58979040021c4e15859a139.json diff --git a/docs/groovy/snapshots/df5a083869378962413e5e1ec88361d6.json b/docs/groovy/snapshots/df5a083869378962413e5e1ec88361d6.json new file mode 100644 index 00000000000..3ddb3f43ca3 --- /dev/null +++ b/docs/groovy/snapshots/df5a083869378962413e5e1ec88361d6.json @@ -0,0 +1 @@ +{"file":"reference/table-operations/create/rollup.md","objects":{"source":{"type":"Table","data":{"columns":[{"name":"Key","type":"java.lang.String"},{"name":"Value","type":"int"}],"rows":[[{"value":"Alpha"},{"value":"10"}],[{"value":"Alpha"},{"value":"20"}],[{"value":"Alpha"},{"value":"15"}],[{"value":"Bravo"},{"value":"20"}],[{"value":"Bravo"},{"value":"15"}],[{"value":"Charlie"},{"value":"25"}],[{"value":"Charlie"},{"value":"35"}]]}},"cappedSum":{"type":"HierarchicalTable","data":{"columns":[{"name":"Key","type":"java.lang.String"},{"name":"Value","type":"long"}],"rows":[[{"value":""},{"value":"115"}],[{"value":"Alpha"},{"value":"40"}],[{"value":"Bravo"},{"value":"35"}],[{"value":"Charlie"},{"value":"40"}]],"rowDepths":[1,2,2,2]}}}} \ No newline at end of file diff --git a/docs/groovy/snapshots/e2ecbb33eca1c30622522fbd85188e95.json b/docs/groovy/snapshots/f1ee5c94013279dcc75428cd1e365ea4.json similarity index 68% rename from docs/groovy/snapshots/e2ecbb33eca1c30622522fbd85188e95.json rename to docs/groovy/snapshots/f1ee5c94013279dcc75428cd1e365ea4.json index ec80137999d..a532e456acd 100644 --- a/docs/groovy/snapshots/e2ecbb33eca1c30622522fbd85188e95.json +++ b/docs/groovy/snapshots/f1ee5c94013279dcc75428cd1e365ea4.json @@ -1 +1 @@ -{"file":"reference/table-operations/create/rollup.md","objects":{"source":{"type":"Table","data":{"columns":[{"name":"Key","type":"java.lang.String"},{"name":"Value","type":"int"}],"rows":[[{"value":"Alpha"},{"value":"10"}],[{"value":"Alpha"},{"value":"10"}],[{"value":"Alpha"},{"value":"10"}],[{"value":"Bravo"},{"value":"20"}],[{"value":"Bravo"},{"value":"20"}],[{"value":"Charlie"},{"value":"30"}],[{"value":"Charlie"},{"value":"30"}]]}},"firstThenSum":{"type":"HierarchicalTable","data":{"columns":[{"name":"Key","type":"java.lang.String"},{"name":"Result","type":"long"}],"rows":[[{"value":""},{"value":"130"}],[{"value":"Alpha"},{"value":"10"}],[{"value":"Bravo"},{"value":"20"}],[{"value":"Charlie"},{"value":"30"}]],"rowDepths":[1,2,2,2]}}}} \ No newline at end of file +{"file":"reference/table-operations/create/rollup.md","objects":{"source":{"type":"Table","data":{"columns":[{"name":"Key","type":"java.lang.String"},{"name":"Value","type":"int"}],"rows":[[{"value":"Alpha"},{"value":"10"}],[{"value":"Alpha"},{"value":"10"}],[{"value":"Alpha"},{"value":"10"}],[{"value":"Bravo"},{"value":"20"}],[{"value":"Bravo"},{"value":"20"}],[{"value":"Charlie"},{"value":"30"}],[{"value":"Charlie"},{"value":"30"}]]}},"firstThenSum":{"type":"HierarchicalTable","data":{"columns":[{"name":"Key","type":"java.lang.String"},{"name":"Value","type":"long"}],"rows":[[{"value":""},{"value":"130"}],[{"value":"Alpha"},{"value":"10"}],[{"value":"Bravo"},{"value":"20"}],[{"value":"Charlie"},{"value":"30"}]],"rowDepths":[1,2,2,2]}}}} \ No newline at end of file diff --git a/docs/groovy/snapshots/f2208a297db36f58594c7ecc25d3713f.json b/docs/groovy/snapshots/f2208a297db36f58594c7ecc25d3713f.json new file mode 100644 index 00000000000..f081a76cd49 --- /dev/null +++ b/docs/groovy/snapshots/f2208a297db36f58594c7ecc25d3713f.json @@ -0,0 +1 @@ +{"file":"reference/table-operations/create/rollup.md","objects":{"source":{"type":"Table","data":{"columns":[{"name":"Key","type":"java.lang.String"},{"name":"Value","type":"int"}],"rows":[[{"value":"Alpha"},{"value":"10"}],[{"value":"Alpha"},{"value":"10"}],[{"value":"Alpha"},{"value":"10"}],[{"value":"Bravo"},{"value":"20"}],[{"value":"Bravo"},{"value":"20"}],[{"value":"Charlie"},{"value":"30"}],[{"value":"Charlie"},{"value":"30"}]]}},"firstThenSum":{"type":"HierarchicalTable","data":{"columns":[{"name":"Key","type":"java.lang.String"},{"name":"Value","type":"long"}],"rows":[[{"value":""},{"value":"60"}],[{"value":"Alpha"},{"value":"10"}],[{"value":"Bravo"},{"value":"20"}],[{"value":"Charlie"},{"value":"30"}]],"rowDepths":[1,2,2,2]}}}} \ No newline at end of file diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/hierarchical/RollupTableImpl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/hierarchical/RollupTableImpl.java index d3986d77ef6..a072a75844d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/hierarchical/RollupTableImpl.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/hierarchical/RollupTableImpl.java @@ -703,7 +703,8 @@ private static void rollupFromBase( private static Stream> filterRollupInternalColumns( @NotNull final Stream> columnDefinitions) { - return columnDefinitions.filter(cd -> !cd.getName().endsWith(ROLLUP_COLUMN_SUFFIX) && !cd.getName().equals(EXPOSED_GROUP_ROW_SETS.name())); + return columnDefinitions.filter(cd -> !cd.getName().endsWith(ROLLUP_COLUMN_SUFFIX) + && !cd.getName().equals(EXPOSED_GROUP_ROW_SETS.name())); } @Override diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java index b4aa1db0fea..e1f213901b0 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java @@ -369,7 +369,7 @@ public void testRollupGroupStatic() { TableTools.showWithRowSet(snapshot); final Table expected = initialExpectedGrouped(rollup1); - assertTableEquals(expected, snapshot.dropColumns("__EXPOSED_GROUP_ROW_SETS__")); + assertTableEquals(expected, snapshot); freeSnapshotTableChunks(snapshot); } @@ -416,7 +416,7 @@ private void testRollupFormulaStatic(boolean withGroup) { final Table expectedBase = initialExpectedGrouped(rollup1); final Table expectedSentinel = withGroup ? expectedBase : expectedBase.dropColumns("Sentinel"); final Table expected = expectedSentinel.update("FSum=ii == 0 ? 7 : 1 + Sum"); - assertTableEquals(expected, snapshot.dropColumns("__EXPOSED_GROUP_ROW_SETS__")); + assertTableEquals(expected, snapshot); freeSnapshotTableChunks(snapshot); } @@ -517,8 +517,7 @@ private void testRollupFormulaStatic3(boolean hasGroup) { TableTools.show(expected); - assertTableEquals(hasGroup ? expected : expected.dropColumns("gqty"), - hasGroup ? snapshot.dropColumns("__EXPOSED_GROUP_ROW_SETS__") : snapshot); + assertTableEquals(hasGroup ? expected : expected.dropColumns("gqty"), snapshot); freeSnapshotTableChunks(snapshot); } @@ -576,7 +575,7 @@ public void testRollupGroupIncremental() { TableTools.showWithRowSet(snapshot); final Table expected = initialExpectedGrouped(rollup1); - assertTableEquals(expected, snapshot.dropColumns("__EXPOSED_GROUP_ROW_SETS__")); + assertTableEquals(expected, snapshot); freeSnapshotTableChunks(snapshot); final ControlledUpdateGraph cug = source.getUpdateGraph().cast(); @@ -593,7 +592,7 @@ public void testRollupGroupIncremental() { TableTools.showWithRowSet(snapshot2); Table expected2 = secondExpectedGrouped(rollup1); TableTools.showWithRowSet(expected2); - assertTableEquals(expected2, snapshot2.dropColumns("__EXPOSED_GROUP_ROW_SETS__")); + assertTableEquals(expected2, snapshot2); freeSnapshotTableChunks(snapshot2); } @@ -621,7 +620,7 @@ public void testReusedGrouping() { TableTools.showWithRowSet(snapshot); final Table expected = initialExpectedGrouped(rollup1).update("S2=Sentinel"); - assertTableEquals(expected, snapshot.dropColumns("__EXPOSED_GROUP_ROW_SETS__")); + assertTableEquals(expected, snapshot); freeSnapshotTableChunks(snapshot); final ControlledUpdateGraph cug = source.getUpdateGraph().cast(); @@ -638,7 +637,7 @@ public void testReusedGrouping() { TableTools.showWithRowSet(snapshot2); Table expected2 = secondExpectedGrouped(rollup1).update("S2=Sentinel"); TableTools.showWithRowSet(expected2); - assertTableEquals(expected2, snapshot2.dropColumns("__EXPOSED_GROUP_ROW_SETS__")); + assertTableEquals(expected2, snapshot2); freeSnapshotTableChunks(snapshot2); } } From f0d41b303ca7541a770d58494aeff7aef7a59beb Mon Sep 17 00:00:00 2001 From: "Charles P. Wright" Date: Tue, 13 Jan 2026 09:05:06 -0500 Subject: [PATCH 22/24] copilot nits --- .../deephaven/engine/table/impl/by/AggregationProcessor.java | 3 ++- .../engine/table/impl/by/GroupByReaggregateOperator.java | 3 ++- .../engine/table/impl/hierarchical/RollupTableImpl.java | 2 +- 3 files changed, 5 insertions(+), 3 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index c1572d89f60..cb91fea14a4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -200,6 +200,7 @@ public static AggregationContextFactory forRollupBase( * mutated by {@link AggregationProcessor}. * @param nullColumns Map of group-by column names and data types to aggregate with a null-column aggregation * @param rollupColumn the name of the rollup column in the result, used to traverse to the next lower level nodes + * @param source the original source table of the rollup (not the table we are reaggregating) * @return The {@link AggregationContextFactory} */ public static AggregationContextFactory forRollupReaggregated( @@ -1348,7 +1349,7 @@ public void visit(Formula formula) { MatchPair.fromPairs(Pair.from(inputNonKeyColumns))); delegate = false; } else { - // We we are reaggregating, we do not expose the rowsets, because the next level creates a + // When we are reaggregating, we do not expose the rowsets, because the next level creates a // completely fresh operator groupByChunkedOperator = makeGroupByOperatorForFormula(inputNonKeyColumns, table, null); // the operator is not added, so there is delegation diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java index 940bad9092e..73b8c6448a1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java @@ -209,7 +209,8 @@ public boolean shiftChunk(final SingletonContext singletonContext, final Chunk postShiftRowKeys, final long destination) { // we don't need to deal with these yet - throw new IllegalStateException(); + throw new IllegalStateException( + "Reaggregations should not require shifts, as aggregations have fixed output slots."); } private void addChunk(@NotNull final ObjectChunk rowSets, final int start, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/hierarchical/RollupTableImpl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/hierarchical/RollupTableImpl.java index a072a75844d..7893d758537 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/hierarchical/RollupTableImpl.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/hierarchical/RollupTableImpl.java @@ -671,7 +671,7 @@ private static ColumnSource
[] makeLevelNodeTableSourcesArray( * already filled * @param aggregations The aggregations * @param groupByColumns The group-by columns - * @param source + * @param source the source table for the rollup */ private static void rollupFromBase( @NotNull final QueryTable[] levelTables, From 9240d29d86f9edb5d1ea9b03b5fb441313475dcb Mon Sep 17 00:00:00 2001 From: "Charles P. Wright" Date: Tue, 13 Jan 2026 17:08:43 -0500 Subject: [PATCH 23/24] another test, more rename tomfoolery. --- .../java/io/deephaven/engine/table/Table.java | 5 ++ .../table/impl/by/AggregationProcessor.java | 65 +++++++++++-------- .../by/FormulaMultiColumnChunkedOperator.java | 12 +++- .../impl/by/GroupByReaggregateOperator.java | 2 +- .../engine/table/impl/TestRollupTable.java | 31 ++++++++- 5 files changed, 84 insertions(+), 31 deletions(-) diff --git a/engine/api/src/main/java/io/deephaven/engine/table/Table.java b/engine/api/src/main/java/io/deephaven/engine/table/Table.java index f297cda94b6..6703718ace8 100644 --- a/engine/api/src/main/java/io/deephaven/engine/table/Table.java +++ b/engine/api/src/main/java/io/deephaven/engine/table/Table.java @@ -567,6 +567,11 @@ CloseableIterator objectColumnIterator(@NotNull String co *

* *

+ * A blink table can be converted to an append-only table with + * {@link io.deephaven.engine.table.impl.BlinkTableTools#blinkToAppendOnly(io.deephaven.engine.table.Table)}. + *

+ * + *

* Some aggregations (in particular {@link #groupBy} and {@link #partitionBy} cannot provide the desired blink table * aggregation semantics because doing so would require storing the entire stream of blink updates in memory. If * that behavior is desired, use {@code blinkToAppendOnly}. If on the other hand, you would like to group or diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index cb91fea14a4..e6fdc635974 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -952,11 +952,9 @@ public void visit(@NotNull final Formula formula) { final int existingGroupByOperatorIndex = existingGroupByOperatorIndex(); if (existingGroupByOperatorIndex >= 0) { // if we have an existing group by operator, then use it (or update it to reflect our input columns) - final MatchPair[] matchPairs = - Arrays.stream(inputNonKeyColumns).map(cn -> new MatchPair(cn, cn)).toArray(MatchPair[]::new); - groupByChunkedOperator = ensureGroupByOperator(table, existingGroupByOperatorIndex, null, matchPairs); + groupByChunkedOperator = ensureGroupByOperator(table, existingGroupByOperatorIndex, null, makeSymmetricMatchPairs(inputNonKeyColumns)); } else { - groupByChunkedOperator = makeGroupByOperatorForFormula(inputNonKeyColumns, table, null); + groupByChunkedOperator = makeGroupByOperatorForFormula(makeSymmetricMatchPairs(inputNonKeyColumns), table, null); } final FormulaMultiColumnChunkedOperator op = new FormulaMultiColumnChunkedOperator(table, @@ -1146,26 +1144,34 @@ private void maybeInitializeVectorColumns(Set groupByColumnSet, final Ta } - private @NotNull GroupByChunkedOperator makeGroupByOperatorForFormula(String[] inputNonKeyColumns, + private @NotNull GroupByChunkedOperator makeGroupByOperatorForFormula(final MatchPair[] pairs, final QueryTable table, final String exposedRowsets) { - final MatchPair[] pairs; - final boolean register; - if (exposedRowsets == null) { - register = false; - pairs = Arrays.stream(inputNonKeyColumns).map(col -> MatchPair.of(Pair.parse(col))) - .toArray(MatchPair[]::new); - } else { - register = true; - pairs = Arrays - .stream(inputNonKeyColumns).map(col -> MatchPair.of( - Pair - .of(ColumnName.of(col), - ColumnName.of(col + ROLLUP_GRP_COLUMN_ID + ROLLUP_COLUMN_SUFFIX)))) - .toArray(MatchPair[]::new); - } + final boolean register = exposedRowsets != null; return new GroupByChunkedOperator(table, register, exposedRowsets, null, pairs); } + /** + * Convert the array of column names to MatchPairs of the form {@code Col_GRP__ROLLUP__} + * + * @param cols the columns to convert + * @return the mangled name matchpairs + */ + private static MatchPair @NotNull [] makeMangledMatchPairs(String[] cols) { + return Arrays + .stream(cols).map(col -> new MatchPair(col + ROLLUP_GRP_COLUMN_ID + ROLLUP_COLUMN_SUFFIX, col)) + .toArray(MatchPair[]::new); + } + + /** + * Convert the array of strings to MatchPairs of the form Col=Col + * + * @param columns the columns to convert to MatchPairs + * @return an array of MatchPairs + */ + private static MatchPair @NotNull [] makeSymmetricMatchPairs(String[] columns) { + return Arrays.stream(columns).map(col -> new MatchPair(col, col)).toArray(MatchPair[]::new); + } + // ----------------------------------------------------------------------------------------------------------------- // Rollup Unsupported Operations // ----------------------------------------------------------------------------------------------------------------- @@ -1318,6 +1324,8 @@ public void visit(AggSpecGroup group) { @Override public void visit(Formula formula) { + unsupportedForBlinkTables("Formula for rollup"); + final SelectColumn selectColumn = SelectColumn.of(formula.selectable()); // Get or create a column definition map composed of vectors of the original column types (or scalars when @@ -1343,27 +1351,30 @@ public void visit(Formula formula) { final boolean delegate; final int existingGroupByOperatorIndex = existingGroupByOperatorIndex(); + final MatchPair[] mangledMatchPairs = makeMangledMatchPairs(inputNonKeyColumns); + if (formula.reaggregateAggregatedValues()) { if (existingGroupByOperatorIndex >= 0) { - groupByChunkedOperator = ensureGroupByOperator(table, existingGroupByOperatorIndex, null, - MatchPair.fromPairs(Pair.from(inputNonKeyColumns))); + groupByChunkedOperator = ensureGroupByOperator(table, existingGroupByOperatorIndex, null, mangledMatchPairs); + // TODO: do this better with an existing reaggregated op delegate = false; } else { // When we are reaggregating, we do not expose the rowsets, because the next level creates a // completely fresh operator - groupByChunkedOperator = makeGroupByOperatorForFormula(inputNonKeyColumns, table, null); + groupByChunkedOperator = makeGroupByOperatorForFormula(mangledMatchPairs, table, null); // the operator is not added, so there is delegation delegate = true; } } else { if (existingGroupByOperatorIndex >= 0) { - groupByChunkedOperator = ensureGroupByOperator(table, existingGroupByOperatorIndex, - EXPOSED_GROUP_ROW_SETS.name(), MatchPair.fromPairs(Pair.from(inputNonKeyColumns))); + groupByChunkedOperator = ensureGroupByOperator(table, existingGroupByOperatorIndex, EXPOSED_GROUP_ROW_SETS.name(), mangledMatchPairs); + // TODO: do this better + List asPairs = Arrays.stream(mangledMatchPairs).map(mp -> Pair.of(mp.input(), mp.output())).collect(Collectors.toList()); + addNoInputOperator(groupByChunkedOperator.resultExtractor(asPairs)); delegate = false; } else { // When we do not reaggregate, the next level needs access to our exposed group row sets - groupByChunkedOperator = - makeGroupByOperatorForFormula(inputNonKeyColumns, table, EXPOSED_GROUP_ROW_SETS.name()); + groupByChunkedOperator = makeGroupByOperatorForFormula(mangledMatchPairs, table, EXPOSED_GROUP_ROW_SETS.name()); addNoInputOperator(groupByChunkedOperator); // we added the operator, so we cannot delegate delegate = false; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaMultiColumnChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaMultiColumnChunkedOperator.java index a1543c88b69..531ec4c7787 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaMultiColumnChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaMultiColumnChunkedOperator.java @@ -3,6 +3,7 @@ // package io.deephaven.engine.table.impl.by; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.ChunkLengths; import io.deephaven.chunk.attributes.ChunkPositions; @@ -239,8 +240,15 @@ public void propagateInitialState(@NotNull final QueryTable resultTable, int sta } else { final Map> columnSourceMap = resultTable.getColumnSourceMap(); sourceColumns = new HashMap<>(groupBy.getInputResultColumns().size() + 1); - groupBy.getInputResultColumns() - .forEach((k, v) -> sourceColumns.put(renames == null ? k : renames.get(k), v)); + for (Map.Entry> entry : groupBy.getInputResultColumns().entrySet()) { + final String columnName = entry.getKey(); + final String renamed; + if (renames != null && (renamed = renames.get(columnName)) != null) { + sourceColumns.put(renamed, entry.getValue()); + } else { + sourceColumns.put(columnName, entry.getValue()); + } + } Arrays.stream(inputKeyColumns).forEach(col -> sourceColumns.put(col, columnSourceMap.get(col))); sourceColumns.put(AggregationProcessor.ROLLUP_FORMULA_DEPTH.name(), formulaDepthSource); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java index 73b8c6448a1..779edd80380 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByReaggregateOperator.java @@ -327,7 +327,7 @@ public UnaryOperator initializeRefreshing( @Override public Map> getInputResultColumns() { - return resultAggregatedColumns; + return inputAggregatedColumns; } @Override diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java index e1f213901b0..67e1c47f0a2 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java @@ -375,8 +375,8 @@ public void testRollupGroupStatic() { @Test public void testRollupFormulaStatic() { - testRollupFormulaStatic(true); testRollupFormulaStatic(false); + testRollupFormulaStatic(true); } private void testRollupFormulaStatic(boolean withGroup) { @@ -522,6 +522,35 @@ private void testRollupFormulaStatic3(boolean hasGroup) { freeSnapshotTableChunks(snapshot); } + @Test + public void testRollupFormulaGroupRenames() { + final int[] allValues = {10, 10, 10, 20, 20, 30, 30}; + final Table source = newTable( + stringCol("Key", "Alpha", "Alpha", "Alpha", "Bravo", "Bravo", "Charlie", "Charlie"), + intCol("Value", allValues)); + final RollupTable simpleSum = + source.rollup(List.of(AggGroup("Values=Value"), AggFormula("Sum = sum(Value)")), "Key"); + + final String[] arrayWithNull = new String[1]; + final Table keyTable = newTable( + intCol(simpleSum.getRowDepthColumn().name(), 0), + stringCol("Key", arrayWithNull), + byteCol("Action", HierarchicalTable.KEY_TABLE_ACTION_EXPAND_ALL)); + + final HierarchicalTable.SnapshotState ss1 = simpleSum.makeSnapshotState(); + final Table snapshot = + snapshotToTable(simpleSum, ss1, keyTable, ColumnName.of("Action"), null, RowSetFactory.flat(30)); + TableTools.showWithRowSet(snapshot); + + assertTableEquals(TableTools.newTable(intCol(simpleSum.getRowDepthColumn().name(), 1, 2, 2, 2), + booleanCol(simpleSum.getRowExpandedColumn().name(), true, null, null, null), + stringCol("Key", null, "Alpha", "Bravo", "Charlie"), + col("Values", iv(allValues), iv(10, 10, 10), iv(20, 20), iv(30, 30)), longCol("Sum", 130, 30, 40, 60)), + snapshot); + + freeSnapshotTableChunks(snapshot); + } + private static Table initialExpectedGrouped(RollupTable rollup1) { return TableTools.newTable(intCol(rollup1.getRowDepthColumn().name(), 1, 2, 3, 3, 2, 3, 3, 2, 3), booleanCol(rollup1.getRowExpandedColumn().name(), true, true, null, null, true, null, null, From b855a737e43b4b36837d23885510886dd7d29aea Mon Sep 17 00:00:00 2001 From: "Charles P. Wright" Date: Wed, 14 Jan 2026 16:04:35 -0500 Subject: [PATCH 24/24] operator should always be updated for a formula, produce mangled results. --- .../table/impl/by/AggregationProcessor.java | 54 +++++++++++++------ .../by/FormulaMultiColumnChunkedOperator.java | 9 ++++ .../engine/table/impl/TestRollupTable.java | 2 +- 3 files changed, 48 insertions(+), 17 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index e6fdc635974..9812aaf7ae6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -760,10 +760,21 @@ int existingGroupByReaggregateIndex() { return -1; } + /** + * Ensures that the existing GroupByChunkedOperator has the required input/output columns + * + * @param createExtraPairs When true, create all of the pairs for the group by operator. When false, if there + * are any inputs that match the pairs we'll pass the operator through. The AggGroup aggregation can't + * just tack pairs onto an existing operator, because the order would be incorrect. Formulas in a rollup + * don't expose the results of the shared grouping, so just tacking them on is fine. + * @param hideExtras true if the extra pairs should be hidden from results, false otherwise + */ GroupByChunkedOperator ensureGroupByOperator(final QueryTable table, final int existingOperatorIndex, final String exposeRowSetAs, - final MatchPair[] matchPairs) { + final MatchPair[] matchPairs, + final boolean createExtraPairs, + final boolean hideExtras) { boolean recreate = false; final GroupByChunkedOperator existing = (GroupByChunkedOperator) operators.get(existingOperatorIndex); if (exposeRowSetAs != null) { @@ -783,9 +794,17 @@ GroupByChunkedOperator ensureGroupByOperator(final QueryTable table, for (MatchPair matchPair : matchPairs) { final String input = matchPair.input().name(); if (Arrays.stream(existing.getAggregatedColumnPairs()).noneMatch(p -> p.input().name().equals(input))) { + // we didn't have this in the input at all newPairs.add(matchPair); hiddenResults.add(matchPair.output().name()); recreate = true; + } else if (createExtraPairs + && Arrays.stream(existing.getAggregatedColumnPairs()).noneMatch(p -> p.equals(matchPair))) { + newPairs.add(matchPair); + if (hideExtras) { + hiddenResults.add(matchPair.output().name()); + } + recreate = true; } } if (!recreate) { @@ -952,9 +971,11 @@ public void visit(@NotNull final Formula formula) { final int existingGroupByOperatorIndex = existingGroupByOperatorIndex(); if (existingGroupByOperatorIndex >= 0) { // if we have an existing group by operator, then use it (or update it to reflect our input columns) - groupByChunkedOperator = ensureGroupByOperator(table, existingGroupByOperatorIndex, null, makeSymmetricMatchPairs(inputNonKeyColumns)); + groupByChunkedOperator = ensureGroupByOperator(table, existingGroupByOperatorIndex, null, + makeSymmetricMatchPairs(inputNonKeyColumns), false, false); } else { - groupByChunkedOperator = makeGroupByOperatorForFormula(makeSymmetricMatchPairs(inputNonKeyColumns), table, null); + groupByChunkedOperator = + makeGroupByOperatorForFormula(makeSymmetricMatchPairs(inputNonKeyColumns), table, null); } final FormulaMultiColumnChunkedOperator op = new FormulaMultiColumnChunkedOperator(table, @@ -1024,7 +1045,8 @@ public void visit(@NotNull final AggSpecGroup group) { if (existingOperator >= 0) { // Reuse the operator, adding a result extractor for the new result pairs GroupByChunkedOperator existing = - ensureGroupByOperator(table, existingOperator, null, MatchPair.fromPairs(resultPairs)); + ensureGroupByOperator(table, existingOperator, null, MatchPair.fromPairs(resultPairs), false, + false); addNoInputOperator(existing.resultExtractor(resultPairs)); } else { addNoInputOperator( @@ -1152,7 +1174,7 @@ private void maybeInitializeVectorColumns(Set groupByColumnSet, final Ta /** * Convert the array of column names to MatchPairs of the form {@code Col_GRP__ROLLUP__} - * + * * @param cols the columns to convert * @return the mangled name matchpairs */ @@ -1164,7 +1186,7 @@ private void maybeInitializeVectorColumns(Set groupByColumnSet, final Ta /** * Convert the array of strings to MatchPairs of the form Col=Col - * + * * @param columns the columns to convert to MatchPairs * @return an array of MatchPairs */ @@ -1313,7 +1335,7 @@ public void visit(AggSpecGroup group) { if (indexOfExistingOperator >= 0) { // share the existing operator for groupBy in a rollup base final GroupByChunkedOperator existing = ensureGroupByOperator(table, indexOfExistingOperator, - EXPOSED_GROUP_ROW_SETS.name(), MatchPair.fromPairs(resultPairs)); + EXPOSED_GROUP_ROW_SETS.name(), MatchPair.fromPairs(resultPairs), false, false); addNoInputOperator(existing.resultExtractor(resultPairs)); } else { addNoInputOperator(new GroupByChunkedOperator(table, true, EXPOSED_GROUP_ROW_SETS.name(), @@ -1355,8 +1377,9 @@ public void visit(Formula formula) { if (formula.reaggregateAggregatedValues()) { if (existingGroupByOperatorIndex >= 0) { - groupByChunkedOperator = ensureGroupByOperator(table, existingGroupByOperatorIndex, null, mangledMatchPairs); - // TODO: do this better with an existing reaggregated op + groupByChunkedOperator = + ensureGroupByOperator(table, existingGroupByOperatorIndex, null, mangledMatchPairs, true, + true); delegate = false; } else { // When we are reaggregating, we do not expose the rowsets, because the next level creates a @@ -1367,14 +1390,13 @@ public void visit(Formula formula) { } } else { if (existingGroupByOperatorIndex >= 0) { - groupByChunkedOperator = ensureGroupByOperator(table, existingGroupByOperatorIndex, EXPOSED_GROUP_ROW_SETS.name(), mangledMatchPairs); - // TODO: do this better - List asPairs = Arrays.stream(mangledMatchPairs).map(mp -> Pair.of(mp.input(), mp.output())).collect(Collectors.toList()); - addNoInputOperator(groupByChunkedOperator.resultExtractor(asPairs)); + groupByChunkedOperator = ensureGroupByOperator(table, existingGroupByOperatorIndex, + EXPOSED_GROUP_ROW_SETS.name(), mangledMatchPairs, true, false); delegate = false; } else { // When we do not reaggregate, the next level needs access to our exposed group row sets - groupByChunkedOperator = makeGroupByOperatorForFormula(mangledMatchPairs, table, EXPOSED_GROUP_ROW_SETS.name()); + groupByChunkedOperator = + makeGroupByOperatorForFormula(mangledMatchPairs, table, EXPOSED_GROUP_ROW_SETS.name()); addNoInputOperator(groupByChunkedOperator); // we added the operator, so we cannot delegate delegate = false; @@ -1625,7 +1647,7 @@ public void visit(Formula formula) { final int existingIndex = existingGroupByOperatorIndex(); if (existingIndex >= 0) { - groupByOperator = ensureGroupByOperator(table, existingIndex, null, groupPairs); + groupByOperator = ensureGroupByOperator(table, existingIndex, null, groupPairs, true, true); } else { final List hiddenPairs = Arrays.stream(groupPairs).map(mp -> mp.left().name()).collect(Collectors.toList()); @@ -1635,7 +1657,7 @@ public void visit(Formula formula) { // everything gets hidden final FormulaMultiColumnChunkedOperator op = new FormulaMultiColumnChunkedOperator(table, groupByOperator, - true, selectColumn, inputKeyColumns, null, depthSource); + true, selectColumn, inputKeyColumns, renames, depthSource); addOperator(op, null, inputNonKeyColumns); } else { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaMultiColumnChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaMultiColumnChunkedOperator.java index 531ec4c7787..7cdffaba562 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaMultiColumnChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaMultiColumnChunkedOperator.java @@ -24,8 +24,10 @@ import java.util.Arrays; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.function.UnaryOperator; +import java.util.stream.Collectors; import static io.deephaven.engine.table.impl.sources.ArrayBackedColumnSource.BLOCK_SIZE; @@ -252,6 +254,13 @@ public void propagateInitialState(@NotNull final QueryTable resultTable, int sta Arrays.stream(inputKeyColumns).forEach(col -> sourceColumns.put(col, columnSourceMap.get(col))); sourceColumns.put(AggregationProcessor.ROLLUP_FORMULA_DEPTH.name(), formulaDepthSource); } + final List missingColumns = selectColumn.getColumns().stream() + .filter(column -> !sourceColumns.containsKey(column)).collect(Collectors.toList()); + if (!missingColumns.isEmpty()) { + throw new IllegalStateException( + "Columns " + missingColumns + " not found, available columns are: " + sourceColumns.keySet()); + } + selectColumn.initInputs(resultTable.getRowSet(), sourceColumns); formulaDataSource = selectColumn.getDataView(); diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java index 67e1c47f0a2..18b5993f5f5 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestRollupTable.java @@ -458,8 +458,8 @@ public void testRollupFormulaStatic2() { @Test public void testRollupFormulaStatic3() { - testRollupFormulaStatic3(true); testRollupFormulaStatic3(false); + testRollupFormulaStatic3(true); } private void testRollupFormulaStatic3(boolean hasGroup) {