From 326f0a43c5dd6b119af0d1d98b4d8429427d3996 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Thu, 4 Aug 2022 15:25:28 -0700 Subject: [PATCH 001/123] added UpdateByCumulativeOperator class, tests still pass --- .../engine/table/impl/UpdateByOperator.java | 43 ++++--- .../engine/table/impl/ZeroKeyUpdateBy.java | 106 ++++++++++-------- .../impl/updateby/ema/ByteEMAOperator.java | 2 +- .../impl/updateby/ema/IntEMAOperator.java | 2 +- .../impl/updateby/ema/LongEMAOperator.java | 2 +- .../updateby/hashing/UpdateBySlotTracker.java | 2 +- .../internal/BaseByteUpdateByOperator.java | 5 +- .../internal/BaseCharUpdateByOperator.java | 5 +- .../internal/BaseDoubleUpdateByOperator.java | 5 +- .../internal/BaseFloatUpdateByOperator.java | 5 +- .../internal/BaseIntUpdateByOperator.java | 5 +- .../internal/BaseLongUpdateByOperator.java | 5 +- .../internal/BaseObjectUpdateByOperator.java | 5 +- .../internal/BaseShortUpdateByOperator.java | 5 +- .../LongRecordingUpdateByOperator.java | 5 +- 15 files changed, 118 insertions(+), 84 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java index 158ad91cab4..4655c6d4114 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java @@ -89,20 +89,6 @@ static boolean isAppendOnly(@NotNull final TableUpdate update, final long lastKn update.added().firstRowKey() > lastKnownKey; } - /** - * Find the smallest valued key that participated in the upstream {@link TableUpdate}. - * - * @param upstream the update - * @return the smallest key that participated in any part of the update. - */ - static long determineSmallestVisitedKey(@NotNull final TableUpdate upstream, @NotNull final RowSet affected) { - return determineSmallestVisitedKey(upstream.added(), - upstream.modified(), - upstream.removed(), - upstream.shifted(), - affected); - } - /** * Find the smallest valued key that participated in the upstream {@link TableUpdate}. * @@ -113,7 +99,7 @@ static long determineSmallestVisitedKey(@NotNull final TableUpdate upstream, @No * * @return the smallest key that participated in any part of the update. */ - static long determineSmallestVisitedKey(@NotNull final RowSet added, + static long smallestAffectedKey(@NotNull final RowSet added, @NotNull final RowSet modified, @NotNull final RowSet removed, @NotNull final RowSetShiftData shifted, @@ -181,6 +167,33 @@ static long determineSmallestVisitedKey(@NotNull final RowSet added, * updates. */ interface UpdateContext extends SafeCloseable { + /** + * Determine all the rows affected by the {@link TableUpdate} that need to be recomputed + * + * @param upstream the update + * @param source the rowset of the parent table (affected rows will be a subset) + */ + default void determineAffectedRows(@NotNull final TableUpdate upstream, @NotNull final RowSet source, + final boolean upstreamAppendOnly) { + throw (new IllegalStateException("A raw UpdateContext cannot execute determineAffectedRows()")); + } + + /** + * Return the rows computed by the {@Code determineAffectedRows()} + */ + default RowSet getAffectedRows() { + throw (new IllegalStateException("A raw UpdateContext cannot execute getAffectedRows()")); + } + + /** + * Set all the rows that need to be computed by this operator + * + * @param affected the rowset of the parent table (affected is subset) + */ + default void setAffectedRows(@NotNull final RowSet affected) { + throw (new IllegalStateException("A raw UpdateContext cannot execute setAffectedRows()")); + + } } /** diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java index 9a15d6260c0..377cd9f303b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java @@ -127,7 +127,7 @@ private class UpdateContext implements SafeCloseable { /** A Long Chunk for previous keys */ WritableLongChunk prevKeyChunk; - final long smallestModifiedKey; + final RowSet affectedRows; @SuppressWarnings("resource") UpdateContext(@NotNull final TableUpdate upstream, @@ -158,8 +158,6 @@ private class UpdateContext implements SafeCloseable { final boolean upstreamAppendOnly = isInitializeStep || UpdateByOperator.isAppendOnly(upstream, source.getRowSet().lastRowKeyPrev()); - smallestModifiedKey = upstreamAppendOnly ? Long.MAX_VALUE - : UpdateByOperator.determineSmallestVisitedKey(upstream, source.getRowSet()); // noinspection unchecked this.postWorkingChunks = new SizedSafeCloseable[operators.length]; @@ -198,6 +196,20 @@ private class UpdateContext implements SafeCloseable { operators[opIdx].initializeForUpdate(opContext[opIdx], upstream, source.getRowSet(), false, upstreamAppendOnly); } + + // retrieve the affected rows from all operator update contexts + WritableRowSet tmp = RowSetFactory.empty(); + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (!opAffected[opIdx]) { + continue; + } + // trigger the operator to determine its own set of affected rows (window-specific) + opContext[opIdx].determineAffectedRows(upstream, source.getRowSet(), upstreamAppendOnly); + + // union the operator rowsets together to get a global set + tmp.insert(opContext[opIdx].getAffectedRows()); + } + affectedRows = tmp; } public SharedContext getSharedContext() { @@ -267,6 +279,7 @@ void finishFor(@NotNull final UpdateType type) { public void close() { sharedContext.close(); keyChunk.close(); + affectedRows.close(); if (prevKeyChunk != null) { prevKeyChunk.close(); @@ -380,23 +393,23 @@ void doUpdate(@NotNull final RowSet updateRowSet, postWorkingChunks[slotPosition].get(), 0); } else if (type == UpdateType.Reprocess) { - // TODO: When we reprocess rows, we are basically re-adding the entire table starting at the - // lowest key. - // Since every operator might start at a different key, we could try to be efficient and not - // replay - // chunks of rows to operators that don't actually need them. - // - // At the time of writing, any op that reprocesses uses the same logic to decide when, - // so there is no need for fancyness deciding if we need to push this particular set - // of RowSequence through. - prepareValuesChunkFor(opIdx, slotPosition, false, true, chunkOk, null, - null, postWorkingChunks[slotPosition].get(), - null, fillContexts[slotPosition].get()); - currentOp.reprocessChunk(opContext[opIdx], - chunkOk, - keyChunk.get(), - postWorkingChunks[slotPosition].get(), - source.getRowSet()); + // is this chunk relevant to this operator? If so, then intersect and process only the + // relevant rows + if (chunkOk.firstRowKey() <= opContext[opIdx].getAffectedRows().lastRowKey() + && chunkOk.lastRowKey() >= opContext[opIdx].getAffectedRows().firstRowKey()) { + try (final RowSet rs = chunkOk.asRowSet(); + final RowSet intersect = rs.intersect(opContext[opIdx].getAffectedRows())) { + + prepareValuesChunkFor(opIdx, slotPosition, false, true, intersect, intersect, + null, postWorkingChunks[slotPosition].get(), + null, fillContexts[slotPosition].get()); + currentOp.reprocessChunk(opContext[opIdx], + intersect, + keyChunk.get(), + postWorkingChunks[slotPosition].get(), + source.getRowSet()); + } + } } } } @@ -409,43 +422,42 @@ void doUpdate(@NotNull final RowSet updateRowSet, * Locate the smallest key that requires reprocessing and then replay the table from that point */ private void reprocessRows(RowSetShiftData shifted) { + // Get a sub-index of the source from that minimum reprocessing index and make sure we update our // Get a sub-index of the source from that minimum reprocessing index and make sure we update our // contextual chunks and FillContexts to an appropriate size for this step. final RowSet sourceRowSet = source.getRowSet(); - try (final RowSet indexToReprocess = - sourceRowSet.subSetByKeyRange(smallestModifiedKey, sourceRowSet.lastRowKey())) { - final int newChunkSize = (int) Math.min(control.chunkCapacityOrDefault(), indexToReprocess.size()); - setChunkSize(newChunkSize); - - final long keyBefore; - try (final RowSet.SearchIterator sit = sourceRowSet.searchIterator()) { - keyBefore = sit.binarySearchValue( - (compareTo, ignored) -> Long.compare(smallestModifiedKey - 1, compareTo), 1); - } - for (int opRowSet = 0; opRowSet < operators.length; opRowSet++) { - if (opAffected[opRowSet]) { - operators[opRowSet].resetForReprocess(opContext[opRowSet], sourceRowSet, keyBefore); + final int newChunkSize = (int) Math.min(control.chunkCapacityOrDefault(), affectedRows.size()); + setChunkSize(newChunkSize); + + for (int opIndex = 0; opIndex < operators.length; opIndex++) { + if (opAffected[opIndex]) { + final long keyStart = opContext[opIndex].getAffectedRows().firstRowKey(); + final long keyBefore; + try (final RowSet.SearchIterator sit = sourceRowSet.searchIterator()) { + keyBefore = sit.binarySearchValue( + (compareTo, ignored) -> Long.compare(keyStart - 1, compareTo), 1); } + operators[opIndex].resetForReprocess(opContext[opIndex], sourceRowSet, keyBefore); } + } - // We will not mess with shifts if we are using a redirection because we'll have applied the shift - // to the redirection index already by now. - if (rowRedirection == null && shifted.nonempty()) { - try (final RowSet prevIdx = source.getRowSet().copyPrev()) { - shifted.apply((begin, end, delta) -> { - try (final RowSet subRowSet = prevIdx.subSetByKeyRange(begin, end)) { - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - operators[opIdx].applyOutputShift(opContext[opIdx], subRowSet, delta); - } + // We will not mess with shifts if we are using a redirection because we'll have applied the shift + // to the redirection index already by now. + if (rowRedirection == null && shifted.nonempty()) { + try (final RowSet prevIdx = source.getRowSet().copyPrev()) { + shifted.apply((begin, end, delta) -> { + try (final RowSet subRowSet = prevIdx.subSetByKeyRange(begin, end)) { + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + operators[opIdx].applyOutputShift(opContext[opIdx], subRowSet, delta); } - }); - } + } + }); } - - // Now iterate index to reprocess. - doUpdate(indexToReprocess, indexToReprocess, UpdateType.Reprocess); } + + // Now iterate rowset to reprocess. + doUpdate(affectedRows, affectedRows, UpdateType.Reprocess); } /** diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java index 4067dad979c..f8ac5847f43 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java @@ -5,12 +5,12 @@ */ package io.deephaven.engine.table.impl.updateby.ema; +import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ByteChunk; import io.deephaven.chunk.WritableDoubleChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.ColumnSource; -import io.deephaven.api.updateby.OperationControl; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java index 5f391dde775..87620966e84 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java @@ -5,12 +5,12 @@ */ package io.deephaven.engine.table.impl.updateby.ema; +import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.WritableDoubleChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.ColumnSource; -import io.deephaven.api.updateby.OperationControl; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java index d0e35e2dba0..c183adf706c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java @@ -5,12 +5,12 @@ */ package io.deephaven.engine.table.impl.updateby.ema; +import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableDoubleChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.ColumnSource; -import io.deephaven.api.updateby.OperationControl; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/hashing/UpdateBySlotTracker.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/hashing/UpdateBySlotTracker.java index a59f361de5f..eca00aca813 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/hashing/UpdateBySlotTracker.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/hashing/UpdateBySlotTracker.java @@ -174,7 +174,7 @@ private void applyTo(@NotNull final WritableRowSet slotindex, @NotNull final Row wasAppendOnly = false; } - this.smallestModifiedKey = UpdateByOperator.determineSmallestVisitedKey(added, + this.smallestModifiedKey = UpdateByOperator.smallestAffectedKey(added, modified, removed, shifts, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java index 1fd0c385d46..d9682e0e7c7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java @@ -17,6 +17,7 @@ import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.util.RowRedirection; @@ -30,7 +31,7 @@ import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; -public abstract class BaseByteUpdateByOperator implements UpdateByOperator { +public abstract class BaseByteUpdateByOperator extends UpdateByCumulativeOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; protected final MatchPair pair; @@ -49,7 +50,7 @@ public abstract class BaseByteUpdateByOperator implements UpdateByOperator { final byte nullValue; // endregion extra-fields - protected class Context implements UpdateContext { + protected class Context extends UpdateCumulativeContext { public final SizedSafeCloseable fillContext; public final SizedByteChunk outputValues; public boolean canProcessDirectly; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java index 0e21db78eed..fe6c7f1780e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java @@ -12,6 +12,7 @@ import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.util.RowRedirection; @@ -25,7 +26,7 @@ import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; -public abstract class BaseCharUpdateByOperator implements UpdateByOperator { +public abstract class BaseCharUpdateByOperator extends UpdateByCumulativeOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; protected final MatchPair pair; @@ -43,7 +44,7 @@ public abstract class BaseCharUpdateByOperator implements UpdateByOperator { // region extra-fields // endregion extra-fields - protected class Context implements UpdateContext { + protected class Context extends UpdateCumulativeContext { public final SizedSafeCloseable fillContext; public final SizedCharChunk outputValues; public boolean canProcessDirectly; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java index 354860af0ee..3a0b7e6d278 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java @@ -17,6 +17,7 @@ import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.sources.DoubleArraySource; import io.deephaven.engine.table.impl.sources.DoubleSparseArraySource; @@ -33,7 +34,7 @@ import static io.deephaven.util.QueryConstants.NULL_DOUBLE; import static io.deephaven.util.QueryConstants.NULL_LONG; -public abstract class BaseDoubleUpdateByOperator implements UpdateByOperator { +public abstract class BaseDoubleUpdateByOperator extends UpdateByCumulativeOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; @@ -49,7 +50,7 @@ public abstract class BaseDoubleUpdateByOperator implements UpdateByOperator { protected boolean initialized = false; - protected class Context implements UpdateContext { + protected class Context extends UpdateCumulativeContext { public final SizedSafeCloseable fillContext; public final SizedDoubleChunk outputValues; public boolean canProcessDirectly; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java index 60ea52d2548..8e3dc96020d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java @@ -12,6 +12,7 @@ import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.sources.FloatArraySource; import io.deephaven.engine.table.impl.sources.FloatSparseArraySource; @@ -28,7 +29,7 @@ import static io.deephaven.util.QueryConstants.NULL_FLOAT; import static io.deephaven.util.QueryConstants.NULL_LONG; -public abstract class BaseFloatUpdateByOperator implements UpdateByOperator { +public abstract class BaseFloatUpdateByOperator extends UpdateByCumulativeOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; @@ -44,7 +45,7 @@ public abstract class BaseFloatUpdateByOperator implements UpdateByOperator { protected boolean initialized = false; - protected class Context implements UpdateContext { + protected class Context extends UpdateCumulativeContext { public final SizedSafeCloseable fillContext; public final SizedFloatChunk outputValues; public boolean canProcessDirectly; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java index 49871e2dad3..4b5c875a0f2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java @@ -17,6 +17,7 @@ import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.util.RowRedirection; @@ -30,7 +31,7 @@ import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; -public abstract class BaseIntUpdateByOperator implements UpdateByOperator { +public abstract class BaseIntUpdateByOperator extends UpdateByCumulativeOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; protected final MatchPair pair; @@ -48,7 +49,7 @@ public abstract class BaseIntUpdateByOperator implements UpdateByOperator { // region extra-fields // endregion extra-fields - protected class Context implements UpdateContext { + protected class Context extends UpdateCumulativeContext { public final SizedSafeCloseable fillContext; public final SizedIntChunk outputValues; public boolean canProcessDirectly; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java index cad71cddca9..d1b13b1f1ec 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java @@ -17,6 +17,7 @@ import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.util.RowRedirection; @@ -30,7 +31,7 @@ import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; -public abstract class BaseLongUpdateByOperator implements UpdateByOperator { +public abstract class BaseLongUpdateByOperator extends UpdateByCumulativeOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; protected final MatchPair pair; @@ -48,7 +49,7 @@ public abstract class BaseLongUpdateByOperator implements UpdateByOperator { // region extra-fields // endregion extra-fields - protected class Context implements UpdateContext { + protected class Context extends UpdateCumulativeContext { public final SizedSafeCloseable fillContext; public final SizedLongChunk outputValues; public boolean canProcessDirectly; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java index 477c356fe30..e9a9c6fe1f5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java @@ -17,6 +17,7 @@ import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.util.RowRedirection; @@ -30,7 +31,7 @@ import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; -public abstract class BaseObjectUpdateByOperator implements UpdateByOperator { +public abstract class BaseObjectUpdateByOperator extends UpdateByCumulativeOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; protected final MatchPair pair; @@ -49,7 +50,7 @@ public abstract class BaseObjectUpdateByOperator implements UpdateByOperator private final Class colType; // endregion extra-fields - protected class Context implements UpdateContext { + protected class Context extends UpdateCumulativeContext { public final SizedSafeCloseable fillContext; public final SizedObjectChunk outputValues; public boolean canProcessDirectly; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java index 073cf9360eb..233124ffbee 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java @@ -17,6 +17,7 @@ import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.util.RowRedirection; @@ -30,7 +31,7 @@ import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; -public abstract class BaseShortUpdateByOperator implements UpdateByOperator { +public abstract class BaseShortUpdateByOperator extends UpdateByCumulativeOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; protected final MatchPair pair; @@ -48,7 +49,7 @@ public abstract class BaseShortUpdateByOperator implements UpdateByOperator { // region extra-fields // endregion extra-fields - protected class Context implements UpdateContext { + protected class Context extends UpdateCumulativeContext { public final SizedSafeCloseable fillContext; public final SizedShortChunk outputValues; public boolean canProcessDirectly; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/LongRecordingUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/LongRecordingUpdateByOperator.java index 08081ee57f3..13ff966d45c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/LongRecordingUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/LongRecordingUpdateByOperator.java @@ -17,6 +17,7 @@ import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.sources.ReinterpretUtils; import org.jetbrains.annotations.NotNull; @@ -28,7 +29,7 @@ /** * An operator that simply remembers the current chunks during the add and reprocess phases. */ -public class LongRecordingUpdateByOperator implements UpdateByOperator { +public class LongRecordingUpdateByOperator extends UpdateByCumulativeOperator { private final String inputColumnName; private final String[] affectingColumns; private final ColumnSource columnSource; @@ -42,7 +43,7 @@ public LongRecordingUpdateByOperator(@NotNull final String inputColumnName, this.columnSource = ReinterpretUtils.maybeConvertToPrimitive(columnSource); } - private class RecordingContext implements UpdateContext { + private class RecordingContext extends UpdateCumulativeContext { private LongChunk addedChunk; @Override From d8f941ce85a36287c531fa798fd81cf618dbaa15 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Thu, 4 Aug 2022 16:23:54 -0700 Subject: [PATCH 002/123] adding in windowed operators, tests still passing --- .../table/impl/UpdateByOperatorFactory.java | 74 +++++++++++++++++++ .../api/updateby/spec/UpdateBySpec.java | 2 + 2 files changed, 76 insertions(+) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java index 429cf2644d5..32d7ef0856c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java @@ -205,6 +205,29 @@ public Void visit(CumProdSpec p) { return null; } + @Override + public Void visit(@NotNull final RollingSumSpec rs) { + final LongRecordingUpdateByOperator timeStampRecorder; + final boolean isTimeBased = rs.prevTimeScale().isTimeBased(); + final String timestampCol = rs.prevTimeScale().timestampCol(); + + if (isTimeBased) { + timeStampRecorder = makeLongRecordingOperator(source, timestampCol); + ops.add(timeStampRecorder); + } else { + timeStampRecorder = null; + } + + Arrays.stream(pairs) + .filter(p -> !isTimeBased || !p.rightColumn().equals(timestampCol)) + .map(fc -> makeRollingSumOperator(fc, + source, + timeStampRecorder, + rs)) + .forEach(ops::add); + return null; + } + @SuppressWarnings("unchecked") private UpdateByOperator makeEmaOperator(@NotNull final MatchPair pair, @NotNull final TableWithDefaults source, @@ -364,5 +387,56 @@ private UpdateByOperator makeForwardFillOperator(MatchPair fc, TableWithDefaults return new ObjectFillByOperator<>(fc, rowRedirection, csType); } } + + private UpdateByOperator makeRollingSumOperator(@NotNull final MatchPair pair, + @NotNull final TableWithDefaults source, + @Nullable final LongRecordingUpdateByOperator recorder, + @NotNull final RollingSumSpec rs) { + // noinspection rawtypes + final ColumnSource columnSource = source.getColumnSource(pair.rightColumn); + final Class csType = columnSource.getType(); + + final String[] affectingColumns; + if (recorder == null) { + affectingColumns = new String[] {pair.rightColumn}; + } else { + affectingColumns = new String[] {rs.prevTimeScale().timestampCol(), pair.rightColumn}; + } + + // use the correct units from the EmaSpec (depending on was Time or Tick based) + final long prevTimeScaleUnits = rs.prevTimeScale().timescaleUnits(); + final long fwdTimeScaleUnits = rs.fwdTimeScale().timescaleUnits(); + +// if (csType == Boolean.class || csType == boolean.class) { +// return new ByteRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, prevTimeScaleUnits, fwdTimeScaleUnits, +// columnSource, rowRedirection, NULL_BOOLEAN_AS_BYTE); +// } else if (csType == byte.class || csType == Byte.class) { +// return new ByteRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, prevTimeScaleUnits, fwdTimeScaleUnits, +// columnSource, rowRedirection, NULL_BYTE); +// } else if (csType == short.class || csType == Short.class) { +// return new ShortRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, prevTimeScaleUnits, fwdTimeScaleUnits, +// columnSource, rowRedirection); +// } else if (csType == int.class || csType == Integer.class) { +// return new IntRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, prevTimeScaleUnits, fwdTimeScaleUnits, +// columnSource, rowRedirection); +// } else if (csType == long.class || csType == Long.class) { +// return new LongRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, prevTimeScaleUnits, fwdTimeScaleUnits, +// columnSource, rowRedirection); +// } else if (csType == float.class || csType == Float.class) { +// return new FloatRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, prevTimeScaleUnits, fwdTimeScaleUnits, +// columnSource, rowRedirection); +// } else if (csType == double.class || csType == Double.class) { +// return new DoubleRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, +// prevTimeScaleUnits, fwdTimeScaleUnits, columnSource, rowRedirection); +// } else if (csType == BigDecimal.class) { +// return new BigDecimalRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, +// prevTimeScaleUnits, fwdTimeScaleUnits, columnSource, rowRedirection, control.mathContextOrDefault()); +// } else if (csType == BigInteger.class) { +// return new BigIntegerRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, +// prevTimeScaleUnits, fwdTimeScaleUnits, columnSource, rowRedirection); +// } + + throw new IllegalArgumentException("Can not perform RollingSum on type " + csType); + } } } diff --git a/table-api/src/main/java/io/deephaven/api/updateby/spec/UpdateBySpec.java b/table-api/src/main/java/io/deephaven/api/updateby/spec/UpdateBySpec.java index e8fb67759f5..633a1fbd415 100644 --- a/table-api/src/main/java/io/deephaven/api/updateby/spec/UpdateBySpec.java +++ b/table-api/src/main/java/io/deephaven/api/updateby/spec/UpdateBySpec.java @@ -71,6 +71,8 @@ interface Visitor { T visit(CumMinMaxSpec m); T visit(CumProdSpec p); + + T visit(RollingSumSpec p); } // endregion } From 6ddc5ed27755536e817e880253bcd795def353c6 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 5 Aug 2022 15:14:24 -0700 Subject: [PATCH 003/123] wip --- .../table/impl/UpdateByOperatorFactory.java | 54 ++++++++++++++++--- .../LongRecordingUpdateByOperator.java | 4 +- .../replicators/ReplicateUpdateBy.java | 26 +++++++++ .../api/updateby/UpdateByOperation.java | 25 +++++++++ 4 files changed, 99 insertions(+), 10 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java index 32d7ef0856c..6bd1bf379b4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java @@ -11,8 +11,10 @@ import io.deephaven.engine.table.impl.updateby.ema.*; import io.deephaven.engine.table.impl.updateby.fill.*; import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; +import io.deephaven.engine.table.impl.updateby.internal.WindowMangerUpdateByOperator; import io.deephaven.engine.table.impl.updateby.minmax.*; import io.deephaven.engine.table.impl.updateby.prod.*; +import io.deephaven.engine.table.impl.updateby.rollingsum.ShortRollingSumOperator; import io.deephaven.engine.table.impl.updateby.sum.*; import io.deephaven.engine.table.impl.util.WritableRowRedirection; import io.deephaven.time.DateTime; @@ -207,22 +209,28 @@ public Void visit(CumProdSpec p) { @Override public Void visit(@NotNull final RollingSumSpec rs) { - final LongRecordingUpdateByOperator timeStampRecorder; + final WindowMangerUpdateByOperator windowManager; final boolean isTimeBased = rs.prevTimeScale().isTimeBased(); - final String timestampCol = rs.prevTimeScale().timestampCol(); + final String timestampCol = rs.prevTimeScale().timestampCol(); if (isTimeBased) { - timeStampRecorder = makeLongRecordingOperator(source, timestampCol); - ops.add(timeStampRecorder); + windowManager = makeWindowManagerUpdateByOperator(source, + timestampCol, + rs.prevTimeScale().timescaleUnits(), + rs.fwdTimeScale().timescaleUnits()); } else { - timeStampRecorder = null; + windowManager = makeWindowManagerUpdateByOperator(source, + null, + rs.prevTimeScale().timescaleUnits(), + rs.fwdTimeScale().timescaleUnits()); } + ops.add(windowManager); Arrays.stream(pairs) .filter(p -> !isTimeBased || !p.rightColumn().equals(timestampCol)) .map(fc -> makeRollingSumOperator(fc, source, - timeStampRecorder, + windowManager, rs)) .forEach(ops::add); return null; @@ -388,16 +396,42 @@ private UpdateByOperator makeForwardFillOperator(MatchPair fc, TableWithDefaults } } + private WindowMangerUpdateByOperator makeWindowManagerUpdateByOperator(TableWithDefaults source, + String colName, + final long reverseTimeScaleUnits, + final long forwardTimeScaleUnits) { + if (colName == null) { + // not using a time source, this will manage our fixed offset calcs + final String[] inputColumns = Arrays.stream(pairs).map(MatchPair::rightColumn).toArray(String[]::new); + return new WindowMangerUpdateByOperator(colName, inputColumns, reverseTimeScaleUnits, forwardTimeScaleUnits, null); + } else { + final ColumnSource columnSource = source.getColumnSource(colName); + final Class colType = columnSource.getType(); + if (colType != long.class && + colType != Long.class && + colType != DateTime.class && + colType != Instant.class && + !columnSource.allowsReinterpret(long.class)) { + throw new IllegalArgumentException("Column " + colName + " cannot be interpreted as a long"); + } + + final String[] inputColumns = Stream.concat(Stream.of(colName), + Arrays.stream(pairs).map(MatchPair::rightColumn)).toArray(String[]::new); + + return new WindowMangerUpdateByOperator(colName, inputColumns, reverseTimeScaleUnits, forwardTimeScaleUnits, columnSource); + } + } + private UpdateByOperator makeRollingSumOperator(@NotNull final MatchPair pair, @NotNull final TableWithDefaults source, - @Nullable final LongRecordingUpdateByOperator recorder, + @NotNull final WindowMangerUpdateByOperator windowManager, @NotNull final RollingSumSpec rs) { // noinspection rawtypes final ColumnSource columnSource = source.getColumnSource(pair.rightColumn); final Class csType = columnSource.getType(); final String[] affectingColumns; - if (recorder == null) { + if (!windowManager.isTimeBased()) { affectingColumns = new String[] {pair.rightColumn}; } else { affectingColumns = new String[] {rs.prevTimeScale().timestampCol(), pair.rightColumn}; @@ -407,6 +441,10 @@ private UpdateByOperator makeRollingSumOperator(@NotNull final MatchPair pair, final long prevTimeScaleUnits = rs.prevTimeScale().timescaleUnits(); final long fwdTimeScaleUnits = rs.fwdTimeScale().timescaleUnits(); + if (csType == short.class || csType == Short.class) { + return new ShortRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), windowManager, prevTimeScaleUnits, fwdTimeScaleUnits, + columnSource, rowRedirection); + } // if (csType == Boolean.class || csType == boolean.class) { // return new ByteRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, prevTimeScaleUnits, fwdTimeScaleUnits, // columnSource, rowRedirection, NULL_BOOLEAN_AS_BYTE); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/LongRecordingUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/LongRecordingUpdateByOperator.java index 13ff966d45c..8a74ba83326 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/LongRecordingUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/LongRecordingUpdateByOperator.java @@ -29,7 +29,7 @@ /** * An operator that simply remembers the current chunks during the add and reprocess phases. */ -public class LongRecordingUpdateByOperator extends UpdateByCumulativeOperator { +public class LongRecordingUpdateByOperator implements UpdateByOperator { private final String inputColumnName; private final String[] affectingColumns; private final ColumnSource columnSource; @@ -43,7 +43,7 @@ public LongRecordingUpdateByOperator(@NotNull final String inputColumnName, this.columnSource = ReinterpretUtils.maybeConvertToPrimitive(columnSource); } - private class RecordingContext extends UpdateCumulativeContext { + private class RecordingContext implements UpdateContext { private LongChunk addedChunk; @Override diff --git a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java index 7a8961d3a65..911ba17acfe 100644 --- a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java +++ b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java @@ -80,6 +80,32 @@ public static void main(String[] args) throws IOException { replicateNumericOperator( "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java", "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java"); + + + + // Replicate the rolling versions of UpdateByOperations + files = ReplicatePrimitiveCode.charToAllButBooleanAndFloats( + "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java", + exemptions); + for (final String f : files) { + if (f.contains("Int")) { + fixupInteger(f); + } + + if (f.contains("Byte")) { + fixupByteBase(f); + } + } +// objectResult = ReplicatePrimitiveCode.charToObject( +// "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java"); +// fixupStandardObject(objectResult, "BaseWindowedObjectUpdateByOperator", true); +// ReplicatePrimitiveCode.floatToAllFloatingPoints( +// "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java", +// exemptions); +// +// replicateNumericOperator( +// "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java", +// "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java"); } private static void replicateNumericOperator(@NotNull final String shortClass, @NotNull final String floatClass) diff --git a/table-api/src/main/java/io/deephaven/api/updateby/UpdateByOperation.java b/table-api/src/main/java/io/deephaven/api/updateby/UpdateByOperation.java index 6269c471059..5441048abe7 100644 --- a/table-api/src/main/java/io/deephaven/api/updateby/UpdateByOperation.java +++ b/table-api/src/main/java/io/deephaven/api/updateby/UpdateByOperation.java @@ -215,6 +215,31 @@ static UpdateByOperation Ema(OperationControl control, String timestampColumn, D return EmaSpec.ofTime(control, timestampColumn, emaDuration).clause(pairs); } + /** + * Create an {@link RollingSumSpec rolling sum} for the supplied column name pairs, using ticks as + * the windowing unit. Uses the default OperationControl settings. + * + * @param prevTimeTicks the look-behind window size (in rows/ticks) + * @param pairs The input/output column name pairs + * @return The aggregation + */ + static UpdateByOperation RollingSum(long prevTimeTicks, String... pairs) { + return RollingSumSpec.ofTicks(prevTimeTicks).clause(pairs); + } + + /** + * Create an {@link RollingSumSpec rolling sum} for the supplied column name pairs, using ticks as + * the windowing unit. Uses the default OperationControl settings. + * + * @param prevTimeTicks the look-behind window size (in rows/ticks) + * @param fwdTimeTicks the look-ahead window size (in rows/ticks) + * @param pairs The input/output column name pairs + * @return The aggregation + */ + static UpdateByOperation RollingSum(long prevTimeTicks, long fwdTimeTicks, String... pairs) { + return RollingSumSpec.ofTicks(prevTimeTicks, fwdTimeTicks).clause(pairs); + } + T walk(Visitor visitor); interface Visitor { From c8d322460c086cb8804817df17c7256159cff5cd Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 10 Aug 2022 12:15:19 -0700 Subject: [PATCH 004/123] WIP, good save point. ZeroKey working, bucketed not --- .../engine/table/impl/BucketedUpdateBy.java | 7 +- .../deephaven/engine/table/impl/UpdateBy.java | 4 +- .../impl/UpdateByCumulativeOperator.java | 42 ++ .../engine/table/impl/UpdateByOperator.java | 84 ++- .../table/impl/UpdateByOperatorFactory.java | 53 +- .../table/impl/UpdateByWindowedOperator.java | 221 ++++++++ .../engine/table/impl/ZeroKeyUpdateBy.java | 25 +- .../ema/BasePrimitiveEMAOperator.java | 24 +- .../updateby/ema/BigNumberEMAOperator.java | 24 +- .../updateby/fill/BooleanFillByOperator.java | 12 +- .../updateby/fill/ByteFillByOperator.java | 12 +- .../updateby/fill/CharFillByOperator.java | 12 +- .../updateby/fill/DoubleFillByOperator.java | 12 +- .../updateby/fill/FloatFillByOperator.java | 12 +- .../impl/updateby/fill/IntFillByOperator.java | 12 +- .../updateby/fill/LongFillByOperator.java | 12 +- .../updateby/fill/ObjectFillByOperator.java | 12 +- .../updateby/fill/ShortFillByOperator.java | 12 +- .../internal/BaseByteUpdateByOperator.java | 48 +- .../internal/BaseCharUpdateByOperator.java | 43 +- .../internal/BaseDoubleUpdateByOperator.java | 41 +- .../internal/BaseFloatUpdateByOperator.java | 41 +- .../internal/BaseIntUpdateByOperator.java | 43 +- .../internal/BaseLongUpdateByOperator.java | 43 +- .../internal/BaseObjectBinaryOperator.java | 12 +- .../internal/BaseObjectUpdateByOperator.java | 43 +- .../internal/BaseShortUpdateByOperator.java | 43 +- .../BaseWindowedByteUpdateByOperator.java | 412 ++++++++++++++ .../BaseWindowedCharUpdateByOperator.java | 388 ++++++++++++++ .../BaseWindowedIntUpdateByOperator.java | 393 ++++++++++++++ .../BaseWindowedLongUpdateByOperator.java | 393 ++++++++++++++ .../BaseWindowedShortUpdateByOperator.java | 393 ++++++++++++++ .../LongRecordingUpdateByOperator.java | 44 +- .../minmax/ByteCumMinMaxOperator.java | 12 +- .../minmax/DoubleCumMinMaxOperator.java | 12 +- .../minmax/FloatCumMinMaxOperator.java | 12 +- .../updateby/minmax/IntCumMinMaxOperator.java | 12 +- .../minmax/LongCumMinMaxOperator.java | 12 +- .../minmax/ShortCumMinMaxOperator.java | 12 +- .../updateby/prod/ByteCumProdOperator.java | 12 +- .../updateby/prod/DoubleCumProdOperator.java | 12 +- .../updateby/prod/FloatCumProdOperator.java | 12 +- .../updateby/prod/IntCumProdOperator.java | 12 +- .../updateby/prod/LongCumProdOperator.java | 12 +- .../updateby/prod/ShortCumProdOperator.java | 12 +- .../rollingsum/ShortRollingSumOperator.java | 225 ++++++++ .../impl/updateby/sum/ByteCumSumOperator.java | 12 +- .../updateby/sum/DoubleCumSumOperator.java | 12 +- .../updateby/sum/FloatCumSumOperator.java | 12 +- .../impl/updateby/sum/IntCumSumOperator.java | 12 +- .../impl/updateby/sum/LongCumSumOperator.java | 12 +- .../updateby/sum/ShortCumSumOperator.java | 12 +- .../table/impl/updateby/TestRollingSum.java | 504 ++++++++++++++++++ .../replicators/ReplicateUpdateBy.java | 6 + .../api/updateby/spec/RollingSumSpec.java | 125 +++++ 55 files changed, 3549 insertions(+), 499 deletions(-) create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java create mode 100644 engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java create mode 100644 table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedUpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedUpdateBy.java index 1bd6653c2a2..9e84f421231 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedUpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedUpdateBy.java @@ -9,7 +9,6 @@ import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.sized.SizedIntChunk; import io.deephaven.chunk.sized.SizedLongChunk; -import io.deephaven.configuration.Configuration; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.rowset.chunkattributes.RowKeys; @@ -690,7 +689,7 @@ private void accumulateIndexToReprocess(@NotNull final UpdateBySlotTracker.Updat for (int opIdx = 0; opIdx < operators.length; opIdx++) { opAffected[opIdx] |= bucketHasAddedOrRemoved; if (opAffected[opIdx]) { - operators[opIdx].resetForReprocess(opContext[opIdx], slotIndex, tracker.getSlot(), keyBefore); + operators[opIdx].resetForReprocessBucketed(opContext[opIdx], slotIndex, tracker.getSlot(), keyBefore); } } } @@ -808,7 +807,7 @@ public void processBucketedUpdates() { localPermutedWorkingChunk, fillContexts[slotPosition].get()); - operators[opIdx].reprocessChunk(opContext[opIdx], + operators[opIdx].reprocessChunkBucketed(opContext[opIdx], chunkOk, permuteRequired ? localPermutedWorkingChunk : localPostWorkingChunk, permuteRequired ? localPermutedKeyChunk : localKeyChunk, @@ -903,7 +902,7 @@ private void doAppendOnlyAdds(final boolean initialBuild, @NotNull final RowSet localPermutedPostWorkingChunk, fillContexts[slotPosition].get()); - operators[opIdx].addChunk(opContext[opIdx], + operators[opIdx].addChunkBucketed(opContext[opIdx], permuteRequired ? localPermutedPostWorkingChunk : localPostWorkingChunk, permuteRequired ? localPermutedKeys : localKeys, localOutputPositions, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java index 7a92f970842..7c6fa0428f7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java @@ -2,8 +2,6 @@ import gnu.trove.map.hash.TObjectIntHashMap; import io.deephaven.api.ColumnName; -import io.deephaven.api.Selectable; -import io.deephaven.api.Strings; import io.deephaven.api.updateby.UpdateByOperation; import io.deephaven.api.updateby.UpdateByControl; import io.deephaven.chunk.Chunk; @@ -243,7 +241,7 @@ private boolean shiftRedirectedKey(@NotNull final RowSet.SearchIterator iterator public enum UpdateType { /** * Indicates that rows are being - * {@link UpdateByOperator#addChunk(UpdateByOperator.UpdateContext, RowSequence, LongChunk, Chunk, long)} added} + * {@link UpdateByOperator#addChunkBucketed(UpdateByOperator.UpdateContext, RowSequence, LongChunk, Chunk, long)} added} * to the operator. */ Add, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java new file mode 100644 index 00000000000..7e283e1a352 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java @@ -0,0 +1,42 @@ +package io.deephaven.engine.table.impl; + +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.RowSetFactory; +import io.deephaven.engine.table.TableUpdate; +import org.jetbrains.annotations.NotNull; + +public abstract class UpdateByCumulativeOperator implements UpdateByOperator { + + public abstract class UpdateCumulativeContext implements UpdateContext { + // store the current subset of rows that need computation + protected RowSet affectedRows = RowSetFactory.empty(); + + public RowSet determineAffectedRows(@NotNull final TableUpdate upstream, @NotNull final RowSet source, final boolean upstreamAppendOnly) { + if (upstreamAppendOnly) { + // cumulative operators do not need to reprocess any rows on append-only updates + try (final RowSet ignored = affectedRows) { + affectedRows = RowSetFactory.empty(); + } + return affectedRows; + } + + long smallestModifiedKey = UpdateByOperator.smallestAffectedKey(upstream.added(), upstream.modified(), upstream.removed(), upstream.shifted(), source); + + try (final RowSet ignored = affectedRows) { + affectedRows = smallestModifiedKey == Long.MAX_VALUE + ? RowSetFactory.empty() + : source.subSetByKeyRange(smallestModifiedKey, source.lastRowKey()); + } + return affectedRows; + } + + public RowSet getAffectedRows() { + return affectedRows; + } + + @Override + public void close() { + affectedRows.close(); + } + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java index 4655c6d4114..0063f85cc69 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java @@ -51,7 +51,7 @@ *
  • Adds *
      *
    • {@link #initializeFor(UpdateContext, RowSet, UpdateBy.UpdateType)}
    • - *
    • {@link #addChunk(UpdateContext, RowSequence, LongChunk, Chunk, long)}
    • + *
    • {@link #addChunkBucketed(UpdateContext, RowSequence, LongChunk, Chunk, long)}
    • *
    • {@link #finishFor(UpdateContext, UpdateBy.UpdateType)}
    • *
    *
  • @@ -60,7 +60,7 @@ *
      *
    • {@link #resetForReprocess(UpdateContext, RowSet, long)}
    • *
    • {@link #initializeFor(UpdateContext, RowSet, UpdateBy.UpdateType)}
    • - *
    • {@link #reprocessChunk(UpdateContext, RowSequence, Chunk, LongChunk, IntChunk, IntChunk, IntChunk)}
    • + *
    • {@link #reprocessChunkBucketed(UpdateContext, RowSequence, Chunk, LongChunk, IntChunk, IntChunk, IntChunk)}
    • *
    • {@link #finishFor(UpdateContext, UpdateBy.UpdateType)}
    • *
    * @@ -168,32 +168,18 @@ static long smallestAffectedKey(@NotNull final RowSet added, */ interface UpdateContext extends SafeCloseable { /** - * Determine all the rows affected by the {@link TableUpdate} that need to be recomputed + * Determine all the rows affected by the {@link TableUpdate} that need to be reprocessed * * @param upstream the update * @param source the rowset of the parent table (affected rows will be a subset) */ - default void determineAffectedRows(@NotNull final TableUpdate upstream, @NotNull final RowSet source, - final boolean upstreamAppendOnly) { - throw (new IllegalStateException("A raw UpdateContext cannot execute determineAffectedRows()")); - } + RowSet determineAffectedRows(@NotNull final TableUpdate upstream, @NotNull final RowSet source, + final boolean upstreamAppendOnly); /** * Return the rows computed by the {@Code determineAffectedRows()} */ - default RowSet getAffectedRows() { - throw (new IllegalStateException("A raw UpdateContext cannot execute getAffectedRows()")); - } - - /** - * Set all the rows that need to be computed by this operator - * - * @param affected the rowset of the parent table (affected is subset) - */ - default void setAffectedRows(@NotNull final RowSet affected) { - throw (new IllegalStateException("A raw UpdateContext cannot execute setAffectedRows()")); - - } + RowSet getAffectedRows(); } /** @@ -271,11 +257,11 @@ void initializeForUpdate(@NotNull final UpdateContext context, *

    * * @param context the context object - * @param updateIndex the index of rows associated with the update. + * @param updateRowSet the index of rows associated with the update. * @param type the type of update being applied */ void initializeFor(@NotNull final UpdateContext context, - @NotNull final RowSet updateIndex, + @NotNull final RowSet updateRowSet, @NotNull final UpdateBy.UpdateType type); /** @@ -362,10 +348,10 @@ void initializeFor(@NotNull final UpdateContext context, * @param bucketPosition the group position */ void addChunk(@NotNull final UpdateContext context, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk values, - final long bucketPosition); + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk values, + final long bucketPosition); /** * Add a chunk of bucketed items to the operation. @@ -377,12 +363,12 @@ void addChunk(@NotNull final UpdateContext context, * @param runLengths the runLengths of each run of bucket values * @param startPositions the start position of a run within the chunk */ - void addChunk(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths); + void addChunkBucketed(@NotNull final UpdateContext context, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk startPositions, + @NotNull final IntChunk runLengths); /** * Modify a chunk of values with the operation. @@ -444,10 +430,10 @@ void applyOutputShift(@NotNull final UpdateContext context, * @param postUpdateSourceIndex the resulting source index af */ void reprocessChunk(@NotNull final UpdateContext context, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex); + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk valuesChunk, + @NotNull final RowSet postUpdateSourceIndex); /** * Reprocess a chunk of data for a bucketed table. @@ -461,17 +447,17 @@ void reprocessChunk(@NotNull final UpdateContext context, * @param runStartPositions the starting positions of each run within the key and value chunk * @param runLengths the run runLengths of each run in the key and value chunks. Parallel to `runStartPositions` */ - void reprocessChunk(@NotNull final UpdateContext context, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk runStartPositions, - @NotNull final IntChunk runLengths); + void reprocessChunkBucketed(@NotNull final UpdateContext context, + @NotNull final RowSequence inputKeys, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk runStartPositions, + @NotNull final IntChunk runLengths); /** * Reset the operator to the state at the `firstModifiedKey` for non-bucketed operation. This is invoked immediately - * prior to calls to {@link #reprocessChunk(UpdateContext, RowSequence, LongChunk, Chunk, RowSet)}.
    + * prior to calls to {@link #resetForReprocess(UpdateContext, RowSet, long)}.
    *
    * A `firstUnmodifiedKey` of {@link RowSet#NULL_ROW_KEY} indicates that the entire table needs to be recomputed. * @@ -486,14 +472,14 @@ void resetForReprocess(@NotNull final UpdateContext context, /** * Reset the operator to the state at the `firstModifiedKey` for the specified bucket. This is invoked immediately * prior to calls to - * {@link #reprocessChunk(UpdateContext, RowSequence, Chunk, LongChunk, IntChunk, IntChunk, IntChunk)}. + * {@link #reprocessChunkBucketed(UpdateContext, RowSequence, Chunk, LongChunk, IntChunk, IntChunk, IntChunk)}. * * @param context the context object * @param bucketIndex the current index of the specified bucket * @param firstUnmodifiedKey the first unmodified key in the bucket after which we will reprocess rows. */ - void resetForReprocess(@NotNull final UpdateContext context, - @NotNull final RowSet bucketIndex, - final long bucketPosition, - final long firstUnmodifiedKey); + void resetForReprocessBucketed(@NotNull final UpdateContext context, + @NotNull final RowSet bucketIndex, + final long bucketPosition, + final long firstUnmodifiedKey); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java index 6bd1bf379b4..b8003373386 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java @@ -11,7 +11,6 @@ import io.deephaven.engine.table.impl.updateby.ema.*; import io.deephaven.engine.table.impl.updateby.fill.*; import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; -import io.deephaven.engine.table.impl.updateby.internal.WindowMangerUpdateByOperator; import io.deephaven.engine.table.impl.updateby.minmax.*; import io.deephaven.engine.table.impl.updateby.prod.*; import io.deephaven.engine.table.impl.updateby.rollingsum.ShortRollingSumOperator; @@ -209,28 +208,22 @@ public Void visit(CumProdSpec p) { @Override public Void visit(@NotNull final RollingSumSpec rs) { - final WindowMangerUpdateByOperator windowManager; + final LongRecordingUpdateByOperator timeStampRecorder; final boolean isTimeBased = rs.prevTimeScale().isTimeBased(); - final String timestampCol = rs.prevTimeScale().timestampCol(); + if (isTimeBased) { - windowManager = makeWindowManagerUpdateByOperator(source, - timestampCol, - rs.prevTimeScale().timescaleUnits(), - rs.fwdTimeScale().timescaleUnits()); + timeStampRecorder = makeLongRecordingOperator(source, timestampCol); + ops.add(timeStampRecorder); } else { - windowManager = makeWindowManagerUpdateByOperator(source, - null, - rs.prevTimeScale().timescaleUnits(), - rs.fwdTimeScale().timescaleUnits()); + timeStampRecorder = null; } - ops.add(windowManager); Arrays.stream(pairs) .filter(p -> !isTimeBased || !p.rightColumn().equals(timestampCol)) .map(fc -> makeRollingSumOperator(fc, source, - windowManager, + timeStampRecorder, rs)) .forEach(ops::add); return null; @@ -396,54 +389,26 @@ private UpdateByOperator makeForwardFillOperator(MatchPair fc, TableWithDefaults } } - private WindowMangerUpdateByOperator makeWindowManagerUpdateByOperator(TableWithDefaults source, - String colName, - final long reverseTimeScaleUnits, - final long forwardTimeScaleUnits) { - if (colName == null) { - // not using a time source, this will manage our fixed offset calcs - final String[] inputColumns = Arrays.stream(pairs).map(MatchPair::rightColumn).toArray(String[]::new); - return new WindowMangerUpdateByOperator(colName, inputColumns, reverseTimeScaleUnits, forwardTimeScaleUnits, null); - } else { - final ColumnSource columnSource = source.getColumnSource(colName); - final Class colType = columnSource.getType(); - if (colType != long.class && - colType != Long.class && - colType != DateTime.class && - colType != Instant.class && - !columnSource.allowsReinterpret(long.class)) { - throw new IllegalArgumentException("Column " + colName + " cannot be interpreted as a long"); - } - - final String[] inputColumns = Stream.concat(Stream.of(colName), - Arrays.stream(pairs).map(MatchPair::rightColumn)).toArray(String[]::new); - - return new WindowMangerUpdateByOperator(colName, inputColumns, reverseTimeScaleUnits, forwardTimeScaleUnits, columnSource); - } - } - private UpdateByOperator makeRollingSumOperator(@NotNull final MatchPair pair, @NotNull final TableWithDefaults source, - @NotNull final WindowMangerUpdateByOperator windowManager, + @Nullable final LongRecordingUpdateByOperator recorder, @NotNull final RollingSumSpec rs) { // noinspection rawtypes final ColumnSource columnSource = source.getColumnSource(pair.rightColumn); final Class csType = columnSource.getType(); final String[] affectingColumns; - if (!windowManager.isTimeBased()) { + if (recorder == null) { affectingColumns = new String[] {pair.rightColumn}; } else { affectingColumns = new String[] {rs.prevTimeScale().timestampCol(), pair.rightColumn}; } - // use the correct units from the EmaSpec (depending on was Time or Tick based) final long prevTimeScaleUnits = rs.prevTimeScale().timescaleUnits(); final long fwdTimeScaleUnits = rs.fwdTimeScale().timescaleUnits(); if (csType == short.class || csType == Short.class) { - return new ShortRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), windowManager, prevTimeScaleUnits, fwdTimeScaleUnits, - columnSource, rowRedirection); + return new ShortRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, prevTimeScaleUnits, fwdTimeScaleUnits, columnSource, rowRedirection); } // if (csType == Boolean.class || csType == boolean.class) { // return new ByteRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, prevTimeScaleUnits, fwdTimeScaleUnits, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java new file mode 100644 index 00000000000..0760b44c8f8 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java @@ -0,0 +1,221 @@ +package io.deephaven.engine.table.impl; + +import io.deephaven.api.updateby.OperationControl; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.RowSetBuilderRandom; +import io.deephaven.engine.rowset.RowSetFactory; +import io.deephaven.engine.rowset.WritableRowSet; +import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; +import io.deephaven.engine.table.impl.util.RowRedirection; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.util.Arrays; +import java.util.List; + +public abstract class UpdateByWindowedOperator implements UpdateByOperator { + protected final OperationControl control; + protected final LongRecordingUpdateByOperator recorder; + protected final long reverseTimeScaleUnits; + protected final long forwardTimeScaleUnits; + + protected final MatchPair pair; + protected final String[] affectingColumns; + + protected final boolean isRedirected; + + + public abstract class UpdateWindowedContext implements UpdateContext { + // store the current subset of rows that need computation + protected RowSet affectedRows = RowSetFactory.empty(); + + /*** + * This function is only correct if the proper {@code source} rowset is provided. If using buckets, then the + * provided rowset must be limited to the rows in the current bucket + * only + * + * @param upstream the update + * @param source the rowset of the parent table (affected rows will be a subset) + * @param upstreamAppendOnly + */ + public RowSet determineAffectedRows(@NotNull final TableUpdate upstream, @NotNull final RowSet source, + final boolean upstreamAppendOnly) { + + // NOTE: this is fast rather than bounding to the smallest set possible (i.e. unaware of buckets and + // over-represents sparse data). Will result in computing far more than actually necessary + + // TODO: return the minimal set of data for this update + + RowSetBuilderRandom builder = RowSetFactory.builderRandom(); + + if (upstream.removed().isNonempty()) { + // need removes in post-shift space to determine rows to recompute + try (final WritableRowSet shiftedRemoves = upstream.removed().copy()) { + upstream.shifted().apply(shiftedRemoves); + + builder.addRange(computeFirstAffectedKey(shiftedRemoves.firstRowKey(), source), + computeLastAffectedKey(shiftedRemoves.lastRowKey(), source)); + } + } + + if (upstream.added().isNonempty()) { + // all the new rows need computed + builder.addRowSet(upstream.added()); + + // add the rows affected by the adds + builder.addRange(computeFirstAffectedKey(upstream.added().firstRowKey(), source), + computeLastAffectedKey(upstream.added().lastRowKey(), source)); + } + + if (upstream.modified().isNonempty()) { + // TODO: make this more efficient + final List cols = List.of(affectingColumns); + boolean modifiedAffectingColumn = Arrays.stream(upstream.modifiedColumnSet().dirtyColumnNames()).anyMatch(cols::contains); + + if (modifiedAffectingColumn) { + // add the rows affected by the mods + builder.addRange(computeFirstAffectedKey(upstream.modified().firstRowKey(), source), + computeLastAffectedKey(upstream.modified().lastRowKey(), source)); + } + } + + try (final RowSet ignored = affectedRows) { + affectedRows = builder.build(); + } + return affectedRows; + } + + public RowSet getAffectedRows() { + return affectedRows; + } + + @Override + public void close() { + affectedRows.close(); + } + } + + /** + * An operator that computes a windowed operation from a column + * + * @param pair the {@link MatchPair} that defines the input/output for this operation + * @param affectingColumns the names of the columns that affect this operation + * @param control the control parameters for operation + * @param timeRecorder an optional recorder for a timestamp column. If this is null, it will be assumed time is + * measured in integer ticks. + * @param rowRedirection the row redirection to use for the operation + */ + public UpdateByWindowedOperator(@NotNull final MatchPair pair, + @NotNull final String[] affectingColumns, + @NotNull final OperationControl control, + @Nullable final LongRecordingUpdateByOperator timeRecorder, + final long reverseTimeScaleUnits, + final long forwardTimeScaleUnits, + @Nullable final RowRedirection rowRedirection) { + this.pair = pair; + this.affectingColumns = affectingColumns; + this.control = control; + this.recorder = timeRecorder; + this.reverseTimeScaleUnits = reverseTimeScaleUnits; + this.forwardTimeScaleUnits = forwardTimeScaleUnits; + this.isRedirected = rowRedirection != null; + } + + // return the first row that affects this key + public long computeFirstAffectingKey(long key, @NotNull final RowSet source) { + + if (recorder == null) { + // ticks + final long keyPos = source.find(key); + final long idx = keyPos - (long) reverseTimeScaleUnits + 1; + if (idx < 0) { + return source.firstRowKey(); + } + return source.get(idx); + } + return -1; + } + + // return the last row that affects this key + public long computeLastAffectingKey(long key, @NotNull final RowSet source) { + if (recorder == null) { + // ticks + final long keyPos = source.find(key); + final long idx = keyPos + (long)forwardTimeScaleUnits; + if (idx >= source.size()) { + return source.lastRowKey(); + } + return source.get(idx); + } + return -1; + } + + // return the first row affected by this key + public long computeFirstAffectedKey(long key, @NotNull final RowSet source) { + if (recorder == null) { + // ticks + final long keyPos = source.find(key); + final long idx = keyPos - (long) forwardTimeScaleUnits; + if (idx >= source.size()) { + return source.lastRowKey(); + } + return source.get(idx); + } + return -1; + } + + // return the last row affected by this key + public long computeLastAffectedKey(long key, @NotNull final RowSet source) { + if (recorder == null) { + // ticks + final long keyPos = source.find(key); + final long idx = keyPos + (long) reverseTimeScaleUnits; + if (idx >= source.size()) { + return source.lastRowKey(); + } + return source.get(idx); + } + return -1; + } + + /*** + * Windowed operators must always reprocess + */ + @Override + public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { + return false; + } + + @NotNull + @Override + public String getInputColumnName() { + return pair.rightColumn; + } + + @NotNull + @Override + public String[] getAffectingColumnNames() { + return affectingColumns; + } + + @NotNull + @Override + public String[] getOutputColumnNames() { + return new String[] { pair.leftColumn }; + } + + @Override + public boolean requiresKeys() { + return true; + } + + @Override + public void setBucketCapacity(int capacity) { + } + + @Override + public void onBucketsRemoved(@NotNull final RowSet removedBuckets) { + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java index 377cd9f303b..28412dfb88c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java @@ -183,8 +183,7 @@ private class UpdateContext implements SafeCloseable { postWorkingChunks[slotPosition].ensureCapacity(chunkSize); // Note that these don't participate in setChunkSize() because nothing will use them. If that - // changes - // then these must also become SizedSafeCloseables. + // changes then these must also become SizedSafeCloseables. if (hasModifies) { prevFillContexts[slotPosition] = inputSources[opIdx].makeFillContext(chunkSize, prevSharedContext); @@ -203,11 +202,12 @@ private class UpdateContext implements SafeCloseable { if (!opAffected[opIdx]) { continue; } - // trigger the operator to determine its own set of affected rows (window-specific) - opContext[opIdx].determineAffectedRows(upstream, source.getRowSet(), upstreamAppendOnly); + // trigger the operator to determine its own set of affected rows (window-specific), do not close() + // since this is managed by the operator context + final RowSet rs = opContext[opIdx].determineAffectedRows(upstream, source.getRowSet(), upstreamAppendOnly); // union the operator rowsets together to get a global set - tmp.insert(opContext[opIdx].getAffectedRows()); + tmp.insert(rs); } affectedRows = tmp; } @@ -422,7 +422,6 @@ void doUpdate(@NotNull final RowSet updateRowSet, * Locate the smallest key that requires reprocessing and then replay the table from that point */ private void reprocessRows(RowSetShiftData shifted) { - // Get a sub-index of the source from that minimum reprocessing index and make sure we update our // Get a sub-index of the source from that minimum reprocessing index and make sure we update our // contextual chunks and FillContexts to an appropriate size for this step. final RowSet sourceRowSet = source.getRowSet(); @@ -463,12 +462,12 @@ private void reprocessRows(RowSetShiftData shifted) { /** * Prepare the specified chunk for use. * - * @param opRowSet the operator index + * @param opIdx the operator index * @param usePrev if previous values should be fetched * @param chunkOk the {@link RowSequence} for current values * @param prevChunkOk the {@link RowSequence} for previous values. */ - private void prepareValuesChunkFor(final int opRowSet, + private void prepareValuesChunkFor(final int opIdx, final int inputSlot, final boolean usePrev, final boolean useCurrent, @@ -478,20 +477,20 @@ private void prepareValuesChunkFor(final int opRowSet, final WritableChunk postWorkingChunk, final ChunkSource.FillContext prevFillContext, final ChunkSource.FillContext postFillContext) { - if (!operators[opRowSet].requiresValues(opContext[opRowSet])) { + if (!operators[opIdx].requiresValues(opContext[opIdx])) { return; } if (!inputChunkPopulated[inputSlot]) { - // Using opRowSet below is OK, because if we are sharing an input slot, we are referring to the same - // input source. Instead of maintaining another array of sourced by slot, we'll just use the opRowSet + // Using opIdx below is OK, because if we are sharing an input slot, we are referring to the same + // input source. Instead of maintaining another array of sourced by slot, we'll just use the opIdx inputChunkPopulated[inputSlot] = true; if (usePrev) { - inputSources[opRowSet].fillPrevChunk(prevFillContext, prevWorkingChunk, prevChunkOk); + inputSources[opIdx].fillPrevChunk(prevFillContext, prevWorkingChunk, prevChunkOk); } if (useCurrent) { - inputSources[opRowSet].fillChunk(postFillContext, postWorkingChunk, chunkOk); + inputSources[opIdx].fillChunk(postFillContext, postWorkingChunk, chunkOk); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java index 5dec7ff97bd..273d71da9d8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java @@ -115,9 +115,9 @@ public void initializeForUpdate(@NotNull final UpdateByOperator.UpdateContext co @Override public void initializeFor(@NotNull final UpdateByOperator.UpdateContext updateContext, - @NotNull final RowSet updateIndex, + @NotNull final RowSet updateRowSet, @NotNull final UpdateBy.UpdateType type) { - super.initializeFor(updateContext, updateIndex, type); + super.initializeFor(updateContext, updateRowSet, type); ((EmaContext) updateContext).lastStamp = NULL_LONG; } @@ -159,12 +159,12 @@ protected void doAddChunk(@NotNull final Context context, } @Override - public void addChunk(@NotNull final UpdateByOperator.UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { + public void addChunkBucketed(@NotNull final UpdateByOperator.UpdateContext context, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk startPositions, + @NotNull final IntChunk runLengths) { final EmaContext ctx = (EmaContext) context; for (int runIdx = 0; runIdx < startPositions.size(); runIdx++) { final int runStart = startPositions.get(runIdx); @@ -220,10 +220,10 @@ public void resetForReprocess(@NotNull final UpdateByOperator.UpdateContext cont } @Override - public void resetForReprocess(@NotNull final UpdateByOperator.UpdateContext ctx, - @NotNull final RowSet bucketIndex, - final long bucketPosition, - final long firstUnmodifiedKey) { + public void resetForReprocessBucketed(@NotNull final UpdateByOperator.UpdateContext ctx, + @NotNull final RowSet bucketIndex, + final long bucketPosition, + final long firstUnmodifiedKey) { final double previousVal = firstUnmodifiedKey == NULL_ROW_KEY ? QueryConstants.NULL_DOUBLE : outputSource.getDouble(firstUnmodifiedKey); bucketLastVal.set(bucketPosition, previousVal); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java index a0b93df7f53..747934a1d75 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java @@ -126,9 +126,9 @@ public void initializeForUpdate(@NotNull UpdateContext context, @SuppressWarnings("unchecked") @Override public void initializeFor(@NotNull final UpdateContext updateContext, - @NotNull final RowSet updateIndex, + @NotNull final RowSet updateRowSet, @NotNull final UpdateBy.UpdateType type) { - super.initializeFor(updateContext, updateIndex, type); + super.initializeFor(updateContext, updateRowSet, type); ((EmaContext) updateContext).lastStamp = NULL_LONG; } @@ -161,12 +161,12 @@ protected void doAddChunk(@NotNull final Context updateContext, } @Override - public void addChunk(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { + public void addChunkBucketed(@NotNull final UpdateContext context, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk startPositions, + @NotNull final IntChunk runLengths) { final ObjectChunk asObjects = values.asObjectChunk(); // noinspection unchecked final EmaContext ctx = (EmaContext) context; @@ -223,10 +223,10 @@ public void resetForReprocess(@NotNull final UpdateContext context, } @Override - public void resetForReprocess(@NotNull final UpdateContext ctx, - @NotNull final RowSet bucketIndex, - final long bucketPosition, - final long firstUnmodifiedKey) { + public void resetForReprocessBucketed(@NotNull final UpdateContext ctx, + @NotNull final RowSet bucketIndex, + final long bucketPosition, + final long firstUnmodifiedKey) { final BigDecimal previousVal = firstUnmodifiedKey == NULL_ROW_KEY ? null : outputSource.get(firstUnmodifiedKey); bucketLastVal.set(bucketPosition, previousVal); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java index 8caedbe4f9e..b6525a5c469 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java @@ -63,12 +63,12 @@ public Map> getOutputColumns() { // endregion extra-methods @Override - public void addChunk(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { + public void addChunkBucketed(@NotNull final UpdateContext context, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk startPositions, + @NotNull final IntChunk runLengths) { final ByteChunk asBooleans = values.asByteChunk(); final Context ctx = (Context) context; for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java index 918886a02ac..ccf2dfbc3f7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java @@ -37,12 +37,12 @@ public ByteFillByOperator(@NotNull final MatchPair fillPair, // endregion extra-methods @Override - public void addChunk(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { + public void addChunkBucketed(@NotNull final UpdateContext context, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk startPositions, + @NotNull final IntChunk runLengths) { final ByteChunk asBytes = values.asByteChunk(); final Context ctx = (Context) context; for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java index c5d03ac7cde..8e78172c311 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java @@ -32,12 +32,12 @@ public CharFillByOperator(@NotNull final MatchPair fillPair, // endregion extra-methods @Override - public void addChunk(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { + public void addChunkBucketed(@NotNull final UpdateContext context, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk startPositions, + @NotNull final IntChunk runLengths) { final CharChunk asChars = values.asCharChunk(); final Context ctx = (Context) context; for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java index 9d836da5ab2..9664c60eab9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java @@ -37,12 +37,12 @@ public DoubleFillByOperator(@NotNull final MatchPair fillPair, // endregion extra-methods @Override - public void addChunk(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { + public void addChunkBucketed(@NotNull final UpdateContext context, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk startPositions, + @NotNull final IntChunk runLengths) { final DoubleChunk asDoubles = values.asDoubleChunk(); final Context ctx = (Context) context; for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java index c2dd0a4303a..5c91e5596fc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java @@ -37,12 +37,12 @@ public FloatFillByOperator(@NotNull final MatchPair fillPair, // endregion extra-methods @Override - public void addChunk(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { + public void addChunkBucketed(@NotNull final UpdateContext context, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk startPositions, + @NotNull final IntChunk runLengths) { final FloatChunk asFloats = values.asFloatChunk(); final Context ctx = (Context) context; for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java index 6b14668b53e..0a508c13678 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java @@ -37,12 +37,12 @@ public IntFillByOperator(@NotNull final MatchPair fillPair, // endregion extra-methods @Override - public void addChunk(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { + public void addChunkBucketed(@NotNull final UpdateContext context, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk startPositions, + @NotNull final IntChunk runLengths) { final IntChunk asInts = values.asIntChunk(); final Context ctx = (Context) context; for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java index c53d0a64217..366c7d718e7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java @@ -58,12 +58,12 @@ public Map> getOutputColumns() { // endregion extra-methods @Override - public void addChunk(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { + public void addChunkBucketed(@NotNull final UpdateContext context, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk startPositions, + @NotNull final IntChunk runLengths) { final LongChunk asLongs = values.asLongChunk(); final Context ctx = (Context) context; for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java index 8daf5b4e2b2..4b530b4b863 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java @@ -37,12 +37,12 @@ public ObjectFillByOperator(@NotNull final MatchPair fillPair, // endregion extra-methods @Override - public void addChunk(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { + public void addChunkBucketed(@NotNull final UpdateContext context, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk startPositions, + @NotNull final IntChunk runLengths) { final ObjectChunk asObjects = values.asObjectChunk(); final Context ctx = (Context) context; for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java index d895abda7e0..28be54315bb 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java @@ -37,12 +37,12 @@ public ShortFillByOperator(@NotNull final MatchPair fillPair, // endregion extra-methods @Override - public void addChunk(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { + public void addChunkBucketed(@NotNull final UpdateContext context, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk startPositions, + @NotNull final IntChunk runLengths) { final ShortChunk asShorts = values.asShortChunk(); final Context ctx = (Context) context; for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java index d9682e0e7c7..132298c3a96 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java @@ -5,6 +5,11 @@ */ package io.deephaven.engine.table.impl.updateby.internal; +import io.deephaven.util.QueryConstants; +import io.deephaven.engine.table.impl.sources.ByteArraySource; +import io.deephaven.engine.table.impl.sources.ByteSparseArraySource; +import io.deephaven.engine.table.WritableColumnSource; + import io.deephaven.chunk.Chunk; import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.LongChunk; @@ -18,7 +23,6 @@ import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; -import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.util.RowRedirection; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; @@ -205,7 +209,7 @@ public void initializeForUpdate(@NotNull final UpdateContext context, @Override public void initializeFor(@NotNull final UpdateContext updateContext, - @NotNull final RowSet updateIndex, + @NotNull final RowSet updateRowSet, @NotNull final UpdateBy.UpdateType type) { ((Context)updateContext).currentUpdateType = type; ((Context)updateContext).curVal = nullValue; @@ -268,10 +272,10 @@ public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { // region Addition @Override public void addChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk values, - long bucketPosition) { + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk values, + long bucketPosition) { final Context ctx = (Context) updateContext; if (ctx.canProcessDirectly) { doAddChunk(ctx, inputKeys, values, bucketPosition); @@ -319,34 +323,34 @@ public void resetForReprocess(@NotNull final UpdateContext context, @Override - public void resetForReprocess(@NotNull final UpdateContext ctx, - @NotNull final RowSet bucketIndex, - final long bucketPosition, - final long firstUnmodifiedKey) { + public void resetForReprocessBucketed(@NotNull final UpdateContext ctx, + @NotNull final RowSet bucketIndex, + final long bucketPosition, + final long firstUnmodifiedKey) { final byte previousVal = firstUnmodifiedKey == NULL_ROW_KEY ? nullValue : outputSource.getByte(firstUnmodifiedKey); bucketLastVal.set(bucketPosition, previousVal); } @Override public void reprocessChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex) { + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk valuesChunk, + @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; doAddChunk(ctx, inputKeys, valuesChunk, 0); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } @Override - public void reprocessChunk(@NotNull UpdateContext updateContext, - @NotNull final RowSequence chunkOk, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk runStartPositions, - @NotNull final IntChunk runLengths) { - addChunk(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); + public void reprocessChunkBucketed(@NotNull UpdateContext updateContext, + @NotNull final RowSequence chunkOk, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk runStartPositions, + @NotNull final IntChunk runLengths) { + addChunkBucketed(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); ((Context)updateContext).getModifiedBuilder().appendRowSequence(chunkOk); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java index fe6c7f1780e..217e2324016 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java @@ -13,7 +13,6 @@ import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; -import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.util.RowRedirection; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; @@ -186,7 +185,7 @@ public void initializeForUpdate(@NotNull final UpdateContext context, @Override public void initializeFor(@NotNull final UpdateContext updateContext, - @NotNull final RowSet updateIndex, + @NotNull final RowSet updateRowSet, @NotNull final UpdateBy.UpdateType type) { ((Context)updateContext).currentUpdateType = type; ((Context)updateContext).curVal = QueryConstants.NULL_CHAR; @@ -249,10 +248,10 @@ public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { // region Addition @Override public void addChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk values, - long bucketPosition) { + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk values, + long bucketPosition) { final Context ctx = (Context) updateContext; if (ctx.canProcessDirectly) { doAddChunk(ctx, inputKeys, values, bucketPosition); @@ -298,34 +297,34 @@ public void resetForReprocess(@NotNull final UpdateContext context, @Override - public void resetForReprocess(@NotNull final UpdateContext ctx, - @NotNull final RowSet bucketIndex, - final long bucketPosition, - final long firstUnmodifiedKey) { + public void resetForReprocessBucketed(@NotNull final UpdateContext ctx, + @NotNull final RowSet bucketIndex, + final long bucketPosition, + final long firstUnmodifiedKey) { final char previousVal = firstUnmodifiedKey == NULL_ROW_KEY ? QueryConstants.NULL_CHAR : outputSource.getChar(firstUnmodifiedKey); bucketLastVal.set(bucketPosition, previousVal); } @Override public void reprocessChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex) { + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk valuesChunk, + @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; doAddChunk(ctx, inputKeys, valuesChunk, 0); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } @Override - public void reprocessChunk(@NotNull UpdateContext updateContext, - @NotNull final RowSequence chunkOk, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk runStartPositions, - @NotNull final IntChunk runLengths) { - addChunk(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); + public void reprocessChunkBucketed(@NotNull UpdateContext updateContext, + @NotNull final RowSequence chunkOk, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk runStartPositions, + @NotNull final IntChunk runLengths) { + addChunkBucketed(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); ((Context)updateContext).getModifiedBuilder().appendRowSequence(chunkOk); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java index 3a0b7e6d278..5850d36c7e3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java @@ -18,7 +18,6 @@ import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; -import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.sources.DoubleArraySource; import io.deephaven.engine.table.impl.sources.DoubleSparseArraySource; import io.deephaven.engine.table.impl.sources.WritableRedirectedColumnSource; @@ -238,10 +237,10 @@ public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { // region Addition @Override public void addChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk values, - long bucketPosition) { + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk values, + long bucketPosition) { final Context ctx = (Context) updateContext; if (ctx.canProcessDirectly) { doAddChunk(ctx, inputKeys, values, bucketPosition); @@ -278,34 +277,34 @@ public void resetForReprocess(@NotNull final UpdateContext context, @Override - public void resetForReprocess(@NotNull final UpdateContext ctx, - @NotNull final RowSet bucketRowSet, - final long bucketPosition, - final long firstUnmodifiedKey) { + public void resetForReprocessBucketed(@NotNull final UpdateContext ctx, + @NotNull final RowSet bucketRowSet, + final long bucketPosition, + final long firstUnmodifiedKey) { final double previousVal = firstUnmodifiedKey == NULL_ROW_KEY ? NULL_DOUBLE : outputSource.getDouble(firstUnmodifiedKey); bucketLastVal.set(bucketPosition, previousVal); } @Override public void reprocessChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceRowSet) { + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk valuesChunk, + @NotNull final RowSet postUpdateSourceRowSet) { final Context ctx = (Context) updateContext; doAddChunk(ctx, inputKeys, valuesChunk, 0); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } @Override - public void reprocessChunk(@NotNull UpdateContext updateContext, - @NotNull final RowSequence chunkOk, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk runStartPositions, - @NotNull final IntChunk runLengths) { - addChunk(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); + public void reprocessChunkBucketed(@NotNull UpdateContext updateContext, + @NotNull final RowSequence chunkOk, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk runStartPositions, + @NotNull final IntChunk runLengths) { + addChunkBucketed(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); ((Context)updateContext).getModifiedBuilder().appendRowSequence(chunkOk); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java index 8e3dc96020d..6654c974852 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java @@ -13,7 +13,6 @@ import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; -import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.sources.FloatArraySource; import io.deephaven.engine.table.impl.sources.FloatSparseArraySource; import io.deephaven.engine.table.impl.sources.WritableRedirectedColumnSource; @@ -233,10 +232,10 @@ public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { // region Addition @Override public void addChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk values, - long bucketPosition) { + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk values, + long bucketPosition) { final Context ctx = (Context) updateContext; if (ctx.canProcessDirectly) { doAddChunk(ctx, inputKeys, values, bucketPosition); @@ -273,34 +272,34 @@ public void resetForReprocess(@NotNull final UpdateContext context, @Override - public void resetForReprocess(@NotNull final UpdateContext ctx, - @NotNull final RowSet bucketRowSet, - final long bucketPosition, - final long firstUnmodifiedKey) { + public void resetForReprocessBucketed(@NotNull final UpdateContext ctx, + @NotNull final RowSet bucketRowSet, + final long bucketPosition, + final long firstUnmodifiedKey) { final float previousVal = firstUnmodifiedKey == NULL_ROW_KEY ? NULL_FLOAT : outputSource.getFloat(firstUnmodifiedKey); bucketLastVal.set(bucketPosition, previousVal); } @Override public void reprocessChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceRowSet) { + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk valuesChunk, + @NotNull final RowSet postUpdateSourceRowSet) { final Context ctx = (Context) updateContext; doAddChunk(ctx, inputKeys, valuesChunk, 0); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } @Override - public void reprocessChunk(@NotNull UpdateContext updateContext, - @NotNull final RowSequence chunkOk, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk runStartPositions, - @NotNull final IntChunk runLengths) { - addChunk(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); + public void reprocessChunkBucketed(@NotNull UpdateContext updateContext, + @NotNull final RowSequence chunkOk, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk runStartPositions, + @NotNull final IntChunk runLengths) { + addChunkBucketed(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); ((Context)updateContext).getModifiedBuilder().appendRowSequence(chunkOk); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java index 4b5c875a0f2..7f8ffa847a7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java @@ -18,7 +18,6 @@ import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; -import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.util.RowRedirection; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; @@ -191,7 +190,7 @@ public void initializeForUpdate(@NotNull final UpdateContext context, @Override public void initializeFor(@NotNull final UpdateContext updateContext, - @NotNull final RowSet updateIndex, + @NotNull final RowSet updateRowSet, @NotNull final UpdateBy.UpdateType type) { ((Context)updateContext).currentUpdateType = type; ((Context)updateContext).curVal = QueryConstants.NULL_INT; @@ -254,10 +253,10 @@ public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { // region Addition @Override public void addChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk values, - long bucketPosition) { + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk values, + long bucketPosition) { final Context ctx = (Context) updateContext; if (ctx.canProcessDirectly) { doAddChunk(ctx, inputKeys, values, bucketPosition); @@ -303,34 +302,34 @@ public void resetForReprocess(@NotNull final UpdateContext context, @Override - public void resetForReprocess(@NotNull final UpdateContext ctx, - @NotNull final RowSet bucketIndex, - final long bucketPosition, - final long firstUnmodifiedKey) { + public void resetForReprocessBucketed(@NotNull final UpdateContext ctx, + @NotNull final RowSet bucketIndex, + final long bucketPosition, + final long firstUnmodifiedKey) { final int previousVal = firstUnmodifiedKey == NULL_ROW_KEY ? QueryConstants.NULL_INT : outputSource.getInt(firstUnmodifiedKey); bucketLastVal.set(bucketPosition, previousVal); } @Override public void reprocessChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex) { + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk valuesChunk, + @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; doAddChunk(ctx, inputKeys, valuesChunk, 0); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } @Override - public void reprocessChunk(@NotNull UpdateContext updateContext, - @NotNull final RowSequence chunkOk, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk runStartPositions, - @NotNull final IntChunk runLengths) { - addChunk(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); + public void reprocessChunkBucketed(@NotNull UpdateContext updateContext, + @NotNull final RowSequence chunkOk, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk runStartPositions, + @NotNull final IntChunk runLengths) { + addChunkBucketed(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); ((Context)updateContext).getModifiedBuilder().appendRowSequence(chunkOk); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java index d1b13b1f1ec..db99a87c21e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java @@ -18,7 +18,6 @@ import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; -import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.util.RowRedirection; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; @@ -191,7 +190,7 @@ public void initializeForUpdate(@NotNull final UpdateContext context, @Override public void initializeFor(@NotNull final UpdateContext updateContext, - @NotNull final RowSet updateIndex, + @NotNull final RowSet updateRowSet, @NotNull final UpdateBy.UpdateType type) { ((Context)updateContext).currentUpdateType = type; ((Context)updateContext).curVal = QueryConstants.NULL_LONG; @@ -254,10 +253,10 @@ public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { // region Addition @Override public void addChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk values, - long bucketPosition) { + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk values, + long bucketPosition) { final Context ctx = (Context) updateContext; if (ctx.canProcessDirectly) { doAddChunk(ctx, inputKeys, values, bucketPosition); @@ -303,34 +302,34 @@ public void resetForReprocess(@NotNull final UpdateContext context, @Override - public void resetForReprocess(@NotNull final UpdateContext ctx, - @NotNull final RowSet bucketIndex, - final long bucketPosition, - final long firstUnmodifiedKey) { + public void resetForReprocessBucketed(@NotNull final UpdateContext ctx, + @NotNull final RowSet bucketIndex, + final long bucketPosition, + final long firstUnmodifiedKey) { final long previousVal = firstUnmodifiedKey == NULL_ROW_KEY ? QueryConstants.NULL_LONG : outputSource.getLong(firstUnmodifiedKey); bucketLastVal.set(bucketPosition, previousVal); } @Override public void reprocessChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex) { + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk valuesChunk, + @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; doAddChunk(ctx, inputKeys, valuesChunk, 0); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } @Override - public void reprocessChunk(@NotNull UpdateContext updateContext, - @NotNull final RowSequence chunkOk, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk runStartPositions, - @NotNull final IntChunk runLengths) { - addChunk(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); + public void reprocessChunkBucketed(@NotNull UpdateContext updateContext, + @NotNull final RowSequence chunkOk, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk runStartPositions, + @NotNull final IntChunk runLengths) { + addChunkBucketed(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); ((Context)updateContext).getModifiedBuilder().appendRowSequence(chunkOk); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java index d960dc4cd27..687bb30c7e4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java @@ -24,12 +24,12 @@ public BaseObjectBinaryOperator(@NotNull final Class type, // region Addition @Override - public void addChunk(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { + public void addChunkBucketed(@NotNull final UpdateContext context, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk startPositions, + @NotNull final IntChunk runLengths) { final ObjectChunk asObject = values.asObjectChunk(); //noinspection unchecked final Context ctx = (Context) context; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java index e9a9c6fe1f5..affaeeb8d29 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java @@ -18,7 +18,6 @@ import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; -import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.util.RowRedirection; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; @@ -194,7 +193,7 @@ public void initializeForUpdate(@NotNull final UpdateContext context, @Override public void initializeFor(@NotNull final UpdateContext updateContext, - @NotNull final RowSet updateIndex, + @NotNull final RowSet updateRowSet, @NotNull final UpdateBy.UpdateType type) { ((Context)updateContext).currentUpdateType = type; ((Context)updateContext).curVal = null; @@ -257,10 +256,10 @@ public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { // region Addition @Override public void addChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk values, - long bucketPosition) { + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk values, + long bucketPosition) { final Context ctx = (Context) updateContext; if (ctx.canProcessDirectly) { doAddChunk(ctx, inputKeys, values, bucketPosition); @@ -306,34 +305,34 @@ public void resetForReprocess(@NotNull final UpdateContext context, @Override - public void resetForReprocess(@NotNull final UpdateContext ctx, - @NotNull final RowSet bucketIndex, - final long bucketPosition, - final long firstUnmodifiedKey) { + public void resetForReprocessBucketed(@NotNull final UpdateContext ctx, + @NotNull final RowSet bucketIndex, + final long bucketPosition, + final long firstUnmodifiedKey) { final T previousVal = firstUnmodifiedKey == NULL_ROW_KEY ? null : outputSource.get(firstUnmodifiedKey); bucketLastVal.set(bucketPosition, previousVal); } @Override public void reprocessChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex) { + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk valuesChunk, + @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; doAddChunk(ctx, inputKeys, valuesChunk, 0); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } @Override - public void reprocessChunk(@NotNull UpdateContext updateContext, - @NotNull final RowSequence chunkOk, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk runStartPositions, - @NotNull final IntChunk runLengths) { - addChunk(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); + public void reprocessChunkBucketed(@NotNull UpdateContext updateContext, + @NotNull final RowSequence chunkOk, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk runStartPositions, + @NotNull final IntChunk runLengths) { + addChunkBucketed(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); ((Context)updateContext).getModifiedBuilder().appendRowSequence(chunkOk); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java index 233124ffbee..169d51bbfc9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java @@ -18,7 +18,6 @@ import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; -import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.util.RowRedirection; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; @@ -191,7 +190,7 @@ public void initializeForUpdate(@NotNull final UpdateContext context, @Override public void initializeFor(@NotNull final UpdateContext updateContext, - @NotNull final RowSet updateIndex, + @NotNull final RowSet updateRowSet, @NotNull final UpdateBy.UpdateType type) { ((Context)updateContext).currentUpdateType = type; ((Context)updateContext).curVal = QueryConstants.NULL_SHORT; @@ -254,10 +253,10 @@ public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { // region Addition @Override public void addChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk values, - long bucketPosition) { + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk values, + long bucketPosition) { final Context ctx = (Context) updateContext; if (ctx.canProcessDirectly) { doAddChunk(ctx, inputKeys, values, bucketPosition); @@ -303,34 +302,34 @@ public void resetForReprocess(@NotNull final UpdateContext context, @Override - public void resetForReprocess(@NotNull final UpdateContext ctx, - @NotNull final RowSet bucketIndex, - final long bucketPosition, - final long firstUnmodifiedKey) { + public void resetForReprocessBucketed(@NotNull final UpdateContext ctx, + @NotNull final RowSet bucketIndex, + final long bucketPosition, + final long firstUnmodifiedKey) { final short previousVal = firstUnmodifiedKey == NULL_ROW_KEY ? QueryConstants.NULL_SHORT : outputSource.getShort(firstUnmodifiedKey); bucketLastVal.set(bucketPosition, previousVal); } @Override public void reprocessChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex) { + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk valuesChunk, + @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; doAddChunk(ctx, inputKeys, valuesChunk, 0); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } @Override - public void reprocessChunk(@NotNull UpdateContext updateContext, - @NotNull final RowSequence chunkOk, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk runStartPositions, - @NotNull final IntChunk runLengths) { - addChunk(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); + public void reprocessChunkBucketed(@NotNull UpdateContext updateContext, + @NotNull final RowSequence chunkOk, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk runStartPositions, + @NotNull final IntChunk runLengths) { + addChunkBucketed(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); ((Context)updateContext).getModifiedBuilder().appendRowSequence(chunkOk); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java new file mode 100644 index 00000000000..4ec891fd6eb --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java @@ -0,0 +1,412 @@ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit BaseWindowedCharUpdateByOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +package io.deephaven.engine.table.impl.updateby.internal; + +import io.deephaven.util.QueryConstants; +import io.deephaven.engine.table.impl.sources.ByteArraySource; +import io.deephaven.engine.table.impl.sources.ByteSparseArraySource; +import io.deephaven.engine.table.WritableColumnSource; + +import io.deephaven.api.updateby.OperationControl; +import io.deephaven.base.LongRingBuffer; +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.rowset.*; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.table.ChunkSource; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.UpdateByWindowedOperator; +import io.deephaven.engine.table.impl.util.RowRedirection; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +public abstract class BaseWindowedByteUpdateByOperator extends UpdateByWindowedOperator { + protected final ColumnSource valueSource; + + protected boolean initialized = false; + + // region extra-fields + final byte nullValue; + // endregion extra-fields + + protected class Context extends UpdateWindowedContext { + public RowSet workingRowSet = null; + + // candidate data for the window + public final int WINDOW_CHUNK_SIZE = 1024; + + // data that is actually in the current window + public LongRingBuffer windowRowKeys = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); + // the selector that determines whether this value should be in the window, positions for tick-based and + // timestamps for time-based operators + public LongRingBuffer windowSelector = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); + + public RowSequence.Iterator windowIterator = null; + + public WritableByteChunk candidateValuesChunk; + public WritableLongChunk candidateRowKeysChunk; + public WritableLongChunk candidatePositionsChunk; + public WritableLongChunk candidateTimestampsChunk; + + public int candidateWindowIndex = 0; + + // position data for the chunk being currently processed + public WritableLongChunk valuePositionChunk; + + // other useful stuff + public UpdateBy.UpdateType currentUpdateType; + + public RowSetBuilderSequential modifiedBuilder; + public RowSet newModified; + + public RowSetBuilderSequential getModifiedBuilder() { + if(modifiedBuilder == null) { + modifiedBuilder = RowSetFactory.builderSequential(); + } + return modifiedBuilder; + } + + @Override + public void close() { + if (windowIterator != null) { + windowIterator.close(); + windowIterator = null; + } + + if (candidateValuesChunk != null) { + candidateValuesChunk.close(); + candidateValuesChunk = null; + } + + if (candidateRowKeysChunk != null) { + candidateRowKeysChunk.close(); + candidateRowKeysChunk = null; + } + + if (candidatePositionsChunk != null) { + candidatePositionsChunk.close(); + candidatePositionsChunk = null; + } + + if (valuePositionChunk != null) { + valuePositionChunk.close(); + valuePositionChunk = null; + } + + // no need to close, just release the reference + workingRowSet = null; + } + + /*** + * Fill the working chunks with data for this key + * + * @param startKey the key for which we want to + */ + public void loadWindowChunks(final long startKey) { + // TODO: make sure this works for bucketed + if (windowIterator == null) { + windowIterator = workingRowSet.getRowSequenceIterator(); + } + windowIterator.advance(startKey); + + RowSequence windowRowSequence = windowIterator.getNextRowSequenceWithLength(WINDOW_CHUNK_SIZE); + + // fill the window values chunk + if (candidateValuesChunk == null) { + candidateValuesChunk = WritableByteChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + } + try (ChunkSource.FillContext fc = valueSource.makeFillContext(WINDOW_CHUNK_SIZE)){ + valueSource.fillChunk(fc, candidateValuesChunk, windowRowSequence); + } + + // fill the window keys chunk + if (candidateRowKeysChunk == null) { + candidateRowKeysChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + } + windowRowSequence.fillRowKeyChunk(candidateRowKeysChunk); + + if (recorder == null) { + // get position data for the window items (relative to the table or bucket rowset) + if (candidatePositionsChunk == null) { + candidatePositionsChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + } + + // TODO: gotta be a better way than creating two rowsets + try (final RowSet rs = windowRowSequence.asRowSet(); + final RowSet positions = workingRowSet.invert(rs)) { + positions.fillRowKeyChunk(candidatePositionsChunk); + } + } else { + // get timestamp values from the recorder column source + if (candidateTimestampsChunk == null) { + candidateTimestampsChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + } + try (final ChunkSource.FillContext fc = recorder.getColumnSource().makeFillContext(WINDOW_CHUNK_SIZE)) { + recorder.getColumnSource().fillChunk(fc, candidateTimestampsChunk, windowRowSequence); + } + } + + // reset the index to beginning of the chunks + candidateWindowIndex = 0; + } + + /*** + * Fill the working chunks with data for this key + * + * @param inputKeys the keys for which we want to get position or timestamp values + */ + public void loadDataChunks(final RowSequence inputKeys) { + if (recorder != null) { + // timestamp data will be available from the recorder + return; + } + + if (valuePositionChunk == null) { + valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); + } else if (valuePositionChunk.capacity() < inputKeys.size()) { + valuePositionChunk.setSize(inputKeys.intSize()); + } + + // produce position data for the window (will be timestamps for time-based) + // TODO: gotta be a better way than creating two rowsets + try (final RowSet rs = inputKeys.asRowSet(); + final RowSet positions = workingRowSet.invert(rs)) { + positions.fillRowKeyChunk(valuePositionChunk); + } + } + + public void fillWindowTicks(UpdateContext context, long currentPos) { + // compute the head and tail (inclusive) + final long tail = Math.max(0, currentPos - reverseTimeScaleUnits + 1); + final long head = Math.min(workingRowSet.size() - 1, currentPos + forwardTimeScaleUnits); + + while (windowSelector.peek(Long.MAX_VALUE) < tail) { + final long pos = windowSelector.remove(); + final long key = windowRowKeys.remove(); + + pop(context, key); + } + + + // look at the window data and push until satisfied or at the end of the rowset + while (candidatePositionsChunk.size() > 0 && candidatePositionsChunk.get(candidateWindowIndex) <= head) { + final long pos = candidatePositionsChunk.get(candidateWindowIndex); + final long key = candidateRowKeysChunk.get(candidateWindowIndex); + final byte val = candidateValuesChunk.get(candidateWindowIndex); + + push(context, key, val); + + windowSelector.add(pos); + windowRowKeys.add(key); + + if (++candidateWindowIndex >= candidatePositionsChunk.size()) { + // load the next chunk in order + loadWindowChunks(key + 1); + } + } + + if (windowSelector.isEmpty()) { + reset(context); + } + } + } + + public BaseWindowedByteUpdateByOperator(@NotNull final MatchPair pair, + @NotNull final String[] affectingColumns, + @NotNull final OperationControl control, + @Nullable final LongRecordingUpdateByOperator timeRecorder, + final long reverseTimeScaleUnits, + final long forwardTimeScaleUnits, + @Nullable final RowRedirection rowRedirection, + @NotNull final ColumnSource valueSource + // region extra-constructor-args + // endregion extra-constructor-args + ) { + super(pair, affectingColumns, control, timeRecorder, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); + this.valueSource = valueSource; + // region constructor + this.nullValue = getNullValue(); + // endregion constructor + } + + // region extra-methods + protected byte getNullValue() { + return QueryConstants.NULL_BYTE; + } + + // region extra-methods + protected WritableColumnSource makeSparseSource() { + return new ByteSparseArraySource(); + } + + protected WritableColumnSource makeDenseSource() { + return new ByteArraySource(); + } + // endregion extra-methods + + public abstract void push(UpdateContext context, long key, byte val); + public abstract void pop(UpdateContext context, long key); + public abstract void reset(UpdateContext context); + + @Override + public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, boolean usingBuckets, boolean isUpstreamAppendOnly) { + final Context ctx = (Context) context; + ctx.workingRowSet = resultSourceRowSet; + } + + + @Override + public void initializeFor(@NotNull final UpdateContext context, + @NotNull final RowSet updateRowSet, + @NotNull final UpdateBy.UpdateType type) { + final Context ctx = (Context) context; + ctx.currentUpdateType = type; + + if (type == UpdateBy.UpdateType.Add || type == UpdateBy.UpdateType.Reprocess) { + long windowStartKey = computeFirstAffectingKey(updateRowSet.firstRowKey(), ctx.workingRowSet); + ctx.loadWindowChunks(windowStartKey); + } + } + + @Override + public void finishFor(@NotNull final UpdateContext updateContext, @NotNull final UpdateBy.UpdateType type) { + final Context ctx = (Context) updateContext; + if(type == UpdateBy.UpdateType.Reprocess && ctx.modifiedBuilder != null) { + ctx.newModified = ctx.modifiedBuilder.build(); + } + } + + @Override + public boolean requiresValues(@NotNull final UpdateContext context) { + // windowed operators don't need current values supplied to them, they only care about windowed values which + // may or may not intersect with the column values + return false; + } + + @NotNull + @Override + final public RowSet getAdditionalModifications(@NotNull final UpdateContext ctx) { + return ((Context)ctx).newModified == null ? RowSetFactory.empty() : ((Context)ctx).newModified; + } + + @Override + final public boolean anyModified(@NotNull final UpdateContext ctx) { + return ((Context)ctx).newModified != null; + } + + // region Addition + @Override + public void addChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk values, + long bucketPosition) { + final Context ctx = (Context) updateContext; + + if (recorder == null) { + // use position data to determine the windows + ctx.loadDataChunks(inputKeys); + } else { + // use timestamp data to determine the windows + } + doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); + } + + /** + * Add a chunk of values to the operator. + * + * @param ctx the context object + * @param inputKeys the input keys for the chunk + * @param workingChunk the chunk of values + * @param bucketPosition the bucket position that the values belong to. + */ + protected abstract void doAddChunk(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk workingChunk, + final long bucketPosition); + + // endregion + + // region Reprocessing + + public void resetForReprocess(@NotNull final UpdateContext context, + @NotNull final RowSet sourceRowSet, + long firstUnmodifiedKey) { + final Context ctx = (Context) context; + ctx.workingRowSet = sourceRowSet; + } + + @Override + public void resetForReprocessBucketed(@NotNull final UpdateContext context, + @NotNull final RowSet bucketRowSet, + final long bucketPosition, + final long firstUnmodifiedKey) { + final Context ctx = (Context) context; + } + + @Override + public void reprocessChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk valuesChunk, + @NotNull final RowSet postUpdateSourceIndex) { + final Context ctx = (Context) updateContext; + if (recorder == null) { + // use position data to determine the windows + ctx.loadDataChunks(inputKeys); + } else { + // use timestamp data to determine the windows + } + doAddChunk(ctx, inputKeys, keyChunk, valuesChunk, 0); + ctx.getModifiedBuilder().appendRowSequence(inputKeys); + } + + @Override + public void reprocessChunkBucketed(@NotNull UpdateContext updateContext, + @NotNull final RowSequence chunkOk, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk runStartPositions, + @NotNull final IntChunk runLengths) { + addChunkBucketed(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); + ((Context)updateContext).getModifiedBuilder().appendRowSequence(chunkOk); + } + + // endregion + + // region No-Op Operations + + @Override + final public void modifyChunk(@NotNull final UpdateContext updateContext, + @Nullable final LongChunk prevKeyChunk, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk prevValuesChunk, + @NotNull final Chunk postValuesChunk, + long bucketPosition) { + } + + @Override + final public void removeChunk(@NotNull final UpdateContext updateContext, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk prevValuesChunk, + long bucketPosition) { + } + + @Override + final public void applyShift(@NotNull final UpdateContext updateContext, + @NotNull final RowSet prevIndex, + @NotNull final RowSetShiftData shifted) { + } + // endregion +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java new file mode 100644 index 00000000000..996a2a72882 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java @@ -0,0 +1,388 @@ +package io.deephaven.engine.table.impl.updateby.internal; + +import io.deephaven.api.updateby.OperationControl; +import io.deephaven.base.LongRingBuffer; +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.rowset.*; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.table.ChunkSource; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.UpdateByWindowedOperator; +import io.deephaven.engine.table.impl.util.RowRedirection; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +public abstract class BaseWindowedCharUpdateByOperator extends UpdateByWindowedOperator { + protected final ColumnSource valueSource; + + protected boolean initialized = false; + + // region extra-fields + // endregion extra-fields + + protected class Context extends UpdateWindowedContext { + public RowSet workingRowSet = null; + + // candidate data for the window + public final int WINDOW_CHUNK_SIZE = 1024; + + // data that is actually in the current window + public LongRingBuffer windowRowKeys = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); + // the selector that determines whether this value should be in the window, positions for tick-based and + // timestamps for time-based operators + public LongRingBuffer windowSelector = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); + + public RowSequence.Iterator windowIterator = null; + + public WritableCharChunk candidateValuesChunk; + public WritableLongChunk candidateRowKeysChunk; + public WritableLongChunk candidatePositionsChunk; + public WritableLongChunk candidateTimestampsChunk; + + public int candidateWindowIndex = 0; + + // position data for the chunk being currently processed + public WritableLongChunk valuePositionChunk; + + // other useful stuff + public UpdateBy.UpdateType currentUpdateType; + + public RowSetBuilderSequential modifiedBuilder; + public RowSet newModified; + + public RowSetBuilderSequential getModifiedBuilder() { + if(modifiedBuilder == null) { + modifiedBuilder = RowSetFactory.builderSequential(); + } + return modifiedBuilder; + } + + @Override + public void close() { + if (windowIterator != null) { + windowIterator.close(); + windowIterator = null; + } + + if (candidateValuesChunk != null) { + candidateValuesChunk.close(); + candidateValuesChunk = null; + } + + if (candidateRowKeysChunk != null) { + candidateRowKeysChunk.close(); + candidateRowKeysChunk = null; + } + + if (candidatePositionsChunk != null) { + candidatePositionsChunk.close(); + candidatePositionsChunk = null; + } + + if (valuePositionChunk != null) { + valuePositionChunk.close(); + valuePositionChunk = null; + } + + // no need to close, just release the reference + workingRowSet = null; + } + + /*** + * Fill the working chunks with data for this key + * + * @param startKey the key for which we want to + */ + public void loadWindowChunks(final long startKey) { + // TODO: make sure this works for bucketed + if (windowIterator == null) { + windowIterator = workingRowSet.getRowSequenceIterator(); + } + windowIterator.advance(startKey); + + RowSequence windowRowSequence = windowIterator.getNextRowSequenceWithLength(WINDOW_CHUNK_SIZE); + + // fill the window values chunk + if (candidateValuesChunk == null) { + candidateValuesChunk = WritableCharChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + } + try (ChunkSource.FillContext fc = valueSource.makeFillContext(WINDOW_CHUNK_SIZE)){ + valueSource.fillChunk(fc, candidateValuesChunk, windowRowSequence); + } + + // fill the window keys chunk + if (candidateRowKeysChunk == null) { + candidateRowKeysChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + } + windowRowSequence.fillRowKeyChunk(candidateRowKeysChunk); + + if (recorder == null) { + // get position data for the window items (relative to the table or bucket rowset) + if (candidatePositionsChunk == null) { + candidatePositionsChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + } + + // TODO: gotta be a better way than creating two rowsets + try (final RowSet rs = windowRowSequence.asRowSet(); + final RowSet positions = workingRowSet.invert(rs)) { + positions.fillRowKeyChunk(candidatePositionsChunk); + } + } else { + // get timestamp values from the recorder column source + if (candidateTimestampsChunk == null) { + candidateTimestampsChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + } + try (final ChunkSource.FillContext fc = recorder.getColumnSource().makeFillContext(WINDOW_CHUNK_SIZE)) { + recorder.getColumnSource().fillChunk(fc, candidateTimestampsChunk, windowRowSequence); + } + } + + // reset the index to beginning of the chunks + candidateWindowIndex = 0; + } + + /*** + * Fill the working chunks with data for this key + * + * @param inputKeys the keys for which we want to get position or timestamp values + */ + public void loadDataChunks(final RowSequence inputKeys) { + if (recorder != null) { + // timestamp data will be available from the recorder + return; + } + + if (valuePositionChunk == null) { + valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); + } else if (valuePositionChunk.capacity() < inputKeys.size()) { + valuePositionChunk.setSize(inputKeys.intSize()); + } + + // produce position data for the window (will be timestamps for time-based) + // TODO: gotta be a better way than creating two rowsets + try (final RowSet rs = inputKeys.asRowSet(); + final RowSet positions = workingRowSet.invert(rs)) { + positions.fillRowKeyChunk(valuePositionChunk); + } + } + + public void fillWindowTicks(UpdateContext context, long currentPos) { + // compute the head and tail (inclusive) + final long tail = Math.max(0, currentPos - reverseTimeScaleUnits + 1); + final long head = Math.min(workingRowSet.size() - 1, currentPos + forwardTimeScaleUnits); + + while (windowSelector.peek(Long.MAX_VALUE) < tail) { + final long pos = windowSelector.remove(); + final long key = windowRowKeys.remove(); + + pop(context, key); + } + + + // look at the window data and push until satisfied or at the end of the rowset + while (candidatePositionsChunk.size() > 0 && candidatePositionsChunk.get(candidateWindowIndex) <= head) { + final long pos = candidatePositionsChunk.get(candidateWindowIndex); + final long key = candidateRowKeysChunk.get(candidateWindowIndex); + final char val = candidateValuesChunk.get(candidateWindowIndex); + + push(context, key, val); + + windowSelector.add(pos); + windowRowKeys.add(key); + + if (++candidateWindowIndex >= candidatePositionsChunk.size()) { + // load the next chunk in order + loadWindowChunks(key + 1); + } + } + + if (windowSelector.isEmpty()) { + reset(context); + } + } + } + + public BaseWindowedCharUpdateByOperator(@NotNull final MatchPair pair, + @NotNull final String[] affectingColumns, + @NotNull final OperationControl control, + @Nullable final LongRecordingUpdateByOperator timeRecorder, + final long reverseTimeScaleUnits, + final long forwardTimeScaleUnits, + @Nullable final RowRedirection rowRedirection, + @NotNull final ColumnSource valueSource + // region extra-constructor-args + // endregion extra-constructor-args + ) { + super(pair, affectingColumns, control, timeRecorder, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); + this.valueSource = valueSource; + // region constructor + // endregion constructor + } + + // region extra-methods + // endregion extra-methods + + public abstract void push(UpdateContext context, long key, char val); + public abstract void pop(UpdateContext context, long key); + public abstract void reset(UpdateContext context); + + @Override + public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, boolean usingBuckets, boolean isUpstreamAppendOnly) { + final Context ctx = (Context) context; + ctx.workingRowSet = resultSourceRowSet; + } + + + @Override + public void initializeFor(@NotNull final UpdateContext context, + @NotNull final RowSet updateRowSet, + @NotNull final UpdateBy.UpdateType type) { + final Context ctx = (Context) context; + ctx.currentUpdateType = type; + + if (type == UpdateBy.UpdateType.Add || type == UpdateBy.UpdateType.Reprocess) { + long windowStartKey = computeFirstAffectingKey(updateRowSet.firstRowKey(), ctx.workingRowSet); + ctx.loadWindowChunks(windowStartKey); + } + } + + @Override + public void finishFor(@NotNull final UpdateContext updateContext, @NotNull final UpdateBy.UpdateType type) { + final Context ctx = (Context) updateContext; + if(type == UpdateBy.UpdateType.Reprocess && ctx.modifiedBuilder != null) { + ctx.newModified = ctx.modifiedBuilder.build(); + } + } + + @Override + public boolean requiresValues(@NotNull final UpdateContext context) { + // windowed operators don't need current values supplied to them, they only care about windowed values which + // may or may not intersect with the column values + return false; + } + + @NotNull + @Override + final public RowSet getAdditionalModifications(@NotNull final UpdateContext ctx) { + return ((Context)ctx).newModified == null ? RowSetFactory.empty() : ((Context)ctx).newModified; + } + + @Override + final public boolean anyModified(@NotNull final UpdateContext ctx) { + return ((Context)ctx).newModified != null; + } + + // region Addition + @Override + public void addChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk values, + long bucketPosition) { + final Context ctx = (Context) updateContext; + + if (recorder == null) { + // use position data to determine the windows + ctx.loadDataChunks(inputKeys); + } else { + // use timestamp data to determine the windows + } + doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); + } + + /** + * Add a chunk of values to the operator. + * + * @param ctx the context object + * @param inputKeys the input keys for the chunk + * @param workingChunk the chunk of values + * @param bucketPosition the bucket position that the values belong to. + */ + protected abstract void doAddChunk(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk workingChunk, + final long bucketPosition); + + // endregion + + // region Reprocessing + + public void resetForReprocess(@NotNull final UpdateContext context, + @NotNull final RowSet sourceRowSet, + long firstUnmodifiedKey) { + final Context ctx = (Context) context; + ctx.workingRowSet = sourceRowSet; + } + + @Override + public void resetForReprocessBucketed(@NotNull final UpdateContext context, + @NotNull final RowSet bucketRowSet, + final long bucketPosition, + final long firstUnmodifiedKey) { + final Context ctx = (Context) context; + } + + @Override + public void reprocessChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk valuesChunk, + @NotNull final RowSet postUpdateSourceIndex) { + final Context ctx = (Context) updateContext; + if (recorder == null) { + // use position data to determine the windows + ctx.loadDataChunks(inputKeys); + } else { + // use timestamp data to determine the windows + } + doAddChunk(ctx, inputKeys, keyChunk, valuesChunk, 0); + ctx.getModifiedBuilder().appendRowSequence(inputKeys); + } + + @Override + public void reprocessChunkBucketed(@NotNull UpdateContext updateContext, + @NotNull final RowSequence chunkOk, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk runStartPositions, + @NotNull final IntChunk runLengths) { + addChunkBucketed(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); + ((Context)updateContext).getModifiedBuilder().appendRowSequence(chunkOk); + } + + // endregion + + // region No-Op Operations + + @Override + final public void modifyChunk(@NotNull final UpdateContext updateContext, + @Nullable final LongChunk prevKeyChunk, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk prevValuesChunk, + @NotNull final Chunk postValuesChunk, + long bucketPosition) { + } + + @Override + final public void removeChunk(@NotNull final UpdateContext updateContext, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk prevValuesChunk, + long bucketPosition) { + } + + @Override + final public void applyShift(@NotNull final UpdateContext updateContext, + @NotNull final RowSet prevIndex, + @NotNull final RowSetShiftData shifted) { + } + // endregion +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java new file mode 100644 index 00000000000..8799ccd290d --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java @@ -0,0 +1,393 @@ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit BaseWindowedCharUpdateByOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +package io.deephaven.engine.table.impl.updateby.internal; + +import io.deephaven.api.updateby.OperationControl; +import io.deephaven.base.LongRingBuffer; +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.rowset.*; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.table.ChunkSource; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.UpdateByWindowedOperator; +import io.deephaven.engine.table.impl.util.RowRedirection; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +public abstract class BaseWindowedIntUpdateByOperator extends UpdateByWindowedOperator { + protected final ColumnSource valueSource; + + protected boolean initialized = false; + + // region extra-fields + // endregion extra-fields + + protected class Context extends UpdateWindowedContext { + public RowSet workingRowSet = null; + + // candidate data for the window + public final int WINDOW_CHUNK_SIZE = 1024; + + // data that is actually in the current window + public LongRingBuffer windowRowKeys = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); + // the selector that determines whether this value should be in the window, positions for tick-based and + // timestamps for time-based operators + public LongRingBuffer windowSelector = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); + + public RowSequence.Iterator windowIterator = null; + + public WritableIntChunk candidateValuesChunk; + public WritableLongChunk candidateRowKeysChunk; + public WritableLongChunk candidatePositionsChunk; + public WritableLongChunk candidateTimestampsChunk; + + public int candidateWindowIndex = 0; + + // position data for the chunk being currently processed + public WritableLongChunk valuePositionChunk; + + // other useful stuff + public UpdateBy.UpdateType currentUpdateType; + + public RowSetBuilderSequential modifiedBuilder; + public RowSet newModified; + + public RowSetBuilderSequential getModifiedBuilder() { + if(modifiedBuilder == null) { + modifiedBuilder = RowSetFactory.builderSequential(); + } + return modifiedBuilder; + } + + @Override + public void close() { + if (windowIterator != null) { + windowIterator.close(); + windowIterator = null; + } + + if (candidateValuesChunk != null) { + candidateValuesChunk.close(); + candidateValuesChunk = null; + } + + if (candidateRowKeysChunk != null) { + candidateRowKeysChunk.close(); + candidateRowKeysChunk = null; + } + + if (candidatePositionsChunk != null) { + candidatePositionsChunk.close(); + candidatePositionsChunk = null; + } + + if (valuePositionChunk != null) { + valuePositionChunk.close(); + valuePositionChunk = null; + } + + // no need to close, just release the reference + workingRowSet = null; + } + + /*** + * Fill the working chunks with data for this key + * + * @param startKey the key for which we want to + */ + public void loadWindowChunks(final long startKey) { + // TODO: make sure this works for bucketed + if (windowIterator == null) { + windowIterator = workingRowSet.getRowSequenceIterator(); + } + windowIterator.advance(startKey); + + RowSequence windowRowSequence = windowIterator.getNextRowSequenceWithLength(WINDOW_CHUNK_SIZE); + + // fill the window values chunk + if (candidateValuesChunk == null) { + candidateValuesChunk = WritableIntChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + } + try (ChunkSource.FillContext fc = valueSource.makeFillContext(WINDOW_CHUNK_SIZE)){ + valueSource.fillChunk(fc, candidateValuesChunk, windowRowSequence); + } + + // fill the window keys chunk + if (candidateRowKeysChunk == null) { + candidateRowKeysChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + } + windowRowSequence.fillRowKeyChunk(candidateRowKeysChunk); + + if (recorder == null) { + // get position data for the window items (relative to the table or bucket rowset) + if (candidatePositionsChunk == null) { + candidatePositionsChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + } + + // TODO: gotta be a better way than creating two rowsets + try (final RowSet rs = windowRowSequence.asRowSet(); + final RowSet positions = workingRowSet.invert(rs)) { + positions.fillRowKeyChunk(candidatePositionsChunk); + } + } else { + // get timestamp values from the recorder column source + if (candidateTimestampsChunk == null) { + candidateTimestampsChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + } + try (final ChunkSource.FillContext fc = recorder.getColumnSource().makeFillContext(WINDOW_CHUNK_SIZE)) { + recorder.getColumnSource().fillChunk(fc, candidateTimestampsChunk, windowRowSequence); + } + } + + // reset the index to beginning of the chunks + candidateWindowIndex = 0; + } + + /*** + * Fill the working chunks with data for this key + * + * @param inputKeys the keys for which we want to get position or timestamp values + */ + public void loadDataChunks(final RowSequence inputKeys) { + if (recorder != null) { + // timestamp data will be available from the recorder + return; + } + + if (valuePositionChunk == null) { + valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); + } else if (valuePositionChunk.capacity() < inputKeys.size()) { + valuePositionChunk.setSize(inputKeys.intSize()); + } + + // produce position data for the window (will be timestamps for time-based) + // TODO: gotta be a better way than creating two rowsets + try (final RowSet rs = inputKeys.asRowSet(); + final RowSet positions = workingRowSet.invert(rs)) { + positions.fillRowKeyChunk(valuePositionChunk); + } + } + + public void fillWindowTicks(UpdateContext context, long currentPos) { + // compute the head and tail (inclusive) + final long tail = Math.max(0, currentPos - reverseTimeScaleUnits + 1); + final long head = Math.min(workingRowSet.size() - 1, currentPos + forwardTimeScaleUnits); + + while (windowSelector.peek(Long.MAX_VALUE) < tail) { + final long pos = windowSelector.remove(); + final long key = windowRowKeys.remove(); + + pop(context, key); + } + + + // look at the window data and push until satisfied or at the end of the rowset + while (candidatePositionsChunk.size() > 0 && candidatePositionsChunk.get(candidateWindowIndex) <= head) { + final long pos = candidatePositionsChunk.get(candidateWindowIndex); + final long key = candidateRowKeysChunk.get(candidateWindowIndex); + final int val = candidateValuesChunk.get(candidateWindowIndex); + + push(context, key, val); + + windowSelector.add(pos); + windowRowKeys.add(key); + + if (++candidateWindowIndex >= candidatePositionsChunk.size()) { + // load the next chunk in order + loadWindowChunks(key + 1); + } + } + + if (windowSelector.isEmpty()) { + reset(context); + } + } + } + + public BaseWindowedIntUpdateByOperator(@NotNull final MatchPair pair, + @NotNull final String[] affectingColumns, + @NotNull final OperationControl control, + @Nullable final LongRecordingUpdateByOperator timeRecorder, + final long reverseTimeScaleUnits, + final long forwardTimeScaleUnits, + @Nullable final RowRedirection rowRedirection, + @NotNull final ColumnSource valueSource + // region extra-constructor-args + // endregion extra-constructor-args + ) { + super(pair, affectingColumns, control, timeRecorder, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); + this.valueSource = valueSource; + // region constructor + // endregion constructor + } + + // region extra-methods + // endregion extra-methods + + public abstract void push(UpdateContext context, long key, int val); + public abstract void pop(UpdateContext context, long key); + public abstract void reset(UpdateContext context); + + @Override + public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, boolean usingBuckets, boolean isUpstreamAppendOnly) { + final Context ctx = (Context) context; + ctx.workingRowSet = resultSourceRowSet; + } + + + @Override + public void initializeFor(@NotNull final UpdateContext context, + @NotNull final RowSet updateRowSet, + @NotNull final UpdateBy.UpdateType type) { + final Context ctx = (Context) context; + ctx.currentUpdateType = type; + + if (type == UpdateBy.UpdateType.Add || type == UpdateBy.UpdateType.Reprocess) { + long windowStartKey = computeFirstAffectingKey(updateRowSet.firstRowKey(), ctx.workingRowSet); + ctx.loadWindowChunks(windowStartKey); + } + } + + @Override + public void finishFor(@NotNull final UpdateContext updateContext, @NotNull final UpdateBy.UpdateType type) { + final Context ctx = (Context) updateContext; + if(type == UpdateBy.UpdateType.Reprocess && ctx.modifiedBuilder != null) { + ctx.newModified = ctx.modifiedBuilder.build(); + } + } + + @Override + public boolean requiresValues(@NotNull final UpdateContext context) { + // windowed operators don't need current values supplied to them, they only care about windowed values which + // may or may not intersect with the column values + return false; + } + + @NotNull + @Override + final public RowSet getAdditionalModifications(@NotNull final UpdateContext ctx) { + return ((Context)ctx).newModified == null ? RowSetFactory.empty() : ((Context)ctx).newModified; + } + + @Override + final public boolean anyModified(@NotNull final UpdateContext ctx) { + return ((Context)ctx).newModified != null; + } + + // region Addition + @Override + public void addChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk values, + long bucketPosition) { + final Context ctx = (Context) updateContext; + + if (recorder == null) { + // use position data to determine the windows + ctx.loadDataChunks(inputKeys); + } else { + // use timestamp data to determine the windows + } + doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); + } + + /** + * Add a chunk of values to the operator. + * + * @param ctx the context object + * @param inputKeys the input keys for the chunk + * @param workingChunk the chunk of values + * @param bucketPosition the bucket position that the values belong to. + */ + protected abstract void doAddChunk(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk workingChunk, + final long bucketPosition); + + // endregion + + // region Reprocessing + + public void resetForReprocess(@NotNull final UpdateContext context, + @NotNull final RowSet sourceRowSet, + long firstUnmodifiedKey) { + final Context ctx = (Context) context; + ctx.workingRowSet = sourceRowSet; + } + + @Override + public void resetForReprocessBucketed(@NotNull final UpdateContext context, + @NotNull final RowSet bucketRowSet, + final long bucketPosition, + final long firstUnmodifiedKey) { + final Context ctx = (Context) context; + } + + @Override + public void reprocessChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk valuesChunk, + @NotNull final RowSet postUpdateSourceIndex) { + final Context ctx = (Context) updateContext; + if (recorder == null) { + // use position data to determine the windows + ctx.loadDataChunks(inputKeys); + } else { + // use timestamp data to determine the windows + } + doAddChunk(ctx, inputKeys, keyChunk, valuesChunk, 0); + ctx.getModifiedBuilder().appendRowSequence(inputKeys); + } + + @Override + public void reprocessChunkBucketed(@NotNull UpdateContext updateContext, + @NotNull final RowSequence chunkOk, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk runStartPositions, + @NotNull final IntChunk runLengths) { + addChunkBucketed(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); + ((Context)updateContext).getModifiedBuilder().appendRowSequence(chunkOk); + } + + // endregion + + // region No-Op Operations + + @Override + final public void modifyChunk(@NotNull final UpdateContext updateContext, + @Nullable final LongChunk prevKeyChunk, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk prevValuesChunk, + @NotNull final Chunk postValuesChunk, + long bucketPosition) { + } + + @Override + final public void removeChunk(@NotNull final UpdateContext updateContext, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk prevValuesChunk, + long bucketPosition) { + } + + @Override + final public void applyShift(@NotNull final UpdateContext updateContext, + @NotNull final RowSet prevIndex, + @NotNull final RowSetShiftData shifted) { + } + // endregion +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java new file mode 100644 index 00000000000..4dceaf9bd51 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java @@ -0,0 +1,393 @@ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit BaseWindowedCharUpdateByOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +package io.deephaven.engine.table.impl.updateby.internal; + +import io.deephaven.api.updateby.OperationControl; +import io.deephaven.base.LongRingBuffer; +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.rowset.*; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.table.ChunkSource; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.UpdateByWindowedOperator; +import io.deephaven.engine.table.impl.util.RowRedirection; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +public abstract class BaseWindowedLongUpdateByOperator extends UpdateByWindowedOperator { + protected final ColumnSource valueSource; + + protected boolean initialized = false; + + // region extra-fields + // endregion extra-fields + + protected class Context extends UpdateWindowedContext { + public RowSet workingRowSet = null; + + // candidate data for the window + public final int WINDOW_CHUNK_SIZE = 1024; + + // data that is actually in the current window + public LongRingBuffer windowRowKeys = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); + // the selector that determines whether this value should be in the window, positions for tick-based and + // timestamps for time-based operators + public LongRingBuffer windowSelector = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); + + public RowSequence.Iterator windowIterator = null; + + public WritableLongChunk candidateValuesChunk; + public WritableLongChunk candidateRowKeysChunk; + public WritableLongChunk candidatePositionsChunk; + public WritableLongChunk candidateTimestampsChunk; + + public int candidateWindowIndex = 0; + + // position data for the chunk being currently processed + public WritableLongChunk valuePositionChunk; + + // other useful stuff + public UpdateBy.UpdateType currentUpdateType; + + public RowSetBuilderSequential modifiedBuilder; + public RowSet newModified; + + public RowSetBuilderSequential getModifiedBuilder() { + if(modifiedBuilder == null) { + modifiedBuilder = RowSetFactory.builderSequential(); + } + return modifiedBuilder; + } + + @Override + public void close() { + if (windowIterator != null) { + windowIterator.close(); + windowIterator = null; + } + + if (candidateValuesChunk != null) { + candidateValuesChunk.close(); + candidateValuesChunk = null; + } + + if (candidateRowKeysChunk != null) { + candidateRowKeysChunk.close(); + candidateRowKeysChunk = null; + } + + if (candidatePositionsChunk != null) { + candidatePositionsChunk.close(); + candidatePositionsChunk = null; + } + + if (valuePositionChunk != null) { + valuePositionChunk.close(); + valuePositionChunk = null; + } + + // no need to close, just release the reference + workingRowSet = null; + } + + /*** + * Fill the working chunks with data for this key + * + * @param startKey the key for which we want to + */ + public void loadWindowChunks(final long startKey) { + // TODO: make sure this works for bucketed + if (windowIterator == null) { + windowIterator = workingRowSet.getRowSequenceIterator(); + } + windowIterator.advance(startKey); + + RowSequence windowRowSequence = windowIterator.getNextRowSequenceWithLength(WINDOW_CHUNK_SIZE); + + // fill the window values chunk + if (candidateValuesChunk == null) { + candidateValuesChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + } + try (ChunkSource.FillContext fc = valueSource.makeFillContext(WINDOW_CHUNK_SIZE)){ + valueSource.fillChunk(fc, candidateValuesChunk, windowRowSequence); + } + + // fill the window keys chunk + if (candidateRowKeysChunk == null) { + candidateRowKeysChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + } + windowRowSequence.fillRowKeyChunk(candidateRowKeysChunk); + + if (recorder == null) { + // get position data for the window items (relative to the table or bucket rowset) + if (candidatePositionsChunk == null) { + candidatePositionsChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + } + + // TODO: gotta be a better way than creating two rowsets + try (final RowSet rs = windowRowSequence.asRowSet(); + final RowSet positions = workingRowSet.invert(rs)) { + positions.fillRowKeyChunk(candidatePositionsChunk); + } + } else { + // get timestamp values from the recorder column source + if (candidateTimestampsChunk == null) { + candidateTimestampsChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + } + try (final ChunkSource.FillContext fc = recorder.getColumnSource().makeFillContext(WINDOW_CHUNK_SIZE)) { + recorder.getColumnSource().fillChunk(fc, candidateTimestampsChunk, windowRowSequence); + } + } + + // reset the index to beginning of the chunks + candidateWindowIndex = 0; + } + + /*** + * Fill the working chunks with data for this key + * + * @param inputKeys the keys for which we want to get position or timestamp values + */ + public void loadDataChunks(final RowSequence inputKeys) { + if (recorder != null) { + // timestamp data will be available from the recorder + return; + } + + if (valuePositionChunk == null) { + valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); + } else if (valuePositionChunk.capacity() < inputKeys.size()) { + valuePositionChunk.setSize(inputKeys.intSize()); + } + + // produce position data for the window (will be timestamps for time-based) + // TODO: gotta be a better way than creating two rowsets + try (final RowSet rs = inputKeys.asRowSet(); + final RowSet positions = workingRowSet.invert(rs)) { + positions.fillRowKeyChunk(valuePositionChunk); + } + } + + public void fillWindowTicks(UpdateContext context, long currentPos) { + // compute the head and tail (inclusive) + final long tail = Math.max(0, currentPos - reverseTimeScaleUnits + 1); + final long head = Math.min(workingRowSet.size() - 1, currentPos + forwardTimeScaleUnits); + + while (windowSelector.peek(Long.MAX_VALUE) < tail) { + final long pos = windowSelector.remove(); + final long key = windowRowKeys.remove(); + + pop(context, key); + } + + + // look at the window data and push until satisfied or at the end of the rowset + while (candidatePositionsChunk.size() > 0 && candidatePositionsChunk.get(candidateWindowIndex) <= head) { + final long pos = candidatePositionsChunk.get(candidateWindowIndex); + final long key = candidateRowKeysChunk.get(candidateWindowIndex); + final long val = candidateValuesChunk.get(candidateWindowIndex); + + push(context, key, val); + + windowSelector.add(pos); + windowRowKeys.add(key); + + if (++candidateWindowIndex >= candidatePositionsChunk.size()) { + // load the next chunk in order + loadWindowChunks(key + 1); + } + } + + if (windowSelector.isEmpty()) { + reset(context); + } + } + } + + public BaseWindowedLongUpdateByOperator(@NotNull final MatchPair pair, + @NotNull final String[] affectingColumns, + @NotNull final OperationControl control, + @Nullable final LongRecordingUpdateByOperator timeRecorder, + final long reverseTimeScaleUnits, + final long forwardTimeScaleUnits, + @Nullable final RowRedirection rowRedirection, + @NotNull final ColumnSource valueSource + // region extra-constructor-args + // endregion extra-constructor-args + ) { + super(pair, affectingColumns, control, timeRecorder, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); + this.valueSource = valueSource; + // region constructor + // endregion constructor + } + + // region extra-methods + // endregion extra-methods + + public abstract void push(UpdateContext context, long key, long val); + public abstract void pop(UpdateContext context, long key); + public abstract void reset(UpdateContext context); + + @Override + public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, boolean usingBuckets, boolean isUpstreamAppendOnly) { + final Context ctx = (Context) context; + ctx.workingRowSet = resultSourceRowSet; + } + + + @Override + public void initializeFor(@NotNull final UpdateContext context, + @NotNull final RowSet updateRowSet, + @NotNull final UpdateBy.UpdateType type) { + final Context ctx = (Context) context; + ctx.currentUpdateType = type; + + if (type == UpdateBy.UpdateType.Add || type == UpdateBy.UpdateType.Reprocess) { + long windowStartKey = computeFirstAffectingKey(updateRowSet.firstRowKey(), ctx.workingRowSet); + ctx.loadWindowChunks(windowStartKey); + } + } + + @Override + public void finishFor(@NotNull final UpdateContext updateContext, @NotNull final UpdateBy.UpdateType type) { + final Context ctx = (Context) updateContext; + if(type == UpdateBy.UpdateType.Reprocess && ctx.modifiedBuilder != null) { + ctx.newModified = ctx.modifiedBuilder.build(); + } + } + + @Override + public boolean requiresValues(@NotNull final UpdateContext context) { + // windowed operators don't need current values supplied to them, they only care about windowed values which + // may or may not intersect with the column values + return false; + } + + @NotNull + @Override + final public RowSet getAdditionalModifications(@NotNull final UpdateContext ctx) { + return ((Context)ctx).newModified == null ? RowSetFactory.empty() : ((Context)ctx).newModified; + } + + @Override + final public boolean anyModified(@NotNull final UpdateContext ctx) { + return ((Context)ctx).newModified != null; + } + + // region Addition + @Override + public void addChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk values, + long bucketPosition) { + final Context ctx = (Context) updateContext; + + if (recorder == null) { + // use position data to determine the windows + ctx.loadDataChunks(inputKeys); + } else { + // use timestamp data to determine the windows + } + doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); + } + + /** + * Add a chunk of values to the operator. + * + * @param ctx the context object + * @param inputKeys the input keys for the chunk + * @param workingChunk the chunk of values + * @param bucketPosition the bucket position that the values belong to. + */ + protected abstract void doAddChunk(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk workingChunk, + final long bucketPosition); + + // endregion + + // region Reprocessing + + public void resetForReprocess(@NotNull final UpdateContext context, + @NotNull final RowSet sourceRowSet, + long firstUnmodifiedKey) { + final Context ctx = (Context) context; + ctx.workingRowSet = sourceRowSet; + } + + @Override + public void resetForReprocessBucketed(@NotNull final UpdateContext context, + @NotNull final RowSet bucketRowSet, + final long bucketPosition, + final long firstUnmodifiedKey) { + final Context ctx = (Context) context; + } + + @Override + public void reprocessChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk valuesChunk, + @NotNull final RowSet postUpdateSourceIndex) { + final Context ctx = (Context) updateContext; + if (recorder == null) { + // use position data to determine the windows + ctx.loadDataChunks(inputKeys); + } else { + // use timestamp data to determine the windows + } + doAddChunk(ctx, inputKeys, keyChunk, valuesChunk, 0); + ctx.getModifiedBuilder().appendRowSequence(inputKeys); + } + + @Override + public void reprocessChunkBucketed(@NotNull UpdateContext updateContext, + @NotNull final RowSequence chunkOk, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk runStartPositions, + @NotNull final IntChunk runLengths) { + addChunkBucketed(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); + ((Context)updateContext).getModifiedBuilder().appendRowSequence(chunkOk); + } + + // endregion + + // region No-Op Operations + + @Override + final public void modifyChunk(@NotNull final UpdateContext updateContext, + @Nullable final LongChunk prevKeyChunk, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk prevValuesChunk, + @NotNull final Chunk postValuesChunk, + long bucketPosition) { + } + + @Override + final public void removeChunk(@NotNull final UpdateContext updateContext, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk prevValuesChunk, + long bucketPosition) { + } + + @Override + final public void applyShift(@NotNull final UpdateContext updateContext, + @NotNull final RowSet prevIndex, + @NotNull final RowSetShiftData shifted) { + } + // endregion +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java new file mode 100644 index 00000000000..922eb40a7aa --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java @@ -0,0 +1,393 @@ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit BaseWindowedCharUpdateByOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +package io.deephaven.engine.table.impl.updateby.internal; + +import io.deephaven.api.updateby.OperationControl; +import io.deephaven.base.LongRingBuffer; +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.rowset.*; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.table.ChunkSource; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.UpdateByWindowedOperator; +import io.deephaven.engine.table.impl.util.RowRedirection; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +public abstract class BaseWindowedShortUpdateByOperator extends UpdateByWindowedOperator { + protected final ColumnSource valueSource; + + protected boolean initialized = false; + + // region extra-fields + // endregion extra-fields + + protected class Context extends UpdateWindowedContext { + public RowSet workingRowSet = null; + + // candidate data for the window + public final int WINDOW_CHUNK_SIZE = 1024; + + // data that is actually in the current window + public LongRingBuffer windowRowKeys = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); + // the selector that determines whether this value should be in the window, positions for tick-based and + // timestamps for time-based operators + public LongRingBuffer windowSelector = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); + + public RowSequence.Iterator windowIterator = null; + + public WritableShortChunk candidateValuesChunk; + public WritableLongChunk candidateRowKeysChunk; + public WritableLongChunk candidatePositionsChunk; + public WritableLongChunk candidateTimestampsChunk; + + public int candidateWindowIndex = 0; + + // position data for the chunk being currently processed + public WritableLongChunk valuePositionChunk; + + // other useful stuff + public UpdateBy.UpdateType currentUpdateType; + + public RowSetBuilderSequential modifiedBuilder; + public RowSet newModified; + + public RowSetBuilderSequential getModifiedBuilder() { + if(modifiedBuilder == null) { + modifiedBuilder = RowSetFactory.builderSequential(); + } + return modifiedBuilder; + } + + @Override + public void close() { + if (windowIterator != null) { + windowIterator.close(); + windowIterator = null; + } + + if (candidateValuesChunk != null) { + candidateValuesChunk.close(); + candidateValuesChunk = null; + } + + if (candidateRowKeysChunk != null) { + candidateRowKeysChunk.close(); + candidateRowKeysChunk = null; + } + + if (candidatePositionsChunk != null) { + candidatePositionsChunk.close(); + candidatePositionsChunk = null; + } + + if (valuePositionChunk != null) { + valuePositionChunk.close(); + valuePositionChunk = null; + } + + // no need to close, just release the reference + workingRowSet = null; + } + + /*** + * Fill the working chunks with data for this key + * + * @param startKey the key for which we want to + */ + public void loadWindowChunks(final long startKey) { + // TODO: make sure this works for bucketed + if (windowIterator == null) { + windowIterator = workingRowSet.getRowSequenceIterator(); + } + windowIterator.advance(startKey); + + RowSequence windowRowSequence = windowIterator.getNextRowSequenceWithLength(WINDOW_CHUNK_SIZE); + + // fill the window values chunk + if (candidateValuesChunk == null) { + candidateValuesChunk = WritableShortChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + } + try (ChunkSource.FillContext fc = valueSource.makeFillContext(WINDOW_CHUNK_SIZE)){ + valueSource.fillChunk(fc, candidateValuesChunk, windowRowSequence); + } + + // fill the window keys chunk + if (candidateRowKeysChunk == null) { + candidateRowKeysChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + } + windowRowSequence.fillRowKeyChunk(candidateRowKeysChunk); + + if (recorder == null) { + // get position data for the window items (relative to the table or bucket rowset) + if (candidatePositionsChunk == null) { + candidatePositionsChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + } + + // TODO: gotta be a better way than creating two rowsets + try (final RowSet rs = windowRowSequence.asRowSet(); + final RowSet positions = workingRowSet.invert(rs)) { + positions.fillRowKeyChunk(candidatePositionsChunk); + } + } else { + // get timestamp values from the recorder column source + if (candidateTimestampsChunk == null) { + candidateTimestampsChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + } + try (final ChunkSource.FillContext fc = recorder.getColumnSource().makeFillContext(WINDOW_CHUNK_SIZE)) { + recorder.getColumnSource().fillChunk(fc, candidateTimestampsChunk, windowRowSequence); + } + } + + // reset the index to beginning of the chunks + candidateWindowIndex = 0; + } + + /*** + * Fill the working chunks with data for this key + * + * @param inputKeys the keys for which we want to get position or timestamp values + */ + public void loadDataChunks(final RowSequence inputKeys) { + if (recorder != null) { + // timestamp data will be available from the recorder + return; + } + + if (valuePositionChunk == null) { + valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); + } else if (valuePositionChunk.capacity() < inputKeys.size()) { + valuePositionChunk.setSize(inputKeys.intSize()); + } + + // produce position data for the window (will be timestamps for time-based) + // TODO: gotta be a better way than creating two rowsets + try (final RowSet rs = inputKeys.asRowSet(); + final RowSet positions = workingRowSet.invert(rs)) { + positions.fillRowKeyChunk(valuePositionChunk); + } + } + + public void fillWindowTicks(UpdateContext context, long currentPos) { + // compute the head and tail (inclusive) + final long tail = Math.max(0, currentPos - reverseTimeScaleUnits + 1); + final long head = Math.min(workingRowSet.size() - 1, currentPos + forwardTimeScaleUnits); + + while (windowSelector.peek(Long.MAX_VALUE) < tail) { + final long pos = windowSelector.remove(); + final long key = windowRowKeys.remove(); + + pop(context, key); + } + + + // look at the window data and push until satisfied or at the end of the rowset + while (candidatePositionsChunk.size() > 0 && candidatePositionsChunk.get(candidateWindowIndex) <= head) { + final long pos = candidatePositionsChunk.get(candidateWindowIndex); + final long key = candidateRowKeysChunk.get(candidateWindowIndex); + final short val = candidateValuesChunk.get(candidateWindowIndex); + + push(context, key, val); + + windowSelector.add(pos); + windowRowKeys.add(key); + + if (++candidateWindowIndex >= candidatePositionsChunk.size()) { + // load the next chunk in order + loadWindowChunks(key + 1); + } + } + + if (windowSelector.isEmpty()) { + reset(context); + } + } + } + + public BaseWindowedShortUpdateByOperator(@NotNull final MatchPair pair, + @NotNull final String[] affectingColumns, + @NotNull final OperationControl control, + @Nullable final LongRecordingUpdateByOperator timeRecorder, + final long reverseTimeScaleUnits, + final long forwardTimeScaleUnits, + @Nullable final RowRedirection rowRedirection, + @NotNull final ColumnSource valueSource + // region extra-constructor-args + // endregion extra-constructor-args + ) { + super(pair, affectingColumns, control, timeRecorder, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); + this.valueSource = valueSource; + // region constructor + // endregion constructor + } + + // region extra-methods + // endregion extra-methods + + public abstract void push(UpdateContext context, long key, short val); + public abstract void pop(UpdateContext context, long key); + public abstract void reset(UpdateContext context); + + @Override + public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, boolean usingBuckets, boolean isUpstreamAppendOnly) { + final Context ctx = (Context) context; + ctx.workingRowSet = resultSourceRowSet; + } + + + @Override + public void initializeFor(@NotNull final UpdateContext context, + @NotNull final RowSet updateRowSet, + @NotNull final UpdateBy.UpdateType type) { + final Context ctx = (Context) context; + ctx.currentUpdateType = type; + + if (type == UpdateBy.UpdateType.Add || type == UpdateBy.UpdateType.Reprocess) { + long windowStartKey = computeFirstAffectingKey(updateRowSet.firstRowKey(), ctx.workingRowSet); + ctx.loadWindowChunks(windowStartKey); + } + } + + @Override + public void finishFor(@NotNull final UpdateContext updateContext, @NotNull final UpdateBy.UpdateType type) { + final Context ctx = (Context) updateContext; + if(type == UpdateBy.UpdateType.Reprocess && ctx.modifiedBuilder != null) { + ctx.newModified = ctx.modifiedBuilder.build(); + } + } + + @Override + public boolean requiresValues(@NotNull final UpdateContext context) { + // windowed operators don't need current values supplied to them, they only care about windowed values which + // may or may not intersect with the column values + return false; + } + + @NotNull + @Override + final public RowSet getAdditionalModifications(@NotNull final UpdateContext ctx) { + return ((Context)ctx).newModified == null ? RowSetFactory.empty() : ((Context)ctx).newModified; + } + + @Override + final public boolean anyModified(@NotNull final UpdateContext ctx) { + return ((Context)ctx).newModified != null; + } + + // region Addition + @Override + public void addChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk values, + long bucketPosition) { + final Context ctx = (Context) updateContext; + + if (recorder == null) { + // use position data to determine the windows + ctx.loadDataChunks(inputKeys); + } else { + // use timestamp data to determine the windows + } + doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); + } + + /** + * Add a chunk of values to the operator. + * + * @param ctx the context object + * @param inputKeys the input keys for the chunk + * @param workingChunk the chunk of values + * @param bucketPosition the bucket position that the values belong to. + */ + protected abstract void doAddChunk(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk workingChunk, + final long bucketPosition); + + // endregion + + // region Reprocessing + + public void resetForReprocess(@NotNull final UpdateContext context, + @NotNull final RowSet sourceRowSet, + long firstUnmodifiedKey) { + final Context ctx = (Context) context; + ctx.workingRowSet = sourceRowSet; + } + + @Override + public void resetForReprocessBucketed(@NotNull final UpdateContext context, + @NotNull final RowSet bucketRowSet, + final long bucketPosition, + final long firstUnmodifiedKey) { + final Context ctx = (Context) context; + } + + @Override + public void reprocessChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk valuesChunk, + @NotNull final RowSet postUpdateSourceIndex) { + final Context ctx = (Context) updateContext; + if (recorder == null) { + // use position data to determine the windows + ctx.loadDataChunks(inputKeys); + } else { + // use timestamp data to determine the windows + } + doAddChunk(ctx, inputKeys, keyChunk, valuesChunk, 0); + ctx.getModifiedBuilder().appendRowSequence(inputKeys); + } + + @Override + public void reprocessChunkBucketed(@NotNull UpdateContext updateContext, + @NotNull final RowSequence chunkOk, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk runStartPositions, + @NotNull final IntChunk runLengths) { + addChunkBucketed(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); + ((Context)updateContext).getModifiedBuilder().appendRowSequence(chunkOk); + } + + // endregion + + // region No-Op Operations + + @Override + final public void modifyChunk(@NotNull final UpdateContext updateContext, + @Nullable final LongChunk prevKeyChunk, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk prevValuesChunk, + @NotNull final Chunk postValuesChunk, + long bucketPosition) { + } + + @Override + final public void removeChunk(@NotNull final UpdateContext updateContext, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk prevValuesChunk, + long bucketPosition) { + } + + @Override + final public void applyShift(@NotNull final UpdateContext updateContext, + @NotNull final RowSet prevIndex, + @NotNull final RowSetShiftData shifted) { + } + // endregion +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/LongRecordingUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/LongRecordingUpdateByOperator.java index 8a74ba83326..f8c26d7f6a9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/LongRecordingUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/LongRecordingUpdateByOperator.java @@ -29,7 +29,7 @@ /** * An operator that simply remembers the current chunks during the add and reprocess phases. */ -public class LongRecordingUpdateByOperator implements UpdateByOperator { +public class LongRecordingUpdateByOperator extends UpdateByCumulativeOperator { private final String inputColumnName; private final String[] affectingColumns; private final ColumnSource columnSource; @@ -43,7 +43,7 @@ public LongRecordingUpdateByOperator(@NotNull final String inputColumnName, this.columnSource = ReinterpretUtils.maybeConvertToPrimitive(columnSource); } - private class RecordingContext implements UpdateContext { + private class RecordingContext extends UpdateCumulativeContext { private LongChunk addedChunk; @Override @@ -73,13 +73,22 @@ public long getCurrentLong(final long key) { return columnSource.getLong(key); } + /** + * Get the current underlying {@link ColumnSource}. + * + * @return the current value at the key within the column source. + */ + public ColumnSource getColumnSource() { + return columnSource; + } + @Override public void reprocessChunk(@NotNull UpdateContext updateContext, @NotNull RowSequence inputKeys, @Nullable LongChunk keyChunk, @NotNull Chunk valuesChunk, @NotNull RowSet postUpdateSourceIndex) { currentContext.addedChunk = valuesChunk.asLongChunk(); } @Override - public void reprocessChunk(@NotNull UpdateContext updateContext, @NotNull RowSequence chunkOk, @NotNull Chunk values, @NotNull LongChunk keyChunk, @NotNull IntChunk bucketPositions, @NotNull IntChunk runStartPositions, @NotNull IntChunk runLengths) { + public void reprocessChunkBucketed(@NotNull UpdateContext updateContext, @NotNull RowSequence chunkOk, @NotNull Chunk values, @NotNull LongChunk keyChunk, @NotNull IntChunk bucketPositions, @NotNull IntChunk runStartPositions, @NotNull IntChunk runLengths) { currentContext.addedChunk = values.asLongChunk(); } @@ -91,27 +100,26 @@ public UpdateContext makeUpdateContext(int chunkSize) { @Override public void addChunk(@NotNull UpdateContext updateContext, - @NotNull RowSequence inputKeys, - @Nullable LongChunk keyChunk, - @NotNull Chunk values, - long bucketPosition) { + @NotNull RowSequence inputKeys, + @Nullable LongChunk keyChunk, + @NotNull Chunk values, + long bucketPosition) { currentContext.addedChunk = values.asLongChunk(); } @Override - public void addChunk(@NotNull UpdateContext context, - @NotNull Chunk values, - @NotNull LongChunk keyChunk, - @NotNull IntChunk bucketPositions, - @NotNull IntChunk startPositions, - @NotNull IntChunk runLengths) { + public void addChunkBucketed(@NotNull UpdateContext context, + @NotNull Chunk values, + @NotNull LongChunk keyChunk, + @NotNull IntChunk bucketPositions, + @NotNull IntChunk startPositions, + @NotNull IntChunk runLengths) { currentContext.addedChunk = values.asLongChunk(); } // region Unused methods @Override public void setBucketCapacity(int capacity) { - } @NotNull @@ -140,17 +148,14 @@ public Map> getOutputColumns() { @Override public void initializeForUpdate(@NotNull UpdateContext ctx, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceIndex, boolean usingBuckets, boolean isUpstreamAppendOnly) { - } @Override - public void initializeFor(@NotNull UpdateContext updateContext, @NotNull RowSet updateIndex, @NotNull UpdateBy.UpdateType type) { - + public void initializeFor(@NotNull UpdateContext updateContext, @NotNull RowSet updateRowSet, @NotNull UpdateBy.UpdateType type) { } @Override public void finishFor(@NotNull UpdateContext updateContext, @NotNull UpdateBy.UpdateType type) { - } @NotNull @@ -166,7 +171,6 @@ public boolean anyModified(@NotNull UpdateContext ctx) { @Override public void startTrackingPrev() { - } @Override @@ -220,7 +224,7 @@ public void resetForReprocess(@NotNull UpdateContext context, @NotNull RowSet so } @Override - public void resetForReprocess(@NotNull UpdateContext context, @NotNull RowSet bucketIndex, long bucketPosition, long firstUnmodifiedKey) { + public void resetForReprocessBucketed(@NotNull UpdateContext context, @NotNull RowSet bucketIndex, long bucketPosition, long firstUnmodifiedKey) { } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java index cf3d6c4e3af..7da71c87883 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java @@ -41,12 +41,12 @@ public ByteCumMinMaxOperator(@NotNull final MatchPair inputPair, // endregion extra-methods @Override - public void addChunk(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { + public void addChunkBucketed(@NotNull final UpdateContext context, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk startPositions, + @NotNull final IntChunk runLengths) { final ByteChunk asBytes = values.asByteChunk(); final Context ctx = (Context) context; for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java index 2c61c6e6b00..67a73e77466 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java @@ -35,12 +35,12 @@ public DoubleCumMinMaxOperator(@NotNull final MatchPair inputPair, } @Override - public void addChunk(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { + public void addChunkBucketed(@NotNull final UpdateContext context, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk startPositions, + @NotNull final IntChunk runLengths) { final DoubleChunk asDoubles = values.asDoubleChunk(); final Context ctx = (Context) context; for (int runIdx = 0; runIdx < startPositions.size(); runIdx++) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java index d720fa77344..46eebe3cf8d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java @@ -30,12 +30,12 @@ public FloatCumMinMaxOperator(@NotNull final MatchPair inputPair, } @Override - public void addChunk(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { + public void addChunkBucketed(@NotNull final UpdateContext context, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk startPositions, + @NotNull final IntChunk runLengths) { final FloatChunk asFloats = values.asFloatChunk(); final Context ctx = (Context) context; for (int runIdx = 0; runIdx < startPositions.size(); runIdx++) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java index 775bfa23e8b..6bf964156ca 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java @@ -41,12 +41,12 @@ public IntCumMinMaxOperator(@NotNull final MatchPair inputPair, // endregion extra-methods @Override - public void addChunk(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { + public void addChunkBucketed(@NotNull final UpdateContext context, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk startPositions, + @NotNull final IntChunk runLengths) { final IntChunk asIntegers = values.asIntChunk(); final Context ctx = (Context) context; for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java index 4abc476fe53..277ea9b8e21 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java @@ -62,12 +62,12 @@ public Map> getOutputColumns() { // endregion extra-methods @Override - public void addChunk(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { + public void addChunkBucketed(@NotNull final UpdateContext context, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk startPositions, + @NotNull final IntChunk runLengths) { final LongChunk asLongs = values.asLongChunk(); final Context ctx = (Context) context; for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java index ef2feb287ac..99fee944c04 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java @@ -36,12 +36,12 @@ public ShortCumMinMaxOperator(@NotNull final MatchPair inputPair, // endregion extra-methods @Override - public void addChunk(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { + public void addChunkBucketed(@NotNull final UpdateContext context, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk startPositions, + @NotNull final IntChunk runLengths) { final ShortChunk asShorts = values.asShortChunk(); final Context ctx = (Context) context; for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java index 3cd066dbbe6..6f9373d80e6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java @@ -33,12 +33,12 @@ public ByteCumProdOperator(final @NotNull MatchPair inputPair, } @Override - public void addChunk(final @NotNull UpdateContext context, - final @NotNull Chunk values, - final @NotNull LongChunk keyChunk, - final @NotNull IntChunk bucketPositions, - final @NotNull IntChunk startPositions, - final @NotNull IntChunk runLengths) { + public void addChunkBucketed(final @NotNull UpdateContext context, + final @NotNull Chunk values, + final @NotNull LongChunk keyChunk, + final @NotNull IntChunk bucketPositions, + final @NotNull IntChunk startPositions, + final @NotNull IntChunk runLengths) { final Context ctx = (Context) context; final ByteChunk asBytes = values.asByteChunk(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java index d3df04ce7ad..6b30cf650e6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java @@ -32,12 +32,12 @@ public DoubleCumProdOperator(@NotNull final MatchPair inputPair, } @Override - public void addChunk(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { + public void addChunkBucketed(@NotNull final UpdateContext context, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk startPositions, + @NotNull final IntChunk runLengths) { final Context ctx = (Context) context; for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { final int runStart = startPositions.get(runIdx); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java index 529e0cf4b13..87292ea984e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java @@ -27,12 +27,12 @@ public FloatCumProdOperator(@NotNull final MatchPair inputPair, } @Override - public void addChunk(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { + public void addChunkBucketed(@NotNull final UpdateContext context, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk startPositions, + @NotNull final IntChunk runLengths) { final Context ctx = (Context) context; for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { final int runStart = startPositions.get(runIdx); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java index a84eda124aa..b0a2346f0fd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java @@ -33,12 +33,12 @@ public IntCumProdOperator(final @NotNull MatchPair inputPair, } @Override - public void addChunk(final @NotNull UpdateContext context, - final @NotNull Chunk values, - final @NotNull LongChunk keyChunk, - final @NotNull IntChunk bucketPositions, - final @NotNull IntChunk startPositions, - final @NotNull IntChunk runLengths) { + public void addChunkBucketed(final @NotNull UpdateContext context, + final @NotNull Chunk values, + final @NotNull LongChunk keyChunk, + final @NotNull IntChunk bucketPositions, + final @NotNull IntChunk startPositions, + final @NotNull IntChunk runLengths) { final Context ctx = (Context) context; final IntChunk asIntegers = values.asIntChunk(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java index 72d2d50a883..ce7931bc19f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java @@ -33,12 +33,12 @@ public LongCumProdOperator(final @NotNull MatchPair inputPair, } @Override - public void addChunk(final @NotNull UpdateContext context, - final @NotNull Chunk values, - final @NotNull LongChunk keyChunk, - final @NotNull IntChunk bucketPositions, - final @NotNull IntChunk startPositions, - final @NotNull IntChunk runLengths) { + public void addChunkBucketed(final @NotNull UpdateContext context, + final @NotNull Chunk values, + final @NotNull LongChunk keyChunk, + final @NotNull IntChunk bucketPositions, + final @NotNull IntChunk startPositions, + final @NotNull IntChunk runLengths) { final Context ctx = (Context) context; final LongChunk asLongs = values.asLongChunk(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java index f432b09a061..ea90dd54c9d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java @@ -28,12 +28,12 @@ public ShortCumProdOperator(final @NotNull MatchPair inputPair, } @Override - public void addChunk(final @NotNull UpdateContext context, - final @NotNull Chunk values, - final @NotNull LongChunk keyChunk, - final @NotNull IntChunk bucketPositions, - final @NotNull IntChunk startPositions, - final @NotNull IntChunk runLengths) { + public void addChunkBucketed(final @NotNull UpdateContext context, + final @NotNull Chunk values, + final @NotNull LongChunk keyChunk, + final @NotNull IntChunk bucketPositions, + final @NotNull IntChunk startPositions, + final @NotNull IntChunk runLengths) { final Context ctx = (Context) context; final ShortChunk asShorts = values.asShortChunk(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java new file mode 100644 index 00000000000..700e1fc79be --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java @@ -0,0 +1,225 @@ +package io.deephaven.engine.table.impl.updateby.rollingsum; + +import io.deephaven.api.updateby.OperationControl; +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.chunk.sized.SizedLongChunk; +import io.deephaven.chunk.util.pools.ChunkPoolConstants; +import io.deephaven.engine.rowset.*; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.table.*; +import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.UpdateByOperator; +import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.engine.table.impl.updateby.internal.*; +import io.deephaven.engine.table.impl.util.RowRedirection; +import io.deephaven.engine.table.impl.util.SizedSafeCloseable; +import io.deephaven.util.QueryConstants; +import org.apache.commons.lang3.mutable.MutableLong; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.util.Collections; +import java.util.LinkedList; +import java.util.Map; + +public class ShortRollingSumOperator extends BaseWindowedShortUpdateByOperator { + + // RollingSum will output Long values for integral types + private final WritableColumnSource outputSource; + private final WritableColumnSource maybeInnerSource; + + // region extra-fields + // endregion extra-fields + + protected class Context extends BaseWindowedShortUpdateByOperator.Context { + public final SizedSafeCloseable fillContext; + public final SizedLongChunk outputValues; + public UpdateBy.UpdateType currentUpdateType; + + public LinkedList windowValues = new LinkedList<>(); + + public RowSetBuilderSequential modifiedBuilder; + public RowSet newModified; + + protected Context(final int chunkSize) { + this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); + this.fillContext.ensureCapacity(chunkSize); + this.outputValues = new SizedLongChunk<>(chunkSize); + } + + public RowSetBuilderSequential getModifiedBuilder() { + if(modifiedBuilder == null) { + modifiedBuilder = RowSetFactory.builderSequential(); + } + return modifiedBuilder; + } + + @Override + public void close() { + super.close(); + outputValues.close(); + fillContext.close(); + } + } + + @NotNull + @Override + public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize) { + return new Context(chunkSize); + } + + @Override + public void setChunkSize(@NotNull UpdateContext context, int chunkSize) { + ((Context)context).outputValues.ensureCapacity(chunkSize); + ((Context)context).fillContext.ensureCapacity(chunkSize); + } + + public ShortRollingSumOperator(@NotNull final MatchPair pair, + @NotNull final String[] affectingColumns, + @NotNull final OperationControl control, + @Nullable final LongRecordingUpdateByOperator recorder, + final long reverseTimeScaleUnits, + final long forwardTimeScaleUnits, + @NotNull final ColumnSource valueSource, + @Nullable final RowRedirection rowRedirection + // region extra-constructor-args + // endregion extra-constructor-args + ) { + super(pair, affectingColumns, control, recorder, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection, valueSource); + if(rowRedirection != null) { + // region create-dense + this.maybeInnerSource = new LongArraySource(); + // endregion create-dense + this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); + } else { + this.maybeInnerSource = null; + // region create-sparse + this.outputSource = new LongSparseArraySource(); + // endregion create-sparse + } + + // region constructor + // endregion constructor + } + + @Override + public void addChunkBucketed(final @NotNull UpdateByOperator.UpdateContext context, + final @NotNull Chunk values, + final @NotNull LongChunk keyChunk, + final @NotNull IntChunk bucketPositions, + final @NotNull IntChunk startPositions, + final @NotNull IntChunk runLengths) { + + final Context ctx = (Context) context; + final ShortChunk asShorts = values.asShortChunk(); + for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { + final int runStart = startPositions.get(runIdx); + final int runLength = runLengths.get(runIdx); + final int bucketPosition = bucketPositions.get(runStart); + +// try (RowSequence rs = RowSequenceFactory.wrapRowKeysChunkAsRowSequence((LongChunk) keyChunk)_ + +// RowSetBuilderSequential builder = RowSetFactory.builderSequential(); +// for (int ii = runStart; ii < runStart + runLength; ii++) { +// builder.appendKey(keyChunk.get(ii)); +// } +// +// WritableRowSet bucketRs = bucketRowSet.get(bucketPosition); +// if (bucketRs == null) { +// bucketRs = builder.build(); +// bucketRowSet.set(bucketPosition, bucketRs); +// } else { +// try (final RowSet added = builder.build()) { +// bucketRs.insert(added); +// } +// } +// +// ctx.curVal = NULL_LONG; +// ctx.currentWindow.clear(); + +// accumulate(asShorts, (LongChunk) keyChunk, ctx, runStart, runLength, bucketRs); +// bucketLastVal.set(bucketPosition, ctx.curVal); + } + //noinspection unchecked + outputSource.fillFromChunkUnordered(ctx.fillContext.get(), ctx.outputValues.get(), (LongChunk) keyChunk); + } + + @Override + public void push(UpdateContext context, long key, short val) { + final Context ctx = (Context) context; + ctx.windowValues.addLast(val); + } + + @Override + public void pop(UpdateContext context, long key) { + final Context ctx = (Context) context; + ctx.windowValues.pop(); + } + + @Override + public void reset(UpdateContext context) { + final Context ctx = (Context) context; + } + + @Override + public void doAddChunk(@NotNull final BaseWindowedShortUpdateByOperator.Context context, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk workingChunk, + final long groupPosition) { + final Context ctx = (Context) context; + + computeTicks(ctx, 0, workingChunk.size()); + //noinspection unchecked + outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); + } + + private void computeTicks(@NotNull final Context ctx, + final int runStart, + final int runLength) { + + final WritableLongChunk localOutputValues = ctx.outputValues.get(); + for (int ii = runStart; ii < runStart + runLength; ii++) { + if (recorder == null) { + ctx.fillWindowTicks(ctx, ctx.valuePositionChunk.get(ii)); + } + + MutableLong sum = new MutableLong(0); + ctx.windowValues.forEach(v-> { + if (v != null && v != QueryConstants.NULL_SHORT) { + sum.add(v); + } + }); + + // this call generates the push/pop calls to satisfy the window +// ctx.fillWindow(key, postUpdateSourceIndex); + + localOutputValues.set(ii, sum.getValue()); + } + } + + @NotNull + @Override + public Map> getOutputColumns() { + return Collections.singletonMap(pair.leftColumn, outputSource); + } + + @Override + public void startTrackingPrev() { + outputSource.startTrackingPrevValues(); + if(isRedirected) { + maybeInnerSource.startTrackingPrevValues(); + } + } + + @Override + public void applyOutputShift(@NotNull final UpdateContext context, + @NotNull final RowSet subIndexToShift, + final long delta) { + ((LongSparseArraySource)outputSource).shift(subIndexToShift, delta); + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java index 98c1f7e746c..9429d708dee 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java @@ -38,12 +38,12 @@ public ByteCumSumOperator(@NotNull final MatchPair pair, } @Override - public void addChunk(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { + public void addChunkBucketed(@NotNull final UpdateContext context, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk startPositions, + @NotNull final IntChunk runLengths) { final ByteChunk asBytes = values.asByteChunk(); final Context ctx = (Context) context; for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java index 4d50747e978..6ea95acca2e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java @@ -32,12 +32,12 @@ public DoubleCumSumOperator(@NotNull final MatchPair pair, } @Override - public void addChunk(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { + public void addChunkBucketed(@NotNull final UpdateContext context, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk startPositions, + @NotNull final IntChunk runLengths) { final DoubleChunk asDoubles = values.asDoubleChunk(); final Context ctx = (Context) context; final WritableDoubleChunk localOutputChunk = ctx.outputValues.get(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java index 92803d365fa..225f67e2adb 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java @@ -27,12 +27,12 @@ public FloatCumSumOperator(@NotNull final MatchPair pair, } @Override - public void addChunk(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { + public void addChunkBucketed(@NotNull final UpdateContext context, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk startPositions, + @NotNull final IntChunk runLengths) { final FloatChunk asFloats = values.asFloatChunk(); final Context ctx = (Context) context; final WritableFloatChunk localOutputChunk = ctx.outputValues.get(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java index 699107e7ea3..7c7045b20ae 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java @@ -35,12 +35,12 @@ public IntCumSumOperator(@NotNull final MatchPair pair, } @Override - public void addChunk(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { + public void addChunkBucketed(@NotNull final UpdateContext context, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk startPositions, + @NotNull final IntChunk runLengths) { final IntChunk asIntegers = values.asIntChunk(); final Context ctx = (Context) context; for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java index f09492e12cd..0de21f4705d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java @@ -35,12 +35,12 @@ public LongCumSumOperator(@NotNull final MatchPair pair, } @Override - public void addChunk(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { + public void addChunkBucketed(@NotNull final UpdateContext context, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk startPositions, + @NotNull final IntChunk runLengths) { final LongChunk asLongs = values.asLongChunk(); final Context ctx = (Context) context; for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java index 95113f97a31..9fbf844290b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java @@ -30,12 +30,12 @@ public ShortCumSumOperator(@NotNull final MatchPair pair, } @Override - public void addChunk(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { + public void addChunkBucketed(@NotNull final UpdateContext context, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk startPositions, + @NotNull final IntChunk runLengths) { final ShortChunk asShorts = values.asShortChunk(); final Context ctx = (Context) context; for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java new file mode 100644 index 00000000000..10e4f8b7813 --- /dev/null +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java @@ -0,0 +1,504 @@ +package io.deephaven.engine.table.impl.updateby; + +import io.deephaven.api.updateby.UpdateByControl; +import io.deephaven.api.updateby.UpdateByOperation; +import io.deephaven.engine.table.ColumnDefinition; +import io.deephaven.engine.table.PartitionedTable; +import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.impl.*; +import io.deephaven.engine.updategraph.UpdateGraphProcessor; +import io.deephaven.test.types.OutOfBandTest; +import org.jetbrains.annotations.NotNull; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import java.io.IOException; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.util.Arrays; +import java.util.List; +import java.util.Random; + +import static io.deephaven.engine.table.impl.GenerateTableUpdates.generateAppends; +import static io.deephaven.engine.table.impl.RefreshingTableTestCase.simulateShiftAwareStep; +import static io.deephaven.engine.table.impl.TstUtils.assertTableEquals; +import static io.deephaven.engine.table.impl.TstUtils.testTable; +import static io.deephaven.engine.util.TableTools.*; +import static io.deephaven.function.Basic.isNull; +import static io.deephaven.util.QueryConstants.*; +import static org.junit.Assert.assertArrayEquals; + +@Category(OutOfBandTest.class) +public class TestRollingSum extends BaseUpdateByTest { + // region Zero Key Tests + + @Test + public void testStaticZeroKey() { + final QueryTable t = createTestTable(10000, false, false, false, 0x31313131).t; + t.setRefreshing(false); + + int prevTicks = 500; + int postTicks = 0; + + final Table summed = t.updateBy(UpdateByOperation.RollingSum(prevTicks)); + for (String col : t.getDefinition().getColumnNamesArray()) { + assertWithRollingSumTicks(t.getColumn(col).getDirect(), summed.getColumn(col).getDirect(), + summed.getColumn(col).getType(), prevTicks, postTicks); + } + } + + // endregion + + // region Bucketed Tests + + @Test + public void testNullOnBucketChange() throws IOException { + final TableWithDefaults t = testTable(stringCol("Sym", "A", "A", "B", "B"), + byteCol("ByteVal", (byte) 1, (byte) 2, NULL_BYTE, (byte) 3), + shortCol("ShortVal", (short) 1, (short) 2, NULL_SHORT, (short) 3), + intCol("IntVal", 1, 2, NULL_INT, 3)); + + final TableWithDefaults expected = testTable(stringCol("Sym", "A", "A", "B", "B"), + byteCol("ByteVal", (byte) 1, (byte) 2, NULL_BYTE, (byte) 3), + shortCol("ShortVal", (short) 1, (short) 2, NULL_SHORT, (short) 3), + intCol("IntVal", 1, 2, NULL_INT, 3), + longCol("ByteValSum", 1, 3, NULL_LONG, 3), + longCol("ShortValSum", 1, 3, NULL_LONG, 3), + longCol("IntValSum", 1, 3, NULL_LONG, 3)); + + final Table r = t.updateBy(List.of( + UpdateByOperation.RollingSum(100, "ByteValSum=ByteVal"), + UpdateByOperation.RollingSum(100, "ShortValSum=ShortVal"), + UpdateByOperation.RollingSum(100, "IntValSum=IntVal")), "Sym"); + + assertTableEquals(expected, r); + } + + @Test + public void testStaticBucketed() { + doTestStaticBucketed(false); + } + + @Test + public void testStaticGroupedBucketed() { + doTestStaticBucketed(true); + } + + private void doTestStaticBucketed(boolean grouped) { + final QueryTable t = createTestTable(100000, true, grouped, false, 0x31313131).t; + + int prevTicks = 100; + int postTicks = 0; + + final Table summed = + t.updateBy(UpdateByOperation.RollingSum(prevTicks, "byteCol", "shortCol", "intCol", "longCol", "floatCol", + "doubleCol", "boolCol", "bigIntCol", "bigDecimalCol"), "Sym"); + + + final PartitionedTable preOp = t.partitionBy("Sym"); + final PartitionedTable postOp = summed.partitionBy("Sym"); + + String[] columns = t.getDefinition().getColumnStream().map(ColumnDefinition::getName).toArray(String[]::new); + + preOp.partitionedTransform(postOp, (source, actual) -> { + Arrays.stream(columns).forEach(col -> { + assertWithRollingSumTicks(source.getColumn(col).getDirect(), actual.getColumn(col).getDirect(), + actual.getColumn(col).getType(), prevTicks, postTicks); + }); + return source; + }); + } + + // endregion + + // region Live Tests + + @Test + public void testZeroKeyAppendOnly() { + doTestAppendOnly(false); + } + + @Test + public void testBucketedAppendOnly() { + doTestAppendOnly(true); + } + + private void doTestAppendOnly(boolean bucketed) { + final CreateResult result = createTestTable(10000, bucketed, false, true, 0x31313131); + final QueryTable t = result.t; + t.setAttribute(Table.ADD_ONLY_TABLE_ATTRIBUTE, Boolean.TRUE); + + final EvalNugget[] nuggets = new EvalNugget[] { + new EvalNugget() { + @Override + protected Table e() { + return bucketed ? t.updateBy(UpdateByOperation.RollingSum(100), "Sym") + : t.updateBy(UpdateByOperation.RollingSum(100)); + } + } + }; + + final Random billy = new Random(0xB177B177); + for (int ii = 0; ii < 100; ii++) { + UpdateGraphProcessor.DEFAULT.runWithinUnitTestCycle(() -> generateAppends(100, billy, t, result.infos)); + TstUtils.validate("Table", nuggets); + } + } + + @Test + public void testZeroKeyGeneralTicking() { + final CreateResult result = createTestTable(10000, false, false, true, 0x31313131); + final QueryTable t = result.t; + + final EvalNugget[] nuggets = new EvalNugget[] { + new EvalNugget() { + @Override + protected Table e() { + return t.updateBy(UpdateByOperation.RollingSum(100)); + } + } + }; + + final Random billy = new Random(0xB177B177); + for (int ii = 0; ii < 100; ii++) { + UpdateGraphProcessor.DEFAULT.runWithinUnitTestCycle( + () -> GenerateTableUpdates.generateTableUpdates(100, billy, t, result.infos)); + TstUtils.validate("Table - step " + ii, nuggets); + } + } + + @Test + public void testBucketedGeneralTicking() { + final CreateResult result = createTestTable(10000, true, false, true, 0x31313131); + final QueryTable t = result.t; + + final EvalNugget[] nuggets = new EvalNugget[] { + new EvalNugget() { + @Override + protected Table e() { + return t.updateBy(UpdateByOperation.RollingSum(100), "Sym"); + } + } + }; + + final Random billy = new Random(0xB177B177); + for (int ii = 0; ii < 100; ii++) { + try { + simulateShiftAwareStep(100, billy, t, result.infos, nuggets); + } catch (Throwable ex) { + System.out.println("Crapped out on step " + ii); + throw ex; + } + } + } + + /* + * Ideas for specialized tests: 1) Remove first index 2) Removed everything, add some back 3) Make sandwiches + */ + // endregion + + // implement these calculations as pure rolling sums with local storage + + private long[] rollingSum(byte[] values, int prevTicks, int postTicks) { + if (values == null) { + return null; + } + + if (values.length == 0) { + return new long[0]; + } + + long[] result = new long[values.length]; + + + for (int i = 0; i < values.length; i++) { + result[i] = NULL_LONG; + + // set the head and the tail + final int head = Math.max(0, i - prevTicks + 1); + final int tail = Math.min(values.length - 1, i + postTicks); + + // compute everything in this window + for (int computeIdx = head; computeIdx <= tail; computeIdx++) { + if (!isNull(values[computeIdx])) { + if (result[i] == NULL_LONG) { + result[i] = values[computeIdx]; + } else { + result[i] += values[computeIdx]; + } + } + } + } + + return result; + } + + private long[] rollingSum(short[] values, int prevTicks, int postTicks) { + if (values == null) { + return null; + } + + if (values.length == 0) { + return new long[0]; + } + + long[] result = new long[values.length]; + + + for (int i = 0; i < values.length; i++) { + result[i] = NULL_LONG; + + // set the head and the tail + final int head = Math.max(0, i - prevTicks + 1); + final int tail = Math.min(values.length - 1, i + postTicks); + + // compute everything in this window + for (int computeIdx = head; computeIdx <= tail; computeIdx++) { + if (!isNull(values[computeIdx])) { + if (result[i] == NULL_LONG) { + result[i] = values[computeIdx]; + } else { + result[i] += values[computeIdx]; + } + } + } + } + + return result; + } + + private long[] rollingSum(int[] values, int prevTicks, int postTicks) { + if (values == null) { + return null; + } + + if (values.length == 0) { + return new long[0]; + } + + long[] result = new long[values.length]; + + + for (int i = 0; i < values.length; i++) { + result[i] = NULL_LONG; + + // set the head and the tail + final int head = Math.max(0, i - prevTicks + 1); + final int tail = Math.min(values.length - 1, i + postTicks); + + // compute everything in this window + for (int computeIdx = head; computeIdx <= tail; computeIdx++) { + if (!isNull(values[computeIdx])) { + if (result[i] == NULL_LONG) { + result[i] = values[computeIdx]; + } else { + result[i] += values[computeIdx]; + } + } + } + } + + return result; + } + + private long[] rollingSum(long[] values, int prevTicks, int postTicks) { + if (values == null) { + return null; + } + + if (values.length == 0) { + return new long[0]; + } + + long[] result = new long[values.length]; + + + for (int i = 0; i < values.length; i++) { + result[i] = NULL_LONG; + + // set the head and the tail + final int head = Math.max(0, i - prevTicks + 1); + final int tail = Math.min(values.length - 1, i + postTicks); + + // compute everything in this window + for (int computeIdx = head; computeIdx <= tail; computeIdx++) { + if (!isNull(values[computeIdx])) { + if (result[i] == NULL_LONG) { + result[i] = values[computeIdx]; + } else { + result[i] += values[computeIdx]; + } + } + } + } + + return result; + } + + private float[] rollingSum(float[] values, int prevTicks, int postTicks) { + if (values == null) { + return null; + } + + if (values.length == 0) { + return new float[0]; + } + + float[] result = new float[values.length]; + + + for (int i = 0; i < values.length; i++) { + result[i] = NULL_FLOAT; + + // set the head and the tail + final int head = Math.max(0, i - prevTicks + 1); + final int tail = Math.min(values.length - 1, i + postTicks); + + // compute everything in this window + for (int computeIdx = head; computeIdx <= tail; computeIdx++) { + if (!isNull(values[computeIdx])) { + if (result[i] == NULL_FLOAT) { + result[i] = values[computeIdx]; + } else { + result[i] += values[computeIdx]; + } + } + } + } + + return result; + } + + private double[] rollingSum(double[] values, int prevTicks, int postTicks) { + if (values == null) { + return null; + } + + if (values.length == 0) { + return new double[0]; + } + + double[] result = new double[values.length]; + + + for (int i = 0; i < values.length; i++) { + result[i] = NULL_DOUBLE; + + // set the head and the tail + final int head = Math.max(0, i - prevTicks + 1); + final int tail = Math.min(values.length - 1, i + postTicks); + + // compute everything in this window + for (int computeIdx = head; computeIdx <= tail; computeIdx++) { + if (!isNull(values[computeIdx])) { + if (result[i] == NULL_DOUBLE) { + result[i] = values[computeIdx]; + } else { + result[i] += values[computeIdx]; + } + } + } + } + + return result; + } + + private long[] rollingSum(Boolean[] values, int prevTicks, int postTicks) { + if (values == null) { + return null; + } + + if (values.length == 0) { + return new long[0]; + } + + long[] result = new long[values.length]; + + + for (int i = 0; i < values.length; i++) { + result[i] = NULL_LONG; + + // set the head and the tail + final int head = Math.max(0, i - prevTicks + 1); + final int tail = Math.min(values.length - 1, i + postTicks); + + // compute everything in this window + for (int computeIdx = head; computeIdx <= tail; computeIdx++) { + if (!isNull(values[computeIdx])) { + if (result[i] == NULL_LONG) { + result[i] = values[computeIdx] ? 1 : 0; + } else { + result[i] += (values[computeIdx] ? 1 : 0); + } + } + } + } + + return result; + } + + public static Object[] rollingSum(Object[] values, final boolean isBD, int prevTicks, int postTicks) { + if (values == null) { + return null; + } + + if (values.length == 0) { + return new Object[0]; + } + + Object[] result = new Object[values.length]; + + + for (int i = 0; i < values.length; i++) { + result[i] = null; + + // set the head and the tail + final int head = Math.max(0, i - prevTicks + 1); + final int tail = Math.min(values.length - 1, i + postTicks); + + // compute everything in this window + for (int computeIdx = head; computeIdx <= tail; computeIdx++) { + if (values[computeIdx] != null) { + if (result[i] == null) { + result[i] = values[computeIdx]; + } else { + if (isBD) { + result[i] = ((BigDecimal) result[i]).add((BigDecimal) values[computeIdx], + UpdateByControl.mathContextDefault()); + } else { + result[i] = ((BigInteger) result[i]).add((BigInteger) values[computeIdx]); + } + } + } + } + } + + return result; + } + + final void assertWithRollingSumTicks(final @NotNull Object expected, final @NotNull Object actual, Class type, int prevTicks, int postTicks) { + final float deltaF = .001f; + final double deltaD = .001d; + + if (expected instanceof short[]) { + assertArrayEquals(rollingSum((short[]) expected, prevTicks, postTicks), (long[]) actual); + } +// if (expected instanceof byte[]) { +// assertArrayEquals(rollingSum((byte[]) expected, prevTicks, postTicks), (long[]) actual); +// } else if (expected instanceof short[]) { +// assertArrayEquals(rollingSum((short[]) expected, prevTicks, postTicks), (long[]) actual); +// } else if (expected instanceof int[]) { +// assertArrayEquals(rollingSum((int[]) expected, prevTicks, postTicks), (long[]) actual); +// } else if (expected instanceof long[]) { +// assertArrayEquals(rollingSum((long[]) expected, prevTicks, postTicks), (long[]) actual); +// } else if (expected instanceof float[]) { +// assertArrayEquals(rollingSum((float[]) expected, prevTicks, postTicks), (float[]) actual, deltaF); +// } else if (expected instanceof double[]) { +// assertArrayEquals(rollingSum((double[]) expected, prevTicks, postTicks), (double[]) actual, deltaD); +// } else if (expected instanceof Boolean[]) { +// assertArrayEquals(rollingSum((Boolean[]) expected, prevTicks, postTicks), (long[]) actual); +// } else { +// assertArrayEquals(rollingSum((Object[]) expected, type == BigDecimal.class, prevTicks, postTicks), (Object[]) actual); +// } + } +} diff --git a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java index 911ba17acfe..363246cf8bf 100644 --- a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java +++ b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java @@ -130,6 +130,12 @@ private static void replicateNumericOperator(@NotNull final String shortClass, @ private static void fixupByteBase(String byteResult) throws IOException { final File objectFile = new File(byteResult); List lines = FileUtils.readLines(objectFile, Charset.defaultCharset()); + + lines = addImport(lines, "import io.deephaven.util.QueryConstants;", + "import io.deephaven.engine.table.impl.sources.ByteArraySource;", + "import io.deephaven.engine.table.impl.sources.ByteSparseArraySource;", + "import io.deephaven.engine.table.WritableColumnSource;"); + lines = replaceRegion(lines, "extra-fields", Collections.singletonList(" final byte nullValue;")); lines = replaceRegion(lines, "constructor", Collections.singletonList(" this.nullValue = getNullValue();")); diff --git a/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java b/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java new file mode 100644 index 00000000000..971debbf37a --- /dev/null +++ b/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java @@ -0,0 +1,125 @@ +package io.deephaven.api.updateby.spec; + +import io.deephaven.annotations.BuildableStyle; +import io.deephaven.api.updateby.OperationControl; +import org.immutables.value.Value; +import org.immutables.value.Value.Immutable; + +import java.time.Duration; +import java.util.Optional; + +/** + * A {@link UpdateBySpec} for performing a windowed rolling sum across the specified columns + */ +@Immutable +@BuildableStyle +public abstract class RollingSumSpec extends UpdateBySpecBase { + + // most common usages first, will complete the list later + + public static RollingSumSpec ofTicks(long tickWindow) { + return of(TimeScale.ofTicks(tickWindow)); + } + + public static RollingSumSpec ofTicks(long prevTickWindow, long fwdTickWindow) { + return of(TimeScale.ofTicks(prevTickWindow), TimeScale.ofTicks(fwdTickWindow)); + } + + public static RollingSumSpec ofTime(final String timestampCol, Duration prevWindowDuration) { + return of(TimeScale.ofTime(timestampCol, prevWindowDuration)); + } + + public static RollingSumSpec ofTime(final String timestampCol, Duration prevWindowDuration, Duration fwdWindowDuration) { + return of(TimeScale.ofTime(timestampCol, prevWindowDuration), TimeScale.ofTime(timestampCol, fwdWindowDuration)); + } + + // general use contructors + + public static RollingSumSpec of(TimeScale prevTimeScale) { + return ImmutableRollingSumSpec.builder().prevTimeScale(prevTimeScale).build(); + } + + public static RollingSumSpec of(TimeScale prevTimeScale, TimeScale fwdTimeScale) { + return ImmutableRollingSumSpec.builder().prevTimeScale(prevTimeScale).fwdTimeScale(fwdTimeScale).build(); + } + +// public static RollingSumSpec of(TimeScale prevTimeScale) { +// return ImmutableWindowedOpSpec.builder().prevTimeScale(prevTimeScale).build(); +// } +// +// public static RollingSumSpec of(OperationControl control, TimeScale prevTimeScale, TimeScale fwdTimeScale) { +// return ImmutableWindowedOpSpec.builder().control(control).prevTimeScale(prevTimeScale).fwdTimeScale(fwdTimeScale).build(); +// } +// +// public static RollingSumSpec ofTime(final OperationControl control, +// final String timestampCol, +// long prevWindowTimeScaleNanos) { +// return of(control, TimeScale.ofTime(timestampCol, prevWindowTimeScaleNanos)); +// } +// +// public static RollingSumSpec ofTime(final OperationControl control, +// final String timestampCol, +// long prevWindowTimeScaleNanos, +// long fwdWindowTimeScaleNanos) { +// return of(control, TimeScale.ofTime(timestampCol, prevWindowTimeScaleNanos), TimeScale.ofTime(timestampCol, fwdWindowTimeScaleNanos)); +// } +// +// public static RollingSumSpec ofTime(final OperationControl control, +// final String timestampCol, +// Duration prevWindowDuration) { +// return of(control, TimeScale.ofTime(timestampCol, prevWindowDuration)); +// } +// +// +// public static RollingSumSpec ofTime(final OperationControl control, +// final String timestampCol, +// Duration prevWindowDuration, +// Duration fwdWindowDuration) { +// return of(control, TimeScale.ofTime(timestampCol, prevWindowDuration), TimeScale.ofTime(timestampCol, fwdWindowDuration)); +// } +// +// public static RollingSumSpec ofTicks(OperationControl control, long prevTickWindow) { +// return of(control, TimeScale.ofTicks(prevTickWindow)); +// } +// +// public static RollingSumSpec ofTicks(OperationControl control, long prevTickWindow, long fwdTickWindow) { +// return of(control, TimeScale.ofTicks(prevTickWindow), TimeScale.ofTicks(fwdTickWindow)); +// } + + + public abstract Optional control(); + + public abstract TimeScale prevTimeScale(); + + // provide a default forward-looking timescale + @Value.Default + public TimeScale fwdTimeScale() { + return TimeScale.ofTicks(0); + } + + public final OperationControl controlOrDefault() { + return control().orElseGet(OperationControl::defaultInstance); + } + + @Override + public final boolean applicableTo(Class inputType) { + return inputType.equals(short.class); + +// return +// // is primitive numeric? +// inputType.equals(double.class) || inputType.equals(float.class) +// || inputType.equals(int.class) || inputType.equals(long.class) || inputType.equals(short.class) +// || inputType.equals(byte.class) +// +// // is boxed numeric? +// || Number.class.isAssignableFrom(inputType) +// +// // is boolean? +// || inputType == boolean.class || inputType == Boolean.class; + } + + @Override + public final T walk(Visitor visitor) { + return visitor.visit(this); + } +} From a4707c75f23626c789fc6332b8d775919a1e19d3 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 10 Aug 2022 14:17:59 -0700 Subject: [PATCH 005/123] zerokey working well, bucketed upcoming --- .../table/impl/UpdateByWindowedOperator.java | 39 ++++++------ .../rollingsum/ShortRollingSumOperator.java | 22 +++---- .../table/impl/updateby/TestRollingSum.java | 63 +++++++++++++++++-- 3 files changed, 89 insertions(+), 35 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java index 0760b44c8f8..395b64614a4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java @@ -43,8 +43,8 @@ public abstract class UpdateWindowedContext implements UpdateContext { public RowSet determineAffectedRows(@NotNull final TableUpdate upstream, @NotNull final RowSet source, final boolean upstreamAppendOnly) { - // NOTE: this is fast rather than bounding to the smallest set possible (i.e. unaware of buckets and - // over-represents sparse data). Will result in computing far more than actually necessary + // NOTE: this is fast rather than bounding to the smallest set possible. Will result in computing more than + // actually necessary // TODO: return the minimal set of data for this update @@ -70,19 +70,14 @@ public RowSet determineAffectedRows(@NotNull final TableUpdate upstream, @NotNul } if (upstream.modified().isNonempty()) { - // TODO: make this more efficient - final List cols = List.of(affectingColumns); - boolean modifiedAffectingColumn = Arrays.stream(upstream.modifiedColumnSet().dirtyColumnNames()).anyMatch(cols::contains); - - if (modifiedAffectingColumn) { - // add the rows affected by the mods - builder.addRange(computeFirstAffectedKey(upstream.modified().firstRowKey(), source), - computeLastAffectedKey(upstream.modified().lastRowKey(), source)); - } + // add the rows affected by the mods + builder.addRange(computeFirstAffectedKey(upstream.modified().firstRowKey(), source), + computeLastAffectedKey(upstream.modified().lastRowKey(), source)); } - try (final RowSet ignored = affectedRows) { - affectedRows = builder.build(); + try (final RowSet ignored = affectedRows; + final RowSet brs = builder.build()) { + affectedRows = source.intersect(brs); } return affectedRows; } @@ -129,8 +124,10 @@ public long computeFirstAffectingKey(long key, @NotNull final RowSet source) { if (recorder == null) { // ticks final long keyPos = source.find(key); - final long idx = keyPos - (long) reverseTimeScaleUnits + 1; - if (idx < 0) { + final long idx = (keyPos < 0) ? -keyPos - reverseTimeScaleUnits : keyPos - reverseTimeScaleUnits + 1; + if (idx >= source.size()) { + return source.lastRowKey(); + } else if (idx < 0) { return source.firstRowKey(); } return source.get(idx); @@ -143,9 +140,11 @@ public long computeLastAffectingKey(long key, @NotNull final RowSet source) { if (recorder == null) { // ticks final long keyPos = source.find(key); - final long idx = keyPos + (long)forwardTimeScaleUnits; + final long idx = keyPos + forwardTimeScaleUnits; if (idx >= source.size()) { return source.lastRowKey(); + } else if (idx < 0) { + return source.firstRowKey(); } return source.get(idx); } @@ -157,9 +156,11 @@ public long computeFirstAffectedKey(long key, @NotNull final RowSet source) { if (recorder == null) { // ticks final long keyPos = source.find(key); - final long idx = keyPos - (long) forwardTimeScaleUnits; + final long idx = (keyPos < 0) ? -keyPos - forwardTimeScaleUnits - 1 : keyPos - forwardTimeScaleUnits; if (idx >= source.size()) { return source.lastRowKey(); + } else if (idx < 0) { + return source.firstRowKey(); } return source.get(idx); } @@ -171,9 +172,11 @@ public long computeLastAffectedKey(long key, @NotNull final RowSet source) { if (recorder == null) { // ticks final long keyPos = source.find(key); - final long idx = keyPos + (long) reverseTimeScaleUnits; + final long idx = (keyPos < 0) ? -keyPos + reverseTimeScaleUnits - 1 : keyPos + reverseTimeScaleUnits; if (idx >= source.size()) { return source.lastRowKey(); + } else if (idx < 0) { + return source.firstRowKey(); } return source.get(idx); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java index 700e1fc79be..c6ec76fefae 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java @@ -26,6 +26,8 @@ import java.util.LinkedList; import java.util.Map; +import static io.deephaven.util.QueryConstants.NULL_LONG; + public class ShortRollingSumOperator extends BaseWindowedShortUpdateByOperator { // RollingSum will output Long values for integral types @@ -42,22 +44,12 @@ protected class Context extends BaseWindowedShortUpdateByOperator.Context { public LinkedList windowValues = new LinkedList<>(); - public RowSetBuilderSequential modifiedBuilder; - public RowSet newModified; - protected Context(final int chunkSize) { this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); this.fillContext.ensureCapacity(chunkSize); this.outputValues = new SizedLongChunk<>(chunkSize); } - public RowSetBuilderSequential getModifiedBuilder() { - if(modifiedBuilder == null) { - modifiedBuilder = RowSetFactory.builderSequential(); - } - return modifiedBuilder; - } - @Override public void close() { super.close(); @@ -173,7 +165,7 @@ public void doAddChunk(@NotNull final BaseWindowedShortUpdateByOperator.Context final long groupPosition) { final Context ctx = (Context) context; - computeTicks(ctx, 0, workingChunk.size()); + computeTicks(ctx, 0, inputKeys.intSize()); //noinspection unchecked outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } @@ -188,10 +180,14 @@ private void computeTicks(@NotNull final Context ctx, ctx.fillWindowTicks(ctx, ctx.valuePositionChunk.get(ii)); } - MutableLong sum = new MutableLong(0); + MutableLong sum = new MutableLong(NULL_LONG); ctx.windowValues.forEach(v-> { if (v != null && v != QueryConstants.NULL_SHORT) { - sum.add(v); + if (sum.longValue() == NULL_LONG) { + sum.setValue(v); + } else { + sum.add(v); + } } }); diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java index 10e4f8b7813..252da68d562 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java @@ -37,10 +37,41 @@ public void testStaticZeroKey() { final QueryTable t = createTestTable(10000, false, false, false, 0x31313131).t; t.setRefreshing(false); - int prevTicks = 500; + int prevTicks = 100; int postTicks = 0; - final Table summed = t.updateBy(UpdateByOperation.RollingSum(prevTicks)); + final Table summed = t.updateBy(UpdateByOperation.RollingSum(prevTicks, postTicks)); + for (String col : t.getDefinition().getColumnNamesArray()) { + assertWithRollingSumTicks(t.getColumn(col).getDirect(), summed.getColumn(col).getDirect(), + summed.getColumn(col).getType(), prevTicks, postTicks); + } + } + + @Test + public void testStaticZeroKeyFwdWindow() { + final QueryTable t = createTestTable(10000, false, false, false, 0x31313131).t; + t.setRefreshing(false); + + int prevTicks = 0; + int postTicks = 100; + + final Table summed = t.updateBy(UpdateByOperation.RollingSum(prevTicks, postTicks)); + for (String col : t.getDefinition().getColumnNamesArray()) { + assertWithRollingSumTicks(t.getColumn(col).getDirect(), summed.getColumn(col).getDirect(), + summed.getColumn(col).getType(), prevTicks, postTicks); + } + } + + + @Test + public void testStaticZeroKeyFwdRevWindow() { + final QueryTable t = createTestTable(10000, false, false, false, 0x31313131).t; + t.setRefreshing(false); + + int prevTicks = 100; + int postTicks = 100; + + final Table summed = t.updateBy(UpdateByOperation.RollingSum(prevTicks, postTicks)); for (String col : t.getDefinition().getColumnNamesArray()) { assertWithRollingSumTicks(t.getColumn(col).getDirect(), summed.getColumn(col).getDirect(), summed.getColumn(col).getType(), prevTicks, postTicks); @@ -88,10 +119,10 @@ private void doTestStaticBucketed(boolean grouped) { final QueryTable t = createTestTable(100000, true, grouped, false, 0x31313131).t; int prevTicks = 100; - int postTicks = 0; + int postTicks = 10; final Table summed = - t.updateBy(UpdateByOperation.RollingSum(prevTicks, "byteCol", "shortCol", "intCol", "longCol", "floatCol", + t.updateBy(UpdateByOperation.RollingSum(prevTicks, postTicks, "byteCol", "shortCol", "intCol", "longCol", "floatCol", "doubleCol", "boolCol", "bigIntCol", "bigDecimalCol"), "Sym"); @@ -150,6 +181,30 @@ public void testZeroKeyGeneralTicking() { final CreateResult result = createTestTable(10000, false, false, true, 0x31313131); final QueryTable t = result.t; + final long prevTicks = 100; + + final EvalNugget[] nuggets = new EvalNugget[] { + new EvalNugget() { + @Override + protected Table e() { + return t.updateBy(UpdateByOperation.RollingSum(prevTicks)); + } + } + }; + + final Random billy = new Random(0xB177B177); + for (int ii = 0; ii < 100; ii++) { + UpdateGraphProcessor.DEFAULT.runWithinUnitTestCycle( + () -> GenerateTableUpdates.generateTableUpdates(100, billy, t, result.infos)); + TstUtils.validate("Table - step " + ii, nuggets); + } + } + + @Test + public void testZeroKeyGeneralTickingFwdWindow() { + final CreateResult result = createTestTable(10000, false, false, true, 0x31313131); + final QueryTable t = result.t; + final EvalNugget[] nuggets = new EvalNugget[] { new EvalNugget() { @Override From 3347af17cc8c5496235a479306d02cd11bbeb65f Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 10 Aug 2022 15:55:30 -0700 Subject: [PATCH 006/123] append-only zerokey supported for windowed operators --- .../engine/table/impl/BucketedUpdateBy.java | 2 ++ .../table/impl/UpdateByWindowedOperator.java | 16 ++++----- .../engine/table/impl/ZeroKeyUpdateBy.java | 4 +++ .../BaseWindowedByteUpdateByOperator.java | 35 +++++++++++-------- .../BaseWindowedCharUpdateByOperator.java | 35 +++++++++++-------- .../BaseWindowedIntUpdateByOperator.java | 35 +++++++++++-------- .../BaseWindowedLongUpdateByOperator.java | 35 +++++++++++-------- .../BaseWindowedShortUpdateByOperator.java | 35 +++++++++++-------- 8 files changed, 114 insertions(+), 83 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedUpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedUpdateBy.java index 9e84f421231..b553d12d9be 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedUpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedUpdateBy.java @@ -1131,7 +1131,9 @@ private void doInitialAdditions(final boolean useGrouping, @NotNull final MatchP processUpdateForRedirection(initialUpdate); } +// accumulateUpdatesByBucket(upstream, ctx); ctx.doAppendOnlyAdds(true, source.getRowSet()); + if (slotTracker != null) { // noinspection resource slotTracker.applyUpdates(RowSetShiftData.EMPTY); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java index 395b64614a4..1357dd74447 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java @@ -43,6 +43,14 @@ public abstract class UpdateWindowedContext implements UpdateContext { public RowSet determineAffectedRows(@NotNull final TableUpdate upstream, @NotNull final RowSet source, final boolean upstreamAppendOnly) { + // under certain circumstances, we can process directly and do not need to reprocess + if (upstreamAppendOnly && forwardTimeScaleUnits <= 0 && reverseTimeScaleUnits >= 0) { + try (final RowSet ignored = affectedRows) { + affectedRows = RowSetFactory.empty(); + } + return affectedRows; + } + // NOTE: this is fast rather than bounding to the smallest set possible. Will result in computing more than // actually necessary @@ -183,14 +191,6 @@ public long computeLastAffectedKey(long key, @NotNull final RowSet source) { return -1; } - /*** - * Windowed operators must always reprocess - */ - @Override - public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { - return false; - } - @NotNull @Override public String getInputColumnName() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java index 28412dfb88c..2984cec831c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java @@ -79,7 +79,11 @@ void doInitialAdditions() { if (rowRedirection != null && source.isRefreshing()) { processUpdateForRedirection(fakeUpdate); } + // do an addition phase for all the operators that can add directly (i.e. backwards looking) ctx.doUpdate(source.getRowSet(), source.getRowSet(), UpdateType.Add); + + // do a reprocessing phase for operators that can't add directly + ctx.reprocessRows(RowSetShiftData.EMPTY); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java index 4ec891fd6eb..53727046fe2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java @@ -39,6 +39,7 @@ public abstract class BaseWindowedByteUpdateByOperator extends UpdateByWindowedO // endregion extra-fields protected class Context extends UpdateWindowedContext { + public boolean canProcessDirectly; public RowSet workingRowSet = null; // candidate data for the window @@ -173,7 +174,8 @@ public void loadDataChunks(final RowSequence inputKeys) { if (valuePositionChunk == null) { valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); } else if (valuePositionChunk.capacity() < inputKeys.size()) { - valuePositionChunk.setSize(inputKeys.intSize()); + valuePositionChunk.close(); + valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); } // produce position data for the window (will be timestamps for time-based) @@ -261,8 +263,13 @@ protected WritableColumnSource makeDenseSource() { public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, boolean usingBuckets, boolean isUpstreamAppendOnly) { final Context ctx = (Context) context; ctx.workingRowSet = resultSourceRowSet; - } + if(!usingBuckets) { + // we can only process directly from an update if the window is entire backward-looking. Since we + // allow negative values in fwd/rev ticks and timestamps, we need to check both + ctx.canProcessDirectly = isUpstreamAppendOnly && forwardTimeScaleUnits <= 0 && reverseTimeScaleUnits >= 0;; + } + } @Override public void initializeFor(@NotNull final UpdateContext context, @@ -271,7 +278,8 @@ public void initializeFor(@NotNull final UpdateContext context, final Context ctx = (Context) context; ctx.currentUpdateType = type; - if (type == UpdateBy.UpdateType.Add || type == UpdateBy.UpdateType.Reprocess) { + if ((type == UpdateBy.UpdateType.Add && ctx.canProcessDirectly) + || type == UpdateBy.UpdateType.Reprocess) { long windowStartKey = computeFirstAffectingKey(updateRowSet.firstRowKey(), ctx.workingRowSet); ctx.loadWindowChunks(windowStartKey); } @@ -285,6 +293,12 @@ public void finishFor(@NotNull final UpdateContext updateContext, @NotNull final } } + @Override + public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { + final Context ctx = (Context) context; + return ctx.canProcessDirectly; + } + @Override public boolean requiresValues(@NotNull final UpdateContext context) { // windowed operators don't need current values supplied to them, they only care about windowed values which @@ -311,14 +325,10 @@ public void addChunk(@NotNull final UpdateContext updateContext, @NotNull final Chunk values, long bucketPosition) { final Context ctx = (Context) updateContext; - - if (recorder == null) { - // use position data to determine the windows + if (ctx.canProcessDirectly) { ctx.loadDataChunks(inputKeys); - } else { - // use timestamp data to determine the windows + doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); } - doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); } /** @@ -361,12 +371,7 @@ public void reprocessChunk(@NotNull final UpdateContext updateContext, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; - if (recorder == null) { - // use position data to determine the windows - ctx.loadDataChunks(inputKeys); - } else { - // use timestamp data to determine the windows - } + ctx.loadDataChunks(inputKeys); doAddChunk(ctx, inputKeys, keyChunk, valuesChunk, 0); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java index 996a2a72882..6c3eabd8046 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java @@ -28,6 +28,7 @@ public abstract class BaseWindowedCharUpdateByOperator extends UpdateByWindowedO // endregion extra-fields protected class Context extends UpdateWindowedContext { + public boolean canProcessDirectly; public RowSet workingRowSet = null; // candidate data for the window @@ -162,7 +163,8 @@ public void loadDataChunks(final RowSequence inputKeys) { if (valuePositionChunk == null) { valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); } else if (valuePositionChunk.capacity() < inputKeys.size()) { - valuePositionChunk.setSize(inputKeys.intSize()); + valuePositionChunk.close(); + valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); } // produce position data for the window (will be timestamps for time-based) @@ -237,8 +239,13 @@ public BaseWindowedCharUpdateByOperator(@NotNull final MatchPair pair, public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, boolean usingBuckets, boolean isUpstreamAppendOnly) { final Context ctx = (Context) context; ctx.workingRowSet = resultSourceRowSet; - } + if(!usingBuckets) { + // we can only process directly from an update if the window is entire backward-looking. Since we + // allow negative values in fwd/rev ticks and timestamps, we need to check both + ctx.canProcessDirectly = isUpstreamAppendOnly && forwardTimeScaleUnits <= 0 && reverseTimeScaleUnits >= 0;; + } + } @Override public void initializeFor(@NotNull final UpdateContext context, @@ -247,7 +254,8 @@ public void initializeFor(@NotNull final UpdateContext context, final Context ctx = (Context) context; ctx.currentUpdateType = type; - if (type == UpdateBy.UpdateType.Add || type == UpdateBy.UpdateType.Reprocess) { + if ((type == UpdateBy.UpdateType.Add && ctx.canProcessDirectly) + || type == UpdateBy.UpdateType.Reprocess) { long windowStartKey = computeFirstAffectingKey(updateRowSet.firstRowKey(), ctx.workingRowSet); ctx.loadWindowChunks(windowStartKey); } @@ -261,6 +269,12 @@ public void finishFor(@NotNull final UpdateContext updateContext, @NotNull final } } + @Override + public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { + final Context ctx = (Context) context; + return ctx.canProcessDirectly; + } + @Override public boolean requiresValues(@NotNull final UpdateContext context) { // windowed operators don't need current values supplied to them, they only care about windowed values which @@ -287,14 +301,10 @@ public void addChunk(@NotNull final UpdateContext updateContext, @NotNull final Chunk values, long bucketPosition) { final Context ctx = (Context) updateContext; - - if (recorder == null) { - // use position data to determine the windows + if (ctx.canProcessDirectly) { ctx.loadDataChunks(inputKeys); - } else { - // use timestamp data to determine the windows + doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); } - doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); } /** @@ -337,12 +347,7 @@ public void reprocessChunk(@NotNull final UpdateContext updateContext, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; - if (recorder == null) { - // use position data to determine the windows - ctx.loadDataChunks(inputKeys); - } else { - // use timestamp data to determine the windows - } + ctx.loadDataChunks(inputKeys); doAddChunk(ctx, inputKeys, keyChunk, valuesChunk, 0); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java index 8799ccd290d..1f1be4bcf0c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java @@ -33,6 +33,7 @@ public abstract class BaseWindowedIntUpdateByOperator extends UpdateByWindowedOp // endregion extra-fields protected class Context extends UpdateWindowedContext { + public boolean canProcessDirectly; public RowSet workingRowSet = null; // candidate data for the window @@ -167,7 +168,8 @@ public void loadDataChunks(final RowSequence inputKeys) { if (valuePositionChunk == null) { valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); } else if (valuePositionChunk.capacity() < inputKeys.size()) { - valuePositionChunk.setSize(inputKeys.intSize()); + valuePositionChunk.close(); + valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); } // produce position data for the window (will be timestamps for time-based) @@ -242,8 +244,13 @@ public BaseWindowedIntUpdateByOperator(@NotNull final MatchPair pair, public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, boolean usingBuckets, boolean isUpstreamAppendOnly) { final Context ctx = (Context) context; ctx.workingRowSet = resultSourceRowSet; - } + if(!usingBuckets) { + // we can only process directly from an update if the window is entire backward-looking. Since we + // allow negative values in fwd/rev ticks and timestamps, we need to check both + ctx.canProcessDirectly = isUpstreamAppendOnly && forwardTimeScaleUnits <= 0 && reverseTimeScaleUnits >= 0;; + } + } @Override public void initializeFor(@NotNull final UpdateContext context, @@ -252,7 +259,8 @@ public void initializeFor(@NotNull final UpdateContext context, final Context ctx = (Context) context; ctx.currentUpdateType = type; - if (type == UpdateBy.UpdateType.Add || type == UpdateBy.UpdateType.Reprocess) { + if ((type == UpdateBy.UpdateType.Add && ctx.canProcessDirectly) + || type == UpdateBy.UpdateType.Reprocess) { long windowStartKey = computeFirstAffectingKey(updateRowSet.firstRowKey(), ctx.workingRowSet); ctx.loadWindowChunks(windowStartKey); } @@ -266,6 +274,12 @@ public void finishFor(@NotNull final UpdateContext updateContext, @NotNull final } } + @Override + public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { + final Context ctx = (Context) context; + return ctx.canProcessDirectly; + } + @Override public boolean requiresValues(@NotNull final UpdateContext context) { // windowed operators don't need current values supplied to them, they only care about windowed values which @@ -292,14 +306,10 @@ public void addChunk(@NotNull final UpdateContext updateContext, @NotNull final Chunk values, long bucketPosition) { final Context ctx = (Context) updateContext; - - if (recorder == null) { - // use position data to determine the windows + if (ctx.canProcessDirectly) { ctx.loadDataChunks(inputKeys); - } else { - // use timestamp data to determine the windows + doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); } - doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); } /** @@ -342,12 +352,7 @@ public void reprocessChunk(@NotNull final UpdateContext updateContext, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; - if (recorder == null) { - // use position data to determine the windows - ctx.loadDataChunks(inputKeys); - } else { - // use timestamp data to determine the windows - } + ctx.loadDataChunks(inputKeys); doAddChunk(ctx, inputKeys, keyChunk, valuesChunk, 0); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java index 4dceaf9bd51..8bea2e8f218 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java @@ -33,6 +33,7 @@ public abstract class BaseWindowedLongUpdateByOperator extends UpdateByWindowedO // endregion extra-fields protected class Context extends UpdateWindowedContext { + public boolean canProcessDirectly; public RowSet workingRowSet = null; // candidate data for the window @@ -167,7 +168,8 @@ public void loadDataChunks(final RowSequence inputKeys) { if (valuePositionChunk == null) { valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); } else if (valuePositionChunk.capacity() < inputKeys.size()) { - valuePositionChunk.setSize(inputKeys.intSize()); + valuePositionChunk.close(); + valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); } // produce position data for the window (will be timestamps for time-based) @@ -242,8 +244,13 @@ public BaseWindowedLongUpdateByOperator(@NotNull final MatchPair pair, public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, boolean usingBuckets, boolean isUpstreamAppendOnly) { final Context ctx = (Context) context; ctx.workingRowSet = resultSourceRowSet; - } + if(!usingBuckets) { + // we can only process directly from an update if the window is entire backward-looking. Since we + // allow negative values in fwd/rev ticks and timestamps, we need to check both + ctx.canProcessDirectly = isUpstreamAppendOnly && forwardTimeScaleUnits <= 0 && reverseTimeScaleUnits >= 0;; + } + } @Override public void initializeFor(@NotNull final UpdateContext context, @@ -252,7 +259,8 @@ public void initializeFor(@NotNull final UpdateContext context, final Context ctx = (Context) context; ctx.currentUpdateType = type; - if (type == UpdateBy.UpdateType.Add || type == UpdateBy.UpdateType.Reprocess) { + if ((type == UpdateBy.UpdateType.Add && ctx.canProcessDirectly) + || type == UpdateBy.UpdateType.Reprocess) { long windowStartKey = computeFirstAffectingKey(updateRowSet.firstRowKey(), ctx.workingRowSet); ctx.loadWindowChunks(windowStartKey); } @@ -266,6 +274,12 @@ public void finishFor(@NotNull final UpdateContext updateContext, @NotNull final } } + @Override + public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { + final Context ctx = (Context) context; + return ctx.canProcessDirectly; + } + @Override public boolean requiresValues(@NotNull final UpdateContext context) { // windowed operators don't need current values supplied to them, they only care about windowed values which @@ -292,14 +306,10 @@ public void addChunk(@NotNull final UpdateContext updateContext, @NotNull final Chunk values, long bucketPosition) { final Context ctx = (Context) updateContext; - - if (recorder == null) { - // use position data to determine the windows + if (ctx.canProcessDirectly) { ctx.loadDataChunks(inputKeys); - } else { - // use timestamp data to determine the windows + doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); } - doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); } /** @@ -342,12 +352,7 @@ public void reprocessChunk(@NotNull final UpdateContext updateContext, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; - if (recorder == null) { - // use position data to determine the windows - ctx.loadDataChunks(inputKeys); - } else { - // use timestamp data to determine the windows - } + ctx.loadDataChunks(inputKeys); doAddChunk(ctx, inputKeys, keyChunk, valuesChunk, 0); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java index 922eb40a7aa..01550d83e42 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java @@ -33,6 +33,7 @@ public abstract class BaseWindowedShortUpdateByOperator extends UpdateByWindowed // endregion extra-fields protected class Context extends UpdateWindowedContext { + public boolean canProcessDirectly; public RowSet workingRowSet = null; // candidate data for the window @@ -167,7 +168,8 @@ public void loadDataChunks(final RowSequence inputKeys) { if (valuePositionChunk == null) { valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); } else if (valuePositionChunk.capacity() < inputKeys.size()) { - valuePositionChunk.setSize(inputKeys.intSize()); + valuePositionChunk.close(); + valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); } // produce position data for the window (will be timestamps for time-based) @@ -242,8 +244,13 @@ public BaseWindowedShortUpdateByOperator(@NotNull final MatchPair pair, public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, boolean usingBuckets, boolean isUpstreamAppendOnly) { final Context ctx = (Context) context; ctx.workingRowSet = resultSourceRowSet; - } + if(!usingBuckets) { + // we can only process directly from an update if the window is entire backward-looking. Since we + // allow negative values in fwd/rev ticks and timestamps, we need to check both + ctx.canProcessDirectly = isUpstreamAppendOnly && forwardTimeScaleUnits <= 0 && reverseTimeScaleUnits >= 0;; + } + } @Override public void initializeFor(@NotNull final UpdateContext context, @@ -252,7 +259,8 @@ public void initializeFor(@NotNull final UpdateContext context, final Context ctx = (Context) context; ctx.currentUpdateType = type; - if (type == UpdateBy.UpdateType.Add || type == UpdateBy.UpdateType.Reprocess) { + if ((type == UpdateBy.UpdateType.Add && ctx.canProcessDirectly) + || type == UpdateBy.UpdateType.Reprocess) { long windowStartKey = computeFirstAffectingKey(updateRowSet.firstRowKey(), ctx.workingRowSet); ctx.loadWindowChunks(windowStartKey); } @@ -266,6 +274,12 @@ public void finishFor(@NotNull final UpdateContext updateContext, @NotNull final } } + @Override + public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { + final Context ctx = (Context) context; + return ctx.canProcessDirectly; + } + @Override public boolean requiresValues(@NotNull final UpdateContext context) { // windowed operators don't need current values supplied to them, they only care about windowed values which @@ -292,14 +306,10 @@ public void addChunk(@NotNull final UpdateContext updateContext, @NotNull final Chunk values, long bucketPosition) { final Context ctx = (Context) updateContext; - - if (recorder == null) { - // use position data to determine the windows + if (ctx.canProcessDirectly) { ctx.loadDataChunks(inputKeys); - } else { - // use timestamp data to determine the windows + doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); } - doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); } /** @@ -342,12 +352,7 @@ public void reprocessChunk(@NotNull final UpdateContext updateContext, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; - if (recorder == null) { - // use position data to determine the windows - ctx.loadDataChunks(inputKeys); - } else { - // use timestamp data to determine the windows - } + ctx.loadDataChunks(inputKeys); doAddChunk(ctx, inputKeys, keyChunk, valuesChunk, 0); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } From cce7e86ad15bfa0ee5503ea96dabea392ef6ade6 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Thu, 11 Aug 2022 08:47:19 -0700 Subject: [PATCH 007/123] WIP, but all cum tests passing, windowed static only --- .../engine/table/impl/BucketedUpdateBy.java | 14 +- .../table/impl/UpdateByOperatorFactory.java | 50 +- .../BaseWindowedByteUpdateByOperator.java | 70 ++- .../BaseWindowedCharUpdateByOperator.java | 70 ++- .../BaseWindowedDoubleUpdateByOperator.java | 440 +++++++++++++++++ .../BaseWindowedFloatUpdateByOperator.java | 435 +++++++++++++++++ .../BaseWindowedIntUpdateByOperator.java | 70 ++- .../BaseWindowedLongUpdateByOperator.java | 70 ++- .../BaseWindowedObjectUpdateByOperator.java | 445 ++++++++++++++++++ .../BaseWindowedShortUpdateByOperator.java | 70 ++- .../rollingsum/ByteRollingSumOperator.java | 187 ++++++++ .../rollingsum/DoubleRollingSumOperator.java | 188 ++++++++ .../rollingsum/FloatRollingSumOperator.java | 183 +++++++ .../rollingsum/IntRollingSumOperator.java | 184 ++++++++ .../rollingsum/LongRollingSumOperator.java | 184 ++++++++ .../rollingsum/ShortRollingSumOperator.java | 42 -- .../table/impl/updateby/TestRollingSum.java | 4 +- .../replicators/ReplicateUpdateBy.java | 25 +- .../api/updateby/spec/RollingSumSpec.java | 18 +- 19 files changed, 2587 insertions(+), 162 deletions(-) create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedUpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedUpdateBy.java index b553d12d9be..d86fd55b986 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedUpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedUpdateBy.java @@ -1073,11 +1073,11 @@ private BucketedUpdateBy(@NotNull final UpdateByOperator[] operators, final int hashTableSize = control.initialHashTableSizeOrDefault(); - if (source.isRefreshing() && !source.isAddOnly()) { +// if (source.isRefreshing() && !source.isAddOnly()) { slotTracker = new UpdateBySlotTracker(control.chunkCapacityOrDefault()); - } else { - slotTracker = null; - } +// } else { +// slotTracker = null; +// } if (!useGrouping) { this.hashTable = TypedHasherFactory.make(UpdateByStateManagerTypedBase.class, keySources, keySources, @@ -1131,9 +1131,13 @@ private void doInitialAdditions(final boolean useGrouping, @NotNull final MatchP processUpdateForRedirection(initialUpdate); } -// accumulateUpdatesByBucket(upstream, ctx); + // do an addition phase for all the operators that can add directly (i.e. backwards looking) ctx.doAppendOnlyAdds(true, source.getRowSet()); + + // do a reprocessing phase for operators that can't add directly +// ctx.reprocessRows(RowSetShiftData.EMPTY); + if (slotTracker != null) { // noinspection resource slotTracker.applyUpdates(RowSetShiftData.EMPTY); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java index b8003373386..4595db158f0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java @@ -8,12 +8,13 @@ import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.impl.sources.ReinterpretUtils; import io.deephaven.engine.table.impl.updateby.ema.*; import io.deephaven.engine.table.impl.updateby.fill.*; import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; import io.deephaven.engine.table.impl.updateby.minmax.*; import io.deephaven.engine.table.impl.updateby.prod.*; -import io.deephaven.engine.table.impl.updateby.rollingsum.ShortRollingSumOperator; +import io.deephaven.engine.table.impl.updateby.rollingsum.*; import io.deephaven.engine.table.impl.updateby.sum.*; import io.deephaven.engine.table.impl.util.WritableRowRedirection; import io.deephaven.time.DateTime; @@ -407,37 +408,34 @@ private UpdateByOperator makeRollingSumOperator(@NotNull final MatchPair pair, final long prevTimeScaleUnits = rs.prevTimeScale().timescaleUnits(); final long fwdTimeScaleUnits = rs.fwdTimeScale().timescaleUnits(); - if (csType == short.class || csType == Short.class) { - return new ShortRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, prevTimeScaleUnits, fwdTimeScaleUnits, columnSource, rowRedirection); - } -// if (csType == Boolean.class || csType == boolean.class) { -// return new ByteRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, prevTimeScaleUnits, fwdTimeScaleUnits, -// columnSource, rowRedirection, NULL_BOOLEAN_AS_BYTE); -// } else if (csType == byte.class || csType == Byte.class) { -// return new ByteRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, prevTimeScaleUnits, fwdTimeScaleUnits, -// columnSource, rowRedirection, NULL_BYTE); -// } else if (csType == short.class || csType == Short.class) { -// return new ShortRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, prevTimeScaleUnits, fwdTimeScaleUnits, -// columnSource, rowRedirection); -// } else if (csType == int.class || csType == Integer.class) { -// return new IntRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, prevTimeScaleUnits, fwdTimeScaleUnits, -// columnSource, rowRedirection); -// } else if (csType == long.class || csType == Long.class) { -// return new LongRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, prevTimeScaleUnits, fwdTimeScaleUnits, -// columnSource, rowRedirection); -// } else if (csType == float.class || csType == Float.class) { -// return new FloatRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, prevTimeScaleUnits, fwdTimeScaleUnits, -// columnSource, rowRedirection); -// } else if (csType == double.class || csType == Double.class) { -// return new DoubleRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, -// prevTimeScaleUnits, fwdTimeScaleUnits, columnSource, rowRedirection); + if (csType == Boolean.class || csType == boolean.class) { + return new ByteRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, prevTimeScaleUnits, fwdTimeScaleUnits, + ReinterpretUtils.booleanToByteSource(columnSource), rowRedirection, NULL_BOOLEAN_AS_BYTE); + } else if (csType == byte.class || csType == Byte.class) { + return new ByteRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, prevTimeScaleUnits, fwdTimeScaleUnits, + columnSource, rowRedirection, NULL_BYTE); + } else if (csType == short.class || csType == Short.class) { + return new ShortRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, prevTimeScaleUnits, fwdTimeScaleUnits, + columnSource, rowRedirection); + } else if (csType == int.class || csType == Integer.class) { + return new IntRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, prevTimeScaleUnits, fwdTimeScaleUnits, + columnSource, rowRedirection); + } else if (csType == long.class || csType == Long.class) { + return new LongRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, prevTimeScaleUnits, fwdTimeScaleUnits, + columnSource, rowRedirection); + } else if (csType == float.class || csType == Float.class) { + return new FloatRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, prevTimeScaleUnits, fwdTimeScaleUnits, + columnSource, rowRedirection); + } else if (csType == double.class || csType == Double.class) { + return new DoubleRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, + prevTimeScaleUnits, fwdTimeScaleUnits, columnSource, rowRedirection); // } else if (csType == BigDecimal.class) { // return new BigDecimalRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, // prevTimeScaleUnits, fwdTimeScaleUnits, columnSource, rowRedirection, control.mathContextOrDefault()); // } else if (csType == BigInteger.class) { // return new BigIntegerRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, // prevTimeScaleUnits, fwdTimeScaleUnits, columnSource, rowRedirection); -// } + } throw new IllegalArgumentException("Can not perform RollingSum on type " + csType); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java index 53727046fe2..885db5f7a66 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java @@ -24,7 +24,9 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; +import io.deephaven.engine.table.impl.updateby.rollingsum.ShortRollingSumOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -318,19 +320,6 @@ final public boolean anyModified(@NotNull final UpdateContext ctx) { } // region Addition - @Override - public void addChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk values, - long bucketPosition) { - final Context ctx = (Context) updateContext; - if (ctx.canProcessDirectly) { - ctx.loadDataChunks(inputKeys); - doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); - } - } - /** * Add a chunk of values to the operator. * @@ -345,6 +334,61 @@ protected abstract void doAddChunk(@NotNull final Context ctx, @NotNull final Chunk workingChunk, final long bucketPosition); + @Override + public void addChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk values, + long bucketPosition) { + final Context ctx = (Context) updateContext; + if (ctx.canProcessDirectly) { + ctx.loadDataChunks(inputKeys); + doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); + } + } + + @Override + public void addChunkBucketed(final @NotNull UpdateByOperator.UpdateContext context, + final @NotNull Chunk values, + final @NotNull LongChunk keyChunk, + final @NotNull IntChunk bucketPositions, + final @NotNull IntChunk startPositions, + final @NotNull IntChunk runLengths) { + final Context ctx = (Context) context; + if (ctx.canProcessDirectly) { +// final ShortChunk asShorts = values.asShortChunk(); + for (int runIdx = 0; runIdx < startPositions.size(); runIdx++) { + final int runStart = startPositions.get(runIdx); + final int runLength = runLengths.get(runIdx); + final int bucketPosition = bucketPositions.get(runStart); +// doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); + +// try (RowSequence rs = RowSequenceFactory.wrapRowKeysChunkAsRowSequence((LongChunk) keyChunk)_ + +// RowSetBuilderSequential builder = RowSetFactory.builderSequential(); +// for (int ii = runStart; ii < runStart + runLength; ii++) { +// builder.appendKey(keyChunk.get(ii)); +// } +// +// WritableRowSet bucketRs = bucketRowSet.get(bucketPosition); +// if (bucketRs == null) { +// bucketRs = builder.build(); +// bucketRowSet.set(bucketPosition, bucketRs); +// } else { +// try (final RowSet added = builder.build()) { +// bucketRs.insert(added); +// } +// } +// +// ctx.curVal = NULL_LONG; +// ctx.currentWindow.clear(); + +// accumulate(asShorts, (LongChunk) keyChunk, ctx, runStart, runLength, bucketRs); +// bucketLastVal.set(bucketPosition, ctx.curVal); + } + } + } + // endregion // region Reprocessing diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java index 6c3eabd8046..88374270f75 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java @@ -14,7 +14,9 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; +import io.deephaven.engine.table.impl.updateby.rollingsum.ShortRollingSumOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -294,19 +296,6 @@ final public boolean anyModified(@NotNull final UpdateContext ctx) { } // region Addition - @Override - public void addChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk values, - long bucketPosition) { - final Context ctx = (Context) updateContext; - if (ctx.canProcessDirectly) { - ctx.loadDataChunks(inputKeys); - doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); - } - } - /** * Add a chunk of values to the operator. * @@ -321,6 +310,61 @@ protected abstract void doAddChunk(@NotNull final Context ctx, @NotNull final Chunk workingChunk, final long bucketPosition); + @Override + public void addChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk values, + long bucketPosition) { + final Context ctx = (Context) updateContext; + if (ctx.canProcessDirectly) { + ctx.loadDataChunks(inputKeys); + doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); + } + } + + @Override + public void addChunkBucketed(final @NotNull UpdateByOperator.UpdateContext context, + final @NotNull Chunk values, + final @NotNull LongChunk keyChunk, + final @NotNull IntChunk bucketPositions, + final @NotNull IntChunk startPositions, + final @NotNull IntChunk runLengths) { + final Context ctx = (Context) context; + if (ctx.canProcessDirectly) { +// final ShortChunk asShorts = values.asShortChunk(); + for (int runIdx = 0; runIdx < startPositions.size(); runIdx++) { + final int runStart = startPositions.get(runIdx); + final int runLength = runLengths.get(runIdx); + final int bucketPosition = bucketPositions.get(runStart); +// doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); + +// try (RowSequence rs = RowSequenceFactory.wrapRowKeysChunkAsRowSequence((LongChunk) keyChunk)_ + +// RowSetBuilderSequential builder = RowSetFactory.builderSequential(); +// for (int ii = runStart; ii < runStart + runLength; ii++) { +// builder.appendKey(keyChunk.get(ii)); +// } +// +// WritableRowSet bucketRs = bucketRowSet.get(bucketPosition); +// if (bucketRs == null) { +// bucketRs = builder.build(); +// bucketRowSet.set(bucketPosition, bucketRs); +// } else { +// try (final RowSet added = builder.build()) { +// bucketRs.insert(added); +// } +// } +// +// ctx.curVal = NULL_LONG; +// ctx.currentWindow.clear(); + +// accumulate(asShorts, (LongChunk) keyChunk, ctx, runStart, runLength, bucketRs); +// bucketLastVal.set(bucketPosition, ctx.curVal); + } + } + } + // endregion // region Reprocessing diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java new file mode 100644 index 00000000000..c0d8dec2746 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java @@ -0,0 +1,440 @@ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit BaseWindowedFloatUpdateByOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +package io.deephaven.engine.table.impl.updateby.internal; + +import io.deephaven.api.updateby.OperationControl; +import io.deephaven.base.LongRingBuffer; +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.rowset.*; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.table.ChunkSource; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.UpdateByWindowedOperator; +import io.deephaven.engine.table.impl.util.RowRedirection; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +public abstract class BaseWindowedDoubleUpdateByOperator extends UpdateByWindowedOperator { + protected final ColumnSource valueSource; + + protected boolean initialized = false; + + // region extra-fields + // endregion extra-fields + + protected class Context extends UpdateWindowedContext { + public boolean canProcessDirectly; + public RowSet workingRowSet = null; + + // candidate data for the window + public final int WINDOW_CHUNK_SIZE = 1024; + + // data that is actually in the current window + public LongRingBuffer windowRowKeys = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); + // the selector that determines whether this value should be in the window, positions for tick-based and + // timestamps for time-based operators + public LongRingBuffer windowSelector = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); + + public RowSequence.Iterator windowIterator = null; + + public WritableDoubleChunk candidateValuesChunk; + public WritableLongChunk candidateRowKeysChunk; + public WritableLongChunk candidatePositionsChunk; + public WritableLongChunk candidateTimestampsChunk; + + public int candidateWindowIndex = 0; + + // position data for the chunk being currently processed + public WritableLongChunk valuePositionChunk; + + // other useful stuff + public UpdateBy.UpdateType currentUpdateType; + + public RowSetBuilderSequential modifiedBuilder; + public RowSet newModified; + + public RowSetBuilderSequential getModifiedBuilder() { + if(modifiedBuilder == null) { + modifiedBuilder = RowSetFactory.builderSequential(); + } + return modifiedBuilder; + } + + @Override + public void close() { + if (windowIterator != null) { + windowIterator.close(); + windowIterator = null; + } + + if (candidateValuesChunk != null) { + candidateValuesChunk.close(); + candidateValuesChunk = null; + } + + if (candidateRowKeysChunk != null) { + candidateRowKeysChunk.close(); + candidateRowKeysChunk = null; + } + + if (candidatePositionsChunk != null) { + candidatePositionsChunk.close(); + candidatePositionsChunk = null; + } + + if (valuePositionChunk != null) { + valuePositionChunk.close(); + valuePositionChunk = null; + } + + // no need to close, just release the reference + workingRowSet = null; + } + + /*** + * Fill the working chunks with data for this key + * + * @param startKey the key for which we want to + */ + public void loadWindowChunks(final long startKey) { + // TODO: make sure this works for bucketed + if (windowIterator == null) { + windowIterator = workingRowSet.getRowSequenceIterator(); + } + windowIterator.advance(startKey); + + RowSequence windowRowSequence = windowIterator.getNextRowSequenceWithLength(WINDOW_CHUNK_SIZE); + + // fill the window values chunk + if (candidateValuesChunk == null) { + candidateValuesChunk = WritableDoubleChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + } + try (ChunkSource.FillContext fc = valueSource.makeFillContext(WINDOW_CHUNK_SIZE)){ + valueSource.fillChunk(fc, candidateValuesChunk, windowRowSequence); + } + + // fill the window keys chunk + if (candidateRowKeysChunk == null) { + candidateRowKeysChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + } + windowRowSequence.fillRowKeyChunk(candidateRowKeysChunk); + + if (recorder == null) { + // get position data for the window items (relative to the table or bucket rowset) + if (candidatePositionsChunk == null) { + candidatePositionsChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + } + + // TODO: gotta be a better way than creating two rowsets + try (final RowSet rs = windowRowSequence.asRowSet(); + final RowSet positions = workingRowSet.invert(rs)) { + positions.fillRowKeyChunk(candidatePositionsChunk); + } + } else { + // get timestamp values from the recorder column source + if (candidateTimestampsChunk == null) { + candidateTimestampsChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + } + try (final ChunkSource.FillContext fc = recorder.getColumnSource().makeFillContext(WINDOW_CHUNK_SIZE)) { + recorder.getColumnSource().fillChunk(fc, candidateTimestampsChunk, windowRowSequence); + } + } + + // reset the index to beginning of the chunks + candidateWindowIndex = 0; + } + + /*** + * Fill the working chunks with data for this key + * + * @param inputKeys the keys for which we want to get position or timestamp values + */ + public void loadDataChunks(final RowSequence inputKeys) { + if (recorder != null) { + // timestamp data will be available from the recorder + return; + } + + if (valuePositionChunk == null) { + valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); + } else if (valuePositionChunk.capacity() < inputKeys.size()) { + valuePositionChunk.close(); + valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); + } + + // produce position data for the window (will be timestamps for time-based) + // TODO: gotta be a better way than creating two rowsets + try (final RowSet rs = inputKeys.asRowSet(); + final RowSet positions = workingRowSet.invert(rs)) { + positions.fillRowKeyChunk(valuePositionChunk); + } + } + + public void fillWindowTicks(UpdateContext context, long currentPos) { + // compute the head and tail (inclusive) + final long tail = Math.max(0, currentPos - reverseTimeScaleUnits + 1); + final long head = Math.min(workingRowSet.size() - 1, currentPos + forwardTimeScaleUnits); + + while (windowSelector.peek(Long.MAX_VALUE) < tail) { + final long pos = windowSelector.remove(); + final long key = windowRowKeys.remove(); + + pop(context, key); + } + + + // look at the window data and push until satisfied or at the end of the rowset + while (candidatePositionsChunk.size() > 0 && candidatePositionsChunk.get(candidateWindowIndex) <= head) { + final long pos = candidatePositionsChunk.get(candidateWindowIndex); + final long key = candidateRowKeysChunk.get(candidateWindowIndex); + final double val = candidateValuesChunk.get(candidateWindowIndex); + + push(context, key, val); + + windowSelector.add(pos); + windowRowKeys.add(key); + + if (++candidateWindowIndex >= candidatePositionsChunk.size()) { + // load the next chunk in order + loadWindowChunks(key + 1); + } + } + + if (windowSelector.isEmpty()) { + reset(context); + } + } + } + + public BaseWindowedDoubleUpdateByOperator(@NotNull final MatchPair pair, + @NotNull final String[] affectingColumns, + @NotNull final OperationControl control, + @Nullable final LongRecordingUpdateByOperator timeRecorder, + final long reverseTimeScaleUnits, + final long forwardTimeScaleUnits, + @Nullable final RowRedirection rowRedirection, + @NotNull final ColumnSource valueSource + // region extra-constructor-args + // endregion extra-constructor-args + ) { + super(pair, affectingColumns, control, timeRecorder, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); + this.valueSource = valueSource; + // region constructor + // endregion constructor + } + + // region extra-methods + // endregion extra-methods + + public abstract void push(UpdateContext context, long key, double val); + public abstract void pop(UpdateContext context, long key); + public abstract void reset(UpdateContext context); + + @Override + public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, boolean usingBuckets, boolean isUpstreamAppendOnly) { + final Context ctx = (Context) context; + ctx.workingRowSet = resultSourceRowSet; + + if(!usingBuckets) { + // we can only process directly from an update if the window is entire backward-looking. Since we + // allow negative values in fwd/rev ticks and timestamps, we need to check both + ctx.canProcessDirectly = isUpstreamAppendOnly && forwardTimeScaleUnits <= 0 && reverseTimeScaleUnits >= 0;; + } + } + + @Override + public void initializeFor(@NotNull final UpdateContext context, + @NotNull final RowSet updateRowSet, + @NotNull final UpdateBy.UpdateType type) { + final Context ctx = (Context) context; + ctx.currentUpdateType = type; + + if ((type == UpdateBy.UpdateType.Add && ctx.canProcessDirectly) + || type == UpdateBy.UpdateType.Reprocess) { + long windowStartKey = computeFirstAffectingKey(updateRowSet.firstRowKey(), ctx.workingRowSet); + ctx.loadWindowChunks(windowStartKey); + } + } + + @Override + public void finishFor(@NotNull final UpdateContext updateContext, @NotNull final UpdateBy.UpdateType type) { + final Context ctx = (Context) updateContext; + if(type == UpdateBy.UpdateType.Reprocess && ctx.modifiedBuilder != null) { + ctx.newModified = ctx.modifiedBuilder.build(); + } + } + + @Override + public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { + final Context ctx = (Context) context; + return ctx.canProcessDirectly; + } + + @Override + public boolean requiresValues(@NotNull final UpdateContext context) { + // windowed operators don't need current values supplied to them, they only care about windowed values which + // may or may not intersect with the column values + return false; + } + + @NotNull + @Override + final public RowSet getAdditionalModifications(@NotNull final UpdateContext ctx) { + return ((Context)ctx).newModified == null ? RowSetFactory.empty() : ((Context)ctx).newModified; + } + + @Override + final public boolean anyModified(@NotNull final UpdateContext ctx) { + return ((Context)ctx).newModified != null; + } + + // region Addition + /** + * Add a chunk of values to the operator. + * + * @param ctx the context object + * @param inputKeys the input keys for the chunk + * @param workingChunk the chunk of values + * @param bucketPosition the bucket position that the values belong to. + */ + protected abstract void doAddChunk(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk workingChunk, + final long bucketPosition); + + @Override + public void addChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk values, + long bucketPosition) { + final Context ctx = (Context) updateContext; + if (ctx.canProcessDirectly) { + ctx.loadDataChunks(inputKeys); + doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); + } + } + + @Override + public void addChunkBucketed(final @NotNull UpdateContext context, + final @NotNull Chunk values, + final @NotNull LongChunk keyChunk, + final @NotNull IntChunk bucketPositions, + final @NotNull IntChunk startPositions, + final @NotNull IntChunk runLengths) { + final Context ctx = (Context) context; + if (ctx.canProcessDirectly) { +// final ShortChunk asShorts = values.asShortChunk(); + for (int runIdx = 0; runIdx < startPositions.size(); runIdx++) { + final int runStart = startPositions.get(runIdx); + final int runLength = runLengths.get(runIdx); + final int bucketPosition = bucketPositions.get(runStart); +// doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); + +// try (RowSequence rs = RowSequenceFactory.wrapRowKeysChunkAsRowSequence((LongChunk) keyChunk)_ + +// RowSetBuilderSequential builder = RowSetFactory.builderSequential(); +// for (int ii = runStart; ii < runStart + runLength; ii++) { +// builder.appendKey(keyChunk.get(ii)); +// } +// +// WritableRowSet bucketRs = bucketRowSet.get(bucketPosition); +// if (bucketRs == null) { +// bucketRs = builder.build(); +// bucketRowSet.set(bucketPosition, bucketRs); +// } else { +// try (final RowSet added = builder.build()) { +// bucketRs.insert(added); +// } +// } +// +// ctx.curVal = NULL_LONG; +// ctx.currentWindow.clear(); + +// accumulate(asShorts, (LongChunk) keyChunk, ctx, runStart, runLength, bucketRs); +// bucketLastVal.set(bucketPosition, ctx.curVal); + } + } + } + + // endregion + + // region Reprocessing + + public void resetForReprocess(@NotNull final UpdateContext context, + @NotNull final RowSet sourceRowSet, + long firstUnmodifiedKey) { + final Context ctx = (Context) context; + ctx.workingRowSet = sourceRowSet; + } + + @Override + public void resetForReprocessBucketed(@NotNull final UpdateContext context, + @NotNull final RowSet bucketRowSet, + final long bucketPosition, + final long firstUnmodifiedKey) { + final Context ctx = (Context) context; + } + + @Override + public void reprocessChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk valuesChunk, + @NotNull final RowSet postUpdateSourceIndex) { + final Context ctx = (Context) updateContext; + ctx.loadDataChunks(inputKeys); + doAddChunk(ctx, inputKeys, keyChunk, valuesChunk, 0); + ctx.getModifiedBuilder().appendRowSequence(inputKeys); + } + + @Override + public void reprocessChunkBucketed(@NotNull UpdateContext updateContext, + @NotNull final RowSequence chunkOk, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk runStartPositions, + @NotNull final IntChunk runLengths) { + addChunkBucketed(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); + ((Context)updateContext).getModifiedBuilder().appendRowSequence(chunkOk); + } + + // endregion + + // region No-Op Operations + + @Override + final public void modifyChunk(@NotNull final UpdateContext updateContext, + @Nullable final LongChunk prevKeyChunk, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk prevValuesChunk, + @NotNull final Chunk postValuesChunk, + long bucketPosition) { + } + + @Override + final public void removeChunk(@NotNull final UpdateContext updateContext, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk prevValuesChunk, + long bucketPosition) { + } + + @Override + final public void applyShift(@NotNull final UpdateContext updateContext, + @NotNull final RowSet prevIndex, + @NotNull final RowSetShiftData shifted) { + } + // endregion +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java new file mode 100644 index 00000000000..598a1df0c81 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java @@ -0,0 +1,435 @@ +package io.deephaven.engine.table.impl.updateby.internal; + +import io.deephaven.api.updateby.OperationControl; +import io.deephaven.base.LongRingBuffer; +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.rowset.*; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.table.ChunkSource; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.UpdateByWindowedOperator; +import io.deephaven.engine.table.impl.util.RowRedirection; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +public abstract class BaseWindowedFloatUpdateByOperator extends UpdateByWindowedOperator { + protected final ColumnSource valueSource; + + protected boolean initialized = false; + + // region extra-fields + // endregion extra-fields + + protected class Context extends UpdateWindowedContext { + public boolean canProcessDirectly; + public RowSet workingRowSet = null; + + // candidate data for the window + public final int WINDOW_CHUNK_SIZE = 1024; + + // data that is actually in the current window + public LongRingBuffer windowRowKeys = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); + // the selector that determines whether this value should be in the window, positions for tick-based and + // timestamps for time-based operators + public LongRingBuffer windowSelector = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); + + public RowSequence.Iterator windowIterator = null; + + public WritableFloatChunk candidateValuesChunk; + public WritableLongChunk candidateRowKeysChunk; + public WritableLongChunk candidatePositionsChunk; + public WritableLongChunk candidateTimestampsChunk; + + public int candidateWindowIndex = 0; + + // position data for the chunk being currently processed + public WritableLongChunk valuePositionChunk; + + // other useful stuff + public UpdateBy.UpdateType currentUpdateType; + + public RowSetBuilderSequential modifiedBuilder; + public RowSet newModified; + + public RowSetBuilderSequential getModifiedBuilder() { + if(modifiedBuilder == null) { + modifiedBuilder = RowSetFactory.builderSequential(); + } + return modifiedBuilder; + } + + @Override + public void close() { + if (windowIterator != null) { + windowIterator.close(); + windowIterator = null; + } + + if (candidateValuesChunk != null) { + candidateValuesChunk.close(); + candidateValuesChunk = null; + } + + if (candidateRowKeysChunk != null) { + candidateRowKeysChunk.close(); + candidateRowKeysChunk = null; + } + + if (candidatePositionsChunk != null) { + candidatePositionsChunk.close(); + candidatePositionsChunk = null; + } + + if (valuePositionChunk != null) { + valuePositionChunk.close(); + valuePositionChunk = null; + } + + // no need to close, just release the reference + workingRowSet = null; + } + + /*** + * Fill the working chunks with data for this key + * + * @param startKey the key for which we want to + */ + public void loadWindowChunks(final long startKey) { + // TODO: make sure this works for bucketed + if (windowIterator == null) { + windowIterator = workingRowSet.getRowSequenceIterator(); + } + windowIterator.advance(startKey); + + RowSequence windowRowSequence = windowIterator.getNextRowSequenceWithLength(WINDOW_CHUNK_SIZE); + + // fill the window values chunk + if (candidateValuesChunk == null) { + candidateValuesChunk = WritableFloatChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + } + try (ChunkSource.FillContext fc = valueSource.makeFillContext(WINDOW_CHUNK_SIZE)){ + valueSource.fillChunk(fc, candidateValuesChunk, windowRowSequence); + } + + // fill the window keys chunk + if (candidateRowKeysChunk == null) { + candidateRowKeysChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + } + windowRowSequence.fillRowKeyChunk(candidateRowKeysChunk); + + if (recorder == null) { + // get position data for the window items (relative to the table or bucket rowset) + if (candidatePositionsChunk == null) { + candidatePositionsChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + } + + // TODO: gotta be a better way than creating two rowsets + try (final RowSet rs = windowRowSequence.asRowSet(); + final RowSet positions = workingRowSet.invert(rs)) { + positions.fillRowKeyChunk(candidatePositionsChunk); + } + } else { + // get timestamp values from the recorder column source + if (candidateTimestampsChunk == null) { + candidateTimestampsChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + } + try (final ChunkSource.FillContext fc = recorder.getColumnSource().makeFillContext(WINDOW_CHUNK_SIZE)) { + recorder.getColumnSource().fillChunk(fc, candidateTimestampsChunk, windowRowSequence); + } + } + + // reset the index to beginning of the chunks + candidateWindowIndex = 0; + } + + /*** + * Fill the working chunks with data for this key + * + * @param inputKeys the keys for which we want to get position or timestamp values + */ + public void loadDataChunks(final RowSequence inputKeys) { + if (recorder != null) { + // timestamp data will be available from the recorder + return; + } + + if (valuePositionChunk == null) { + valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); + } else if (valuePositionChunk.capacity() < inputKeys.size()) { + valuePositionChunk.close(); + valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); + } + + // produce position data for the window (will be timestamps for time-based) + // TODO: gotta be a better way than creating two rowsets + try (final RowSet rs = inputKeys.asRowSet(); + final RowSet positions = workingRowSet.invert(rs)) { + positions.fillRowKeyChunk(valuePositionChunk); + } + } + + public void fillWindowTicks(UpdateContext context, long currentPos) { + // compute the head and tail (inclusive) + final long tail = Math.max(0, currentPos - reverseTimeScaleUnits + 1); + final long head = Math.min(workingRowSet.size() - 1, currentPos + forwardTimeScaleUnits); + + while (windowSelector.peek(Long.MAX_VALUE) < tail) { + final long pos = windowSelector.remove(); + final long key = windowRowKeys.remove(); + + pop(context, key); + } + + + // look at the window data and push until satisfied or at the end of the rowset + while (candidatePositionsChunk.size() > 0 && candidatePositionsChunk.get(candidateWindowIndex) <= head) { + final long pos = candidatePositionsChunk.get(candidateWindowIndex); + final long key = candidateRowKeysChunk.get(candidateWindowIndex); + final float val = candidateValuesChunk.get(candidateWindowIndex); + + push(context, key, val); + + windowSelector.add(pos); + windowRowKeys.add(key); + + if (++candidateWindowIndex >= candidatePositionsChunk.size()) { + // load the next chunk in order + loadWindowChunks(key + 1); + } + } + + if (windowSelector.isEmpty()) { + reset(context); + } + } + } + + public BaseWindowedFloatUpdateByOperator(@NotNull final MatchPair pair, + @NotNull final String[] affectingColumns, + @NotNull final OperationControl control, + @Nullable final LongRecordingUpdateByOperator timeRecorder, + final long reverseTimeScaleUnits, + final long forwardTimeScaleUnits, + @Nullable final RowRedirection rowRedirection, + @NotNull final ColumnSource valueSource + // region extra-constructor-args + // endregion extra-constructor-args + ) { + super(pair, affectingColumns, control, timeRecorder, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); + this.valueSource = valueSource; + // region constructor + // endregion constructor + } + + // region extra-methods + // endregion extra-methods + + public abstract void push(UpdateContext context, long key, float val); + public abstract void pop(UpdateContext context, long key); + public abstract void reset(UpdateContext context); + + @Override + public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, boolean usingBuckets, boolean isUpstreamAppendOnly) { + final Context ctx = (Context) context; + ctx.workingRowSet = resultSourceRowSet; + + if(!usingBuckets) { + // we can only process directly from an update if the window is entire backward-looking. Since we + // allow negative values in fwd/rev ticks and timestamps, we need to check both + ctx.canProcessDirectly = isUpstreamAppendOnly && forwardTimeScaleUnits <= 0 && reverseTimeScaleUnits >= 0;; + } + } + + @Override + public void initializeFor(@NotNull final UpdateContext context, + @NotNull final RowSet updateRowSet, + @NotNull final UpdateBy.UpdateType type) { + final Context ctx = (Context) context; + ctx.currentUpdateType = type; + + if ((type == UpdateBy.UpdateType.Add && ctx.canProcessDirectly) + || type == UpdateBy.UpdateType.Reprocess) { + long windowStartKey = computeFirstAffectingKey(updateRowSet.firstRowKey(), ctx.workingRowSet); + ctx.loadWindowChunks(windowStartKey); + } + } + + @Override + public void finishFor(@NotNull final UpdateContext updateContext, @NotNull final UpdateBy.UpdateType type) { + final Context ctx = (Context) updateContext; + if(type == UpdateBy.UpdateType.Reprocess && ctx.modifiedBuilder != null) { + ctx.newModified = ctx.modifiedBuilder.build(); + } + } + + @Override + public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { + final Context ctx = (Context) context; + return ctx.canProcessDirectly; + } + + @Override + public boolean requiresValues(@NotNull final UpdateContext context) { + // windowed operators don't need current values supplied to them, they only care about windowed values which + // may or may not intersect with the column values + return false; + } + + @NotNull + @Override + final public RowSet getAdditionalModifications(@NotNull final UpdateContext ctx) { + return ((Context)ctx).newModified == null ? RowSetFactory.empty() : ((Context)ctx).newModified; + } + + @Override + final public boolean anyModified(@NotNull final UpdateContext ctx) { + return ((Context)ctx).newModified != null; + } + + // region Addition + /** + * Add a chunk of values to the operator. + * + * @param ctx the context object + * @param inputKeys the input keys for the chunk + * @param workingChunk the chunk of values + * @param bucketPosition the bucket position that the values belong to. + */ + protected abstract void doAddChunk(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk workingChunk, + final long bucketPosition); + + @Override + public void addChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk values, + long bucketPosition) { + final Context ctx = (Context) updateContext; + if (ctx.canProcessDirectly) { + ctx.loadDataChunks(inputKeys); + doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); + } + } + + @Override + public void addChunkBucketed(final @NotNull UpdateContext context, + final @NotNull Chunk values, + final @NotNull LongChunk keyChunk, + final @NotNull IntChunk bucketPositions, + final @NotNull IntChunk startPositions, + final @NotNull IntChunk runLengths) { + final Context ctx = (Context) context; + if (ctx.canProcessDirectly) { +// final ShortChunk asShorts = values.asShortChunk(); + for (int runIdx = 0; runIdx < startPositions.size(); runIdx++) { + final int runStart = startPositions.get(runIdx); + final int runLength = runLengths.get(runIdx); + final int bucketPosition = bucketPositions.get(runStart); +// doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); + +// try (RowSequence rs = RowSequenceFactory.wrapRowKeysChunkAsRowSequence((LongChunk) keyChunk)_ + +// RowSetBuilderSequential builder = RowSetFactory.builderSequential(); +// for (int ii = runStart; ii < runStart + runLength; ii++) { +// builder.appendKey(keyChunk.get(ii)); +// } +// +// WritableRowSet bucketRs = bucketRowSet.get(bucketPosition); +// if (bucketRs == null) { +// bucketRs = builder.build(); +// bucketRowSet.set(bucketPosition, bucketRs); +// } else { +// try (final RowSet added = builder.build()) { +// bucketRs.insert(added); +// } +// } +// +// ctx.curVal = NULL_LONG; +// ctx.currentWindow.clear(); + +// accumulate(asShorts, (LongChunk) keyChunk, ctx, runStart, runLength, bucketRs); +// bucketLastVal.set(bucketPosition, ctx.curVal); + } + } + } + + // endregion + + // region Reprocessing + + public void resetForReprocess(@NotNull final UpdateContext context, + @NotNull final RowSet sourceRowSet, + long firstUnmodifiedKey) { + final Context ctx = (Context) context; + ctx.workingRowSet = sourceRowSet; + } + + @Override + public void resetForReprocessBucketed(@NotNull final UpdateContext context, + @NotNull final RowSet bucketRowSet, + final long bucketPosition, + final long firstUnmodifiedKey) { + final Context ctx = (Context) context; + } + + @Override + public void reprocessChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk valuesChunk, + @NotNull final RowSet postUpdateSourceIndex) { + final Context ctx = (Context) updateContext; + ctx.loadDataChunks(inputKeys); + doAddChunk(ctx, inputKeys, keyChunk, valuesChunk, 0); + ctx.getModifiedBuilder().appendRowSequence(inputKeys); + } + + @Override + public void reprocessChunkBucketed(@NotNull UpdateContext updateContext, + @NotNull final RowSequence chunkOk, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk runStartPositions, + @NotNull final IntChunk runLengths) { + addChunkBucketed(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); + ((Context)updateContext).getModifiedBuilder().appendRowSequence(chunkOk); + } + + // endregion + + // region No-Op Operations + + @Override + final public void modifyChunk(@NotNull final UpdateContext updateContext, + @Nullable final LongChunk prevKeyChunk, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk prevValuesChunk, + @NotNull final Chunk postValuesChunk, + long bucketPosition) { + } + + @Override + final public void removeChunk(@NotNull final UpdateContext updateContext, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk prevValuesChunk, + long bucketPosition) { + } + + @Override + final public void applyShift(@NotNull final UpdateContext updateContext, + @NotNull final RowSet prevIndex, + @NotNull final RowSetShiftData shifted) { + } + // endregion +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java index 1f1be4bcf0c..3611e68a05d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java @@ -19,7 +19,9 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; +import io.deephaven.engine.table.impl.updateby.rollingsum.ShortRollingSumOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -299,19 +301,6 @@ final public boolean anyModified(@NotNull final UpdateContext ctx) { } // region Addition - @Override - public void addChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk values, - long bucketPosition) { - final Context ctx = (Context) updateContext; - if (ctx.canProcessDirectly) { - ctx.loadDataChunks(inputKeys); - doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); - } - } - /** * Add a chunk of values to the operator. * @@ -326,6 +315,61 @@ protected abstract void doAddChunk(@NotNull final Context ctx, @NotNull final Chunk workingChunk, final long bucketPosition); + @Override + public void addChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk values, + long bucketPosition) { + final Context ctx = (Context) updateContext; + if (ctx.canProcessDirectly) { + ctx.loadDataChunks(inputKeys); + doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); + } + } + + @Override + public void addChunkBucketed(final @NotNull UpdateByOperator.UpdateContext context, + final @NotNull Chunk values, + final @NotNull LongChunk keyChunk, + final @NotNull IntChunk bucketPositions, + final @NotNull IntChunk startPositions, + final @NotNull IntChunk runLengths) { + final Context ctx = (Context) context; + if (ctx.canProcessDirectly) { +// final ShortChunk asShorts = values.asShortChunk(); + for (int runIdx = 0; runIdx < startPositions.size(); runIdx++) { + final int runStart = startPositions.get(runIdx); + final int runLength = runLengths.get(runIdx); + final int bucketPosition = bucketPositions.get(runStart); +// doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); + +// try (RowSequence rs = RowSequenceFactory.wrapRowKeysChunkAsRowSequence((LongChunk) keyChunk)_ + +// RowSetBuilderSequential builder = RowSetFactory.builderSequential(); +// for (int ii = runStart; ii < runStart + runLength; ii++) { +// builder.appendKey(keyChunk.get(ii)); +// } +// +// WritableRowSet bucketRs = bucketRowSet.get(bucketPosition); +// if (bucketRs == null) { +// bucketRs = builder.build(); +// bucketRowSet.set(bucketPosition, bucketRs); +// } else { +// try (final RowSet added = builder.build()) { +// bucketRs.insert(added); +// } +// } +// +// ctx.curVal = NULL_LONG; +// ctx.currentWindow.clear(); + +// accumulate(asShorts, (LongChunk) keyChunk, ctx, runStart, runLength, bucketRs); +// bucketLastVal.set(bucketPosition, ctx.curVal); + } + } + } + // endregion // region Reprocessing diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java index 8bea2e8f218..64ac0a8ea30 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java @@ -19,7 +19,9 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; +import io.deephaven.engine.table.impl.updateby.rollingsum.ShortRollingSumOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -299,19 +301,6 @@ final public boolean anyModified(@NotNull final UpdateContext ctx) { } // region Addition - @Override - public void addChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk values, - long bucketPosition) { - final Context ctx = (Context) updateContext; - if (ctx.canProcessDirectly) { - ctx.loadDataChunks(inputKeys); - doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); - } - } - /** * Add a chunk of values to the operator. * @@ -326,6 +315,61 @@ protected abstract void doAddChunk(@NotNull final Context ctx, @NotNull final Chunk workingChunk, final long bucketPosition); + @Override + public void addChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk values, + long bucketPosition) { + final Context ctx = (Context) updateContext; + if (ctx.canProcessDirectly) { + ctx.loadDataChunks(inputKeys); + doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); + } + } + + @Override + public void addChunkBucketed(final @NotNull UpdateByOperator.UpdateContext context, + final @NotNull Chunk values, + final @NotNull LongChunk keyChunk, + final @NotNull IntChunk bucketPositions, + final @NotNull IntChunk startPositions, + final @NotNull IntChunk runLengths) { + final Context ctx = (Context) context; + if (ctx.canProcessDirectly) { +// final ShortChunk asShorts = values.asShortChunk(); + for (int runIdx = 0; runIdx < startPositions.size(); runIdx++) { + final int runStart = startPositions.get(runIdx); + final int runLength = runLengths.get(runIdx); + final int bucketPosition = bucketPositions.get(runStart); +// doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); + +// try (RowSequence rs = RowSequenceFactory.wrapRowKeysChunkAsRowSequence((LongChunk) keyChunk)_ + +// RowSetBuilderSequential builder = RowSetFactory.builderSequential(); +// for (int ii = runStart; ii < runStart + runLength; ii++) { +// builder.appendKey(keyChunk.get(ii)); +// } +// +// WritableRowSet bucketRs = bucketRowSet.get(bucketPosition); +// if (bucketRs == null) { +// bucketRs = builder.build(); +// bucketRowSet.set(bucketPosition, bucketRs); +// } else { +// try (final RowSet added = builder.build()) { +// bucketRs.insert(added); +// } +// } +// +// ctx.curVal = NULL_LONG; +// ctx.currentWindow.clear(); + +// accumulate(asShorts, (LongChunk) keyChunk, ctx, runStart, runLength, bucketRs); +// bucketLastVal.set(bucketPosition, ctx.curVal); + } + } + } + // endregion // region Reprocessing diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java new file mode 100644 index 00000000000..eee6c81c0a6 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java @@ -0,0 +1,445 @@ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit BaseWindowedCharUpdateByOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +package io.deephaven.engine.table.impl.updateby.internal; + +import io.deephaven.api.updateby.OperationControl; +import io.deephaven.base.LongRingBuffer; +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.rowset.*; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.table.ChunkSource; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.UpdateByOperator; +import io.deephaven.engine.table.impl.UpdateByWindowedOperator; +import io.deephaven.engine.table.impl.updateby.rollingsum.ShortRollingSumOperator; +import io.deephaven.engine.table.impl.util.RowRedirection; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +public abstract class BaseWindowedObjectUpdateByOperator extends UpdateByWindowedOperator { + protected final ColumnSource valueSource; + + protected boolean initialized = false; + + // region extra-fields + private final Class colType; + // endregion extra-fields + + protected class Context extends UpdateWindowedContext { + public boolean canProcessDirectly; + public RowSet workingRowSet = null; + + // candidate data for the window + public final int WINDOW_CHUNK_SIZE = 1024; + + // data that is actually in the current window + public LongRingBuffer windowRowKeys = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); + // the selector that determines whether this value should be in the window, positions for tick-based and + // timestamps for time-based operators + public LongRingBuffer windowSelector = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); + + public RowSequence.Iterator windowIterator = null; + + public WritableObjectChunk candidateValuesChunk; + public WritableLongChunk candidateRowKeysChunk; + public WritableLongChunk candidatePositionsChunk; + public WritableLongChunk candidateTimestampsChunk; + + public int candidateWindowIndex = 0; + + // position data for the chunk being currently processed + public WritableLongChunk valuePositionChunk; + + // other useful stuff + public UpdateBy.UpdateType currentUpdateType; + + public RowSetBuilderSequential modifiedBuilder; + public RowSet newModified; + + public RowSetBuilderSequential getModifiedBuilder() { + if(modifiedBuilder == null) { + modifiedBuilder = RowSetFactory.builderSequential(); + } + return modifiedBuilder; + } + + @Override + public void close() { + if (windowIterator != null) { + windowIterator.close(); + windowIterator = null; + } + + if (candidateValuesChunk != null) { + candidateValuesChunk.close(); + candidateValuesChunk = null; + } + + if (candidateRowKeysChunk != null) { + candidateRowKeysChunk.close(); + candidateRowKeysChunk = null; + } + + if (candidatePositionsChunk != null) { + candidatePositionsChunk.close(); + candidatePositionsChunk = null; + } + + if (valuePositionChunk != null) { + valuePositionChunk.close(); + valuePositionChunk = null; + } + + // no need to close, just release the reference + workingRowSet = null; + } + + /*** + * Fill the working chunks with data for this key + * + * @param startKey the key for which we want to + */ + public void loadWindowChunks(final long startKey) { + // TODO: make sure this works for bucketed + if (windowIterator == null) { + windowIterator = workingRowSet.getRowSequenceIterator(); + } + windowIterator.advance(startKey); + + RowSequence windowRowSequence = windowIterator.getNextRowSequenceWithLength(WINDOW_CHUNK_SIZE); + + // fill the window values chunk + if (candidateValuesChunk == null) { + candidateValuesChunk = WritableObjectChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + } + try (ChunkSource.FillContext fc = valueSource.makeFillContext(WINDOW_CHUNK_SIZE)){ + valueSource.fillChunk(fc, candidateValuesChunk, windowRowSequence); + } + + // fill the window keys chunk + if (candidateRowKeysChunk == null) { + candidateRowKeysChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + } + windowRowSequence.fillRowKeyChunk(candidateRowKeysChunk); + + if (recorder == null) { + // get position data for the window items (relative to the table or bucket rowset) + if (candidatePositionsChunk == null) { + candidatePositionsChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + } + + // TODO: gotta be a better way than creating two rowsets + try (final RowSet rs = windowRowSequence.asRowSet(); + final RowSet positions = workingRowSet.invert(rs)) { + positions.fillRowKeyChunk(candidatePositionsChunk); + } + } else { + // get timestamp values from the recorder column source + if (candidateTimestampsChunk == null) { + candidateTimestampsChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + } + try (final ChunkSource.FillContext fc = recorder.getColumnSource().makeFillContext(WINDOW_CHUNK_SIZE)) { + recorder.getColumnSource().fillChunk(fc, candidateTimestampsChunk, windowRowSequence); + } + } + + // reset the index to beginning of the chunks + candidateWindowIndex = 0; + } + + /*** + * Fill the working chunks with data for this key + * + * @param inputKeys the keys for which we want to get position or timestamp values + */ + public void loadDataChunks(final RowSequence inputKeys) { + if (recorder != null) { + // timestamp data will be available from the recorder + return; + } + + if (valuePositionChunk == null) { + valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); + } else if (valuePositionChunk.capacity() < inputKeys.size()) { + valuePositionChunk.close(); + valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); + } + + // produce position data for the window (will be timestamps for time-based) + // TODO: gotta be a better way than creating two rowsets + try (final RowSet rs = inputKeys.asRowSet(); + final RowSet positions = workingRowSet.invert(rs)) { + positions.fillRowKeyChunk(valuePositionChunk); + } + } + + public void fillWindowTicks(UpdateContext context, long currentPos) { + // compute the head and tail (inclusive) + final long tail = Math.max(0, currentPos - reverseTimeScaleUnits + 1); + final long head = Math.min(workingRowSet.size() - 1, currentPos + forwardTimeScaleUnits); + + while (windowSelector.peek(Long.MAX_VALUE) < tail) { + final long pos = windowSelector.remove(); + final long key = windowRowKeys.remove(); + + pop(context, key); + } + + + // look at the window data and push until satisfied or at the end of the rowset + while (candidatePositionsChunk.size() > 0 && candidatePositionsChunk.get(candidateWindowIndex) <= head) { + final long pos = candidatePositionsChunk.get(candidateWindowIndex); + final long key = candidateRowKeysChunk.get(candidateWindowIndex); + final Object val = candidateValuesChunk.get(candidateWindowIndex); + + push(context, key, val); + + windowSelector.add(pos); + windowRowKeys.add(key); + + if (++candidateWindowIndex >= candidatePositionsChunk.size()) { + // load the next chunk in order + loadWindowChunks(key + 1); + } + } + + if (windowSelector.isEmpty()) { + reset(context); + } + } + } + + public BaseWindowedObjectUpdateByOperator(@NotNull final MatchPair pair, + @NotNull final String[] affectingColumns, + @NotNull final OperationControl control, + @Nullable final LongRecordingUpdateByOperator timeRecorder, + final long reverseTimeScaleUnits, + final long forwardTimeScaleUnits, + @Nullable final RowRedirection rowRedirection, + @NotNull final ColumnSource valueSource + // region extra-constructor-args + , final Class colType + // endregion extra-constructor-args + ) { + super(pair, affectingColumns, control, timeRecorder, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); + this.valueSource = valueSource; + // region constructor + this.colType = colType; + // endregion constructor + } + + // region extra-methods + // endregion extra-methods + + public abstract void push(UpdateContext context, long key, Object val); + public abstract void pop(UpdateContext context, long key); + public abstract void reset(UpdateContext context); + + @Override + public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, boolean usingBuckets, boolean isUpstreamAppendOnly) { + final Context ctx = (Context) context; + ctx.workingRowSet = resultSourceRowSet; + + if(!usingBuckets) { + // we can only process directly from an update if the window is entire backward-looking. Since we + // allow negative values in fwd/rev ticks and timestamps, we need to check both + ctx.canProcessDirectly = isUpstreamAppendOnly && forwardTimeScaleUnits <= 0 && reverseTimeScaleUnits >= 0;; + } + } + + @Override + public void initializeFor(@NotNull final UpdateContext context, + @NotNull final RowSet updateRowSet, + @NotNull final UpdateBy.UpdateType type) { + final Context ctx = (Context) context; + ctx.currentUpdateType = type; + + if ((type == UpdateBy.UpdateType.Add && ctx.canProcessDirectly) + || type == UpdateBy.UpdateType.Reprocess) { + long windowStartKey = computeFirstAffectingKey(updateRowSet.firstRowKey(), ctx.workingRowSet); + ctx.loadWindowChunks(windowStartKey); + } + } + + @Override + public void finishFor(@NotNull final UpdateContext updateContext, @NotNull final UpdateBy.UpdateType type) { + final Context ctx = (Context) updateContext; + if(type == UpdateBy.UpdateType.Reprocess && ctx.modifiedBuilder != null) { + ctx.newModified = ctx.modifiedBuilder.build(); + } + } + + @Override + public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { + final Context ctx = (Context) context; + return ctx.canProcessDirectly; + } + + @Override + public boolean requiresValues(@NotNull final UpdateContext context) { + // windowed operators don't need current values supplied to them, they only care about windowed values which + // may or may not intersect with the column values + return false; + } + + @NotNull + @Override + final public RowSet getAdditionalModifications(@NotNull final UpdateContext ctx) { + return ((Context)ctx).newModified == null ? RowSetFactory.empty() : ((Context)ctx).newModified; + } + + @Override + final public boolean anyModified(@NotNull final UpdateContext ctx) { + return ((Context)ctx).newModified != null; + } + + // region Addition + /** + * Add a chunk of values to the operator. + * + * @param ctx the context object + * @param inputKeys the input keys for the chunk + * @param workingChunk the chunk of values + * @param bucketPosition the bucket position that the values belong to. + */ + protected abstract void doAddChunk(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk workingChunk, + final long bucketPosition); + + @Override + public void addChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk values, + long bucketPosition) { + final Context ctx = (Context) updateContext; + if (ctx.canProcessDirectly) { + ctx.loadDataChunks(inputKeys); + doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); + } + } + + @Override + public void addChunkBucketed(final @NotNull UpdateByOperator.UpdateContext context, + final @NotNull Chunk values, + final @NotNull LongChunk keyChunk, + final @NotNull IntChunk bucketPositions, + final @NotNull IntChunk startPositions, + final @NotNull IntChunk runLengths) { + final Context ctx = (Context) context; + if (ctx.canProcessDirectly) { +// final ShortChunk asShorts = values.asShortChunk(); + for (int runIdx = 0; runIdx < startPositions.size(); runIdx++) { + final int runStart = startPositions.get(runIdx); + final int runLength = runLengths.get(runIdx); + final int bucketPosition = bucketPositions.get(runStart); +// doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); + +// try (RowSequence rs = RowSequenceFactory.wrapRowKeysChunkAsRowSequence((LongChunk) keyChunk)_ + +// RowSetBuilderSequential builder = RowSetFactory.builderSequential(); +// for (int ii = runStart; ii < runStart + runLength; ii++) { +// builder.appendKey(keyChunk.get(ii)); +// } +// +// WritableRowSet bucketRs = bucketRowSet.get(bucketPosition); +// if (bucketRs == null) { +// bucketRs = builder.build(); +// bucketRowSet.set(bucketPosition, bucketRs); +// } else { +// try (final RowSet added = builder.build()) { +// bucketRs.insert(added); +// } +// } +// +// ctx.curVal = NULL_LONG; +// ctx.currentWindow.clear(); + +// accumulate(asShorts, (LongChunk) keyChunk, ctx, runStart, runLength, bucketRs); +// bucketLastVal.set(bucketPosition, ctx.curVal); + } + } + } + + // endregion + + // region Reprocessing + + public void resetForReprocess(@NotNull final UpdateContext context, + @NotNull final RowSet sourceRowSet, + long firstUnmodifiedKey) { + final Context ctx = (Context) context; + ctx.workingRowSet = sourceRowSet; + } + + @Override + public void resetForReprocessBucketed(@NotNull final UpdateContext context, + @NotNull final RowSet bucketRowSet, + final long bucketPosition, + final long firstUnmodifiedKey) { + final Context ctx = (Context) context; + } + + @Override + public void reprocessChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk valuesChunk, + @NotNull final RowSet postUpdateSourceIndex) { + final Context ctx = (Context) updateContext; + ctx.loadDataChunks(inputKeys); + doAddChunk(ctx, inputKeys, keyChunk, valuesChunk, 0); + ctx.getModifiedBuilder().appendRowSequence(inputKeys); + } + + @Override + public void reprocessChunkBucketed(@NotNull UpdateContext updateContext, + @NotNull final RowSequence chunkOk, + @NotNull final Chunk values, + @NotNull final LongChunk keyChunk, + @NotNull final IntChunk bucketPositions, + @NotNull final IntChunk runStartPositions, + @NotNull final IntChunk runLengths) { + addChunkBucketed(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); + ((Context)updateContext).getModifiedBuilder().appendRowSequence(chunkOk); + } + + // endregion + + // region No-Op Operations + + @Override + final public void modifyChunk(@NotNull final UpdateContext updateContext, + @Nullable final LongChunk prevKeyChunk, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk prevValuesChunk, + @NotNull final Chunk postValuesChunk, + long bucketPosition) { + } + + @Override + final public void removeChunk(@NotNull final UpdateContext updateContext, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk prevValuesChunk, + long bucketPosition) { + } + + @Override + final public void applyShift(@NotNull final UpdateContext updateContext, + @NotNull final RowSet prevIndex, + @NotNull final RowSetShiftData shifted) { + } + // endregion +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java index 01550d83e42..7c1c05eacd4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java @@ -19,7 +19,9 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; +import io.deephaven.engine.table.impl.updateby.rollingsum.ShortRollingSumOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -299,19 +301,6 @@ final public boolean anyModified(@NotNull final UpdateContext ctx) { } // region Addition - @Override - public void addChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk values, - long bucketPosition) { - final Context ctx = (Context) updateContext; - if (ctx.canProcessDirectly) { - ctx.loadDataChunks(inputKeys); - doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); - } - } - /** * Add a chunk of values to the operator. * @@ -326,6 +315,61 @@ protected abstract void doAddChunk(@NotNull final Context ctx, @NotNull final Chunk workingChunk, final long bucketPosition); + @Override + public void addChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk values, + long bucketPosition) { + final Context ctx = (Context) updateContext; + if (ctx.canProcessDirectly) { + ctx.loadDataChunks(inputKeys); + doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); + } + } + + @Override + public void addChunkBucketed(final @NotNull UpdateByOperator.UpdateContext context, + final @NotNull Chunk values, + final @NotNull LongChunk keyChunk, + final @NotNull IntChunk bucketPositions, + final @NotNull IntChunk startPositions, + final @NotNull IntChunk runLengths) { + final Context ctx = (Context) context; + if (ctx.canProcessDirectly) { +// final ShortChunk asShorts = values.asShortChunk(); + for (int runIdx = 0; runIdx < startPositions.size(); runIdx++) { + final int runStart = startPositions.get(runIdx); + final int runLength = runLengths.get(runIdx); + final int bucketPosition = bucketPositions.get(runStart); +// doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); + +// try (RowSequence rs = RowSequenceFactory.wrapRowKeysChunkAsRowSequence((LongChunk) keyChunk)_ + +// RowSetBuilderSequential builder = RowSetFactory.builderSequential(); +// for (int ii = runStart; ii < runStart + runLength; ii++) { +// builder.appendKey(keyChunk.get(ii)); +// } +// +// WritableRowSet bucketRs = bucketRowSet.get(bucketPosition); +// if (bucketRs == null) { +// bucketRs = builder.build(); +// bucketRowSet.set(bucketPosition, bucketRs); +// } else { +// try (final RowSet added = builder.build()) { +// bucketRs.insert(added); +// } +// } +// +// ctx.curVal = NULL_LONG; +// ctx.currentWindow.clear(); + +// accumulate(asShorts, (LongChunk) keyChunk, ctx, runStart, runLength, bucketRs); +// bucketLastVal.set(bucketPosition, ctx.curVal); + } + } + } + // endregion // region Reprocessing diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java new file mode 100644 index 00000000000..4ba0573ab15 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java @@ -0,0 +1,187 @@ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ShortRollingSumOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +package io.deephaven.engine.table.impl.updateby.rollingsum; + +import io.deephaven.api.updateby.OperationControl; +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.chunk.sized.SizedLongChunk; +import io.deephaven.chunk.util.pools.ChunkPoolConstants; +import io.deephaven.engine.rowset.*; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.table.*; +import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.UpdateByOperator; +import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.engine.table.impl.updateby.internal.*; +import io.deephaven.engine.table.impl.util.RowRedirection; +import io.deephaven.engine.table.impl.util.SizedSafeCloseable; +import io.deephaven.util.QueryConstants; +import org.apache.commons.lang3.mutable.MutableLong; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.util.Collections; +import java.util.LinkedList; +import java.util.Map; + +import static io.deephaven.util.QueryConstants.NULL_LONG; + +public class ByteRollingSumOperator extends BaseWindowedByteUpdateByOperator { + + // RollingSum will output Long values for integral types + private final WritableColumnSource outputSource; + private final WritableColumnSource maybeInnerSource; + + // region extra-fields + final byte nullValue; + // endregion extra-fields + + protected class Context extends BaseWindowedByteUpdateByOperator.Context { + public final SizedSafeCloseable fillContext; + public final SizedLongChunk outputValues; + public UpdateBy.UpdateType currentUpdateType; + + public LinkedList windowValues = new LinkedList<>(); + + protected Context(final int chunkSize) { + this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); + this.fillContext.ensureCapacity(chunkSize); + this.outputValues = new SizedLongChunk<>(chunkSize); + } + + @Override + public void close() { + super.close(); + outputValues.close(); + fillContext.close(); + } + } + + @NotNull + @Override + public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize) { + return new Context(chunkSize); + } + + @Override + public void setChunkSize(@NotNull UpdateContext context, int chunkSize) { + ((Context)context).outputValues.ensureCapacity(chunkSize); + ((Context)context).fillContext.ensureCapacity(chunkSize); + } + + public ByteRollingSumOperator(@NotNull final MatchPair pair, + @NotNull final String[] affectingColumns, + @NotNull final OperationControl control, + @Nullable final LongRecordingUpdateByOperator recorder, + final long reverseTimeScaleUnits, + final long forwardTimeScaleUnits, + @NotNull final ColumnSource valueSource, + @Nullable final RowRedirection rowRedirection + // region extra-constructor-args + ,final byte nullValue + // endregion extra-constructor-args + ) { + super(pair, affectingColumns, control, recorder, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection, valueSource); + if(rowRedirection != null) { + // region create-dense + this.maybeInnerSource = new LongArraySource(); + // endregion create-dense + this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); + } else { + this.maybeInnerSource = null; + // region create-sparse + this.outputSource = new LongSparseArraySource(); + // endregion create-sparse + } + + // region constructor + this.nullValue = nullValue; + // endregion constructor + } + + @Override + public void push(UpdateContext context, long key, byte val) { + final Context ctx = (Context) context; + ctx.windowValues.addLast(val); + } + + @Override + public void pop(UpdateContext context, long key) { + final Context ctx = (Context) context; + ctx.windowValues.pop(); + } + + @Override + public void reset(UpdateContext context) { + final Context ctx = (Context) context; + } + + @Override + public void doAddChunk(@NotNull final BaseWindowedByteUpdateByOperator.Context context, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk workingChunk, + final long groupPosition) { + final Context ctx = (Context) context; + + computeTicks(ctx, 0, inputKeys.intSize()); + //noinspection unchecked + outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); + } + + private void computeTicks(@NotNull final Context ctx, + final int runStart, + final int runLength) { + + final WritableLongChunk localOutputValues = ctx.outputValues.get(); + for (int ii = runStart; ii < runStart + runLength; ii++) { + if (recorder == null) { + ctx.fillWindowTicks(ctx, ctx.valuePositionChunk.get(ii)); + } + + MutableLong sum = new MutableLong(NULL_LONG); + ctx.windowValues.forEach(v-> { + if (v != null && v != QueryConstants.NULL_BYTE) { + if (sum.longValue() == NULL_LONG) { + sum.setValue(v); + } else { + sum.add(v); + } + } + }); + + // this call generates the push/pop calls to satisfy the window +// ctx.fillWindow(key, postUpdateSourceIndex); + + localOutputValues.set(ii, sum.getValue()); + } + } + + @NotNull + @Override + public Map> getOutputColumns() { + return Collections.singletonMap(pair.leftColumn, outputSource); + } + + @Override + public void startTrackingPrev() { + outputSource.startTrackingPrevValues(); + if(isRedirected) { + maybeInnerSource.startTrackingPrevValues(); + } + } + + @Override + public void applyOutputShift(@NotNull final UpdateContext context, + @NotNull final RowSet subIndexToShift, + final long delta) { + ((LongSparseArraySource)outputSource).shift(subIndexToShift, delta); + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java new file mode 100644 index 00000000000..d8965f97a97 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java @@ -0,0 +1,188 @@ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit FloatRollingSumOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +package io.deephaven.engine.table.impl.updateby.rollingsum; + +import io.deephaven.api.updateby.OperationControl; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.WritableDoubleChunk; +import io.deephaven.chunk.WritableLongChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.chunk.sized.SizedDoubleChunk; +import io.deephaven.chunk.sized.SizedLongChunk; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.table.ChunkSink; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.WritableColumnSource; +import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedDoubleUpdateByOperator; +import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; +import io.deephaven.engine.table.impl.util.RowRedirection; +import io.deephaven.engine.table.impl.util.SizedSafeCloseable; +import io.deephaven.util.QueryConstants; +import org.apache.commons.lang3.mutable.MutableLong; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.util.Collections; +import java.util.LinkedList; +import java.util.Map; + +import static io.deephaven.util.QueryConstants.NULL_LONG; + +public class DoubleRollingSumOperator extends BaseWindowedDoubleUpdateByOperator { + + // RollingSum will output Long values for integral types + private final WritableColumnSource outputSource; + private final WritableColumnSource maybeInnerSource; + + // region extra-fields + // endregion extra-fields + + protected class Context extends BaseWindowedDoubleUpdateByOperator.Context { + public final SizedSafeCloseable fillContext; + public final SizedDoubleChunk outputValues; + public UpdateBy.UpdateType currentUpdateType; + + public LinkedList windowValues = new LinkedList<>(); + + protected Context(final int chunkSize) { + this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); + this.fillContext.ensureCapacity(chunkSize); + this.outputValues = new SizedDoubleChunk<>(chunkSize); + } + + @Override + public void close() { + super.close(); + outputValues.close(); + fillContext.close(); + } + } + + @NotNull + @Override + public UpdateContext makeUpdateContext(final int chunkSize) { + return new Context(chunkSize); + } + + @Override + public void setChunkSize(@NotNull UpdateContext context, int chunkSize) { + ((Context)context).outputValues.ensureCapacity(chunkSize); + ((Context)context).fillContext.ensureCapacity(chunkSize); + } + + public DoubleRollingSumOperator(@NotNull final MatchPair pair, + @NotNull final String[] affectingColumns, + @NotNull final OperationControl control, + @Nullable final LongRecordingUpdateByOperator recorder, + final long reverseTimeScaleUnits, + final long forwardTimeScaleUnits, + @NotNull final ColumnSource valueSource, + @Nullable final RowRedirection rowRedirection + // region extra-constructor-args + // endregion extra-constructor-args + ) { + super(pair, affectingColumns, control, recorder, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection, valueSource); + if(rowRedirection != null) { + // region create-dense + this.maybeInnerSource = new DoubleArraySource(); + // endregion create-dense + this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); + } else { + this.maybeInnerSource = null; + // region create-sparse + this.outputSource = new DoubleSparseArraySource(); + // endregion create-sparse + } + + // region constructor + // endregion constructor + } + + @Override + public void push(UpdateContext context, long key, double val) { + final Context ctx = (Context) context; + ctx.windowValues.addLast(val); + } + + @Override + public void pop(UpdateContext context, long key) { + final Context ctx = (Context) context; + ctx.windowValues.pop(); + } + + @Override + public void reset(UpdateContext context) { + final Context ctx = (Context) context; + } + + @Override + public void doAddChunk(@NotNull final BaseWindowedDoubleUpdateByOperator.Context context, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk workingChunk, + final long groupPosition) { + final Context ctx = (Context) context; + + computeTicks(ctx, 0, inputKeys.intSize()); + //noinspection unchecked + outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); + } + + private void computeTicks(@NotNull final Context ctx, + final int runStart, + final int runLength) { + + final WritableDoubleChunk localOutputValues = ctx.outputValues.get(); + for (int ii = runStart; ii < runStart + runLength; ii++) { + if (recorder == null) { + ctx.fillWindowTicks(ctx, ctx.valuePositionChunk.get(ii)); + } + + MutableLong sum = new MutableLong(NULL_LONG); + ctx.windowValues.forEach(v-> { + if (v != null && v != QueryConstants.NULL_SHORT) { + if (sum.longValue() == NULL_LONG) { + sum.setValue(v); + } else { + sum.add(v); + } + } + }); + + // this call generates the push/pop calls to satisfy the window +// ctx.fillWindow(key, postUpdateSourceIndex); + + localOutputValues.set(ii, sum.getValue()); + } + } + + @NotNull + @Override + public Map> getOutputColumns() { + return Collections.singletonMap(pair.leftColumn, outputSource); + } + + @Override + public void startTrackingPrev() { + outputSource.startTrackingPrevValues(); + if(isRedirected) { + maybeInnerSource.startTrackingPrevValues(); + } + } + + @Override + public void applyOutputShift(@NotNull final UpdateContext context, + @NotNull final RowSet subIndexToShift, + final long delta) { + ((DoubleSparseArraySource)outputSource).shift(subIndexToShift, delta); + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java new file mode 100644 index 00000000000..271e8870a36 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java @@ -0,0 +1,183 @@ +package io.deephaven.engine.table.impl.updateby.rollingsum; + +import io.deephaven.api.updateby.OperationControl; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.WritableFloatChunk; +import io.deephaven.chunk.WritableLongChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.chunk.sized.SizedFloatChunk; +import io.deephaven.chunk.sized.SizedLongChunk; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.table.ChunkSink; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.WritableColumnSource; +import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedFloatUpdateByOperator; +import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; +import io.deephaven.engine.table.impl.util.RowRedirection; +import io.deephaven.engine.table.impl.util.SizedSafeCloseable; +import io.deephaven.util.QueryConstants; +import org.apache.commons.lang3.mutable.MutableLong; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.util.Collections; +import java.util.LinkedList; +import java.util.Map; + +import static io.deephaven.util.QueryConstants.NULL_LONG; + +public class FloatRollingSumOperator extends BaseWindowedFloatUpdateByOperator { + + // RollingSum will output Long values for integral types + private final WritableColumnSource outputSource; + private final WritableColumnSource maybeInnerSource; + + // region extra-fields + // endregion extra-fields + + protected class Context extends BaseWindowedFloatUpdateByOperator.Context { + public final SizedSafeCloseable fillContext; + public final SizedFloatChunk outputValues; + public UpdateBy.UpdateType currentUpdateType; + + public LinkedList windowValues = new LinkedList<>(); + + protected Context(final int chunkSize) { + this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); + this.fillContext.ensureCapacity(chunkSize); + this.outputValues = new SizedFloatChunk<>(chunkSize); + } + + @Override + public void close() { + super.close(); + outputValues.close(); + fillContext.close(); + } + } + + @NotNull + @Override + public UpdateContext makeUpdateContext(final int chunkSize) { + return new Context(chunkSize); + } + + @Override + public void setChunkSize(@NotNull UpdateContext context, int chunkSize) { + ((Context)context).outputValues.ensureCapacity(chunkSize); + ((Context)context).fillContext.ensureCapacity(chunkSize); + } + + public FloatRollingSumOperator(@NotNull final MatchPair pair, + @NotNull final String[] affectingColumns, + @NotNull final OperationControl control, + @Nullable final LongRecordingUpdateByOperator recorder, + final long reverseTimeScaleUnits, + final long forwardTimeScaleUnits, + @NotNull final ColumnSource valueSource, + @Nullable final RowRedirection rowRedirection + // region extra-constructor-args + // endregion extra-constructor-args + ) { + super(pair, affectingColumns, control, recorder, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection, valueSource); + if(rowRedirection != null) { + // region create-dense + this.maybeInnerSource = new FloatArraySource(); + // endregion create-dense + this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); + } else { + this.maybeInnerSource = null; + // region create-sparse + this.outputSource = new FloatSparseArraySource(); + // endregion create-sparse + } + + // region constructor + // endregion constructor + } + + @Override + public void push(UpdateContext context, long key, float val) { + final Context ctx = (Context) context; + ctx.windowValues.addLast(val); + } + + @Override + public void pop(UpdateContext context, long key) { + final Context ctx = (Context) context; + ctx.windowValues.pop(); + } + + @Override + public void reset(UpdateContext context) { + final Context ctx = (Context) context; + } + + @Override + public void doAddChunk(@NotNull final BaseWindowedFloatUpdateByOperator.Context context, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk workingChunk, + final long groupPosition) { + final Context ctx = (Context) context; + + computeTicks(ctx, 0, inputKeys.intSize()); + //noinspection unchecked + outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); + } + + private void computeTicks(@NotNull final Context ctx, + final int runStart, + final int runLength) { + + final WritableFloatChunk localOutputValues = ctx.outputValues.get(); + for (int ii = runStart; ii < runStart + runLength; ii++) { + if (recorder == null) { + ctx.fillWindowTicks(ctx, ctx.valuePositionChunk.get(ii)); + } + + MutableLong sum = new MutableLong(NULL_LONG); + ctx.windowValues.forEach(v-> { + if (v != null && v != QueryConstants.NULL_SHORT) { + if (sum.longValue() == NULL_LONG) { + sum.setValue(v); + } else { + sum.add(v); + } + } + }); + + // this call generates the push/pop calls to satisfy the window +// ctx.fillWindow(key, postUpdateSourceIndex); + + localOutputValues.set(ii, sum.getValue()); + } + } + + @NotNull + @Override + public Map> getOutputColumns() { + return Collections.singletonMap(pair.leftColumn, outputSource); + } + + @Override + public void startTrackingPrev() { + outputSource.startTrackingPrevValues(); + if(isRedirected) { + maybeInnerSource.startTrackingPrevValues(); + } + } + + @Override + public void applyOutputShift(@NotNull final UpdateContext context, + @NotNull final RowSet subIndexToShift, + final long delta) { + ((FloatSparseArraySource)outputSource).shift(subIndexToShift, delta); + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java new file mode 100644 index 00000000000..4df3f391ba5 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java @@ -0,0 +1,184 @@ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ShortRollingSumOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +package io.deephaven.engine.table.impl.updateby.rollingsum; + +import io.deephaven.api.updateby.OperationControl; +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.chunk.sized.SizedLongChunk; +import io.deephaven.chunk.util.pools.ChunkPoolConstants; +import io.deephaven.engine.rowset.*; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.table.*; +import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.UpdateByOperator; +import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.engine.table.impl.updateby.internal.*; +import io.deephaven.engine.table.impl.util.RowRedirection; +import io.deephaven.engine.table.impl.util.SizedSafeCloseable; +import io.deephaven.util.QueryConstants; +import org.apache.commons.lang3.mutable.MutableLong; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.util.Collections; +import java.util.LinkedList; +import java.util.Map; + +import static io.deephaven.util.QueryConstants.NULL_LONG; + +public class IntRollingSumOperator extends BaseWindowedIntUpdateByOperator { + + // RollingSum will output Long values for integral types + private final WritableColumnSource outputSource; + private final WritableColumnSource maybeInnerSource; + + // region extra-fields + // endregion extra-fields + + protected class Context extends BaseWindowedIntUpdateByOperator.Context { + public final SizedSafeCloseable fillContext; + public final SizedLongChunk outputValues; + public UpdateBy.UpdateType currentUpdateType; + + public LinkedList windowValues = new LinkedList<>(); + + protected Context(final int chunkSize) { + this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); + this.fillContext.ensureCapacity(chunkSize); + this.outputValues = new SizedLongChunk<>(chunkSize); + } + + @Override + public void close() { + super.close(); + outputValues.close(); + fillContext.close(); + } + } + + @NotNull + @Override + public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize) { + return new Context(chunkSize); + } + + @Override + public void setChunkSize(@NotNull UpdateContext context, int chunkSize) { + ((Context)context).outputValues.ensureCapacity(chunkSize); + ((Context)context).fillContext.ensureCapacity(chunkSize); + } + + public IntRollingSumOperator(@NotNull final MatchPair pair, + @NotNull final String[] affectingColumns, + @NotNull final OperationControl control, + @Nullable final LongRecordingUpdateByOperator recorder, + final long reverseTimeScaleUnits, + final long forwardTimeScaleUnits, + @NotNull final ColumnSource valueSource, + @Nullable final RowRedirection rowRedirection + // region extra-constructor-args + // endregion extra-constructor-args + ) { + super(pair, affectingColumns, control, recorder, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection, valueSource); + if(rowRedirection != null) { + // region create-dense + this.maybeInnerSource = new LongArraySource(); + // endregion create-dense + this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); + } else { + this.maybeInnerSource = null; + // region create-sparse + this.outputSource = new LongSparseArraySource(); + // endregion create-sparse + } + + // region constructor + // endregion constructor + } + + @Override + public void push(UpdateContext context, long key, int val) { + final Context ctx = (Context) context; + ctx.windowValues.addLast(val); + } + + @Override + public void pop(UpdateContext context, long key) { + final Context ctx = (Context) context; + ctx.windowValues.pop(); + } + + @Override + public void reset(UpdateContext context) { + final Context ctx = (Context) context; + } + + @Override + public void doAddChunk(@NotNull final BaseWindowedIntUpdateByOperator.Context context, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk workingChunk, + final long groupPosition) { + final Context ctx = (Context) context; + + computeTicks(ctx, 0, inputKeys.intSize()); + //noinspection unchecked + outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); + } + + private void computeTicks(@NotNull final Context ctx, + final int runStart, + final int runLength) { + + final WritableLongChunk localOutputValues = ctx.outputValues.get(); + for (int ii = runStart; ii < runStart + runLength; ii++) { + if (recorder == null) { + ctx.fillWindowTicks(ctx, ctx.valuePositionChunk.get(ii)); + } + + MutableLong sum = new MutableLong(NULL_LONG); + ctx.windowValues.forEach(v-> { + if (v != null && v != QueryConstants.NULL_INT) { + if (sum.longValue() == NULL_LONG) { + sum.setValue(v); + } else { + sum.add(v); + } + } + }); + + // this call generates the push/pop calls to satisfy the window +// ctx.fillWindow(key, postUpdateSourceIndex); + + localOutputValues.set(ii, sum.getValue()); + } + } + + @NotNull + @Override + public Map> getOutputColumns() { + return Collections.singletonMap(pair.leftColumn, outputSource); + } + + @Override + public void startTrackingPrev() { + outputSource.startTrackingPrevValues(); + if(isRedirected) { + maybeInnerSource.startTrackingPrevValues(); + } + } + + @Override + public void applyOutputShift(@NotNull final UpdateContext context, + @NotNull final RowSet subIndexToShift, + final long delta) { + ((LongSparseArraySource)outputSource).shift(subIndexToShift, delta); + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java new file mode 100644 index 00000000000..aee9474b61e --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java @@ -0,0 +1,184 @@ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ShortRollingSumOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +package io.deephaven.engine.table.impl.updateby.rollingsum; + +import io.deephaven.api.updateby.OperationControl; +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.chunk.sized.SizedLongChunk; +import io.deephaven.chunk.util.pools.ChunkPoolConstants; +import io.deephaven.engine.rowset.*; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.table.*; +import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.UpdateByOperator; +import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.engine.table.impl.updateby.internal.*; +import io.deephaven.engine.table.impl.util.RowRedirection; +import io.deephaven.engine.table.impl.util.SizedSafeCloseable; +import io.deephaven.util.QueryConstants; +import org.apache.commons.lang3.mutable.MutableLong; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.util.Collections; +import java.util.LinkedList; +import java.util.Map; + +import static io.deephaven.util.QueryConstants.NULL_LONG; + +public class LongRollingSumOperator extends BaseWindowedLongUpdateByOperator { + + // RollingSum will output Long values for integral types + private final WritableColumnSource outputSource; + private final WritableColumnSource maybeInnerSource; + + // region extra-fields + // endregion extra-fields + + protected class Context extends BaseWindowedLongUpdateByOperator.Context { + public final SizedSafeCloseable fillContext; + public final SizedLongChunk outputValues; + public UpdateBy.UpdateType currentUpdateType; + + public LinkedList windowValues = new LinkedList<>(); + + protected Context(final int chunkSize) { + this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); + this.fillContext.ensureCapacity(chunkSize); + this.outputValues = new SizedLongChunk<>(chunkSize); + } + + @Override + public void close() { + super.close(); + outputValues.close(); + fillContext.close(); + } + } + + @NotNull + @Override + public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize) { + return new Context(chunkSize); + } + + @Override + public void setChunkSize(@NotNull UpdateContext context, int chunkSize) { + ((Context)context).outputValues.ensureCapacity(chunkSize); + ((Context)context).fillContext.ensureCapacity(chunkSize); + } + + public LongRollingSumOperator(@NotNull final MatchPair pair, + @NotNull final String[] affectingColumns, + @NotNull final OperationControl control, + @Nullable final LongRecordingUpdateByOperator recorder, + final long reverseTimeScaleUnits, + final long forwardTimeScaleUnits, + @NotNull final ColumnSource valueSource, + @Nullable final RowRedirection rowRedirection + // region extra-constructor-args + // endregion extra-constructor-args + ) { + super(pair, affectingColumns, control, recorder, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection, valueSource); + if(rowRedirection != null) { + // region create-dense + this.maybeInnerSource = new LongArraySource(); + // endregion create-dense + this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); + } else { + this.maybeInnerSource = null; + // region create-sparse + this.outputSource = new LongSparseArraySource(); + // endregion create-sparse + } + + // region constructor + // endregion constructor + } + + @Override + public void push(UpdateContext context, long key, long val) { + final Context ctx = (Context) context; + ctx.windowValues.addLast(val); + } + + @Override + public void pop(UpdateContext context, long key) { + final Context ctx = (Context) context; + ctx.windowValues.pop(); + } + + @Override + public void reset(UpdateContext context) { + final Context ctx = (Context) context; + } + + @Override + public void doAddChunk(@NotNull final BaseWindowedLongUpdateByOperator.Context context, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk workingChunk, + final long groupPosition) { + final Context ctx = (Context) context; + + computeTicks(ctx, 0, inputKeys.intSize()); + //noinspection unchecked + outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); + } + + private void computeTicks(@NotNull final Context ctx, + final int runStart, + final int runLength) { + + final WritableLongChunk localOutputValues = ctx.outputValues.get(); + for (int ii = runStart; ii < runStart + runLength; ii++) { + if (recorder == null) { + ctx.fillWindowTicks(ctx, ctx.valuePositionChunk.get(ii)); + } + + MutableLong sum = new MutableLong(NULL_LONG); + ctx.windowValues.forEach(v-> { + if (v != null && v != QueryConstants.NULL_LONG) { + if (sum.longValue() == NULL_LONG) { + sum.setValue(v); + } else { + sum.add(v); + } + } + }); + + // this call generates the push/pop calls to satisfy the window +// ctx.fillWindow(key, postUpdateSourceIndex); + + localOutputValues.set(ii, sum.getValue()); + } + } + + @NotNull + @Override + public Map> getOutputColumns() { + return Collections.singletonMap(pair.leftColumn, outputSource); + } + + @Override + public void startTrackingPrev() { + outputSource.startTrackingPrevValues(); + if(isRedirected) { + maybeInnerSource.startTrackingPrevValues(); + } + } + + @Override + public void applyOutputShift(@NotNull final UpdateContext context, + @NotNull final RowSet subIndexToShift, + final long delta) { + ((LongSparseArraySource)outputSource).shift(subIndexToShift, delta); + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java index c6ec76fefae..fc13fdd56a3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java @@ -98,48 +98,6 @@ public ShortRollingSumOperator(@NotNull final MatchPair pair, // endregion constructor } - @Override - public void addChunkBucketed(final @NotNull UpdateByOperator.UpdateContext context, - final @NotNull Chunk values, - final @NotNull LongChunk keyChunk, - final @NotNull IntChunk bucketPositions, - final @NotNull IntChunk startPositions, - final @NotNull IntChunk runLengths) { - - final Context ctx = (Context) context; - final ShortChunk asShorts = values.asShortChunk(); - for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int bucketPosition = bucketPositions.get(runStart); - -// try (RowSequence rs = RowSequenceFactory.wrapRowKeysChunkAsRowSequence((LongChunk) keyChunk)_ - -// RowSetBuilderSequential builder = RowSetFactory.builderSequential(); -// for (int ii = runStart; ii < runStart + runLength; ii++) { -// builder.appendKey(keyChunk.get(ii)); -// } -// -// WritableRowSet bucketRs = bucketRowSet.get(bucketPosition); -// if (bucketRs == null) { -// bucketRs = builder.build(); -// bucketRowSet.set(bucketPosition, bucketRs); -// } else { -// try (final RowSet added = builder.build()) { -// bucketRs.insert(added); -// } -// } -// -// ctx.curVal = NULL_LONG; -// ctx.currentWindow.clear(); - -// accumulate(asShorts, (LongChunk) keyChunk, ctx, runStart, runLength, bucketRs); -// bucketLastVal.set(bucketPosition, ctx.curVal); - } - //noinspection unchecked - outputSource.fillFromChunkUnordered(ctx.fillContext.get(), ctx.outputValues.get(), (LongChunk) keyChunk); - } - @Override public void push(UpdateContext context, long key, short val) { final Context ctx = (Context) context; diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java index 252da68d562..e83694e5e0c 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java @@ -123,7 +123,9 @@ private void doTestStaticBucketed(boolean grouped) { final Table summed = t.updateBy(UpdateByOperation.RollingSum(prevTicks, postTicks, "byteCol", "shortCol", "intCol", "longCol", "floatCol", - "doubleCol", "boolCol", "bigIntCol", "bigDecimalCol"), "Sym"); + "doubleCol", "boolCol" +// TODO: put these back ,"bigIntCol", "bigDecimalCol" + ), "Sym"); final PartitionedTable preOp = t.partitionBy("Sym"); diff --git a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java index 363246cf8bf..6faaf3c1e9f 100644 --- a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java +++ b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java @@ -81,8 +81,6 @@ public static void main(String[] args) throws IOException { "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java", "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java"); - - // Replicate the rolling versions of UpdateByOperations files = ReplicatePrimitiveCode.charToAllButBooleanAndFloats( "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java", @@ -96,16 +94,16 @@ public static void main(String[] args) throws IOException { fixupByteBase(f); } } -// objectResult = ReplicatePrimitiveCode.charToObject( -// "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java"); -// fixupStandardObject(objectResult, "BaseWindowedObjectUpdateByOperator", true); -// ReplicatePrimitiveCode.floatToAllFloatingPoints( -// "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java", -// exemptions); -// -// replicateNumericOperator( -// "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java", -// "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java"); + objectResult = ReplicatePrimitiveCode.charToObject( + "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java"); + fixupStandardObject(objectResult, "BaseWindowedObjectUpdateByOperator", true); + ReplicatePrimitiveCode.floatToAllFloatingPoints( + "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java", + exemptions); + + replicateNumericOperator( + "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java", + "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java"); } private static void replicateNumericOperator(@NotNull final String shortClass, @NotNull final String floatClass) @@ -115,7 +113,7 @@ private static void replicateNumericOperator(@NotNull final String shortClass, @ fixupInteger(f); } - if (f.contains("Byte") && f.contains("CumSum")) { + if (f.contains("Byte") && (f.contains("CumSum") || f.contains("RollingSum"))) { fixupByte(f); } @@ -248,6 +246,7 @@ private static void fixupInteger(String intResult) throws IOException { List lines = FileUtils.readLines(objectFile, Charset.defaultCharset()); lines = ReplicationUtils.globalReplacements(lines, "BaseIntegerUpdateByOperator", "BaseIntUpdateByOperator", + "BaseWindowedIntegerUpdateByOperator", "BaseWindowedIntUpdateByOperator", "public class Integer", "public class Int", "public Integer", "public Int", "WritableIntegerChunk", "WritableIntChunk", diff --git a/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java b/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java index 971debbf37a..906b519b949 100644 --- a/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java +++ b/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java @@ -103,19 +103,17 @@ public final OperationControl controlOrDefault() { @Override public final boolean applicableTo(Class inputType) { - return inputType.equals(short.class); + return + // is primitive numeric? + inputType.equals(double.class) || inputType.equals(float.class) + || inputType.equals(int.class) || inputType.equals(long.class) || inputType.equals(short.class) + || inputType.equals(byte.class) -// return -// // is primitive numeric? -// inputType.equals(double.class) || inputType.equals(float.class) -// || inputType.equals(int.class) || inputType.equals(long.class) || inputType.equals(short.class) -// || inputType.equals(byte.class) -// // // is boxed numeric? // || Number.class.isAssignableFrom(inputType) -// -// // is boolean? -// || inputType == boolean.class || inputType == Boolean.class; + + // is boolean? + || inputType == boolean.class || inputType == Boolean.class; } @Override From 97d85b9e1aca37f4255bdf58e9f04bb90a8668fa Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 12 Aug 2022 08:38:29 -0700 Subject: [PATCH 008/123] rewrote to use PartitionBy as engine for bucketed updateBy --- .../java/io/deephaven/engine/table/Table.java | 14 + .../engine/table/impl/BucketedUpdateBy.java | 1372 ----------------- .../engine/table/impl/QueryTable.java | 27 +- .../engine/table/impl/TableWithDefaults.java | 7 + .../engine/table/impl/UncoalescedTable.java | 6 + .../deephaven/engine/table/impl/UpdateBy.java | 44 +- .../engine/table/impl/UpdateByOperator.java | 88 +- .../table/impl/UpdateByWindowedOperator.java | 8 - .../engine/table/impl/ZeroKeyUpdateBy.java | 27 +- .../ema/BasePrimitiveEMAOperator.java | 117 +- .../updateby/ema/BigNumberEMAOperator.java | 131 +- .../updateby/fill/BooleanFillByOperator.java | 28 +- .../updateby/fill/ByteFillByOperator.java | 28 +- .../updateby/fill/CharFillByOperator.java | 28 +- .../updateby/fill/DoubleFillByOperator.java | 28 +- .../updateby/fill/FloatFillByOperator.java | 28 +- .../impl/updateby/fill/IntFillByOperator.java | 28 +- .../updateby/fill/LongFillByOperator.java | 28 +- .../updateby/fill/ObjectFillByOperator.java | 28 +- .../updateby/fill/ShortFillByOperator.java | 28 +- .../internal/BaseByteUpdateByOperator.java | 96 +- .../internal/BaseCharUpdateByOperator.java | 96 +- .../internal/BaseDoubleUpdateByOperator.java | 91 +- .../internal/BaseFloatUpdateByOperator.java | 91 +- .../internal/BaseIntUpdateByOperator.java | 96 +- .../internal/BaseLongUpdateByOperator.java | 96 +- .../internal/BaseObjectBinaryOperator.java | 39 +- .../internal/BaseObjectUpdateByOperator.java | 96 +- .../internal/BaseShortUpdateByOperator.java | 96 +- .../BaseWindowedByteUpdateByOperator.java | 91 +- .../BaseWindowedCharUpdateByOperator.java | 91 +- .../BaseWindowedDoubleUpdateByOperator.java | 91 +- .../BaseWindowedFloatUpdateByOperator.java | 91 +- .../BaseWindowedIntUpdateByOperator.java | 91 +- .../BaseWindowedLongUpdateByOperator.java | 91 +- .../BaseWindowedObjectUpdateByOperator.java | 91 +- .../BaseWindowedShortUpdateByOperator.java | 91 +- .../LongRecordingUpdateByOperator.java | 37 +- .../minmax/ByteCumMinMaxOperator.java | 29 +- .../minmax/DoubleCumMinMaxOperator.java | 38 +- .../minmax/FloatCumMinMaxOperator.java | 38 +- .../updateby/minmax/IntCumMinMaxOperator.java | 29 +- .../minmax/LongCumMinMaxOperator.java | 29 +- .../minmax/ShortCumMinMaxOperator.java | 29 +- .../updateby/prod/ByteCumProdOperator.java | 32 +- .../updateby/prod/DoubleCumProdOperator.java | 36 +- .../updateby/prod/FloatCumProdOperator.java | 36 +- .../updateby/prod/IntCumProdOperator.java | 32 +- .../updateby/prod/LongCumProdOperator.java | 32 +- .../updateby/prod/ShortCumProdOperator.java | 32 +- .../rollingsum/ByteRollingSumOperator.java | 8 +- .../rollingsum/DoubleRollingSumOperator.java | 3 +- .../rollingsum/FloatRollingSumOperator.java | 3 +- .../rollingsum/IntRollingSumOperator.java | 8 +- .../rollingsum/LongRollingSumOperator.java | 8 +- .../rollingsum/ShortRollingSumOperator.java | 8 +- .../impl/updateby/sum/ByteCumSumOperator.java | 32 +- .../updateby/sum/DoubleCumSumOperator.java | 40 +- .../updateby/sum/FloatCumSumOperator.java | 40 +- .../impl/updateby/sum/IntCumSumOperator.java | 32 +- .../impl/updateby/sum/LongCumSumOperator.java | 32 +- .../updateby/sum/ShortCumSumOperator.java | 32 +- 62 files changed, 453 insertions(+), 3840 deletions(-) delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedUpdateBy.java 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 6605df24864..89fc8f02f0e 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 @@ -1587,6 +1587,19 @@ Table join(Table rightTable, Collection columnsToMatch, @ConcurrentMethod PartitionedTable partitionBy(boolean dropKeys, String... keyColumnNames); + /** + * Create a {@link PartitionedTable} from this table, partitioned according to the specified key columns. + *

    + * The underlying partitioned table backing the result contains each row in {@code this} table in exactly one of the + * result's constituent tables. + * + * @param dropKeys Whether to drop key columns in the output constituent tables + * @param keyColumnNames The name of the key columns to partition by + * @return A {@link PartitionedTable} keyed by {@code keyColumnNames} + */ + @ConcurrentMethod + PartitionedTable partitionBy(boolean dropKeys, @NotNull Collection keyColumnNames); + /** * Equivalent to {@code partitionBy(false, keyColumnNames)} *

    @@ -1602,6 +1615,7 @@ Table join(Table rightTable, Collection columnsToMatch, @ConcurrentMethod PartitionedTable partitionBy(String... keyColumnNames); + // ----------------------------------------------------------------------------------------------------------------- // Hierarchical table operations (rollup and treeTable). // ----------------------------------------------------------------------------------------------------------------- diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedUpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedUpdateBy.java deleted file mode 100644 index d86fd55b986..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedUpdateBy.java +++ /dev/null @@ -1,1372 +0,0 @@ -package io.deephaven.engine.table.impl; - -import io.deephaven.api.updateby.UpdateByControl; -import io.deephaven.base.verify.Assert; -import io.deephaven.base.verify.Require; -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.chunk.sized.SizedIntChunk; -import io.deephaven.chunk.sized.SizedLongChunk; -import io.deephaven.engine.rowset.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; -import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.by.ChunkedOperatorAggregationHelper; -import io.deephaven.engine.table.impl.by.HashedRunFinder; -import io.deephaven.engine.table.impl.by.typed.TypedHasherFactory; -import io.deephaven.engine.table.impl.sort.permute.LongPermuteKernel; -import io.deephaven.engine.table.impl.sort.permute.PermuteKernel; -import io.deephaven.engine.table.impl.updateby.hashing.*; -import io.deephaven.engine.table.impl.util.SizedSafeCloseable; -import io.deephaven.engine.table.impl.util.UpdateSizeCalculator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; -import io.deephaven.util.SafeCloseable; -import io.deephaven.util.SafeCloseableList; -import org.apache.commons.lang3.mutable.MutableInt; -import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; - -import javax.annotation.OverridingMethodsMustInvokeSuper; -import java.util.Arrays; -import java.util.Map; - -import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; - -public class BucketedUpdateBy extends UpdateBy { - /** The column sources that are used as keys for bucketing values. */ - private final ColumnSource[] keySources; - - /** The hash table object to store key tuples -> bucket positions */ - private UpdateByStateManager hashTable; - - /** A tracker object to manage indices and updates on a per-bucket basis */ - private final UpdateBySlotTracker slotTracker; - - /** An object to keep track of the next available output position to use */ - final MutableInt nextOutputPosition = new MutableInt(0); - - /** - * An object to hold the transient state during a single {@link BaseTable.ListenerImpl#onUpdate(TableUpdate)} update - * cycle. - */ - private class BucketedContext implements SafeCloseable { - /** The expected size of chunks to the various update stages */ - int chunkSize; - - /** An indicator of if each slot has been populated with data or not for this phase. */ - boolean[] inputChunkPopulated; - - /** An array of boolean denoting which operators are affected by the current update. */ - final boolean[] opAffected; - - /** A simplem indicator if any operator had modified input columns, adds, or removes. */ - boolean anyAffected; - - /** True if any of the key columns contained modifications. */ - final boolean keysModified; - /** An array of context objects for each underlying operator */ - final UpdateByOperator.UpdateContext[] opContext; - - /** A {@link SharedContext} to be used while creating other contexts */ - SharedContext sharedContext = SharedContext.makeSharedContext(); - - /** An array of {@link ChunkSource.FillContext}s for each input column */ - final SizedSafeCloseable[] fillContexts; - - /** The kernels used to permute chunks when dealing with updates. */ - final PermuteKernel[] permuteKernels; - - /** A set of chunks used to store post-shift working values */ - final SizedSafeCloseable>[] postWorkingChunks; - - /** A set of chunks used to store post-shift working values after a permutation */ - final SizedSafeCloseable>[] permutedPostWorkingChunks; - - /** A Chunk of longs to store the keys being updated */ - final SizedLongChunk keyChunk; - - /** A Chunk of longs to store the keys being updated after a permutation */ - final SizedLongChunk permutedKeyChunk; - - /** A chunk to store the bucket position of individual values in the key and value chunks */ - final SizedIntChunk outputPositions; - - /** A chunk to store the start positions of runs after a run detection and possible permutation */ - final SizedIntChunk runStarts; - - /** - * A chunk to store the lengths of runs after a run detection and possible permutation. Parallel to runStarts - */ - final SizedIntChunk runLengths; - - /** A chunk to store the chunk positions of values in the original chunks after a permutation */ - final SizedIntChunk chunkPositions; - - /** The context to use when finding runs of values in buckets within a chunk of data */ - final SizedSafeCloseable findCtx; - - /** The {@link UpdateByStateManager hash table} build context */ - final SafeCloseable bc; - - /** The {@link UpdateByStateManager hash table} probe context */ - SafeCloseable pc; - - /** The index of keys that need to be revisited during the reprocess phase. */ - RowSetBuilderRandom accumulator; - - // TODO: If I write my own I can avoid extra object creation for the pairs and all of the - // virtual comparator method calls. I will make it work this way first so I'm not chasing - // data structure bugs. - /** - * A Priority queue that we'll use to populate the output position chunk when we are iterating changes by - * affected bucket. - */ - TrackerPriorityQueue bucketQ; - - @SuppressWarnings("resource") - BucketedContext(@NotNull final TableUpdate upstream, - @NotNull final ModifiedColumnSet keyModifiedColumnSet, - @Nullable final ModifiedColumnSet[] inputModifiedColumnSets) { - final int updateSize = UpdateSizeCalculator.chunkSize(upstream, control.chunkCapacityOrDefault()); - - this.inputChunkPopulated = new boolean[operators.length]; - this.keysModified = upstream.modifiedColumnSet().containsAny(keyModifiedColumnSet); - this.chunkSize = - UpdateSizeCalculator.chunkSize(updateSize, upstream.shifted(), control.chunkCapacityOrDefault()); - this.opAffected = new boolean[operators.length]; - // noinspection unchecked - this.fillContexts = new SizedSafeCloseable[operators.length]; - this.opContext = new UpdateByOperator.UpdateContext[operators.length]; - this.keyChunk = new SizedLongChunk<>(chunkSize); - this.permuteKernels = new PermuteKernel[operators.length]; - this.permutedKeyChunk = new SizedLongChunk<>(chunkSize); - - // noinspection unchecked - this.postWorkingChunks = new SizedSafeCloseable[operators.length]; - // noinspection unchecked - this.permutedPostWorkingChunks = new SizedSafeCloseable[operators.length]; - - this.outputPositions = new SizedIntChunk<>(chunkSize); - this.runStarts = new SizedIntChunk<>(chunkSize); - this.runLengths = new SizedIntChunk<>(chunkSize); - this.chunkPositions = new SizedIntChunk<>(chunkSize); - - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - opAffected[opIdx] = upstream.added().isNonempty() || - upstream.removed().isNonempty() || - upstream.shifted().nonempty() || - (upstream.modifiedColumnSet().nonempty() && (inputModifiedColumnSets == null - || upstream.modifiedColumnSet().containsAny(inputModifiedColumnSets[opIdx]))); - - // TODO: Would be nice to abort if the thing is unaffected here and let us recreate later on if we need - // it - // during reprocess. - opContext[opIdx] = operators[opIdx].makeUpdateContext(chunkSize); - if (opAffected[opIdx]) { - operators[opIdx].initializeForUpdate(opContext[opIdx], upstream, source.getRowSet(), true, true); - anyAffected = true; - } - - final int slotPosition = inputSourceSlots[opIdx]; - if (fillContexts[slotPosition] == null) { - fillContexts[slotPosition] = new SizedSafeCloseable<>( - sz -> inputSources[slotPosition].makeFillContext(sz, getSharedContext())); - fillContexts[slotPosition].ensureCapacity(chunkSize); - - postWorkingChunks[slotPosition] = new SizedSafeCloseable<>( - sz -> inputSources[slotPosition].getChunkType().makeWritableChunk(sz)); - postWorkingChunks[slotPosition].ensureCapacity(chunkSize); - - // TODO: Maybe can not allocate this all the time. - permutedPostWorkingChunks[slotPosition] = new SizedSafeCloseable<>( - sz -> inputSources[slotPosition].getChunkType().makeWritableChunk(sz)); - permutedPostWorkingChunks[slotPosition].ensureCapacity(chunkSize); - permuteKernels[slotPosition] = - PermuteKernel.makePermuteKernel(inputSources[slotPosition].getChunkType()); - } - } - - findCtx = new SizedSafeCloseable<>(HashedRunFinder.HashedRunContext::new); - findCtx.ensureCapacity(chunkSize); - bc = hashTable.makeUpdateByBuildContext(keySources, chunkSize); - } - - public SharedContext getSharedContext() { - return sharedContext; - } - - @Override - public void close() { - keyChunk.close(); - permutedKeyChunk.close(); - outputPositions.close(); - runStarts.close(); - runLengths.close(); - chunkPositions.close(); - findCtx.close(); - bc.close(); - - if (pc != null) { - pc.close(); - } - - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - opContext[opIdx].close(); - - if (fillContexts[opIdx] != null) { - fillContexts[opIdx].close(); - } - - if (postWorkingChunks[opIdx] != null) { - postWorkingChunks[opIdx].close(); - } - - if (permutedPostWorkingChunks[opIdx] != null) { - permutedPostWorkingChunks[opIdx].close(); - } - } - sharedContext.close(); - } - - /** - * Get (and potentially create) the probe context object for use with - * {@link UpdateByStateManager#remove(SafeCloseable, RowSequence, ColumnSource[], WritableIntChunk)} and - * {@link UpdateByStateManager#findModifications(SafeCloseable, RowSequence, ColumnSource[], WritableIntChunk)}. - * - * @return an appropriate probe context. - */ - private SafeCloseable getProbeContet() { - if (pc == null) { - pc = hashTable.makeUpdateByProbeContext(keySources, chunkSize); - } - - return pc; - } - - /** - * TableUpdate the chunk sizes used for all internal state objects, for use when reprocessing keys is required - * and the initial selected chunk size is insufficient. - * - * @param newChunkSize the new chunk size to use. - */ - void setChunkSize(final int newChunkSize) { - if (newChunkSize <= this.chunkSize) { - return; - } - - this.chunkSize = newChunkSize; - - // We have to close and recreate the shared context because a .reset() is not enough to ensure that any - // cached chunks that something stuffed into there are resized. - this.sharedContext.close(); - this.sharedContext = SharedContext.makeSharedContext(); - - keyChunk.ensureCapacity(newChunkSize); - permutedKeyChunk.ensureCapacity(newChunkSize); - outputPositions.ensureCapacity(newChunkSize); - runStarts.ensureCapacity(newChunkSize); - runLengths.ensureCapacity(newChunkSize); - chunkPositions.ensureCapacity(newChunkSize); - findCtx.ensureCapacity(newChunkSize); - - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - operators[opIdx].setChunkSize(opContext[opIdx], newChunkSize); - if (fillContexts[opIdx] != null) { - fillContexts[opIdx].ensureCapacity(newChunkSize); - postWorkingChunks[opIdx].ensureCapacity(newChunkSize); - permutedPostWorkingChunks[opIdx].ensureCapacity(newChunkSize); - } - } - } - } - - /** - * Ensure that all operators have enough space to support the proper number of buckets. - * - * @param nBuckets the number of buckets - */ - public void setBucketCapacity(final int nBuckets) { - for (final UpdateByOperator op : operators) { - op.setBucketCapacity(nBuckets); - } - } - - /** - * Notify all operators that some buckets were removed entirely. - * - * @param removedBuckets the buckets that were removed. - */ - private void onBucketsRemoved(@NotNull final RowSet removedBuckets) { - for (final UpdateByOperator operator : operators) { - operator.onBucketsRemoved(removedBuckets); - } - } - - /** - * Mark all columns as affected - */ - public void setAllAffected() { - Arrays.fill(opAffected, true); - } - - /** - * Check if any of the operators produced modifications beyond the set specified by the upstream udpate. - * - * @return true if additional updates were produced. - */ - boolean anyModified() { - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx] && operators[opIdx].anyModified(opContext[opIdx])) { - return true; - } - } - return false; - } - - /** - * Initialize the operators for the specified operation. This is always going to be add or reprocess. - * - * @param updateIndex the index that will be processed - * @param type the type of update to prepare for. - */ - void initializeFor(@NotNull final RowSet updateIndex, - @NotNull final UpdateType type) { - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - operators[opIdx].initializeFor(opContext[opIdx], updateIndex, type); - } - } - } - - /** - * Finish the operation specified. This will always be preceded by a single call to - * {@link #initializeFor(RowSet, UpdateType)} - * - * @param type the type - */ - void finishFor(@NotNull final UpdateType type) { - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - operators[opIdx].finishFor(opContext[opIdx], type); - } - } - } - - /** - * Prepare the specified chunk for use. - * - * @param inputSlot the slot containing the input source - * @param permuteRequired if a permutation of the copied values is required - * @param chunkOk the chunk of keys to read - * @param workingChunk the working chunk of values to fill - * @param chunkPositions the positions within the original chunk -- used when permuting values - * @param permutedWorkingChunk the permuted result chunk, if permutation was required. - * @param fillContext the {@link ChunkSource.FillContext} - */ - private void prepareValuesChunkFor(final int inputSlot, - final boolean permuteRequired, - final RowSequence chunkOk, - final WritableChunk workingChunk, - final IntChunk chunkPositions, - final WritableChunk permutedWorkingChunk, - final ChunkSource.FillContext fillContext) { - if (!inputChunkPopulated[inputSlot]) { - inputChunkPopulated[inputSlot] = true; - inputSources[inputSlot].fillChunk(fillContext, workingChunk, chunkOk); - if (permuteRequired) { - permuteKernels[inputSlot].permuteInput(workingChunk, - chunkPositions, - permutedWorkingChunk); - } - } - } - - /** - * Process the shifts and modifies of the upstream and separate rows that had key changes from those that did - * not. For rows that did, treat them as removed from one bucket and added to another. This method is taken from - * {@link ChunkedOperatorAggregationHelper} and modified to suit the needs of updateBy. - */ - private void processModifiesForChangedKeys(@NotNull final TableUpdate upstream) { - Require.requirement(keysModified, "keysModified"); - - final boolean shifted = upstream.shifted().nonempty(); - try (final RowSequence.Iterator modifiedPreShiftIterator = - upstream.getModifiedPreShift().getRowSequenceIterator(); - final RowSequence.Iterator modifiedPostShiftIterator = - shifted ? upstream.modified().getRowSequenceIterator() : null; - final WritableIntChunk postSlots = WritableIntChunk.makeWritableChunk(chunkSize)) { - - // NULL_ROW_KEY is lower than any other possible index, so we should not trip on the first pass - long lastCurrentIndex = NULL_ROW_KEY; - long lastPreviousIndex = NULL_ROW_KEY; - - final WritableIntChunk localOutputPositions = outputPositions.get(); - - // We will repurpose keyChunk and permutedKeyChunk here to avoid creating even more chunks - final WritableLongChunk preShiftIndicesChunk = keyChunk.get(); - final WritableLongChunk postShiftIndicesChunk = - WritableLongChunk.downcast(permutedKeyChunk.get()); - while (modifiedPreShiftIterator.hasMore()) { - final RowSequence modifiedPreShiftOk = - modifiedPreShiftIterator.getNextRowSequenceWithLength(chunkSize); - final RowSequence modifiedPostShiftOk = - shifted ? modifiedPostShiftIterator.getNextRowSequenceWithLength(chunkSize) - : modifiedPreShiftOk; - - hashTable.remove(getProbeContet(), modifiedPreShiftOk, keySources, localOutputPositions); - hashTable.add(false, bc, modifiedPostShiftOk, keySources, nextOutputPosition, postSlots); - - modifiedPreShiftOk.fillRowKeyChunk(preShiftIndicesChunk); - final LongChunk postShiftIndices; - if (shifted) { - modifiedPostShiftOk.fillRowKeyChunk(postShiftIndicesChunk); - postShiftIndices = postShiftIndicesChunk; - } else { - postShiftIndices = preShiftIndicesChunk; - } - - final int chunkSize = localOutputPositions.size(); - Assert.eq(chunkSize, "postSlots.size()", postSlots.size(), "postSlots.size()"); - Assert.eq(chunkSize, "chunkSize", preShiftIndicesChunk.size(), "preShiftIndices.size()"); - Assert.eq(chunkSize, "chunkSize", postShiftIndices.size(), "postShiftIndices.size()"); - for (int si = 0; si < chunkSize; ++si) { - final int previousSlot = localOutputPositions.get(si); - final int currentSlot = postSlots.get(si); - final long previousIndex = preShiftIndicesChunk.get(si); - final long currentIndex = postShiftIndices.get(si); - - Assert.gt(currentIndex, "currentIndex", lastCurrentIndex, "lastCurrentIndex"); - Assert.gt(previousIndex, "previousIndex", lastPreviousIndex, "lastPreviousIndex"); - - // TODO: Would it be better to do this in chunks? This method requires: - // 1) LCS get() - // 2) comparison - // 3) subtraction - // 4) LCS get() - // 5) randombuilder insert - // - // Using chunks would require - // 1) 2 more chunks allocated - // 2) Hashed run finder x2 (one for modified only, one for removed) - // 3) permute x2 - if (previousSlot == currentSlot) { - slotTracker.modifyBucket(currentSlot, currentIndex); - } else { - slotTracker.removeFromBucket(previousSlot, previousIndex); - slotTracker.addToBucket(currentSlot, currentIndex); - } - - lastCurrentIndex = currentIndex; - lastPreviousIndex = previousIndex; - } - } - } - } - - /** - * Process the upstream additions and accumulate them to the {@link UpdateBySlotTracker} - * - * @param additions the additions. - */ - private void accumulateAdditions(@NotNull final RowSequence additions) { - try (final RowSequence.Iterator okIt = additions.getRowSequenceIterator()) { - final WritableIntChunk localOutputPositions = outputPositions.get(); - final WritableIntChunk localRunStarts = runStarts.get(); - final WritableIntChunk localRunLengths = runLengths.get(); - final WritableLongChunk localPermutedKeyIndicesChunk = permutedKeyChunk.get(); - final WritableLongChunk localKeyIndicesChunk = keyChunk.get(); - while (okIt.hasMore()) { - final RowSequence chunkOk = okIt.getNextRowSequenceWithLength(chunkSize); - hashTable.add(false, bc, chunkOk, keySources, nextOutputPosition, localOutputPositions); - final boolean permuteRequired = findRunsAndPermute(chunkOk); - - for (int runIdx = 0; runIdx < localRunStarts.size(); runIdx++) { - final int runStart = localRunStarts.get(runIdx); - final int runLength = localRunLengths.get(runIdx); - final int bucketPosition = localOutputPositions.get(runStart); - - slotTracker.addToBucket(bucketPosition, - permuteRequired ? localPermutedKeyIndicesChunk : localKeyIndicesChunk, - runStart, - runLength); - } - } - } - } - - /** - * Process the upstream removals and accumulate them to the {@link UpdateBySlotTracker} - * - * @param removals the removals. - */ - private void accumulateRemovals(@NotNull final RowSequence removals) { - try (final RowSequence.Iterator okIt = removals.getRowSequenceIterator()) { - final WritableIntChunk localOutputPositions = outputPositions.get(); - final WritableIntChunk localRunStarts = runStarts.get(); - final WritableIntChunk localRunLengths = runLengths.get(); - final WritableLongChunk localPermutedKeyIndicesChunk = permutedKeyChunk.get(); - final WritableLongChunk localKeyIndicesChunk = keyChunk.get(); - while (okIt.hasMore()) { - final RowSequence chunkOk = okIt.getNextRowSequenceWithLength(chunkSize); - hashTable.remove(getProbeContet(), chunkOk, keySources, localOutputPositions); - final boolean permuteRequired = findRunsAndPermute(chunkOk); - - for (int runIdx = 0; runIdx < localRunStarts.size(); runIdx++) { - final int runStart = localRunStarts.get(runIdx); - final int runLength = localRunLengths.get(runIdx); - final int bucketPosition = localOutputPositions.get(runStart); - - slotTracker.removeFromBucket(bucketPosition, - permuteRequired ? localPermutedKeyIndicesChunk : localKeyIndicesChunk, - runStart, - runLength); - } - } - } - } - - /** - * Process the upstream modifications and accumulate them to the {@link UpdateBySlotTracker}. Note that this - * method can only be called if there are no changes to the key columns. Otherwise we must use - * {@link #processModifiesForChangedKeys(TableUpdate)}. - * - * @param modifications the the modifications. - */ - private void accumulateModifications(@NotNull final RowSequence modifications) { - try (final RowSequence.Iterator okIt = modifications.getRowSequenceIterator()) { - final WritableIntChunk localOutputPositions = outputPositions.get(); - final WritableIntChunk localRunStarts = runStarts.get(); - final WritableIntChunk localRunLengths = runLengths.get(); - final WritableLongChunk localPermutedKeyIndicesChunk = permutedKeyChunk.get(); - final WritableLongChunk localKeyIndicesChunk = keyChunk.get(); - while (okIt.hasMore()) { - final RowSequence chunkOk = okIt.getNextRowSequenceWithLength(chunkSize); - hashTable.findModifications(getProbeContet(), chunkOk, keySources, localOutputPositions); - final boolean permuteRequired = findRunsAndPermute(chunkOk); - - for (int runIdx = 0; runIdx < localRunStarts.size(); runIdx++) { - final int runStart = localRunStarts.get(runIdx); - final int runLength = localRunLengths.get(runIdx); - final int bucketPosition = localOutputPositions.get(runStart); - - slotTracker.modifyBucket(bucketPosition, - permuteRequired ? localPermutedKeyIndicesChunk : localKeyIndicesChunk, - runStart, - runLength); - } - } - } - } - - /** - * Proces the upstream shifts and determine which buckets are affected by the shifts. - * - * @param shifted the upstream shifts. - */ - public void accumulateShifts(@NotNull final RowSetShiftData shifted) { - final WritableLongChunk localKeyChunk = keyChunk.get(); - int nextChunkPosition = 0; - - // We don't want to apply shifts later on and be linear in buckets, so let's identify the affected shift - // buckets - // and only apply to those. - try (final RowSet.SearchIterator curIt = source.getRowSet().searchIterator()) { - for (int shiftIdx = 0; shiftIdx < shifted.size(); shiftIdx++) { - final long start = shifted.getBeginRange(shiftIdx); - final long end = shifted.getEndRange(shiftIdx); - final long delta = shifted.getShiftDelta(shiftIdx); - - // If we don't have any more valid keys, we're done. - if (!curIt.advance(start + delta)) { - break; - } - - // Accumulate the keys into a chunk until it is full, - // Then we'll hash the chunk to bucket it and accumulate those into the tracker. - long keyToTrack = curIt.currentValue(); - while (keyToTrack <= (end + delta)) { - localKeyChunk.set(nextChunkPosition++, keyToTrack); - if (nextChunkPosition == localKeyChunk.capacity()) { - localKeyChunk.setSize(nextChunkPosition); - locateAndTrackShiftedBuckets(); - nextChunkPosition = 0; - } - - if (!curIt.hasNext()) { - break; - } - keyToTrack = curIt.nextLong(); - } - } - - // Make sure we hash whatever is left over. - if (nextChunkPosition > 0) { - localKeyChunk.setSize(nextChunkPosition); - locateAndTrackShiftedBuckets(); - } - } - } - - /** - * Hash the output positions chunk to determine which buckets are affected by the shifts, then pass that - * information along to the slot tracker, - */ - private void locateAndTrackShiftedBuckets() { - final WritableIntChunk localOutputPositions = outputPositions.get(); - final WritableIntChunk localRunStarts = runStarts.get(); - hashTable.findModifications(getProbeContet(), - RowSequenceFactory.wrapRowKeysChunkAsRowSequence(keyChunk.get()), - keySources, - localOutputPositions); - - HashedRunFinder.findRunsHashed(findCtx.get(), - localRunStarts, - runLengths.get(), - chunkPositions.get(), - localOutputPositions); - - for (int runIdx = 0; runIdx < localRunStarts.size(); runIdx++) { - final int runStart = localRunStarts.get(runIdx); - final int bucketPosition = localOutputPositions.get(runStart); - slotTracker.markForShift(bucketPosition); - } - } - - /** - * Accumulate all the bucketed changes into an index to reprocess. If we can detect that a particular bucket was - * simply appended to, we can just accumulate the "added part of that bucket's TableUpdate, otherwise we - * determine the smallest affected key for each bucket and insert a sub-index of the bucket's total index - * beginning at that lowest key. This produces a reprocessing index that we can iterate which will only contain - * rows from buckets that were actually affected, yet still allows us to reprocess those rows sequentially. - * - * @param tracker the tracker object for this slot. - * @param slotIndex the index for this slot, after the specified updates have been - * {@link UpdateBySlotTracker#applyUpdates(RowSetShiftData) applied} - */ - private void accumulateIndexToReprocess(@NotNull final UpdateBySlotTracker.UpdateTracker tracker, - @NotNull final RowSet slotIndex) { - if (tracker.wasShiftOnly()) { - // If the bucket was simply shifted, there's nothing to do. None of the data changed, and the output - // sources - // will get adjusted when we call the shiftOutputSources() method later. - return; - } - - final RowSet indexToAccumulate = tracker.wasAppendOnly() ? tracker.getAdded() : slotIndex; - final long smallestModifiedKey = tracker.getSmallestModifiedKey(); - boolean addedRowsToReprocess = false; - - // This will never be less than, but it costs us nothing. - if (smallestModifiedKey <= slotIndex.firstRowKey()) { - addedRowsToReprocess = true; - accumulator.addRowSet(slotIndex); - } else if (smallestModifiedKey <= slotIndex.lastRowKey()) { - addedRowsToReprocess = true; - try (final RowSet indexToInsert = - indexToAccumulate.subSetByKeyRange(smallestModifiedKey, indexToAccumulate.lastRowKey())) { - accumulator.addRowSet(indexToInsert); - } - } - - // It's possible we marked the operator as untouched if it's column was not affected. Then, later on, - // we discovered that a key changed and a row switched buckets. That means one bucket had removed rows - // and the other had added rows, and now the operators -need- to be visited. - final boolean bucketHasAddedOrRemoved = - tracker.getAdded().isNonempty() || tracker.getRemoved().isNonempty(); - - // If the tracker wasn't append only, we'll issue a reset command to the operators for this bucket. - if (!tracker.wasAppendOnly()) { - final long keyBefore; - try (final RowSet.SearchIterator sit = slotIndex.searchIterator()) { - keyBefore = sit.binarySearchValue( - (compareTo, ignored) -> Long.compare(smallestModifiedKey - 1, compareTo), 1); - } - - // Notify the operators to reset their bucket states, and update their affectivity. - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - opAffected[opIdx] |= bucketHasAddedOrRemoved; - if (opAffected[opIdx]) { - operators[opIdx].resetForReprocessBucketed(opContext[opIdx], slotIndex, tracker.getSlot(), keyBefore); - } - } - } - - if (!addedRowsToReprocess) { - // There's nothing to do, the modified key is after our last valid index. - return; - } - - final RowSet.SearchIterator iterator = slotIndex.searchIterator(); - iterator.advance(smallestModifiedKey); - tracker.setBucketIterator(iterator); - - if (bucketQ == null) { - bucketQ = new TrackerPriorityQueue((int) slotTracker.getModifiedBucketCount()); - } - bucketQ.add(tracker); - } - - /** - * Iterate over the set of rows accumulated from the set of modified buckets by - * {@link #accumulateIndexToReprocess(UpdateBySlotTracker.UpdateTracker, RowSet)} This will use the - * {@link #bucketQ priority queue} to associate values with buckets, instead of the hash table and then push the - * update down to the operators. - */ - public void processBucketedUpdates() { - final RowSet modifiedBucketIndex = accumulator.build(); - - // This can happen if all we did was completely remove some buckets. - if (modifiedBucketIndex.isEmpty()) { - return; - } - - try (final RowSequence.Iterator okIt = modifiedBucketIndex.getRowSequenceIterator()) { - final int newChunkSize = (int) Math.min(control.chunkCapacityOrDefault(), modifiedBucketIndex.size()); - setChunkSize(newChunkSize); - initializeFor(modifiedBucketIndex, UpdateType.Reprocess); - - // TODO: This is a lot of iteration. Maybe a RangeIterator would be more better for the buckets? - UpdateBySlotTracker.UpdateTracker tracker = bucketQ.pop(); - if (tracker == null) { - Assert.statementNeverExecuted("Bucketing queue is empty, but there are still keys to process"); - } - RowSet.SearchIterator trackerIt = tracker.getIterator(); - long trackerCurKey = tracker.getIterator().currentValue(); - - final WritableLongChunk localKeyChunk = keyChunk.get(); - final WritableIntChunk localOutputPositions = outputPositions.get(); - final WritableIntChunk localRunStarts = runStarts.get(); - final WritableIntChunk localRunLengths = runLengths.get(); - final WritableIntChunk localChunkPositions = chunkPositions.get(); - final WritableLongChunk localPermutedKeyChunk = permutedKeyChunk.get(); - - while (okIt.hasMore()) { - sharedContext.reset(); - final RowSequence chunkOk = okIt.getNextRowSequenceWithLength(chunkSize); - chunkOk.fillRowKeyChunk(localKeyChunk); - - // TableUpdate the outputPositions chunk using the PriQ we built during the accumulate step - for (int chunkPosition = 0; chunkPosition < localKeyChunk.size(); chunkPosition++) { - final long keyToInspect = localKeyChunk.get(chunkPosition); - // If the key we're looking for is not part of this bucket, insert it back into - // the queue, so long as there is a next key. - if (keyToInspect != trackerCurKey) { - if (trackerCurKey != NULL_ROW_KEY) { - bucketQ.add(tracker); - } - - // Grab the next bucket from the queue and continue. - tracker = bucketQ.pop(); - if (tracker == null) { - Assert.statementNeverExecuted( - "Bucketing queue is empty, but there are still keys to process"); - } - trackerIt = tracker.getIterator(); - trackerCurKey = trackerIt.currentValue(); - Assert.eq(trackerCurKey, "trackerCurKey", keyToInspect, "keyToInspect"); - } - - // Mark this position with the current bucket's position. - localOutputPositions.set(chunkPosition, tracker.getSlot()); - // Advance the iterator, using NULL_ROW_KEY if there are no further keys. - trackerCurKey = trackerIt.hasNext() ? trackerIt.nextLong() : NULL_ROW_KEY; - } - localOutputPositions.setSize(chunkOk.intSize()); - - // Now, find the runs of buckets and permute them if necessary - final boolean permuteRequired = HashedRunFinder.findRunsHashed(findCtx.get(), - localRunStarts, - localRunLengths, - localChunkPositions, - localOutputPositions); - - // Now permute the keys to match the found ranges - if (permuteRequired) { - localPermutedKeyChunk.setSize(localKeyChunk.size()); - LongPermuteKernel.permuteInput(localKeyChunk, localChunkPositions, localPermutedKeyChunk); - } - - Arrays.fill(inputChunkPopulated, false); - // Finally, push the chunk down into the affected oeprators. - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - final int slotPosition = inputSourceSlots[opIdx]; - final WritableChunk localPostWorkingChunk = postWorkingChunks[slotPosition].get(); - final WritableChunk localPermutedWorkingChunk = - permutedPostWorkingChunks[slotPosition].get(); - - prepareValuesChunkFor( - slotPosition, - permuteRequired, - chunkOk, - localPostWorkingChunk, - localChunkPositions, - localPermutedWorkingChunk, - fillContexts[slotPosition].get()); - - operators[opIdx].reprocessChunkBucketed(opContext[opIdx], - chunkOk, - permuteRequired ? localPermutedWorkingChunk : localPostWorkingChunk, - permuteRequired ? localPermutedKeyChunk : localKeyChunk, - localOutputPositions, - localRunStarts, - localRunLengths); - } - } - } - } - - finishFor(UpdateType.Reprocess); - } - - /** - * Inspect the chunk positions and find sorted runs of individual buckets. If required, permute the input key - * chunk so that input keys remain adjacent to their values. - * - * @param chunkOk the chunk of input keys. - */ - private boolean findRunsAndPermute(@NotNull final RowSequence chunkOk) { - final boolean permuteRequired = HashedRunFinder.findRunsHashed(findCtx.get(), - runStarts.get(), - runLengths.get(), - chunkPositions.get(), - outputPositions.get()); - - chunkOk.fillRowKeyChunk(keyChunk.get()); - // Now permute the keys to match the found ranges - if (permuteRequired) { - permutedKeyChunk.get().setSize(chunkOk.intSize()); - LongPermuteKernel.permuteInput(keyChunk.get(), chunkPositions.get(), permutedKeyChunk.get()); - } - - return permuteRequired; - } - - /** - * Push the specified index as an append-only update to the operators. - * - * @param added the keys added. - */ - private void doAppendOnlyAdds(final boolean initialBuild, @NotNull final RowSet added) { - initializeFor(added, UpdateType.Add); - - try (final RowSequence.Iterator okIt = added.getRowSequenceIterator()) { - final WritableIntChunk localRunStarts = runStarts.get(); - final WritableIntChunk localRunLengths = runLengths.get(); - final WritableIntChunk localOutputPositions = outputPositions.get(); - final WritableLongChunk localPermutedKeys = permutedKeyChunk.get(); - final WritableLongChunk localKeys = keyChunk.get(); - final WritableIntChunk localChunkPositions = chunkPositions.get(); - - while (okIt.hasMore()) { - sharedContext.reset(); - final RowSequence chunkOk = okIt.getNextRowSequenceWithLength(chunkSize); - - // add the values to the hash table, and produce a chunk of the positions each added key mapped to - hashTable.add(initialBuild, bc, chunkOk, keySources, nextOutputPosition, localOutputPositions); - setBucketCapacity(nextOutputPosition.intValue()); - - // Now, organize that chunk by position so we can hand them off to the operators - final boolean permuteRequired = findRunsAndPermute(chunkOk); - - // If we need to track the slot indices, make sure we insert them into the tracker. - if (slotTracker != null) { - for (int runIdx = 0; runIdx < localRunStarts.size(); runIdx++) { - final int runStart = localRunStarts.get(runIdx); - final int runLength = localRunLengths.get(runIdx); - final int bucketPosition = localOutputPositions.get(runStart); - - slotTracker.addToBucket(bucketPosition, - permuteRequired ? localPermutedKeys : localKeys, - runStart, - runLength); - } - } - - Arrays.fill(inputChunkPopulated, false); - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - final int slotPosition = inputSourceSlots[opIdx]; - final WritableChunk localPostWorkingChunk = postWorkingChunks[slotPosition].get(); - final WritableChunk localPermutedPostWorkingChunk = - permutedPostWorkingChunks[slotPosition].get(); - prepareValuesChunkFor( - slotPosition, - permuteRequired, - chunkOk, - localPostWorkingChunk, - localChunkPositions, - localPermutedPostWorkingChunk, - fillContexts[slotPosition].get()); - - operators[opIdx].addChunkBucketed(opContext[opIdx], - permuteRequired ? localPermutedPostWorkingChunk : localPostWorkingChunk, - permuteRequired ? localPermutedKeys : localKeys, - localOutputPositions, - localRunStarts, - localRunLengths); - } - } - } - } - - finishFor(UpdateType.Add); - } - - private void applyShiftsToOutput(@NotNull final RowSetShiftData shifts) { - if (shifts.empty()) { - return; - } - - try (final RowSet prevIdx = source.getRowSet().copyPrev()) { - shifts.apply((begin, end, delta) -> { - try (final RowSet subIndex = prevIdx.subSetByKeyRange(begin, end)) { - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - operators[opIdx].applyOutputShift(opContext[opIdx], subIndex, delta); - } - } - }); - } - } - } - - /** - * An object to hold the transient state during a static grouped bucketed addition - */ - private class GroupedContext implements SafeCloseable { - /** An array of closeables to help with cleanup */ - final SafeCloseableList closeables = new SafeCloseableList(); - - /** The expected size of chunks to the various update stages */ - final int chunkSize; - - /** An indicator of if each slot has been populated with data or not for this phase. */ - boolean[] inputChunkPopulated; - - /** An array of context objects for each underlying operator */ - final UpdateByOperator.UpdateContext[] opContext; - - /** A {@link SharedContext} to be used while creating other contexts */ - final SharedContext sharedContext = SharedContext.makeSharedContext(); - - /** An array of {@link ChunkSource.FillContext}s for each input column */ - final ChunkSource.FillContext[] fillContexts; - - /** A set of chunks used to store post-shift working values */ - final WritableChunk[] postWorkingChunks; - - GroupedContext(final TableUpdate upstream) { - this.chunkSize = Math.min((int) source.size(), control.chunkCapacityOrDefault()); - this.inputChunkPopulated = new boolean[operators.length]; - this.fillContexts = new ChunkSource.FillContext[operators.length]; - this.opContext = new UpdateByOperator.UpdateContext[operators.length]; - - // noinspection unchecked - this.postWorkingChunks = new WritableChunk[operators.length]; - - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - opContext[opIdx] = closeables.add(operators[opIdx].makeUpdateContext(chunkSize)); - final int slotPosition = inputSourceSlots[opIdx]; - if (fillContexts[slotPosition] == null) { - fillContexts[slotPosition] = - closeables.add(inputSources[slotPosition].makeFillContext(chunkSize, sharedContext)); - postWorkingChunks[slotPosition] = - closeables.add(inputSources[slotPosition].getChunkType().makeWritableChunk(chunkSize)); - } - operators[opIdx].initializeForUpdate(opContext[opIdx], upstream, source.getRowSet(), false, true); - } - } - - void initialize(@NotNull final RowSet bucketIndex) { - sharedContext.reset(); - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - operators[opIdx].initializeFor(opContext[opIdx], bucketIndex, UpdateType.Add); - } - } - - void finish() { - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - operators[opIdx].finishFor(opContext[opIdx], UpdateType.Add); - } - } - - @Override - @OverridingMethodsMustInvokeSuper - public void close() { - closeables.close(); - sharedContext.close(); - } - - /** - * Prepare the specified chunk for use. - * - * @param inputSlot the slot holding the input source and contexts - * @param workingChunk the working chunk of values to fill - * @param fillContext the {@link ChunkSource.FillContext} - * @param chunkOk the {@link RowSequence} for current values - */ - private void prepareValuesChunkFor(final int inputSlot, - final RowSequence chunkOk, - final WritableChunk workingChunk, - final ChunkSource.FillContext fillContext) { - if (!inputChunkPopulated[inputSlot]) { - inputChunkPopulated[inputSlot] = true; - inputSources[inputSlot].fillChunk(fillContext, workingChunk, chunkOk); - } - } - } - - /** - * Perform a bucketed updateBy. - * - * @param control the {@link UpdateByControl} to set hashing parameters - * @param description the operation description - * @param source the source table - * @param ops the operations to perform - * @param resultSources the result sources - * @param rowRedirection the redirection index, if one was used. - * @param keySources the sources for key columns. - * @param byColumns the grouping column pairs - * @return the result table - */ - @SuppressWarnings("rawtypes") - public static Table compute(@NotNull final String description, - @NotNull final QueryTable source, - @NotNull final UpdateByOperator[] ops, - @NotNull final Map> resultSources, - @Nullable final WritableRowRedirection rowRedirection, - @NotNull final ColumnSource[] keySources, - @NotNull final ColumnSource[] originalKeySources, - @NotNull final MatchPair[] byColumns, - @NotNull final UpdateByControl control) { - final QueryTable result = new QueryTable(source.getRowSet(), resultSources); - final boolean useGrouping = JoinControl.useGrouping(source, keySources); - final BucketedUpdateBy updateBy = - new BucketedUpdateBy(ops, source, keySources, originalKeySources, useGrouping, rowRedirection, control); - updateBy.doInitialAdditions(useGrouping, byColumns); - - if (source.isRefreshing()) { - if (rowRedirection != null) { - rowRedirection.startTrackingPrevValues(); - } - Arrays.stream(ops).forEach(UpdateByOperator::startTrackingPrev); - final InstrumentedTableUpdateListener listener = updateBy.newListener(description, result, byColumns); - source.listenForUpdates(listener); - result.addParentReference(listener); - } - - return result; - } - - private BucketedUpdateBy(@NotNull final UpdateByOperator[] operators, - @NotNull final QueryTable source, - @NotNull final ColumnSource[] keySources, - @NotNull final ColumnSource[] originalKeySources, - final boolean useGrouping, - @Nullable final WritableRowRedirection rowRedirection, - @NotNull final UpdateByControl control) { - super(operators, source, rowRedirection, control); - this.keySources = keySources; - - final int hashTableSize = control.initialHashTableSizeOrDefault(); - -// if (source.isRefreshing() && !source.isAddOnly()) { - slotTracker = new UpdateBySlotTracker(control.chunkCapacityOrDefault()); -// } else { -// slotTracker = null; -// } - if (!useGrouping) { - this.hashTable = TypedHasherFactory.make(UpdateByStateManagerTypedBase.class, - keySources, keySources, - hashTableSize, control.maximumLoadFactorOrDefault(), - control.targetLoadFactorOrDefault()); - } - } - - /** - * Create an appropriate {@link InstrumentedTableUpdateListener listener} to process upstream updates. - * - * @param description the table description - * @param result the result table - * @param byColumns the grouping column pairs - * @return a listener to process updates. - */ - public InstrumentedTableUpdateListener newListener(@NotNull final String description, - @NotNull final QueryTable result, - @NotNull final MatchPair[] byColumns) { - return new BucketedUpdateByListener(description, source, result, byColumns); - } - - /** - * Insert the original source table as an add-only operation to initialize the result table. - * - * @param useGrouping if grouping should be used. - * @param byColumns the grouping columns - */ - private void doInitialAdditions(final boolean useGrouping, @NotNull final MatchPair[] byColumns) { - if (source.isEmpty()) { - return; - } - - final TableUpdateImpl initialUpdate = new TableUpdateImpl(source.getRowSet(), - RowSetFactory.empty(), - RowSetFactory.empty(), - RowSetShiftData.EMPTY, - ModifiedColumnSet.ALL); - if (useGrouping) { - try (final GroupedContext ctx = new GroupedContext(initialUpdate)) { - doStaticGroupedAddition(ctx, source.getRowSet()); - } - } else { - final ModifiedColumnSet keyModifiedColumnSet = - source.newModifiedColumnSet(MatchPair.getRightColumns(byColumns)); - try (final BucketedContext ctx = new BucketedContext(initialUpdate, keyModifiedColumnSet, null)) { - ctx.setAllAffected(); - ctx.initializeFor(source.getRowSet(), UpdateType.Add); - - if (rowRedirection != null && source.isRefreshing()) { - processUpdateForRedirection(initialUpdate); - } - - // do an addition phase for all the operators that can add directly (i.e. backwards looking) - ctx.doAppendOnlyAdds(true, source.getRowSet()); - - - // do a reprocessing phase for operators that can't add directly -// ctx.reprocessRows(RowSetShiftData.EMPTY); - - if (slotTracker != null) { - // noinspection resource - slotTracker.applyUpdates(RowSetShiftData.EMPTY); - slotTracker.reset(); - } - } - } - } - - /** - * Do a bucketed addition using the precomputed grouping data instead of reading the groupBy column. - * - * @param ctx the context - * @param added the index being added - */ - private void doStaticGroupedAddition(@NotNull final GroupedContext ctx, @NotNull final RowSet added) { - - final Map grouping = keySources[0].getGroupToRange(added); - - if (grouping == null) { - Assert.statementNeverExecuted("Trying to do grouped addition, but no groups exist"); - } - - MutableInt groupPosition = new MutableInt(0); - grouping.forEach((key, groupRowSet) -> { - if (groupRowSet == null) { - Assert.statementNeverExecuted("Found a null RowSet for group key " + key); - } - ctx.initialize(groupRowSet); - try (final RowSequence.Iterator okIt = groupRowSet.getRowSequenceIterator()) { - while (okIt.hasMore()) { - final RowSequence chunkOk = okIt.getNextRowSequenceWithLength(ctx.chunkSize); - final LongChunk keyChunk = chunkOk.asRowKeyChunk(); - Arrays.fill(ctx.inputChunkPopulated, false); - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - final int slotPosition = inputSourceSlots[opIdx]; - ctx.prepareValuesChunkFor( - slotPosition, - chunkOk, - ctx.postWorkingChunks[slotPosition], - ctx.fillContexts[slotPosition]); - - operators[opIdx].addChunk(ctx.opContext[opIdx], - chunkOk, - keyChunk, - ctx.postWorkingChunks[slotPosition], - groupPosition.getValue()); - } - } - } - ctx.finish(); - groupPosition.increment(); - - }); - } - - class BucketedUpdateByListener extends InstrumentedTableUpdateListenerAdapter { - private final QueryTable result; - - private final ModifiedColumnSet[] inputModifiedColumnSets; - private final ModifiedColumnSet[] outputModifiedColumnSets; - private final ModifiedColumnSet.Transformer transformer; - - private final ModifiedColumnSet keyModifiedColumnSet; - - BucketedUpdateByListener(@Nullable final String description, - @NotNull final QueryTable source, - @NotNull final QueryTable result, - @NotNull final MatchPair[] byColumns) { - super(description, source, false); - this.result = result; - - this.inputModifiedColumnSets = new ModifiedColumnSet[operators.length]; - this.outputModifiedColumnSets = new ModifiedColumnSet[operators.length]; - - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - final String[] outputColumnNames = operators[opIdx].getOutputColumnNames(); - inputModifiedColumnSets[opIdx] = - source.newModifiedColumnSet(operators[opIdx].getAffectingColumnNames()); - outputModifiedColumnSets[opIdx] = result.newModifiedColumnSet(outputColumnNames); - } - - this.keyModifiedColumnSet = source.newModifiedColumnSet(MatchPair.getRightColumns(byColumns)); - this.transformer = - source.newModifiedColumnSetTransformer(result, source.getDefinition().getColumnNamesArray()); - } - - @Override - public void onUpdate(@NotNull final TableUpdate upstream) { - try (final BucketedContext ctx = - new BucketedContext(upstream, keyModifiedColumnSet, inputModifiedColumnSets)) { - if (rowRedirection != null) { - processUpdateForRedirection(upstream); - } - - // If the update was, in itself, append-only, we can just push it through. - final boolean isAppendOnly = - UpdateByOperator.isAppendOnly(upstream, source.getRowSet().lastRowKeyPrev()); - if (isAppendOnly) { - ctx.doAppendOnlyAdds(false, upstream.added()); - } else { - accumulateUpdatesByBucket(upstream, ctx); - } - - if (slotTracker != null) { - processAccumulatedUpdates(upstream, ctx, isAppendOnly); - } - - computeDownstreamUpdate(upstream, ctx); - } - } - - /** - * Process the upstream update and divide it into affected buckets, so that we can be smart about how we - * reprocess rows from buckets. We want to ensure that we can re-read only the rows we need from the table and - * on top of that, ensure we can read them sequentially without revisiting blocks of data, - * - * @param upstream the upstream update - * @param ctx the update context - */ - private void accumulateUpdatesByBucket(@NotNull final TableUpdate upstream, - @NotNull final BucketedContext ctx) { - // Build a unique TableUpdate for each bucket affected by the - // modification and issue each of them separately. Hopefully, most buckets - // will shake out as append-only, while the ones that are not can be marked - // and revisited during the reprocessing phase. - if (upstream.removed().isNonempty()) { - ctx.accumulateRemovals(upstream.removed()); - } - - if (upstream.modified().isNonempty()) { - if (ctx.keysModified) { - ctx.processModifiesForChangedKeys(upstream); - } else if (ctx.anyAffected) { - ctx.accumulateModifications(upstream.modified()); - } - } - - if (upstream.shifted().nonempty()) { - ctx.accumulateShifts(upstream.shifted()); - } - - if (upstream.added().isNonempty()) { - ctx.accumulateAdditions(upstream.added()); - } - } - - /** - * Take the updates that have been broken down by bucket and apply them to each bucket sequentially. - * - * @param upstream the upstream update - * @param ctx the update context - * @param isAppendOnly if the update was append-only - */ - private void processAccumulatedUpdates(@NotNull final TableUpdate upstream, - @NotNull final BucketedContext ctx, - final boolean isAppendOnly) { - // Apply the updates to the tracked bucket indices and reset the tracker for the - // next cycle. - final RowSet emptiedBuckets = slotTracker.applyUpdates(upstream.shifted()); - if (emptiedBuckets.isNonempty()) { - ctx.onBucketsRemoved(emptiedBuckets); - } - - if (!isAppendOnly) { - // First, process the bucketed changes and create an index of rows to be processed. - // Explicitly, we are not going to let operators decide to handle remove/modify/shift on their own - // when we are processing general updates like this. Simplifying in this way lets us treat the update - // as a "reset" to a particular state and then adds. See the method doc for more detail - ctx.accumulator = RowSetFactory.builderRandom(); - slotTracker.forAllModifiedSlots(ctx::accumulateIndexToReprocess); - - // Note that we must do this AFTER the accumulateIndexToReprocess to ensure that when we reset - // state, we don't have to worry about shifts messing with our indices. We don't want to do this - // as a bucket because the shifts have to apply to the output column source, and we'd rather not - // iterate the shift O(nBuckets * nOps) times, to do it bucket by bucket. - if (rowRedirection == null) { - ctx.applyShiftsToOutput(upstream.shifted()); - } - - ctx.processBucketedUpdates(); - } - - slotTracker.reset(); - } - - /** - * Compute the downstream update from the result of processing each operator. - * - * @param upstream the upstream update. - * @param ctx the update context - */ - private void computeDownstreamUpdate(@NotNull final TableUpdate upstream, @NotNull final BucketedContext ctx) { - final TableUpdateImpl downstream = new TableUpdateImpl(); - // copy these rowSets since TableUpdateImpl#reset will close them with the upstream update - downstream.added = upstream.added().copy(); - downstream.removed = upstream.removed().copy(); - downstream.shifted = upstream.shifted(); - - if (upstream.modified().isNonempty() || ctx.anyModified()) { - downstream.modifiedColumnSet = result.getModifiedColumnSetForUpdates(); - downstream.modifiedColumnSet.clear(); - - WritableRowSet modifiedRowSet = RowSetFactory.empty(); - downstream.modified = modifiedRowSet; - - if (upstream.modified().isNonempty()) { - // Transform any untouched modified columns to the output. - transformer.clearAndTransform(upstream.modifiedColumnSet(), downstream.modifiedColumnSet); - modifiedRowSet.insert(upstream.modified()); - } - - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (ctx.opAffected[opIdx]) { - downstream.modifiedColumnSet.setAll(outputModifiedColumnSets[opIdx]); - if (operators[opIdx].anyModified(ctx.opContext[opIdx])) { - modifiedRowSet.insert(operators[opIdx].getAdditionalModifications(ctx.opContext[opIdx])); - } - } - } - - if (ctx.anyModified()) { - modifiedRowSet.remove(upstream.added()); - } - } else { - downstream.modified = RowSetFactory.empty(); - downstream.modifiedColumnSet = ModifiedColumnSet.EMPTY; - } - result.notifyListeners(downstream); - } - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java index 43645615a35..62cd0bac68a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java @@ -456,15 +456,14 @@ public Object[] getRecord(long rowNo, String... columnNames) { } @Override - public PartitionedTable partitionBy(final boolean dropKeys, final String... keyColumnNames) { + public PartitionedTable partitionBy(final boolean dropKeys, @NotNull final Collection columns) { if (isStream()) { throw streamUnsupported("partitionBy"); } - final List columns = ColumnName.from(keyColumnNames); return memoizeResult(MemoizedOperationKey.partitionBy(dropKeys, columns), () -> { final Table partitioned = aggBy(Partition.of(CONSTITUENT, !dropKeys), columns); final Set keyColumnNamesSet = - Arrays.stream(keyColumnNames).collect(Collectors.toCollection(LinkedHashSet::new)); + columns.stream().map(ColumnName::name).collect(Collectors.toCollection(LinkedHashSet::new)); final TableDefinition constituentDefinition; if (dropKeys) { constituentDefinition = TableDefinition.of(definition.getColumnStream() @@ -477,6 +476,28 @@ public PartitionedTable partitionBy(final boolean dropKeys, final String... keyC }); } +// @Override +// public PartitionedTable partitionBy(final boolean dropKeys, final String... keyColumnNames) { +// if (isStream()) { +// throw streamUnsupported("partitionBy"); +// } +// final List columns = ColumnName.from(keyColumnNames); +// return memoizeResult(MemoizedOperationKey.partitionBy(dropKeys, columns), () -> { +// final Table partitioned = aggBy(Partition.of(CONSTITUENT, !dropKeys), columns); +// final Set keyColumnNamesSet = +// Arrays.stream(keyColumnNames).collect(Collectors.toCollection(LinkedHashSet::new)); +// final TableDefinition constituentDefinition; +// if (dropKeys) { +// constituentDefinition = TableDefinition.of(definition.getColumnStream() +// .filter(cd -> !keyColumnNamesSet.contains(cd.getName())).toArray(ColumnDefinition[]::new)); +// } else { +// constituentDefinition = definition; +// } +// return new PartitionedTableImpl(partitioned, keyColumnNamesSet, true, CONSTITUENT.name(), +// constituentDefinition, isRefreshing(), false); +// }); +// } + @Override public Table rollup(Collection aggregations, boolean includeConstituents, ColumnName... groupByColumns) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/TableWithDefaults.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/TableWithDefaults.java index dc0bbe6784f..e00954c2491 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/TableWithDefaults.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/TableWithDefaults.java @@ -1057,12 +1057,19 @@ default Table ungroup(boolean nullFill) { // PartitionBy Operations // ----------------------------------------------------------------------------------------------------------------- + @Override + @ConcurrentMethod + default PartitionedTable partitionBy(boolean dropKeys, String... keyColumnNames) { + return partitionBy(dropKeys, ColumnName.from(keyColumnNames)); + } + @Override @ConcurrentMethod default PartitionedTable partitionBy(String... keyColumnNames) { return partitionBy(false, keyColumnNames); } + // ----------------------------------------------------------------------------------------------------------------- // Hierarchical table operations (rollup and treeTable). // ----------------------------------------------------------------------------------------------------------------- diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UncoalescedTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UncoalescedTable.java index 56663b5b2b8..1a05b4e492f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UncoalescedTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UncoalescedTable.java @@ -468,6 +468,12 @@ public PartitionedTable partitionBy(boolean dropKeys, String... keyColumnNames) return coalesce().partitionBy(dropKeys, keyColumnNames); } + @Override + @ConcurrentMethod + public PartitionedTable partitionBy(boolean dropKeys, @NotNull Collection keyColumnNames) { + return coalesce().partitionBy(dropKeys, keyColumnNames); + } + @Override @ConcurrentMethod public Table rollup(Collection aggregations, boolean includeConstituents, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java index 7c6fa0428f7..7091ee01227 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java @@ -85,12 +85,14 @@ public static Table updateBy(@NotNull final QueryTable source, @NotNull final Collection byColumns, @NotNull final UpdateByControl control) { - WritableRowRedirection rowRedirection = null; + final WritableRowRedirection rowRedirection; if (control.useRedirectionOrDefault()) { if (!source.isRefreshing()) { if (!source.isFlat() && SparseConstants.sparseStructureExceedsOverhead(source.getRowSet(), control.maxStaticSparseMemoryOverheadOrDefault())) { rowRedirection = new InverseRowRedirectionImpl(source.getRowSet()); + } else { + rowRedirection = null; } } else { final JoinControl.RedirectionType type = JoinControl.getRedirectionType(source, 4.0, true); @@ -106,6 +108,13 @@ public static Table updateBy(@NotNull final QueryTable source, throw new IllegalStateException("Unsupported redirection type " + type); } } + } else { + rowRedirection = null; + } + + // start tracking previous values + if (rowRedirection != null) { + rowRedirection.startTrackingPrevValues(); } // TODO(deephaven-core#2693): Improve UpdateBy implementation for ColumnName @@ -152,15 +161,11 @@ public static Table updateBy(@NotNull final QueryTable source, descriptionBuilder.append(", pairs={").append(MatchPair.matchString(pairs)).append("})"); - final List> originalKeySources = new ArrayList<>(pairs.length); - final List> keySources = new ArrayList<>(pairs.length); for (final MatchPair byColumn : pairs) { if (!source.hasColumns(byColumn.rightColumn)) { problems.add(byColumn.rightColumn); continue; } - originalKeySources.add(source.getColumnSource(byColumn.rightColumn)); - keySources.add(ReinterpretUtils.maybeConvertToPrimitive(source.getColumnSource(byColumn.rightColumn))); } if (!problems.isEmpty()) { @@ -168,15 +173,36 @@ public static Table updateBy(@NotNull final QueryTable source, String.join(", ", problems) + "}"); } - return BucketedUpdateBy.compute(descriptionBuilder.toString(), + return BucketedPartitionedUpdateBy.compute( + descriptionBuilder.toString(), source, opArr, resultSources, + byColumns, rowRedirection, - keySources.toArray(ColumnSource.ZERO_LENGTH_COLUMN_SOURCE_ARRAY), - originalKeySources.toArray(ColumnSource.ZERO_LENGTH_COLUMN_SOURCE_ARRAY), - pairs, control); + +// return BucketedUpdateBy.compute(descriptionBuilder.toString(), +// source, +// opArr, +// resultSources, +// rowRedirection, +// keySources.toArray(ColumnSource.ZERO_LENGTH_COLUMN_SOURCE_ARRAY), +// originalKeySources.toArray(ColumnSource.ZERO_LENGTH_COLUMN_SOURCE_ARRAY), +// pairs, +// control); + +// +// +// return BucketedUpdateBy.compute(descriptionBuilder.toString(), +// source, +// opArr, +// resultSources, +// rowRedirection, +// keySources.toArray(ColumnSource.ZERO_LENGTH_COLUMN_SOURCE_ARRAY), +// originalKeySources.toArray(ColumnSource.ZERO_LENGTH_COLUMN_SOURCE_ARRAY), +// pairs, +// control); } protected void processUpdateForRedirection(@NotNull final TableUpdate upstream) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java index 0063f85cc69..27ba08f9ae8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java @@ -51,7 +51,7 @@ *

  • Adds *
      *
    • {@link #initializeFor(UpdateContext, RowSet, UpdateBy.UpdateType)}
    • - *
    • {@link #addChunkBucketed(UpdateContext, RowSequence, LongChunk, Chunk, long)}
    • + *
    • {@link #addChunk(UpdateContext, RowSequence, LongChunk, Chunk, long)}
    • *
    • {@link #finishFor(UpdateContext, UpdateBy.UpdateType)}
    • *
    *
  • @@ -159,7 +159,7 @@ static long smallestAffectedKey(@NotNull final RowSet added, } } - return smallestModifiedKey == Long.MAX_VALUE ? RowSet.NULL_ROW_KEY : smallestModifiedKey; + return smallestModifiedKey; } /** @@ -182,13 +182,6 @@ RowSet determineAffectedRows(@NotNull final TableUpdate upstream, @NotNull final RowSet getAffectedRows(); } - /** - * Notify the operator of the current maximum bucket. - * - * @param capacity the capacity - */ - void setBucketCapacity(final int capacity); - /** * Get the name of the input column this operator depends on. * @@ -241,13 +234,13 @@ RowSet determineAffectedRows(@NotNull final TableUpdate upstream, @NotNull final * @param context the context object * @param upstream the upstream update to process * @param resultSourceIndex the result index of the source table - * @param usingBuckets if the update is bucketed or not + * @param lastPrevKey the last key for this table before this update * @param isUpstreamAppendOnly if the upstream update was detected to be append-only. */ void initializeForUpdate(@NotNull final UpdateContext context, @NotNull final TableUpdate upstream, @NotNull final RowSet resultSourceIndex, - final boolean usingBuckets, + final long lastPrevKey, final boolean isUpstreamAppendOnly); /** @@ -313,7 +306,7 @@ void initializeFor(@NotNull final UpdateContext context, /** * Query if this operator can process the update normally, or if it can only reprocess. This method is guaranteed to - * be invoked after {@link #initializeForUpdate(UpdateContext, TableUpdate, RowSet, boolean, boolean)} so the + * be invoked after {@link #initializeForUpdate(UpdateContext, TableUpdate, RowSet, long, boolean)} so the * operator is aware of the upstream {@link TableUpdate}. * * @param context the context @@ -330,14 +323,6 @@ void initializeFor(@NotNull final UpdateContext context, */ void setChunkSize(@NotNull final UpdateContext context, final int chunkSize); - - /** - * Called when some buckets have been completely emptied. Operators can use this to reset internal states. - * - * @param removedBuckets an index of removed bucket positions. - */ - void onBucketsRemoved(@NotNull final RowSet removedBuckets); - /** * Add a chunk of non-bucketed items to the operation. * @@ -345,30 +330,11 @@ void initializeFor(@NotNull final UpdateContext context, * @param keyChunk a chunk of keys for the rows being added. If the operator returns {@code false} for * {@link #requiresKeys()} this will be null. * @param values the chunk of values for the rows being added - * @param bucketPosition the group position */ void addChunk(@NotNull final UpdateContext context, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, - @NotNull final Chunk values, - final long bucketPosition); - - /** - * Add a chunk of bucketed items to the operation. - * - * @param context the context object - * @param values the value chunk - * @param keyChunk a chunk of keys for the rows being added - * @param bucketPositions a chunk of hash bucket positions for each key - * @param runLengths the runLengths of each run of bucket values - * @param startPositions the start position of a run within the chunk - */ - void addChunkBucketed(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths); + @NotNull final Chunk values); /** * Modify a chunk of values with the operation. @@ -378,14 +344,12 @@ void addChunkBucketed(@NotNull final UpdateContext context, * @param keyChunk a chunk of post-shift space keys for the update. * @param prevValuesChunk a chunk of previous values for the update * @param postValuesChunk a chunk of current values for the update - * @param bucketPosition the position of the current group being processed */ void modifyChunk(@NotNull final UpdateContext context, @Nullable final LongChunk prevKeyChunk, @Nullable final LongChunk keyChunk, @NotNull final Chunk prevValuesChunk, - @NotNull final Chunk postValuesChunk, - final long bucketPosition); + @NotNull final Chunk postValuesChunk); /** * Remove a chunk of values from the operation. @@ -393,12 +357,10 @@ void modifyChunk(@NotNull final UpdateContext context, * @param context the context object * @param keyChunk a chunk of keys being removed. * @param prevValuesChunk the chunk of values being removed - * @param bucketPosition the position of the current group being processed */ void removeChunk(@NotNull final UpdateContext context, @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk, - final long bucketPosition); + @NotNull final Chunk prevValuesChunk); /** * Apply a shift to the operation. @@ -435,26 +397,6 @@ void reprocessChunk(@NotNull final UpdateContext context, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceIndex); - /** - * Reprocess a chunk of data for a bucketed table. - * - * @param context the context object - * @param inputKeys the keys contained in the chunk - * @param values the current chunk of working values. - * @param keyChunk a {@link LongChunk} containing the keys. - * @param bucketPositions a {@link IntChunk} containing the bucket position of each key. Parallel to `keyChunk` and - * `values - * @param runStartPositions the starting positions of each run within the key and value chunk - * @param runLengths the run runLengths of each run in the key and value chunks. Parallel to `runStartPositions` - */ - void reprocessChunkBucketed(@NotNull final UpdateContext context, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk runStartPositions, - @NotNull final IntChunk runLengths); - /** * Reset the operator to the state at the `firstModifiedKey` for non-bucketed operation. This is invoked immediately * prior to calls to {@link #resetForReprocess(UpdateContext, RowSet, long)}.
    @@ -468,18 +410,4 @@ void reprocessChunkBucketed(@NotNull final UpdateContext context, void resetForReprocess(@NotNull final UpdateContext context, @NotNull final RowSet sourceIndex, final long firstUnmodifiedKey); - - /** - * Reset the operator to the state at the `firstModifiedKey` for the specified bucket. This is invoked immediately - * prior to calls to - * {@link #reprocessChunkBucketed(UpdateContext, RowSequence, Chunk, LongChunk, IntChunk, IntChunk, IntChunk)}. - * - * @param context the context object - * @param bucketIndex the current index of the specified bucket - * @param firstUnmodifiedKey the first unmodified key in the bucket after which we will reprocess rows. - */ - void resetForReprocessBucketed(@NotNull final UpdateContext context, - @NotNull final RowSet bucketIndex, - final long bucketPosition, - final long firstUnmodifiedKey); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java index 1357dd74447..85fe7800126 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java @@ -213,12 +213,4 @@ public String[] getOutputColumnNames() { public boolean requiresKeys() { return true; } - - @Override - public void setBucketCapacity(int capacity) { - } - - @Override - public void onBucketsRemoved(@NotNull final RowSet removedBuckets) { - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java index 2984cec831c..a016e28166f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java @@ -18,6 +18,8 @@ import java.util.Arrays; import java.util.Map; +import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; + /** * An implementation of {@link UpdateBy} dedicated to zero key computation. */ @@ -45,9 +47,6 @@ public static Table compute(@NotNull final String description, updateBy.doInitialAdditions(); if (source.isRefreshing()) { - if (rowRedirection != null) { - rowRedirection.startTrackingPrevValues(); - } Arrays.stream(ops).forEach(UpdateByOperator::startTrackingPrev); final ZeroKeyUpdateByListener listener = updateBy.newListener(description, result); source.listenForUpdates(listener); @@ -196,8 +195,8 @@ private class UpdateContext implements SafeCloseable { } } - operators[opIdx].initializeForUpdate(opContext[opIdx], upstream, source.getRowSet(), false, - upstreamAppendOnly); + operators[opIdx].initializeForUpdate(opContext[opIdx], upstream, source.getRowSet(), + isInitializeStep ? NULL_ROW_KEY : source.getRowSet().lastRowKeyPrev(), upstreamAppendOnly); } // retrieve the affected rows from all operator update contexts @@ -379,13 +378,13 @@ void doUpdate(@NotNull final RowSet updateRowSet, null, postWorkingChunks[slotPosition].get(), null, fillContexts[slotPosition].get()); currentOp.addChunk(opContext[opIdx], chunkOk, keyChunk.get(), - postWorkingChunks[slotPosition].get(), 0); + postWorkingChunks[slotPosition].get()); } else if (type == UpdateType.Remove) { prepareValuesChunkFor(opIdx, slotPosition, true, false, chunkOk, prevChunkOk, postWorkingChunks[slotPosition].get(), null, fillContexts[slotPosition].get(), null); currentOp.removeChunk(opContext[opIdx], keyChunk.get(), - postWorkingChunks[slotPosition].get(), 0); + postWorkingChunks[slotPosition].get()); } else if (type == UpdateType.Modify) { prepareValuesChunkFor(opIdx, slotPosition, true, true, chunkOk, prevChunkOk, prevWorkingChunks[slotPosition], postWorkingChunks[slotPosition].get(), @@ -394,8 +393,7 @@ void doUpdate(@NotNull final RowSet updateRowSet, prevKeyChunk == null ? keyChunk.get() : prevKeyChunk, keyChunk.get(), prevWorkingChunks[slotPosition], - postWorkingChunks[slotPosition].get(), - 0); + postWorkingChunks[slotPosition].get()); } else if (type == UpdateType.Reprocess) { // is this chunk relevant to this operator? If so, then intersect and process only the // relevant rows @@ -499,19 +497,12 @@ private void prepareValuesChunkFor(final int opIdx, } } - private void onBucketsRemoved(@NotNull final RowSet removedBuckets) { - for (final UpdateByOperator operator : operators) { - operator.onBucketsRemoved(removedBuckets); - } - } - public boolean canAnyProcessNormally() { for (int opIdx = 0; opIdx < operators.length; opIdx++) { if (opAffected[opIdx] && operators[opIdx].canProcessNormalUpdate(opContext[opIdx])) { return true; } } - return false; } } @@ -572,10 +563,6 @@ public void onUpdate(TableUpdate upstream) { ctx.doUpdate(upstream.added(), upstream.added(), UpdateType.Add); } - if (source.getRowSet().isEmpty()) { - ctx.onBucketsRemoved(RowSetFactory.fromKeys(0)); - } - // Now do the reprocessing phase. ctx.reprocessRows(upstream.shifted()); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java index 273d71da9d8..12d7ce4753d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java @@ -33,9 +33,6 @@ public abstract class BasePrimitiveEMAOperator extends BaseDoubleUpdateByOperato protected final OperationControl control; protected final LongRecordingUpdateByOperator timeRecorder; protected final double timeScaleUnits; - private LongArraySource bucketLastTimestamp; - - private long singletonLastStamp = NULL_LONG; class EmaContext extends Context { double alpha; @@ -73,12 +70,6 @@ public BasePrimitiveEMAOperator(@NotNull final MatchPair pair, this.timeScaleUnits = timeScaleUnits; } - @Override - public void setBucketCapacity(final int capacity) { - super.setBucketCapacity(capacity); - bucketLastTimestamp.ensureCapacity(capacity); - } - @NotNull @Override public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize) { @@ -89,16 +80,9 @@ public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize) { public void initializeForUpdate(@NotNull final UpdateByOperator.UpdateContext context, @NotNull final TableUpdate upstream, @NotNull final RowSet resultSourceIndex, - final boolean usingBuckets, + final long lastPrevKey , final boolean isAppendOnly) { final EmaContext ctx = (EmaContext) context; - if (!initialized) { - initialized = true; - if (usingBuckets) { - this.bucketLastVal = new DoubleArraySource(); - this.bucketLastTimestamp = new LongArraySource(); - } - } // If we're redirected we have to make sure we tell the output source it's actual size, or we're going // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not @@ -107,10 +91,8 @@ public void initializeForUpdate(@NotNull final UpdateByOperator.UpdateContext co outputSource.ensureCapacity(resultSourceIndex.size() + 1); } - if (!usingBuckets) { - // If we aren't bucketing, we'll just remember the appendyness. - ctx.canProcessDirectly = isAppendOnly; - } + // If we aren't bucketing, we'll just remember the appendyness. + ctx.canProcessDirectly = isAppendOnly; } @Override @@ -121,74 +103,19 @@ public void initializeFor(@NotNull final UpdateByOperator.UpdateContext updateCo ((EmaContext) updateContext).lastStamp = NULL_LONG; } - @Override - public void onBucketsRemoved(@NotNull final RowSet removedBuckets) { - if (bucketLastVal != null) { - bucketLastVal.setNull(removedBuckets); - bucketLastTimestamp.setNull(removedBuckets); - } else { - singletonVal = QueryConstants.NULL_DOUBLE; - singletonLastStamp = NULL_LONG; - } - } - @Override protected void doAddChunk(@NotNull final Context context, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - long groupPosition) { + @NotNull final Chunk workingChunk) { final EmaContext ctx = (EmaContext) context; - if (groupPosition == singletonGroup) { - ctx.lastStamp = singletonLastStamp; - ctx.curVal = singletonVal; - } else { - ctx.lastStamp = NULL_LONG; - ctx.curVal = NULL_DOUBLE; - } - if (timeRecorder == null) { computeWithTicks(ctx, workingChunk, 0, inputKeys.intSize()); } else { computeWithTime(ctx, workingChunk, 0, inputKeys.intSize()); } - - singletonVal = ctx.curVal; - singletonLastStamp = ctx.lastStamp; - singletonGroup = groupPosition; outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } - @Override - public void addChunkBucketed(@NotNull final UpdateByOperator.UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { - final EmaContext ctx = (EmaContext) context; - for (int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int runEnd = runStart + runLength; - final int bucketPosition = bucketPositions.get(runStart); - - ctx.lastStamp = bucketLastTimestamp.getLong(bucketPosition); - ctx.curVal = bucketLastVal.getDouble(bucketPosition); - if (timeRecorder == null) { - computeWithTicks(ctx, values, runStart, runEnd); - } else { - computeWithTime(ctx, values, runStart, runEnd); - } - - bucketLastVal.set(bucketPosition, ctx.curVal); - bucketLastTimestamp.set(bucketPosition, ctx.lastStamp); - } - - // noinspection unchecked - outputSource.fillFromChunkUnordered(ctx.fillContext.get(), ctx.outputValues.get(), - (LongChunk) keyChunk); - } - @Override public void resetForReprocess(@NotNull final UpdateByOperator.UpdateContext context, @NotNull final RowSet sourceIndex, @@ -203,8 +130,8 @@ public void resetForReprocess(@NotNull final UpdateByOperator.UpdateContext cont if (!ctx.canProcessDirectly) { // If we set the last state to null, then we know it was a reset state and the timestamp must also // have been reset. - if (singletonVal == NULL_DOUBLE || (firstUnmodifiedKey == NULL_ROW_KEY)) { - singletonLastStamp = NULL_LONG; + if (ctx.curVal == NULL_DOUBLE || (firstUnmodifiedKey == NULL_ROW_KEY)) { + ctx.lastStamp = NULL_LONG; } else { // If it hasn't been reset to null, then it's possible that the value at that position was null, in // which case @@ -214,41 +141,11 @@ public void resetForReprocess(@NotNull final UpdateByOperator.UpdateContext cont // Note that it's OK that we are not setting the singletonVal here, because if we had to go back more // rows, then whatever the correct value was, was already set at the initial location. - singletonLastStamp = locateFirstValidPreviousTimestamp(sourceIndex, firstUnmodifiedKey); + ctx.lastStamp = locateFirstValidPreviousTimestamp(sourceIndex, firstUnmodifiedKey); } } } - @Override - public void resetForReprocessBucketed(@NotNull final UpdateByOperator.UpdateContext ctx, - @NotNull final RowSet bucketIndex, - final long bucketPosition, - final long firstUnmodifiedKey) { - final double previousVal = firstUnmodifiedKey == NULL_ROW_KEY ? QueryConstants.NULL_DOUBLE - : outputSource.getDouble(firstUnmodifiedKey); - bucketLastVal.set(bucketPosition, previousVal); - - if (timeRecorder == null) { - return; - } - - long potentialResetTimestamp; - if (previousVal == NULL_DOUBLE) { - potentialResetTimestamp = NULL_LONG; - } else { - // If it hasn't been reset to null, then it's possible that the value at that position was null, in which - // case - // we must have ignored it, and so we have to actually keep looking backwards until we find something - // not null. - - - // Note that it's OK that we are not setting the singletonVal here, because if we had to go back more - // rows, then whatever the correct value was, was already set at the initial location. - potentialResetTimestamp = locateFirstValidPreviousTimestamp(bucketIndex, firstUnmodifiedKey); - } - bucketLastTimestamp.set(bucketPosition, potentialResetTimestamp); - } - private long locateFirstValidPreviousTimestamp(@NotNull final RowSet indexToSearch, final long firstUnmodifiedKey) { long potentialResetTimestamp = timeRecorder.getCurrentLong(firstUnmodifiedKey); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java index 747934a1d75..44e6e9d1908 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java @@ -35,11 +35,6 @@ public abstract class BigNumberEMAOperator extends BaseObjectUpdateByOperator protected final LongRecordingUpdateByOperator timeRecorder; protected final double timeScaleUnits; - private LongArraySource bucketLastTimestamp; - - private long singletonLastStamp = NULL_LONG; - private long singletonGroup = NULL_LONG; - class EmaContext extends Context { BigDecimal alpha = BigDecimal.valueOf(Math.exp(-1 / timeScaleUnits)); BigDecimal oneMinusAlpha = @@ -82,12 +77,6 @@ public BigNumberEMAOperator(@NotNull final MatchPair pair, // endregion constructor } - @Override - public void setBucketCapacity(final int capacity) { - super.setBucketCapacity(capacity); - bucketLastTimestamp.ensureCapacity(capacity); - } - @NotNull @Override public UpdateContext makeUpdateContext(final int chunkSize) { @@ -98,17 +87,10 @@ public UpdateContext makeUpdateContext(final int chunkSize) { public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceIndex, - boolean usingBuckets, + final long lastPrevKey, boolean isAppendOnly) { // noinspection unchecked final EmaContext ctx = (EmaContext) context; - if (!initialized) { - initialized = true; - if (usingBuckets) { - this.bucketLastVal = new ObjectArraySource<>(BigDecimal.class); - this.bucketLastTimestamp = new LongArraySource(); - } - } // If we're redirected we have to make sure we tell the output source it's actual size, or we're going // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not @@ -117,10 +99,8 @@ public void initializeForUpdate(@NotNull UpdateContext context, outputSource.ensureCapacity(resultSourceIndex.size() + 1); } - if (!usingBuckets) { - // If we aren't bucketing, we'll just remember the appendyness. - ctx.canProcessDirectly = isAppendOnly; - } + // If we aren't bucketing, we'll just remember the appendyness. + ctx.canProcessDirectly = isAppendOnly; } @SuppressWarnings("unchecked") @@ -135,62 +115,18 @@ public void initializeFor(@NotNull final UpdateContext updateContext, @Override protected void doAddChunk(@NotNull final Context updateContext, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - long groupPosition) { + @NotNull final Chunk workingChunk) { final ObjectChunk asObjects = workingChunk.asObjectChunk(); final EmaContext ctx = (EmaContext) updateContext; - if (groupPosition == singletonGroup) { - ctx.lastStamp = singletonLastStamp; - ctx.curVal = singletonVal; - } else { - ctx.lastStamp = NULL_LONG; - ctx.curVal = null; - } - if (timeRecorder == null) { computeWithTicks(ctx, asObjects, 0, inputKeys.intSize()); } else { computeWithTime(ctx, asObjects, 0, inputKeys.intSize()); } - - singletonVal = ctx.curVal; - singletonLastStamp = ctx.lastStamp; - singletonGroup = groupPosition; outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } - @Override - public void addChunkBucketed(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { - final ObjectChunk asObjects = values.asObjectChunk(); - // noinspection unchecked - final EmaContext ctx = (EmaContext) context; - for (int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int runEnd = runStart + runLength; - final int bucketPosition = bucketPositions.get(runStart); - - ctx.lastStamp = bucketLastTimestamp.getLong(bucketPosition); - ctx.curVal = bucketLastVal.get(bucketPosition); - if (timeRecorder == null) { - computeWithTicks(ctx, asObjects, runStart, runEnd); - } else { - computeWithTime(ctx, asObjects, runStart, runEnd); - } - - bucketLastVal.set(bucketPosition, ctx.curVal); - bucketLastTimestamp.set(bucketPosition, ctx.lastStamp); - } - // noinspection unchecked - outputSource.fillFromChunkUnordered(ctx.fillContext.get(), ctx.outputValues.get(), - (LongChunk) keyChunk); - } @Override public void resetForReprocess(@NotNull final UpdateContext context, @@ -204,51 +140,22 @@ public void resetForReprocess(@NotNull final UpdateContext context, final EmaContext ctx = (EmaContext) context; if (!ctx.canProcessDirectly) { - // If we set the last state to null, then we know it was a reset state and the timestamp must also - // have been reset. - if (singletonVal == null || (firstUnmodifiedKey == NULL_ROW_KEY)) { - singletonLastStamp = NULL_LONG; - } else { - // If it hasn't been reset to null, then it's possible that the value at that position was null, in - // which case - // we must have ignored it, and so we have to actually keep looking backwards until we find something - // not null. - - - // Note that it's OK that we are not setting the singletonVal here, because if we had to go back more - // rows, then whatever the correct value was, was already set at the initial location. - singletonLastStamp = locateFirstValidPreviousTimestamp(sourceIndex, firstUnmodifiedKey); - } - } - } - - @Override - public void resetForReprocessBucketed(@NotNull final UpdateContext ctx, - @NotNull final RowSet bucketIndex, - final long bucketPosition, - final long firstUnmodifiedKey) { - final BigDecimal previousVal = firstUnmodifiedKey == NULL_ROW_KEY ? null : outputSource.get(firstUnmodifiedKey); - bucketLastVal.set(bucketPosition, previousVal); - - if (timeRecorder == null) { - return; - } - - long potentialResetTimestamp; - if (previousVal == null) { - potentialResetTimestamp = NULL_LONG; - } else { - // If it hasn't been reset to null, then it's possible that the value at that position was null, in which - // case - // we must have ignored it, and so we have to actually keep looking backwards until we find something - // not null. - - - // Note that it's OK that we are not setting the singletonVal here, because if we had to go back more - // rows, then whatever the correct value was, was already set at the initial location. - potentialResetTimestamp = locateFirstValidPreviousTimestamp(bucketIndex, firstUnmodifiedKey); +// // If we set the last state to null, then we know it was a reset state and the timestamp must also +// // have been reset. +// if (singletonVal == null || (firstUnmodifiedKey == NULL_ROW_KEY)) { +// singletonLastStamp = NULL_LONG; +// } else { +// // If it hasn't been reset to null, then it's possible that the value at that position was null, in +// // which case +// // we must have ignored it, and so we have to actually keep looking backwards until we find something +// // not null. +// +// +// // Note that it's OK that we are not setting the singletonVal here, because if we had to go back more +// // rows, then whatever the correct value was, was already set at the initial location. +// singletonLastStamp = locateFirstValidPreviousTimestamp(sourceIndex, firstUnmodifiedKey); +// } } - bucketLastTimestamp.set(bucketPosition, potentialResetTimestamp); } private long locateFirstValidPreviousTimestamp(@NotNull final RowSet indexToSearch, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java index b6525a5c469..c52f5623e18 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java @@ -62,37 +62,11 @@ public Map> getOutputColumns() { } // endregion extra-methods - @Override - public void addChunkBucketed(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { - final ByteChunk asBooleans = values.asByteChunk(); - final Context ctx = (Context) context; - for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int bucketPosition = bucketPositions.get(runStart); - - ctx.curVal = bucketLastVal.getByte(bucketPosition); - accumulate(asBooleans, ctx, runStart, runLength); - bucketLastVal.set(bucketPosition, ctx.curVal); - } - //noinspection unchecked - outputSource.fillFromChunkUnordered(ctx.fillContext.get(), ctx.outputValues.get(), (LongChunk) keyChunk); - } - @Override protected void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - final long bucketPosition) { - ctx.curVal = singletonGroup == bucketPosition ? singletonVal : NULL_BOOLEAN_AS_BYTE; + @NotNull final Chunk workingChunk) { accumulate(workingChunk.asByteChunk(), ctx, 0, workingChunk.size()); - singletonGroup = bucketPosition; - singletonVal = ctx.curVal; outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java index ccf2dfbc3f7..57af903fb06 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java @@ -36,37 +36,11 @@ public ByteFillByOperator(@NotNull final MatchPair fillPair, // region extra-methods // endregion extra-methods - @Override - public void addChunkBucketed(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { - final ByteChunk asBytes = values.asByteChunk(); - final Context ctx = (Context) context; - for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int bucketPosition = bucketPositions.get(runStart); - - ctx.curVal = bucketLastVal.getByte(bucketPosition); - accumulate(asBytes, ctx, runStart, runLength); - bucketLastVal.set(bucketPosition, ctx.curVal); - } - //noinspection unchecked - outputSource.fillFromChunkUnordered(ctx.fillContext.get(), ctx.outputValues.get(), (LongChunk) keyChunk); - } - @Override protected void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - final long bucketPosition) { - ctx.curVal = singletonGroup == bucketPosition ? singletonVal : NULL_BYTE; + @NotNull final Chunk workingChunk) { accumulate(workingChunk.asByteChunk(), ctx, 0, workingChunk.size()); - singletonGroup = bucketPosition; - singletonVal = ctx.curVal; outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java index 8e78172c311..857aa4de476 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java @@ -31,37 +31,11 @@ public CharFillByOperator(@NotNull final MatchPair fillPair, // region extra-methods // endregion extra-methods - @Override - public void addChunkBucketed(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { - final CharChunk asChars = values.asCharChunk(); - final Context ctx = (Context) context; - for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int bucketPosition = bucketPositions.get(runStart); - - ctx.curVal = bucketLastVal.getChar(bucketPosition); - accumulate(asChars, ctx, runStart, runLength); - bucketLastVal.set(bucketPosition, ctx.curVal); - } - //noinspection unchecked - outputSource.fillFromChunkUnordered(ctx.fillContext.get(), ctx.outputValues.get(), (LongChunk) keyChunk); - } - @Override protected void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - final long bucketPosition) { - ctx.curVal = singletonGroup == bucketPosition ? singletonVal : NULL_CHAR; + @NotNull final Chunk workingChunk) { accumulate(workingChunk.asCharChunk(), ctx, 0, workingChunk.size()); - singletonGroup = bucketPosition; - singletonVal = ctx.curVal; outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java index 9664c60eab9..f2060dd5600 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java @@ -36,37 +36,11 @@ public DoubleFillByOperator(@NotNull final MatchPair fillPair, // region extra-methods // endregion extra-methods - @Override - public void addChunkBucketed(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { - final DoubleChunk asDoubles = values.asDoubleChunk(); - final Context ctx = (Context) context; - for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int bucketPosition = bucketPositions.get(runStart); - - ctx.curVal = bucketLastVal.getDouble(bucketPosition); - accumulate(asDoubles, ctx, runStart, runLength); - bucketLastVal.set(bucketPosition, ctx.curVal); - } - //noinspection unchecked - outputSource.fillFromChunkUnordered(ctx.fillContext.get(), ctx.outputValues.get(), (LongChunk) keyChunk); - } - @Override protected void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - final long bucketPosition) { - ctx.curVal = singletonGroup == bucketPosition ? singletonVal : NULL_DOUBLE; + @NotNull final Chunk workingChunk) { accumulate(workingChunk.asDoubleChunk(), ctx, 0, workingChunk.size()); - singletonGroup = bucketPosition; - singletonVal = ctx.curVal; outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java index 5c91e5596fc..c346341e72b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java @@ -36,37 +36,11 @@ public FloatFillByOperator(@NotNull final MatchPair fillPair, // region extra-methods // endregion extra-methods - @Override - public void addChunkBucketed(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { - final FloatChunk asFloats = values.asFloatChunk(); - final Context ctx = (Context) context; - for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int bucketPosition = bucketPositions.get(runStart); - - ctx.curVal = bucketLastVal.getFloat(bucketPosition); - accumulate(asFloats, ctx, runStart, runLength); - bucketLastVal.set(bucketPosition, ctx.curVal); - } - //noinspection unchecked - outputSource.fillFromChunkUnordered(ctx.fillContext.get(), ctx.outputValues.get(), (LongChunk) keyChunk); - } - @Override protected void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - final long bucketPosition) { - ctx.curVal = singletonGroup == bucketPosition ? singletonVal : NULL_FLOAT; + @NotNull final Chunk workingChunk) { accumulate(workingChunk.asFloatChunk(), ctx, 0, workingChunk.size()); - singletonGroup = bucketPosition; - singletonVal = ctx.curVal; outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java index 0a508c13678..a19e9674db6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java @@ -36,37 +36,11 @@ public IntFillByOperator(@NotNull final MatchPair fillPair, // region extra-methods // endregion extra-methods - @Override - public void addChunkBucketed(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { - final IntChunk asInts = values.asIntChunk(); - final Context ctx = (Context) context; - for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int bucketPosition = bucketPositions.get(runStart); - - ctx.curVal = bucketLastVal.getInt(bucketPosition); - accumulate(asInts, ctx, runStart, runLength); - bucketLastVal.set(bucketPosition, ctx.curVal); - } - //noinspection unchecked - outputSource.fillFromChunkUnordered(ctx.fillContext.get(), ctx.outputValues.get(), (LongChunk) keyChunk); - } - @Override protected void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - final long bucketPosition) { - ctx.curVal = singletonGroup == bucketPosition ? singletonVal : NULL_INT; + @NotNull final Chunk workingChunk) { accumulate(workingChunk.asIntChunk(), ctx, 0, workingChunk.size()); - singletonGroup = bucketPosition; - singletonVal = ctx.curVal; outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java index 366c7d718e7..4183bae6887 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java @@ -57,37 +57,11 @@ public Map> getOutputColumns() { } // endregion extra-methods - @Override - public void addChunkBucketed(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { - final LongChunk asLongs = values.asLongChunk(); - final Context ctx = (Context) context; - for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int bucketPosition = bucketPositions.get(runStart); - - ctx.curVal = bucketLastVal.getLong(bucketPosition); - accumulate(asLongs, ctx, runStart, runLength); - bucketLastVal.set(bucketPosition, ctx.curVal); - } - //noinspection unchecked - outputSource.fillFromChunkUnordered(ctx.fillContext.get(), ctx.outputValues.get(), (LongChunk) keyChunk); - } - @Override protected void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - final long bucketPosition) { - ctx.curVal = singletonGroup == bucketPosition ? singletonVal : NULL_LONG; + @NotNull final Chunk workingChunk) { accumulate(workingChunk.asLongChunk(), ctx, 0, workingChunk.size()); - singletonGroup = bucketPosition; - singletonVal = ctx.curVal; outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java index 4b530b4b863..231fb2ce711 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java @@ -36,37 +36,11 @@ public ObjectFillByOperator(@NotNull final MatchPair fillPair, // region extra-methods // endregion extra-methods - @Override - public void addChunkBucketed(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { - final ObjectChunk asObjects = values.asObjectChunk(); - final Context ctx = (Context) context; - for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int bucketPosition = bucketPositions.get(runStart); - - ctx.curVal = bucketLastVal.get(bucketPosition); - accumulate(asObjects, ctx, runStart, runLength); - bucketLastVal.set(bucketPosition, ctx.curVal); - } - //noinspection unchecked - outputSource.fillFromChunkUnordered(ctx.fillContext.get(), ctx.outputValues.get(), (LongChunk) keyChunk); - } - @Override protected void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - final long bucketPosition) { - ctx.curVal = singletonGroup == bucketPosition ? singletonVal : null; + @NotNull final Chunk workingChunk) { accumulate(workingChunk.asObjectChunk(), ctx, 0, workingChunk.size()); - singletonGroup = bucketPosition; - singletonVal = ctx.curVal; outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java index 28be54315bb..77cdd1a1d22 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java @@ -36,37 +36,11 @@ public ShortFillByOperator(@NotNull final MatchPair fillPair, // region extra-methods // endregion extra-methods - @Override - public void addChunkBucketed(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { - final ShortChunk asShorts = values.asShortChunk(); - final Context ctx = (Context) context; - for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int bucketPosition = bucketPositions.get(runStart); - - ctx.curVal = bucketLastVal.getShort(bucketPosition); - accumulate(asShorts, ctx, runStart, runLength); - bucketLastVal.set(bucketPosition, ctx.curVal); - } - //noinspection unchecked - outputSource.fillFromChunkUnordered(ctx.fillContext.get(), ctx.outputValues.get(), (LongChunk) keyChunk); - } - @Override protected void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - final long bucketPosition) { - ctx.curVal = singletonGroup == bucketPosition ? singletonVal : NULL_SHORT; + @NotNull final Chunk workingChunk) { accumulate(workingChunk.asShortChunk(), ctx, 0, workingChunk.size()); - singletonGroup = bucketPosition; - singletonVal = ctx.curVal; outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java index 132298c3a96..eca6d81a3cd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java @@ -38,18 +38,12 @@ public abstract class BaseByteUpdateByOperator extends UpdateByCumulativeOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; + protected boolean trackingPrev = false; + protected final MatchPair pair; protected final String[] affectingColumns; protected final boolean isRedirected; - protected WritableColumnSource bucketLastVal; - - /** These are only used in grouped operations */ - protected byte singletonVal; - protected long singletonGroup = QueryConstants.NULL_LONG; - - protected boolean initialized = false; - // region extra-fields final byte nullValue; // endregion extra-fields @@ -117,8 +111,6 @@ public BaseByteUpdateByOperator(@NotNull final MatchPair pair, // region constructor this.nullValue = getNullValue(); // endregion constructor - - this.singletonVal = nullValue; } // region extra-methods @@ -142,11 +134,6 @@ public void setChunkSize(@NotNull final UpdateContext context, final int chunkSi ((Context)context).fillContext.ensureCapacity(chunkSize); } - @Override - public void setBucketCapacity(final int capacity) { - bucketLastVal.ensureCapacity(capacity); - } - @NotNull @Override public String getInputColumnName() { @@ -181,17 +168,9 @@ public UpdateContext makeUpdateContext(final int chunkSize) { public void initializeForUpdate(@NotNull final UpdateContext context, @NotNull final TableUpdate upstream, @NotNull final RowSet resultSourceIndex, - final boolean usingBuckets, + final long lastPrevKey, final boolean isUpstreamAppendOnly) { final Context ctx = (Context) context; - if(!initialized) { - initialized = true; - if(usingBuckets) { - // region create-bucket - this.bucketLastVal = makeDenseSource(); - // endregion create-bucket - } - } // If we're redirected we have to make sure we tell the output source it's actual size, or we're going // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not @@ -201,10 +180,11 @@ public void initializeForUpdate(@NotNull final UpdateContext context, outputSource.ensureCapacity(resultSourceIndex.size() + 1); } - if(!usingBuckets) { - // If we aren't bucketing, we'll just remember the appendyness. - ctx.canProcessDirectly = isUpstreamAppendOnly; - } + // just remember the appendyness. + ctx.canProcessDirectly = isUpstreamAppendOnly; + + // pre-load the context with the previous last value in the table (if possibe) + ctx.curVal = lastPrevKey == NULL_ROW_KEY ? nullValue : outputSource.getByte(lastPrevKey); } @Override @@ -212,7 +192,6 @@ public void initializeFor(@NotNull final UpdateContext updateContext, @NotNull final RowSet updateRowSet, @NotNull final UpdateBy.UpdateType type) { ((Context)updateContext).currentUpdateType = type; - ((Context)updateContext).curVal = nullValue; } @Override @@ -238,9 +217,12 @@ public boolean requiresValues(@NotNull final UpdateContext context) { @Override public void startTrackingPrev() { - outputSource.startTrackingPrevValues(); - if(isRedirected) { - maybeInnerSource.startTrackingPrevValues(); + if (!trackingPrev) { + trackingPrev = true; + outputSource.startTrackingPrevValues(); + if (isRedirected) { + maybeInnerSource.startTrackingPrevValues(); + } } } @@ -255,15 +237,6 @@ final public boolean anyModified(@NotNull final UpdateContext ctx) { return ((Context)ctx).newModified != null; } - @Override - public void onBucketsRemoved(@NotNull final RowSet removedBuckets) { - if(bucketLastVal != null) { - bucketLastVal.setNull(removedBuckets); - } else { - singletonVal = nullValue; - } - } - @Override public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { return ((Context)context).canProcessDirectly; @@ -274,11 +247,10 @@ public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { public void addChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, - @NotNull final Chunk values, - long bucketPosition) { + @NotNull final Chunk values) { final Context ctx = (Context) updateContext; if (ctx.canProcessDirectly) { - doAddChunk(ctx, inputKeys, values, bucketPosition); + doAddChunk(ctx, inputKeys, values); } } @@ -288,12 +260,10 @@ public void addChunk(@NotNull final UpdateContext updateContext, * @param ctx the context object * @param inputKeys the input keys for the chunk * @param workingChunk the chunk of values - * @param bucketPosition the bucket position that the values belong to. */ protected abstract void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - final long bucketPosition); + @NotNull final Chunk workingChunk); // endregion @@ -317,20 +287,10 @@ public void resetForReprocess(@NotNull final UpdateContext context, long firstUnmodifiedKey) { final Context ctx = (Context) context; if(!ctx.canProcessDirectly) { - singletonVal = firstUnmodifiedKey == NULL_ROW_KEY ? nullValue : outputSource.getByte(firstUnmodifiedKey); + ctx.curVal = firstUnmodifiedKey == NULL_ROW_KEY ? nullValue : outputSource.getByte(firstUnmodifiedKey); } } - - @Override - public void resetForReprocessBucketed(@NotNull final UpdateContext ctx, - @NotNull final RowSet bucketIndex, - final long bucketPosition, - final long firstUnmodifiedKey) { - final byte previousVal = firstUnmodifiedKey == NULL_ROW_KEY ? nullValue : outputSource.getByte(firstUnmodifiedKey); - bucketLastVal.set(bucketPosition, previousVal); - } - @Override public void reprocessChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @@ -338,22 +298,10 @@ public void reprocessChunk(@NotNull final UpdateContext updateContext, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; - doAddChunk(ctx, inputKeys, valuesChunk, 0); + doAddChunk(ctx, inputKeys, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } - @Override - public void reprocessChunkBucketed(@NotNull UpdateContext updateContext, - @NotNull final RowSequence chunkOk, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk runStartPositions, - @NotNull final IntChunk runLengths) { - addChunkBucketed(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); - ((Context)updateContext).getModifiedBuilder().appendRowSequence(chunkOk); - } - // endregion // region No-Op Operations @@ -363,15 +311,13 @@ final public void modifyChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk prevKeyChunk, @Nullable final LongChunk keyChunk, @NotNull final Chunk prevValuesChunk, - @NotNull final Chunk postValuesChunk, - long bucketPosition) { + @NotNull final Chunk postValuesChunk) { } @Override final public void removeChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk, - long bucketPosition) { + @NotNull final Chunk prevValuesChunk) { } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java index 217e2324016..ad3468aa267 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java @@ -28,18 +28,12 @@ public abstract class BaseCharUpdateByOperator extends UpdateByCumulativeOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; + protected boolean trackingPrev = false; + protected final MatchPair pair; protected final String[] affectingColumns; protected final boolean isRedirected; - protected CharacterArraySource bucketLastVal; - - /** These are only used in grouped operations */ - protected char singletonVal; - protected long singletonGroup = QueryConstants.NULL_LONG; - - protected boolean initialized = false; - // region extra-fields // endregion extra-fields @@ -105,8 +99,6 @@ public BaseCharUpdateByOperator(@NotNull final MatchPair pair, // region constructor // endregion constructor - - this.singletonVal = QueryConstants.NULL_CHAR; } // region extra-methods @@ -118,11 +110,6 @@ public void setChunkSize(@NotNull final UpdateContext context, final int chunkSi ((Context)context).fillContext.ensureCapacity(chunkSize); } - @Override - public void setBucketCapacity(final int capacity) { - bucketLastVal.ensureCapacity(capacity); - } - @NotNull @Override public String getInputColumnName() { @@ -157,17 +144,9 @@ public UpdateContext makeUpdateContext(final int chunkSize) { public void initializeForUpdate(@NotNull final UpdateContext context, @NotNull final TableUpdate upstream, @NotNull final RowSet resultSourceIndex, - final boolean usingBuckets, + final long lastPrevKey, final boolean isUpstreamAppendOnly) { final Context ctx = (Context) context; - if(!initialized) { - initialized = true; - if(usingBuckets) { - // region create-bucket - this.bucketLastVal = new CharacterArraySource(); - // endregion create-bucket - } - } // If we're redirected we have to make sure we tell the output source it's actual size, or we're going // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not @@ -177,10 +156,11 @@ public void initializeForUpdate(@NotNull final UpdateContext context, outputSource.ensureCapacity(resultSourceIndex.size() + 1); } - if(!usingBuckets) { - // If we aren't bucketing, we'll just remember the appendyness. - ctx.canProcessDirectly = isUpstreamAppendOnly; - } + // just remember the appendyness. + ctx.canProcessDirectly = isUpstreamAppendOnly; + + // pre-load the context with the previous last value in the table (if possibe) + ctx.curVal = lastPrevKey == NULL_ROW_KEY ? QueryConstants.NULL_CHAR : outputSource.getChar(lastPrevKey); } @Override @@ -188,7 +168,6 @@ public void initializeFor(@NotNull final UpdateContext updateContext, @NotNull final RowSet updateRowSet, @NotNull final UpdateBy.UpdateType type) { ((Context)updateContext).currentUpdateType = type; - ((Context)updateContext).curVal = QueryConstants.NULL_CHAR; } @Override @@ -214,9 +193,12 @@ public boolean requiresValues(@NotNull final UpdateContext context) { @Override public void startTrackingPrev() { - outputSource.startTrackingPrevValues(); - if(isRedirected) { - maybeInnerSource.startTrackingPrevValues(); + if (!trackingPrev) { + trackingPrev = true; + outputSource.startTrackingPrevValues(); + if (isRedirected) { + maybeInnerSource.startTrackingPrevValues(); + } } } @@ -231,15 +213,6 @@ final public boolean anyModified(@NotNull final UpdateContext ctx) { return ((Context)ctx).newModified != null; } - @Override - public void onBucketsRemoved(@NotNull final RowSet removedBuckets) { - if(bucketLastVal != null) { - bucketLastVal.setNull(removedBuckets); - } else { - singletonVal = QueryConstants.NULL_CHAR; - } - } - @Override public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { return ((Context)context).canProcessDirectly; @@ -250,11 +223,10 @@ public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { public void addChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, - @NotNull final Chunk values, - long bucketPosition) { + @NotNull final Chunk values) { final Context ctx = (Context) updateContext; if (ctx.canProcessDirectly) { - doAddChunk(ctx, inputKeys, values, bucketPosition); + doAddChunk(ctx, inputKeys, values); } } @@ -264,12 +236,10 @@ public void addChunk(@NotNull final UpdateContext updateContext, * @param ctx the context object * @param inputKeys the input keys for the chunk * @param workingChunk the chunk of values - * @param bucketPosition the bucket position that the values belong to. */ protected abstract void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - final long bucketPosition); + @NotNull final Chunk workingChunk); // endregion @@ -291,20 +261,10 @@ public void resetForReprocess(@NotNull final UpdateContext context, long firstUnmodifiedKey) { final Context ctx = (Context) context; if(!ctx.canProcessDirectly) { - singletonVal = firstUnmodifiedKey == NULL_ROW_KEY ? QueryConstants.NULL_CHAR : outputSource.getChar(firstUnmodifiedKey); + ctx.curVal = firstUnmodifiedKey == NULL_ROW_KEY ? QueryConstants.NULL_CHAR : outputSource.getChar(firstUnmodifiedKey); } } - - @Override - public void resetForReprocessBucketed(@NotNull final UpdateContext ctx, - @NotNull final RowSet bucketIndex, - final long bucketPosition, - final long firstUnmodifiedKey) { - final char previousVal = firstUnmodifiedKey == NULL_ROW_KEY ? QueryConstants.NULL_CHAR : outputSource.getChar(firstUnmodifiedKey); - bucketLastVal.set(bucketPosition, previousVal); - } - @Override public void reprocessChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @@ -312,22 +272,10 @@ public void reprocessChunk(@NotNull final UpdateContext updateContext, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; - doAddChunk(ctx, inputKeys, valuesChunk, 0); + doAddChunk(ctx, inputKeys, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } - @Override - public void reprocessChunkBucketed(@NotNull UpdateContext updateContext, - @NotNull final RowSequence chunkOk, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk runStartPositions, - @NotNull final IntChunk runLengths) { - addChunkBucketed(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); - ((Context)updateContext).getModifiedBuilder().appendRowSequence(chunkOk); - } - // endregion // region No-Op Operations @@ -337,15 +285,13 @@ final public void modifyChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk prevKeyChunk, @Nullable final LongChunk keyChunk, @NotNull final Chunk prevValuesChunk, - @NotNull final Chunk postValuesChunk, - long bucketPosition) { + @NotNull final Chunk postValuesChunk) { } @Override final public void removeChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk, - long bucketPosition) { + @NotNull final Chunk prevValuesChunk) { } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java index 5850d36c7e3..449dc701946 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java @@ -23,6 +23,7 @@ import io.deephaven.engine.table.impl.sources.WritableRedirectedColumnSource; import io.deephaven.engine.table.impl.util.RowRedirection; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; +import io.deephaven.util.QueryConstants; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -36,19 +37,12 @@ public abstract class BaseDoubleUpdateByOperator extends UpdateByCumulativeOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; + protected boolean trackingPrev = false; private final MatchPair pair; private final String[] affectingColumns; protected final boolean isRedirected; - protected DoubleArraySource bucketLastVal; - - /** These are only used in grouped operations */ - protected double singletonVal = NULL_DOUBLE; - protected long singletonGroup = NULL_LONG; - - protected boolean initialized = false; - protected class Context extends UpdateCumulativeContext { public final SizedSafeCloseable fillContext; public final SizedDoubleChunk outputValues; @@ -111,11 +105,6 @@ public void setChunkSize(@NotNull final UpdateContext context, final int chunkSi ((Context)context).fillContext.ensureCapacity(chunkSize); } - @Override - public void setBucketCapacity(final int capacity) { - bucketLastVal.ensureCapacity(capacity); - } - @NotNull @Override public String getInputColumnName() { @@ -150,15 +139,9 @@ public UpdateContext makeUpdateContext(final int chunkSize) { public void initializeForUpdate(@NotNull final UpdateContext context, @NotNull final TableUpdate upstream, @NotNull final RowSet resultSourceRowSet, - final boolean usingBuckets, + final long lastPrevKey, final boolean isUpstreamAppendOnly) { final Context ctx = (Context) context; - if(!initialized) { - initialized = true; - if(usingBuckets) { - this.bucketLastVal = new DoubleArraySource(); - } - } // If we're redirected we have to make sure we tell the output source it's actual size, or we're going // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not @@ -167,10 +150,11 @@ public void initializeForUpdate(@NotNull final UpdateContext context, outputSource.ensureCapacity(resultSourceRowSet.size() + 1); } - if(!usingBuckets) { - // If we aren't bucketing, we'll just remember the appendyness. - ctx.canProcessDirectly = isUpstreamAppendOnly; - } + // If we aren't bucketing, we'll just remember the appendyness. + ctx.canProcessDirectly = isUpstreamAppendOnly; + + // pre-load the context with the previous last value in the table (if possible) + ctx.curVal = lastPrevKey == NULL_ROW_KEY ? QueryConstants.NULL_DOUBLE : outputSource.getDouble(lastPrevKey); } @Override @@ -178,7 +162,6 @@ public void initializeFor(@NotNull final UpdateContext updateContext, @NotNull final RowSet updateRowSet, @NotNull final UpdateBy.UpdateType type) { ((Context)updateContext).currentUpdateType = type; - ((Context)updateContext).curVal = NULL_DOUBLE; } @Override @@ -204,9 +187,12 @@ public boolean requiresValues(@NotNull final UpdateContext context) { @Override public void startTrackingPrev() { - outputSource.startTrackingPrevValues(); - if(isRedirected) { - maybeInnerSource.startTrackingPrevValues(); + if (!trackingPrev) { + trackingPrev = true; + outputSource.startTrackingPrevValues(); + if (isRedirected) { + maybeInnerSource.startTrackingPrevValues(); + } } } @@ -221,14 +207,6 @@ final public boolean anyModified(@NotNull final UpdateContext ctx) { return ((Context)ctx).newModified != null; } - public void onBucketsRemoved(@NotNull final RowSet removedBuckets) { - if(bucketLastVal != null) { - bucketLastVal.setNull(removedBuckets); - } else { - singletonVal = NULL_DOUBLE; - } - } - @Override public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { return ((Context)context).canProcessDirectly; @@ -239,18 +217,16 @@ public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { public void addChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, - @NotNull final Chunk values, - long bucketPosition) { + @NotNull final Chunk values) { final Context ctx = (Context) updateContext; if (ctx.canProcessDirectly) { - doAddChunk(ctx, inputKeys, values, bucketPosition); + doAddChunk(ctx, inputKeys, values); } } protected abstract void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - final long groupPosition); + @NotNull final Chunk workingChunk); // endregion // region Shifts @@ -271,20 +247,10 @@ public void resetForReprocess(@NotNull final UpdateContext context, final long firstUnmodifiedKey) { final Context ctx = (Context) context; if(!ctx.canProcessDirectly) { - singletonVal = firstUnmodifiedKey == NULL_ROW_KEY ? NULL_DOUBLE : outputSource.getDouble(firstUnmodifiedKey); + ctx.curVal = firstUnmodifiedKey == NULL_ROW_KEY ? NULL_DOUBLE : outputSource.getDouble(firstUnmodifiedKey); } } - - @Override - public void resetForReprocessBucketed(@NotNull final UpdateContext ctx, - @NotNull final RowSet bucketRowSet, - final long bucketPosition, - final long firstUnmodifiedKey) { - final double previousVal = firstUnmodifiedKey == NULL_ROW_KEY ? NULL_DOUBLE : outputSource.getDouble(firstUnmodifiedKey); - bucketLastVal.set(bucketPosition, previousVal); - } - @Override public void reprocessChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @@ -292,22 +258,9 @@ public void reprocessChunk(@NotNull final UpdateContext updateContext, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceRowSet) { final Context ctx = (Context) updateContext; - doAddChunk(ctx, inputKeys, valuesChunk, 0); + doAddChunk(ctx, inputKeys, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } - - @Override - public void reprocessChunkBucketed(@NotNull UpdateContext updateContext, - @NotNull final RowSequence chunkOk, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk runStartPositions, - @NotNull final IntChunk runLengths) { - addChunkBucketed(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); - ((Context)updateContext).getModifiedBuilder().appendRowSequence(chunkOk); - } - // endregion // region No-Op Operations @@ -317,15 +270,13 @@ final public void modifyChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk prevKeyChunk, @Nullable final LongChunk keyChunk, @NotNull final Chunk prevValuesChunk, - @NotNull final Chunk postValuesChunk, - long bucketPosition) { + @NotNull final Chunk postValuesChunk) { } @Override final public void removeChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk, - long bucketPosition) { + @NotNull final Chunk prevValuesChunk) { } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java index 6654c974852..6421d1cff26 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java @@ -18,6 +18,7 @@ import io.deephaven.engine.table.impl.sources.WritableRedirectedColumnSource; import io.deephaven.engine.table.impl.util.RowRedirection; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; +import io.deephaven.util.QueryConstants; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -31,19 +32,12 @@ public abstract class BaseFloatUpdateByOperator extends UpdateByCumulativeOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; + protected boolean trackingPrev = false; private final MatchPair pair; private final String[] affectingColumns; protected final boolean isRedirected; - protected FloatArraySource bucketLastVal; - - /** These are only used in grouped operations */ - protected float singletonVal = NULL_FLOAT; - protected long singletonGroup = NULL_LONG; - - protected boolean initialized = false; - protected class Context extends UpdateCumulativeContext { public final SizedSafeCloseable fillContext; public final SizedFloatChunk outputValues; @@ -106,11 +100,6 @@ public void setChunkSize(@NotNull final UpdateContext context, final int chunkSi ((Context)context).fillContext.ensureCapacity(chunkSize); } - @Override - public void setBucketCapacity(final int capacity) { - bucketLastVal.ensureCapacity(capacity); - } - @NotNull @Override public String getInputColumnName() { @@ -145,15 +134,9 @@ public UpdateContext makeUpdateContext(final int chunkSize) { public void initializeForUpdate(@NotNull final UpdateContext context, @NotNull final TableUpdate upstream, @NotNull final RowSet resultSourceRowSet, - final boolean usingBuckets, + final long lastPrevKey, final boolean isUpstreamAppendOnly) { final Context ctx = (Context) context; - if(!initialized) { - initialized = true; - if(usingBuckets) { - this.bucketLastVal = new FloatArraySource(); - } - } // If we're redirected we have to make sure we tell the output source it's actual size, or we're going // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not @@ -162,10 +145,11 @@ public void initializeForUpdate(@NotNull final UpdateContext context, outputSource.ensureCapacity(resultSourceRowSet.size() + 1); } - if(!usingBuckets) { - // If we aren't bucketing, we'll just remember the appendyness. - ctx.canProcessDirectly = isUpstreamAppendOnly; - } + // If we aren't bucketing, we'll just remember the appendyness. + ctx.canProcessDirectly = isUpstreamAppendOnly; + + // pre-load the context with the previous last value in the table (if possible) + ctx.curVal = lastPrevKey == NULL_ROW_KEY ? QueryConstants.NULL_FLOAT : outputSource.getFloat(lastPrevKey); } @Override @@ -173,7 +157,6 @@ public void initializeFor(@NotNull final UpdateContext updateContext, @NotNull final RowSet updateRowSet, @NotNull final UpdateBy.UpdateType type) { ((Context)updateContext).currentUpdateType = type; - ((Context)updateContext).curVal = NULL_FLOAT; } @Override @@ -199,9 +182,12 @@ public boolean requiresValues(@NotNull final UpdateContext context) { @Override public void startTrackingPrev() { - outputSource.startTrackingPrevValues(); - if(isRedirected) { - maybeInnerSource.startTrackingPrevValues(); + if (!trackingPrev) { + trackingPrev = true; + outputSource.startTrackingPrevValues(); + if (isRedirected) { + maybeInnerSource.startTrackingPrevValues(); + } } } @@ -216,14 +202,6 @@ final public boolean anyModified(@NotNull final UpdateContext ctx) { return ((Context)ctx).newModified != null; } - public void onBucketsRemoved(@NotNull final RowSet removedBuckets) { - if(bucketLastVal != null) { - bucketLastVal.setNull(removedBuckets); - } else { - singletonVal = NULL_FLOAT; - } - } - @Override public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { return ((Context)context).canProcessDirectly; @@ -234,18 +212,16 @@ public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { public void addChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, - @NotNull final Chunk values, - long bucketPosition) { + @NotNull final Chunk values) { final Context ctx = (Context) updateContext; if (ctx.canProcessDirectly) { - doAddChunk(ctx, inputKeys, values, bucketPosition); + doAddChunk(ctx, inputKeys, values); } } protected abstract void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - final long groupPosition); + @NotNull final Chunk workingChunk); // endregion // region Shifts @@ -266,20 +242,10 @@ public void resetForReprocess(@NotNull final UpdateContext context, final long firstUnmodifiedKey) { final Context ctx = (Context) context; if(!ctx.canProcessDirectly) { - singletonVal = firstUnmodifiedKey == NULL_ROW_KEY ? NULL_FLOAT : outputSource.getFloat(firstUnmodifiedKey); + ctx.curVal = firstUnmodifiedKey == NULL_ROW_KEY ? NULL_FLOAT : outputSource.getFloat(firstUnmodifiedKey); } } - - @Override - public void resetForReprocessBucketed(@NotNull final UpdateContext ctx, - @NotNull final RowSet bucketRowSet, - final long bucketPosition, - final long firstUnmodifiedKey) { - final float previousVal = firstUnmodifiedKey == NULL_ROW_KEY ? NULL_FLOAT : outputSource.getFloat(firstUnmodifiedKey); - bucketLastVal.set(bucketPosition, previousVal); - } - @Override public void reprocessChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @@ -287,22 +253,9 @@ public void reprocessChunk(@NotNull final UpdateContext updateContext, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceRowSet) { final Context ctx = (Context) updateContext; - doAddChunk(ctx, inputKeys, valuesChunk, 0); + doAddChunk(ctx, inputKeys, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } - - @Override - public void reprocessChunkBucketed(@NotNull UpdateContext updateContext, - @NotNull final RowSequence chunkOk, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk runStartPositions, - @NotNull final IntChunk runLengths) { - addChunkBucketed(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); - ((Context)updateContext).getModifiedBuilder().appendRowSequence(chunkOk); - } - // endregion // region No-Op Operations @@ -312,15 +265,13 @@ final public void modifyChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk prevKeyChunk, @Nullable final LongChunk keyChunk, @NotNull final Chunk prevValuesChunk, - @NotNull final Chunk postValuesChunk, - long bucketPosition) { + @NotNull final Chunk postValuesChunk) { } @Override final public void removeChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk, - long bucketPosition) { + @NotNull final Chunk prevValuesChunk) { } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java index 7f8ffa847a7..5cb77f3316a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java @@ -33,18 +33,12 @@ public abstract class BaseIntUpdateByOperator extends UpdateByCumulativeOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; + protected boolean trackingPrev = false; + protected final MatchPair pair; protected final String[] affectingColumns; protected final boolean isRedirected; - protected IntegerArraySource bucketLastVal; - - /** These are only used in grouped operations */ - protected int singletonVal; - protected long singletonGroup = QueryConstants.NULL_LONG; - - protected boolean initialized = false; - // region extra-fields // endregion extra-fields @@ -110,8 +104,6 @@ public BaseIntUpdateByOperator(@NotNull final MatchPair pair, // region constructor // endregion constructor - - this.singletonVal = QueryConstants.NULL_INT; } // region extra-methods @@ -123,11 +115,6 @@ public void setChunkSize(@NotNull final UpdateContext context, final int chunkSi ((Context)context).fillContext.ensureCapacity(chunkSize); } - @Override - public void setBucketCapacity(final int capacity) { - bucketLastVal.ensureCapacity(capacity); - } - @NotNull @Override public String getInputColumnName() { @@ -162,17 +149,9 @@ public UpdateContext makeUpdateContext(final int chunkSize) { public void initializeForUpdate(@NotNull final UpdateContext context, @NotNull final TableUpdate upstream, @NotNull final RowSet resultSourceIndex, - final boolean usingBuckets, + final long lastPrevKey, final boolean isUpstreamAppendOnly) { final Context ctx = (Context) context; - if(!initialized) { - initialized = true; - if(usingBuckets) { - // region create-bucket - this.bucketLastVal = new IntegerArraySource(); - // endregion create-bucket - } - } // If we're redirected we have to make sure we tell the output source it's actual size, or we're going // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not @@ -182,10 +161,11 @@ public void initializeForUpdate(@NotNull final UpdateContext context, outputSource.ensureCapacity(resultSourceIndex.size() + 1); } - if(!usingBuckets) { - // If we aren't bucketing, we'll just remember the appendyness. - ctx.canProcessDirectly = isUpstreamAppendOnly; - } + // just remember the appendyness. + ctx.canProcessDirectly = isUpstreamAppendOnly; + + // pre-load the context with the previous last value in the table (if possibe) + ctx.curVal = lastPrevKey == NULL_ROW_KEY ? QueryConstants.NULL_INT : outputSource.getInt(lastPrevKey); } @Override @@ -193,7 +173,6 @@ public void initializeFor(@NotNull final UpdateContext updateContext, @NotNull final RowSet updateRowSet, @NotNull final UpdateBy.UpdateType type) { ((Context)updateContext).currentUpdateType = type; - ((Context)updateContext).curVal = QueryConstants.NULL_INT; } @Override @@ -219,9 +198,12 @@ public boolean requiresValues(@NotNull final UpdateContext context) { @Override public void startTrackingPrev() { - outputSource.startTrackingPrevValues(); - if(isRedirected) { - maybeInnerSource.startTrackingPrevValues(); + if (!trackingPrev) { + trackingPrev = true; + outputSource.startTrackingPrevValues(); + if (isRedirected) { + maybeInnerSource.startTrackingPrevValues(); + } } } @@ -236,15 +218,6 @@ final public boolean anyModified(@NotNull final UpdateContext ctx) { return ((Context)ctx).newModified != null; } - @Override - public void onBucketsRemoved(@NotNull final RowSet removedBuckets) { - if(bucketLastVal != null) { - bucketLastVal.setNull(removedBuckets); - } else { - singletonVal = QueryConstants.NULL_INT; - } - } - @Override public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { return ((Context)context).canProcessDirectly; @@ -255,11 +228,10 @@ public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { public void addChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, - @NotNull final Chunk values, - long bucketPosition) { + @NotNull final Chunk values) { final Context ctx = (Context) updateContext; if (ctx.canProcessDirectly) { - doAddChunk(ctx, inputKeys, values, bucketPosition); + doAddChunk(ctx, inputKeys, values); } } @@ -269,12 +241,10 @@ public void addChunk(@NotNull final UpdateContext updateContext, * @param ctx the context object * @param inputKeys the input keys for the chunk * @param workingChunk the chunk of values - * @param bucketPosition the bucket position that the values belong to. */ protected abstract void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - final long bucketPosition); + @NotNull final Chunk workingChunk); // endregion @@ -296,20 +266,10 @@ public void resetForReprocess(@NotNull final UpdateContext context, long firstUnmodifiedKey) { final Context ctx = (Context) context; if(!ctx.canProcessDirectly) { - singletonVal = firstUnmodifiedKey == NULL_ROW_KEY ? QueryConstants.NULL_INT : outputSource.getInt(firstUnmodifiedKey); + ctx.curVal = firstUnmodifiedKey == NULL_ROW_KEY ? QueryConstants.NULL_INT : outputSource.getInt(firstUnmodifiedKey); } } - - @Override - public void resetForReprocessBucketed(@NotNull final UpdateContext ctx, - @NotNull final RowSet bucketIndex, - final long bucketPosition, - final long firstUnmodifiedKey) { - final int previousVal = firstUnmodifiedKey == NULL_ROW_KEY ? QueryConstants.NULL_INT : outputSource.getInt(firstUnmodifiedKey); - bucketLastVal.set(bucketPosition, previousVal); - } - @Override public void reprocessChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @@ -317,22 +277,10 @@ public void reprocessChunk(@NotNull final UpdateContext updateContext, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; - doAddChunk(ctx, inputKeys, valuesChunk, 0); + doAddChunk(ctx, inputKeys, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } - @Override - public void reprocessChunkBucketed(@NotNull UpdateContext updateContext, - @NotNull final RowSequence chunkOk, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk runStartPositions, - @NotNull final IntChunk runLengths) { - addChunkBucketed(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); - ((Context)updateContext).getModifiedBuilder().appendRowSequence(chunkOk); - } - // endregion // region No-Op Operations @@ -342,15 +290,13 @@ final public void modifyChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk prevKeyChunk, @Nullable final LongChunk keyChunk, @NotNull final Chunk prevValuesChunk, - @NotNull final Chunk postValuesChunk, - long bucketPosition) { + @NotNull final Chunk postValuesChunk) { } @Override final public void removeChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk, - long bucketPosition) { + @NotNull final Chunk prevValuesChunk) { } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java index db99a87c21e..2be4bd95e5d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java @@ -33,18 +33,12 @@ public abstract class BaseLongUpdateByOperator extends UpdateByCumulativeOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; + protected boolean trackingPrev = false; + protected final MatchPair pair; protected final String[] affectingColumns; protected final boolean isRedirected; - protected LongArraySource bucketLastVal; - - /** These are only used in grouped operations */ - protected long singletonVal; - protected long singletonGroup = QueryConstants.NULL_LONG; - - protected boolean initialized = false; - // region extra-fields // endregion extra-fields @@ -110,8 +104,6 @@ public BaseLongUpdateByOperator(@NotNull final MatchPair pair, // region constructor // endregion constructor - - this.singletonVal = QueryConstants.NULL_LONG; } // region extra-methods @@ -123,11 +115,6 @@ public void setChunkSize(@NotNull final UpdateContext context, final int chunkSi ((Context)context).fillContext.ensureCapacity(chunkSize); } - @Override - public void setBucketCapacity(final int capacity) { - bucketLastVal.ensureCapacity(capacity); - } - @NotNull @Override public String getInputColumnName() { @@ -162,17 +149,9 @@ public UpdateContext makeUpdateContext(final int chunkSize) { public void initializeForUpdate(@NotNull final UpdateContext context, @NotNull final TableUpdate upstream, @NotNull final RowSet resultSourceIndex, - final boolean usingBuckets, + final long lastPrevKey, final boolean isUpstreamAppendOnly) { final Context ctx = (Context) context; - if(!initialized) { - initialized = true; - if(usingBuckets) { - // region create-bucket - this.bucketLastVal = new LongArraySource(); - // endregion create-bucket - } - } // If we're redirected we have to make sure we tell the output source it's actual size, or we're going // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not @@ -182,10 +161,11 @@ public void initializeForUpdate(@NotNull final UpdateContext context, outputSource.ensureCapacity(resultSourceIndex.size() + 1); } - if(!usingBuckets) { - // If we aren't bucketing, we'll just remember the appendyness. - ctx.canProcessDirectly = isUpstreamAppendOnly; - } + // just remember the appendyness. + ctx.canProcessDirectly = isUpstreamAppendOnly; + + // pre-load the context with the previous last value in the table (if possibe) + ctx.curVal = lastPrevKey == NULL_ROW_KEY ? QueryConstants.NULL_LONG : outputSource.getLong(lastPrevKey); } @Override @@ -193,7 +173,6 @@ public void initializeFor(@NotNull final UpdateContext updateContext, @NotNull final RowSet updateRowSet, @NotNull final UpdateBy.UpdateType type) { ((Context)updateContext).currentUpdateType = type; - ((Context)updateContext).curVal = QueryConstants.NULL_LONG; } @Override @@ -219,9 +198,12 @@ public boolean requiresValues(@NotNull final UpdateContext context) { @Override public void startTrackingPrev() { - outputSource.startTrackingPrevValues(); - if(isRedirected) { - maybeInnerSource.startTrackingPrevValues(); + if (!trackingPrev) { + trackingPrev = true; + outputSource.startTrackingPrevValues(); + if (isRedirected) { + maybeInnerSource.startTrackingPrevValues(); + } } } @@ -236,15 +218,6 @@ final public boolean anyModified(@NotNull final UpdateContext ctx) { return ((Context)ctx).newModified != null; } - @Override - public void onBucketsRemoved(@NotNull final RowSet removedBuckets) { - if(bucketLastVal != null) { - bucketLastVal.setNull(removedBuckets); - } else { - singletonVal = QueryConstants.NULL_LONG; - } - } - @Override public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { return ((Context)context).canProcessDirectly; @@ -255,11 +228,10 @@ public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { public void addChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, - @NotNull final Chunk values, - long bucketPosition) { + @NotNull final Chunk values) { final Context ctx = (Context) updateContext; if (ctx.canProcessDirectly) { - doAddChunk(ctx, inputKeys, values, bucketPosition); + doAddChunk(ctx, inputKeys, values); } } @@ -269,12 +241,10 @@ public void addChunk(@NotNull final UpdateContext updateContext, * @param ctx the context object * @param inputKeys the input keys for the chunk * @param workingChunk the chunk of values - * @param bucketPosition the bucket position that the values belong to. */ protected abstract void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - final long bucketPosition); + @NotNull final Chunk workingChunk); // endregion @@ -296,20 +266,10 @@ public void resetForReprocess(@NotNull final UpdateContext context, long firstUnmodifiedKey) { final Context ctx = (Context) context; if(!ctx.canProcessDirectly) { - singletonVal = firstUnmodifiedKey == NULL_ROW_KEY ? QueryConstants.NULL_LONG : outputSource.getLong(firstUnmodifiedKey); + ctx.curVal = firstUnmodifiedKey == NULL_ROW_KEY ? QueryConstants.NULL_LONG : outputSource.getLong(firstUnmodifiedKey); } } - - @Override - public void resetForReprocessBucketed(@NotNull final UpdateContext ctx, - @NotNull final RowSet bucketIndex, - final long bucketPosition, - final long firstUnmodifiedKey) { - final long previousVal = firstUnmodifiedKey == NULL_ROW_KEY ? QueryConstants.NULL_LONG : outputSource.getLong(firstUnmodifiedKey); - bucketLastVal.set(bucketPosition, previousVal); - } - @Override public void reprocessChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @@ -317,22 +277,10 @@ public void reprocessChunk(@NotNull final UpdateContext updateContext, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; - doAddChunk(ctx, inputKeys, valuesChunk, 0); + doAddChunk(ctx, inputKeys, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } - @Override - public void reprocessChunkBucketed(@NotNull UpdateContext updateContext, - @NotNull final RowSequence chunkOk, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk runStartPositions, - @NotNull final IntChunk runLengths) { - addChunkBucketed(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); - ((Context)updateContext).getModifiedBuilder().appendRowSequence(chunkOk); - } - // endregion // region No-Op Operations @@ -342,15 +290,13 @@ final public void modifyChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk prevKeyChunk, @Nullable final LongChunk keyChunk, @NotNull final Chunk prevValuesChunk, - @NotNull final Chunk postValuesChunk, - long bucketPosition) { + @NotNull final Chunk postValuesChunk) { } @Override final public void removeChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk, - long bucketPosition) { + @NotNull final Chunk prevValuesChunk) { } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java index 687bb30c7e4..9ae71229dc5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java @@ -23,49 +23,12 @@ public BaseObjectBinaryOperator(@NotNull final Class type, // region Addition - @Override - public void addChunkBucketed(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { - final ObjectChunk asObject = values.asObjectChunk(); - //noinspection unchecked - final Context ctx = (Context) context; - for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int bucketPosition = bucketPositions.get(runStart); - - ctx.curVal = bucketLastVal.get(bucketPosition); - accumulate(asObject, ctx, runStart, runLength); - bucketLastVal.set(bucketPosition, ctx.curVal); - } - - //noinspection unchecked - outputSource.fillFromChunkUnordered(ctx.fillContext.get(), ctx.outputValues.get(), (LongChunk) keyChunk); - } @Override protected void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - final long groupPosition) { - if(bucketLastVal != null) { - ctx.curVal = bucketLastVal.get(groupPosition); - } else { - ctx.curVal = groupPosition == singletonGroup ? singletonVal : null; - } - + @NotNull final Chunk workingChunk) { accumulate(workingChunk.asObjectChunk(), ctx, 0, inputKeys.intSize()); - - if(bucketLastVal != null) { - bucketLastVal.set(groupPosition, ctx.curVal); - } else { - singletonGroup = groupPosition; - singletonVal = ctx.curVal; - } outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java index affaeeb8d29..bf5f822c2da 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java @@ -33,18 +33,12 @@ public abstract class BaseObjectUpdateByOperator extends UpdateByCumulativeOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; + protected boolean trackingPrev = false; + protected final MatchPair pair; protected final String[] affectingColumns; protected final boolean isRedirected; - protected ObjectArraySource bucketLastVal; - - /** These are only used in grouped operations */ - protected T singletonVal; - protected long singletonGroup = QueryConstants.NULL_LONG; - - protected boolean initialized = false; - // region extra-fields private final Class colType; // endregion extra-fields @@ -113,8 +107,6 @@ public BaseObjectUpdateByOperator(@NotNull final MatchPair pair, // region constructor this.colType = colType; // endregion constructor - - this.singletonVal = null; } // region extra-methods @@ -126,11 +118,6 @@ public void setChunkSize(@NotNull final UpdateContext context, final int chunkSi ((Context)context).fillContext.ensureCapacity(chunkSize); } - @Override - public void setBucketCapacity(final int capacity) { - bucketLastVal.ensureCapacity(capacity); - } - @NotNull @Override public String getInputColumnName() { @@ -165,17 +152,9 @@ public UpdateContext makeUpdateContext(final int chunkSize) { public void initializeForUpdate(@NotNull final UpdateContext context, @NotNull final TableUpdate upstream, @NotNull final RowSet resultSourceIndex, - final boolean usingBuckets, + final long lastPrevKey, final boolean isUpstreamAppendOnly) { final Context ctx = (Context) context; - if(!initialized) { - initialized = true; - if(usingBuckets) { - // region create-bucket - this.bucketLastVal = new ObjectArraySource(colType); - // endregion create-bucket - } - } // If we're redirected we have to make sure we tell the output source it's actual size, or we're going // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not @@ -185,10 +164,11 @@ public void initializeForUpdate(@NotNull final UpdateContext context, outputSource.ensureCapacity(resultSourceIndex.size() + 1); } - if(!usingBuckets) { - // If we aren't bucketing, we'll just remember the appendyness. - ctx.canProcessDirectly = isUpstreamAppendOnly; - } + // just remember the appendyness. + ctx.canProcessDirectly = isUpstreamAppendOnly; + + // pre-load the context with the previous last value in the table (if possibe) + ctx.curVal = lastPrevKey == NULL_ROW_KEY ? null : outputSource.get(lastPrevKey); } @Override @@ -196,7 +176,6 @@ public void initializeFor(@NotNull final UpdateContext updateContext, @NotNull final RowSet updateRowSet, @NotNull final UpdateBy.UpdateType type) { ((Context)updateContext).currentUpdateType = type; - ((Context)updateContext).curVal = null; } @Override @@ -222,9 +201,12 @@ public boolean requiresValues(@NotNull final UpdateContext context) { @Override public void startTrackingPrev() { - outputSource.startTrackingPrevValues(); - if(isRedirected) { - maybeInnerSource.startTrackingPrevValues(); + if (!trackingPrev) { + trackingPrev = true; + outputSource.startTrackingPrevValues(); + if (isRedirected) { + maybeInnerSource.startTrackingPrevValues(); + } } } @@ -239,15 +221,6 @@ final public boolean anyModified(@NotNull final UpdateContext ctx) { return ((Context)ctx).newModified != null; } - @Override - public void onBucketsRemoved(@NotNull final RowSet removedBuckets) { - if(bucketLastVal != null) { - bucketLastVal.setNull(removedBuckets); - } else { - singletonVal = null; - } - } - @Override public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { return ((Context)context).canProcessDirectly; @@ -258,11 +231,10 @@ public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { public void addChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, - @NotNull final Chunk values, - long bucketPosition) { + @NotNull final Chunk values) { final Context ctx = (Context) updateContext; if (ctx.canProcessDirectly) { - doAddChunk(ctx, inputKeys, values, bucketPosition); + doAddChunk(ctx, inputKeys, values); } } @@ -272,12 +244,10 @@ public void addChunk(@NotNull final UpdateContext updateContext, * @param ctx the context object * @param inputKeys the input keys for the chunk * @param workingChunk the chunk of values - * @param bucketPosition the bucket position that the values belong to. */ protected abstract void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - final long bucketPosition); + @NotNull final Chunk workingChunk); // endregion @@ -299,20 +269,10 @@ public void resetForReprocess(@NotNull final UpdateContext context, long firstUnmodifiedKey) { final Context ctx = (Context) context; if(!ctx.canProcessDirectly) { - singletonVal = firstUnmodifiedKey == NULL_ROW_KEY ? null : outputSource.get(firstUnmodifiedKey); + ctx.curVal = firstUnmodifiedKey == NULL_ROW_KEY ? null : outputSource.get(firstUnmodifiedKey); } } - - @Override - public void resetForReprocessBucketed(@NotNull final UpdateContext ctx, - @NotNull final RowSet bucketIndex, - final long bucketPosition, - final long firstUnmodifiedKey) { - final T previousVal = firstUnmodifiedKey == NULL_ROW_KEY ? null : outputSource.get(firstUnmodifiedKey); - bucketLastVal.set(bucketPosition, previousVal); - } - @Override public void reprocessChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @@ -320,22 +280,10 @@ public void reprocessChunk(@NotNull final UpdateContext updateContext, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; - doAddChunk(ctx, inputKeys, valuesChunk, 0); + doAddChunk(ctx, inputKeys, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } - @Override - public void reprocessChunkBucketed(@NotNull UpdateContext updateContext, - @NotNull final RowSequence chunkOk, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk runStartPositions, - @NotNull final IntChunk runLengths) { - addChunkBucketed(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); - ((Context)updateContext).getModifiedBuilder().appendRowSequence(chunkOk); - } - // endregion // region No-Op Operations @@ -345,15 +293,13 @@ final public void modifyChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk prevKeyChunk, @Nullable final LongChunk keyChunk, @NotNull final Chunk prevValuesChunk, - @NotNull final Chunk postValuesChunk, - long bucketPosition) { + @NotNull final Chunk postValuesChunk) { } @Override final public void removeChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk, - long bucketPosition) { + @NotNull final Chunk prevValuesChunk) { } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java index 169d51bbfc9..5c407379eb4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java @@ -33,18 +33,12 @@ public abstract class BaseShortUpdateByOperator extends UpdateByCumulativeOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; + protected boolean trackingPrev = false; + protected final MatchPair pair; protected final String[] affectingColumns; protected final boolean isRedirected; - protected ShortArraySource bucketLastVal; - - /** These are only used in grouped operations */ - protected short singletonVal; - protected long singletonGroup = QueryConstants.NULL_LONG; - - protected boolean initialized = false; - // region extra-fields // endregion extra-fields @@ -110,8 +104,6 @@ public BaseShortUpdateByOperator(@NotNull final MatchPair pair, // region constructor // endregion constructor - - this.singletonVal = QueryConstants.NULL_SHORT; } // region extra-methods @@ -123,11 +115,6 @@ public void setChunkSize(@NotNull final UpdateContext context, final int chunkSi ((Context)context).fillContext.ensureCapacity(chunkSize); } - @Override - public void setBucketCapacity(final int capacity) { - bucketLastVal.ensureCapacity(capacity); - } - @NotNull @Override public String getInputColumnName() { @@ -162,17 +149,9 @@ public UpdateContext makeUpdateContext(final int chunkSize) { public void initializeForUpdate(@NotNull final UpdateContext context, @NotNull final TableUpdate upstream, @NotNull final RowSet resultSourceIndex, - final boolean usingBuckets, + final long lastPrevKey, final boolean isUpstreamAppendOnly) { final Context ctx = (Context) context; - if(!initialized) { - initialized = true; - if(usingBuckets) { - // region create-bucket - this.bucketLastVal = new ShortArraySource(); - // endregion create-bucket - } - } // If we're redirected we have to make sure we tell the output source it's actual size, or we're going // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not @@ -182,10 +161,11 @@ public void initializeForUpdate(@NotNull final UpdateContext context, outputSource.ensureCapacity(resultSourceIndex.size() + 1); } - if(!usingBuckets) { - // If we aren't bucketing, we'll just remember the appendyness. - ctx.canProcessDirectly = isUpstreamAppendOnly; - } + // just remember the appendyness. + ctx.canProcessDirectly = isUpstreamAppendOnly; + + // pre-load the context with the previous last value in the table (if possibe) + ctx.curVal = lastPrevKey == NULL_ROW_KEY ? QueryConstants.NULL_SHORT : outputSource.getShort(lastPrevKey); } @Override @@ -193,7 +173,6 @@ public void initializeFor(@NotNull final UpdateContext updateContext, @NotNull final RowSet updateRowSet, @NotNull final UpdateBy.UpdateType type) { ((Context)updateContext).currentUpdateType = type; - ((Context)updateContext).curVal = QueryConstants.NULL_SHORT; } @Override @@ -219,9 +198,12 @@ public boolean requiresValues(@NotNull final UpdateContext context) { @Override public void startTrackingPrev() { - outputSource.startTrackingPrevValues(); - if(isRedirected) { - maybeInnerSource.startTrackingPrevValues(); + if (!trackingPrev) { + trackingPrev = true; + outputSource.startTrackingPrevValues(); + if (isRedirected) { + maybeInnerSource.startTrackingPrevValues(); + } } } @@ -236,15 +218,6 @@ final public boolean anyModified(@NotNull final UpdateContext ctx) { return ((Context)ctx).newModified != null; } - @Override - public void onBucketsRemoved(@NotNull final RowSet removedBuckets) { - if(bucketLastVal != null) { - bucketLastVal.setNull(removedBuckets); - } else { - singletonVal = QueryConstants.NULL_SHORT; - } - } - @Override public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { return ((Context)context).canProcessDirectly; @@ -255,11 +228,10 @@ public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { public void addChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, - @NotNull final Chunk values, - long bucketPosition) { + @NotNull final Chunk values) { final Context ctx = (Context) updateContext; if (ctx.canProcessDirectly) { - doAddChunk(ctx, inputKeys, values, bucketPosition); + doAddChunk(ctx, inputKeys, values); } } @@ -269,12 +241,10 @@ public void addChunk(@NotNull final UpdateContext updateContext, * @param ctx the context object * @param inputKeys the input keys for the chunk * @param workingChunk the chunk of values - * @param bucketPosition the bucket position that the values belong to. */ protected abstract void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - final long bucketPosition); + @NotNull final Chunk workingChunk); // endregion @@ -296,20 +266,10 @@ public void resetForReprocess(@NotNull final UpdateContext context, long firstUnmodifiedKey) { final Context ctx = (Context) context; if(!ctx.canProcessDirectly) { - singletonVal = firstUnmodifiedKey == NULL_ROW_KEY ? QueryConstants.NULL_SHORT : outputSource.getShort(firstUnmodifiedKey); + ctx.curVal = firstUnmodifiedKey == NULL_ROW_KEY ? QueryConstants.NULL_SHORT : outputSource.getShort(firstUnmodifiedKey); } } - - @Override - public void resetForReprocessBucketed(@NotNull final UpdateContext ctx, - @NotNull final RowSet bucketIndex, - final long bucketPosition, - final long firstUnmodifiedKey) { - final short previousVal = firstUnmodifiedKey == NULL_ROW_KEY ? QueryConstants.NULL_SHORT : outputSource.getShort(firstUnmodifiedKey); - bucketLastVal.set(bucketPosition, previousVal); - } - @Override public void reprocessChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @@ -317,22 +277,10 @@ public void reprocessChunk(@NotNull final UpdateContext updateContext, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; - doAddChunk(ctx, inputKeys, valuesChunk, 0); + doAddChunk(ctx, inputKeys, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } - @Override - public void reprocessChunkBucketed(@NotNull UpdateContext updateContext, - @NotNull final RowSequence chunkOk, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk runStartPositions, - @NotNull final IntChunk runLengths) { - addChunkBucketed(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); - ((Context)updateContext).getModifiedBuilder().appendRowSequence(chunkOk); - } - // endregion // region No-Op Operations @@ -342,15 +290,13 @@ final public void modifyChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk prevKeyChunk, @Nullable final LongChunk keyChunk, @NotNull final Chunk prevValuesChunk, - @NotNull final Chunk postValuesChunk, - long bucketPosition) { + @NotNull final Chunk postValuesChunk) { } @Override final public void removeChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk, - long bucketPosition) { + @NotNull final Chunk prevValuesChunk) { } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java index 885db5f7a66..6076d9288ad 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java @@ -34,8 +34,6 @@ public abstract class BaseWindowedByteUpdateByOperator extends UpdateByWindowedOperator { protected final ColumnSource valueSource; - protected boolean initialized = false; - // region extra-fields final byte nullValue; // endregion extra-fields @@ -262,15 +260,13 @@ protected WritableColumnSource makeDenseSource() { public abstract void reset(UpdateContext context); @Override - public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, boolean usingBuckets, boolean isUpstreamAppendOnly) { + public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, final long lastPrevKey, boolean isUpstreamAppendOnly) { final Context ctx = (Context) context; ctx.workingRowSet = resultSourceRowSet; - if(!usingBuckets) { - // we can only process directly from an update if the window is entire backward-looking. Since we - // allow negative values in fwd/rev ticks and timestamps, we need to check both - ctx.canProcessDirectly = isUpstreamAppendOnly && forwardTimeScaleUnits <= 0 && reverseTimeScaleUnits >= 0;; - } + // we can only process directly from an update if the window is entire backward-looking. Since we + // allow negative values in fwd/rev ticks and timestamps, we need to check both + ctx.canProcessDirectly = isUpstreamAppendOnly && forwardTimeScaleUnits <= 0 && reverseTimeScaleUnits >= 0;; } @Override @@ -326,66 +322,21 @@ final public boolean anyModified(@NotNull final UpdateContext ctx) { * @param ctx the context object * @param inputKeys the input keys for the chunk * @param workingChunk the chunk of values - * @param bucketPosition the bucket position that the values belong to. */ protected abstract void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, - @NotNull final Chunk workingChunk, - final long bucketPosition); + @NotNull final Chunk workingChunk); @Override public void addChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, - @NotNull final Chunk values, - long bucketPosition) { + @NotNull final Chunk values) { final Context ctx = (Context) updateContext; if (ctx.canProcessDirectly) { ctx.loadDataChunks(inputKeys); - doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); - } - } - - @Override - public void addChunkBucketed(final @NotNull UpdateByOperator.UpdateContext context, - final @NotNull Chunk values, - final @NotNull LongChunk keyChunk, - final @NotNull IntChunk bucketPositions, - final @NotNull IntChunk startPositions, - final @NotNull IntChunk runLengths) { - final Context ctx = (Context) context; - if (ctx.canProcessDirectly) { -// final ShortChunk asShorts = values.asShortChunk(); - for (int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int bucketPosition = bucketPositions.get(runStart); -// doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); - -// try (RowSequence rs = RowSequenceFactory.wrapRowKeysChunkAsRowSequence((LongChunk) keyChunk)_ - -// RowSetBuilderSequential builder = RowSetFactory.builderSequential(); -// for (int ii = runStart; ii < runStart + runLength; ii++) { -// builder.appendKey(keyChunk.get(ii)); -// } -// -// WritableRowSet bucketRs = bucketRowSet.get(bucketPosition); -// if (bucketRs == null) { -// bucketRs = builder.build(); -// bucketRowSet.set(bucketPosition, bucketRs); -// } else { -// try (final RowSet added = builder.build()) { -// bucketRs.insert(added); -// } -// } -// -// ctx.curVal = NULL_LONG; -// ctx.currentWindow.clear(); - -// accumulate(asShorts, (LongChunk) keyChunk, ctx, runStart, runLength, bucketRs); -// bucketLastVal.set(bucketPosition, ctx.curVal); - } + doAddChunk(ctx, inputKeys, keyChunk, values); } } @@ -400,14 +351,6 @@ public void resetForReprocess(@NotNull final UpdateContext context, ctx.workingRowSet = sourceRowSet; } - @Override - public void resetForReprocessBucketed(@NotNull final UpdateContext context, - @NotNull final RowSet bucketRowSet, - final long bucketPosition, - final long firstUnmodifiedKey) { - final Context ctx = (Context) context; - } - @Override public void reprocessChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @@ -416,22 +359,10 @@ public void reprocessChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; ctx.loadDataChunks(inputKeys); - doAddChunk(ctx, inputKeys, keyChunk, valuesChunk, 0); + doAddChunk(ctx, inputKeys, keyChunk, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } - @Override - public void reprocessChunkBucketed(@NotNull UpdateContext updateContext, - @NotNull final RowSequence chunkOk, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk runStartPositions, - @NotNull final IntChunk runLengths) { - addChunkBucketed(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); - ((Context)updateContext).getModifiedBuilder().appendRowSequence(chunkOk); - } - // endregion // region No-Op Operations @@ -441,15 +372,13 @@ final public void modifyChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk prevKeyChunk, @Nullable final LongChunk keyChunk, @NotNull final Chunk prevValuesChunk, - @NotNull final Chunk postValuesChunk, - long bucketPosition) { + @NotNull final Chunk postValuesChunk) { } @Override final public void removeChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk, - long bucketPosition) { + @NotNull final Chunk prevValuesChunk) { } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java index 88374270f75..9d18723f41a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java @@ -24,8 +24,6 @@ public abstract class BaseWindowedCharUpdateByOperator extends UpdateByWindowedOperator { protected final ColumnSource valueSource; - protected boolean initialized = false; - // region extra-fields // endregion extra-fields @@ -238,15 +236,13 @@ public BaseWindowedCharUpdateByOperator(@NotNull final MatchPair pair, public abstract void reset(UpdateContext context); @Override - public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, boolean usingBuckets, boolean isUpstreamAppendOnly) { + public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, final long lastPrevKey, boolean isUpstreamAppendOnly) { final Context ctx = (Context) context; ctx.workingRowSet = resultSourceRowSet; - if(!usingBuckets) { - // we can only process directly from an update if the window is entire backward-looking. Since we - // allow negative values in fwd/rev ticks and timestamps, we need to check both - ctx.canProcessDirectly = isUpstreamAppendOnly && forwardTimeScaleUnits <= 0 && reverseTimeScaleUnits >= 0;; - } + // we can only process directly from an update if the window is entire backward-looking. Since we + // allow negative values in fwd/rev ticks and timestamps, we need to check both + ctx.canProcessDirectly = isUpstreamAppendOnly && forwardTimeScaleUnits <= 0 && reverseTimeScaleUnits >= 0;; } @Override @@ -302,66 +298,21 @@ final public boolean anyModified(@NotNull final UpdateContext ctx) { * @param ctx the context object * @param inputKeys the input keys for the chunk * @param workingChunk the chunk of values - * @param bucketPosition the bucket position that the values belong to. */ protected abstract void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, - @NotNull final Chunk workingChunk, - final long bucketPosition); + @NotNull final Chunk workingChunk); @Override public void addChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, - @NotNull final Chunk values, - long bucketPosition) { + @NotNull final Chunk values) { final Context ctx = (Context) updateContext; if (ctx.canProcessDirectly) { ctx.loadDataChunks(inputKeys); - doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); - } - } - - @Override - public void addChunkBucketed(final @NotNull UpdateByOperator.UpdateContext context, - final @NotNull Chunk values, - final @NotNull LongChunk keyChunk, - final @NotNull IntChunk bucketPositions, - final @NotNull IntChunk startPositions, - final @NotNull IntChunk runLengths) { - final Context ctx = (Context) context; - if (ctx.canProcessDirectly) { -// final ShortChunk asShorts = values.asShortChunk(); - for (int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int bucketPosition = bucketPositions.get(runStart); -// doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); - -// try (RowSequence rs = RowSequenceFactory.wrapRowKeysChunkAsRowSequence((LongChunk) keyChunk)_ - -// RowSetBuilderSequential builder = RowSetFactory.builderSequential(); -// for (int ii = runStart; ii < runStart + runLength; ii++) { -// builder.appendKey(keyChunk.get(ii)); -// } -// -// WritableRowSet bucketRs = bucketRowSet.get(bucketPosition); -// if (bucketRs == null) { -// bucketRs = builder.build(); -// bucketRowSet.set(bucketPosition, bucketRs); -// } else { -// try (final RowSet added = builder.build()) { -// bucketRs.insert(added); -// } -// } -// -// ctx.curVal = NULL_LONG; -// ctx.currentWindow.clear(); - -// accumulate(asShorts, (LongChunk) keyChunk, ctx, runStart, runLength, bucketRs); -// bucketLastVal.set(bucketPosition, ctx.curVal); - } + doAddChunk(ctx, inputKeys, keyChunk, values); } } @@ -376,14 +327,6 @@ public void resetForReprocess(@NotNull final UpdateContext context, ctx.workingRowSet = sourceRowSet; } - @Override - public void resetForReprocessBucketed(@NotNull final UpdateContext context, - @NotNull final RowSet bucketRowSet, - final long bucketPosition, - final long firstUnmodifiedKey) { - final Context ctx = (Context) context; - } - @Override public void reprocessChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @@ -392,22 +335,10 @@ public void reprocessChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; ctx.loadDataChunks(inputKeys); - doAddChunk(ctx, inputKeys, keyChunk, valuesChunk, 0); + doAddChunk(ctx, inputKeys, keyChunk, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } - @Override - public void reprocessChunkBucketed(@NotNull UpdateContext updateContext, - @NotNull final RowSequence chunkOk, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk runStartPositions, - @NotNull final IntChunk runLengths) { - addChunkBucketed(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); - ((Context)updateContext).getModifiedBuilder().appendRowSequence(chunkOk); - } - // endregion // region No-Op Operations @@ -417,15 +348,13 @@ final public void modifyChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk prevKeyChunk, @Nullable final LongChunk keyChunk, @NotNull final Chunk prevValuesChunk, - @NotNull final Chunk postValuesChunk, - long bucketPosition) { + @NotNull final Chunk postValuesChunk) { } @Override final public void removeChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk, - long bucketPosition) { + @NotNull final Chunk prevValuesChunk) { } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java index c0d8dec2746..722e5ab837c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java @@ -27,8 +27,6 @@ public abstract class BaseWindowedDoubleUpdateByOperator extends UpdateByWindowedOperator { protected final ColumnSource valueSource; - protected boolean initialized = false; - // region extra-fields // endregion extra-fields @@ -241,15 +239,13 @@ public BaseWindowedDoubleUpdateByOperator(@NotNull final MatchPair pair, public abstract void reset(UpdateContext context); @Override - public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, boolean usingBuckets, boolean isUpstreamAppendOnly) { + public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, final long lastPrevKey, boolean isUpstreamAppendOnly) { final Context ctx = (Context) context; ctx.workingRowSet = resultSourceRowSet; - if(!usingBuckets) { - // we can only process directly from an update if the window is entire backward-looking. Since we - // allow negative values in fwd/rev ticks and timestamps, we need to check both - ctx.canProcessDirectly = isUpstreamAppendOnly && forwardTimeScaleUnits <= 0 && reverseTimeScaleUnits >= 0;; - } + // we can only process directly from an update if the window is entire backward-looking. Since we + // allow negative values in fwd/rev ticks and timestamps, we need to check both + ctx.canProcessDirectly = isUpstreamAppendOnly && forwardTimeScaleUnits <= 0 && reverseTimeScaleUnits >= 0;; } @Override @@ -305,66 +301,21 @@ final public boolean anyModified(@NotNull final UpdateContext ctx) { * @param ctx the context object * @param inputKeys the input keys for the chunk * @param workingChunk the chunk of values - * @param bucketPosition the bucket position that the values belong to. */ protected abstract void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, - @NotNull final Chunk workingChunk, - final long bucketPosition); + @NotNull final Chunk workingChunk); @Override public void addChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, - @NotNull final Chunk values, - long bucketPosition) { + @NotNull final Chunk values) { final Context ctx = (Context) updateContext; if (ctx.canProcessDirectly) { ctx.loadDataChunks(inputKeys); - doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); - } - } - - @Override - public void addChunkBucketed(final @NotNull UpdateContext context, - final @NotNull Chunk values, - final @NotNull LongChunk keyChunk, - final @NotNull IntChunk bucketPositions, - final @NotNull IntChunk startPositions, - final @NotNull IntChunk runLengths) { - final Context ctx = (Context) context; - if (ctx.canProcessDirectly) { -// final ShortChunk asShorts = values.asShortChunk(); - for (int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int bucketPosition = bucketPositions.get(runStart); -// doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); - -// try (RowSequence rs = RowSequenceFactory.wrapRowKeysChunkAsRowSequence((LongChunk) keyChunk)_ - -// RowSetBuilderSequential builder = RowSetFactory.builderSequential(); -// for (int ii = runStart; ii < runStart + runLength; ii++) { -// builder.appendKey(keyChunk.get(ii)); -// } -// -// WritableRowSet bucketRs = bucketRowSet.get(bucketPosition); -// if (bucketRs == null) { -// bucketRs = builder.build(); -// bucketRowSet.set(bucketPosition, bucketRs); -// } else { -// try (final RowSet added = builder.build()) { -// bucketRs.insert(added); -// } -// } -// -// ctx.curVal = NULL_LONG; -// ctx.currentWindow.clear(); - -// accumulate(asShorts, (LongChunk) keyChunk, ctx, runStart, runLength, bucketRs); -// bucketLastVal.set(bucketPosition, ctx.curVal); - } + doAddChunk(ctx, inputKeys, keyChunk, values); } } @@ -379,14 +330,6 @@ public void resetForReprocess(@NotNull final UpdateContext context, ctx.workingRowSet = sourceRowSet; } - @Override - public void resetForReprocessBucketed(@NotNull final UpdateContext context, - @NotNull final RowSet bucketRowSet, - final long bucketPosition, - final long firstUnmodifiedKey) { - final Context ctx = (Context) context; - } - @Override public void reprocessChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @@ -395,22 +338,10 @@ public void reprocessChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; ctx.loadDataChunks(inputKeys); - doAddChunk(ctx, inputKeys, keyChunk, valuesChunk, 0); + doAddChunk(ctx, inputKeys, keyChunk, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } - @Override - public void reprocessChunkBucketed(@NotNull UpdateContext updateContext, - @NotNull final RowSequence chunkOk, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk runStartPositions, - @NotNull final IntChunk runLengths) { - addChunkBucketed(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); - ((Context)updateContext).getModifiedBuilder().appendRowSequence(chunkOk); - } - // endregion // region No-Op Operations @@ -420,15 +351,13 @@ final public void modifyChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk prevKeyChunk, @Nullable final LongChunk keyChunk, @NotNull final Chunk prevValuesChunk, - @NotNull final Chunk postValuesChunk, - long bucketPosition) { + @NotNull final Chunk postValuesChunk) { } @Override final public void removeChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk, - long bucketPosition) { + @NotNull final Chunk prevValuesChunk) { } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java index 598a1df0c81..59db0b11f0f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java @@ -22,8 +22,6 @@ public abstract class BaseWindowedFloatUpdateByOperator extends UpdateByWindowedOperator { protected final ColumnSource valueSource; - protected boolean initialized = false; - // region extra-fields // endregion extra-fields @@ -236,15 +234,13 @@ public BaseWindowedFloatUpdateByOperator(@NotNull final MatchPair pair, public abstract void reset(UpdateContext context); @Override - public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, boolean usingBuckets, boolean isUpstreamAppendOnly) { + public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, final long lastPrevKey, boolean isUpstreamAppendOnly) { final Context ctx = (Context) context; ctx.workingRowSet = resultSourceRowSet; - if(!usingBuckets) { - // we can only process directly from an update if the window is entire backward-looking. Since we - // allow negative values in fwd/rev ticks and timestamps, we need to check both - ctx.canProcessDirectly = isUpstreamAppendOnly && forwardTimeScaleUnits <= 0 && reverseTimeScaleUnits >= 0;; - } + // we can only process directly from an update if the window is entire backward-looking. Since we + // allow negative values in fwd/rev ticks and timestamps, we need to check both + ctx.canProcessDirectly = isUpstreamAppendOnly && forwardTimeScaleUnits <= 0 && reverseTimeScaleUnits >= 0;; } @Override @@ -300,66 +296,21 @@ final public boolean anyModified(@NotNull final UpdateContext ctx) { * @param ctx the context object * @param inputKeys the input keys for the chunk * @param workingChunk the chunk of values - * @param bucketPosition the bucket position that the values belong to. */ protected abstract void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, - @NotNull final Chunk workingChunk, - final long bucketPosition); + @NotNull final Chunk workingChunk); @Override public void addChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, - @NotNull final Chunk values, - long bucketPosition) { + @NotNull final Chunk values) { final Context ctx = (Context) updateContext; if (ctx.canProcessDirectly) { ctx.loadDataChunks(inputKeys); - doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); - } - } - - @Override - public void addChunkBucketed(final @NotNull UpdateContext context, - final @NotNull Chunk values, - final @NotNull LongChunk keyChunk, - final @NotNull IntChunk bucketPositions, - final @NotNull IntChunk startPositions, - final @NotNull IntChunk runLengths) { - final Context ctx = (Context) context; - if (ctx.canProcessDirectly) { -// final ShortChunk asShorts = values.asShortChunk(); - for (int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int bucketPosition = bucketPositions.get(runStart); -// doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); - -// try (RowSequence rs = RowSequenceFactory.wrapRowKeysChunkAsRowSequence((LongChunk) keyChunk)_ - -// RowSetBuilderSequential builder = RowSetFactory.builderSequential(); -// for (int ii = runStart; ii < runStart + runLength; ii++) { -// builder.appendKey(keyChunk.get(ii)); -// } -// -// WritableRowSet bucketRs = bucketRowSet.get(bucketPosition); -// if (bucketRs == null) { -// bucketRs = builder.build(); -// bucketRowSet.set(bucketPosition, bucketRs); -// } else { -// try (final RowSet added = builder.build()) { -// bucketRs.insert(added); -// } -// } -// -// ctx.curVal = NULL_LONG; -// ctx.currentWindow.clear(); - -// accumulate(asShorts, (LongChunk) keyChunk, ctx, runStart, runLength, bucketRs); -// bucketLastVal.set(bucketPosition, ctx.curVal); - } + doAddChunk(ctx, inputKeys, keyChunk, values); } } @@ -374,14 +325,6 @@ public void resetForReprocess(@NotNull final UpdateContext context, ctx.workingRowSet = sourceRowSet; } - @Override - public void resetForReprocessBucketed(@NotNull final UpdateContext context, - @NotNull final RowSet bucketRowSet, - final long bucketPosition, - final long firstUnmodifiedKey) { - final Context ctx = (Context) context; - } - @Override public void reprocessChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @@ -390,22 +333,10 @@ public void reprocessChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; ctx.loadDataChunks(inputKeys); - doAddChunk(ctx, inputKeys, keyChunk, valuesChunk, 0); + doAddChunk(ctx, inputKeys, keyChunk, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } - @Override - public void reprocessChunkBucketed(@NotNull UpdateContext updateContext, - @NotNull final RowSequence chunkOk, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk runStartPositions, - @NotNull final IntChunk runLengths) { - addChunkBucketed(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); - ((Context)updateContext).getModifiedBuilder().appendRowSequence(chunkOk); - } - // endregion // region No-Op Operations @@ -415,15 +346,13 @@ final public void modifyChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk prevKeyChunk, @Nullable final LongChunk keyChunk, @NotNull final Chunk prevValuesChunk, - @NotNull final Chunk postValuesChunk, - long bucketPosition) { + @NotNull final Chunk postValuesChunk) { } @Override final public void removeChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk, - long bucketPosition) { + @NotNull final Chunk prevValuesChunk) { } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java index 3611e68a05d..fa3bd2f9a85 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java @@ -29,8 +29,6 @@ public abstract class BaseWindowedIntUpdateByOperator extends UpdateByWindowedOperator { protected final ColumnSource valueSource; - protected boolean initialized = false; - // region extra-fields // endregion extra-fields @@ -243,15 +241,13 @@ public BaseWindowedIntUpdateByOperator(@NotNull final MatchPair pair, public abstract void reset(UpdateContext context); @Override - public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, boolean usingBuckets, boolean isUpstreamAppendOnly) { + public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, final long lastPrevKey, boolean isUpstreamAppendOnly) { final Context ctx = (Context) context; ctx.workingRowSet = resultSourceRowSet; - if(!usingBuckets) { - // we can only process directly from an update if the window is entire backward-looking. Since we - // allow negative values in fwd/rev ticks and timestamps, we need to check both - ctx.canProcessDirectly = isUpstreamAppendOnly && forwardTimeScaleUnits <= 0 && reverseTimeScaleUnits >= 0;; - } + // we can only process directly from an update if the window is entire backward-looking. Since we + // allow negative values in fwd/rev ticks and timestamps, we need to check both + ctx.canProcessDirectly = isUpstreamAppendOnly && forwardTimeScaleUnits <= 0 && reverseTimeScaleUnits >= 0;; } @Override @@ -307,66 +303,21 @@ final public boolean anyModified(@NotNull final UpdateContext ctx) { * @param ctx the context object * @param inputKeys the input keys for the chunk * @param workingChunk the chunk of values - * @param bucketPosition the bucket position that the values belong to. */ protected abstract void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, - @NotNull final Chunk workingChunk, - final long bucketPosition); + @NotNull final Chunk workingChunk); @Override public void addChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, - @NotNull final Chunk values, - long bucketPosition) { + @NotNull final Chunk values) { final Context ctx = (Context) updateContext; if (ctx.canProcessDirectly) { ctx.loadDataChunks(inputKeys); - doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); - } - } - - @Override - public void addChunkBucketed(final @NotNull UpdateByOperator.UpdateContext context, - final @NotNull Chunk values, - final @NotNull LongChunk keyChunk, - final @NotNull IntChunk bucketPositions, - final @NotNull IntChunk startPositions, - final @NotNull IntChunk runLengths) { - final Context ctx = (Context) context; - if (ctx.canProcessDirectly) { -// final ShortChunk asShorts = values.asShortChunk(); - for (int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int bucketPosition = bucketPositions.get(runStart); -// doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); - -// try (RowSequence rs = RowSequenceFactory.wrapRowKeysChunkAsRowSequence((LongChunk) keyChunk)_ - -// RowSetBuilderSequential builder = RowSetFactory.builderSequential(); -// for (int ii = runStart; ii < runStart + runLength; ii++) { -// builder.appendKey(keyChunk.get(ii)); -// } -// -// WritableRowSet bucketRs = bucketRowSet.get(bucketPosition); -// if (bucketRs == null) { -// bucketRs = builder.build(); -// bucketRowSet.set(bucketPosition, bucketRs); -// } else { -// try (final RowSet added = builder.build()) { -// bucketRs.insert(added); -// } -// } -// -// ctx.curVal = NULL_LONG; -// ctx.currentWindow.clear(); - -// accumulate(asShorts, (LongChunk) keyChunk, ctx, runStart, runLength, bucketRs); -// bucketLastVal.set(bucketPosition, ctx.curVal); - } + doAddChunk(ctx, inputKeys, keyChunk, values); } } @@ -381,14 +332,6 @@ public void resetForReprocess(@NotNull final UpdateContext context, ctx.workingRowSet = sourceRowSet; } - @Override - public void resetForReprocessBucketed(@NotNull final UpdateContext context, - @NotNull final RowSet bucketRowSet, - final long bucketPosition, - final long firstUnmodifiedKey) { - final Context ctx = (Context) context; - } - @Override public void reprocessChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @@ -397,22 +340,10 @@ public void reprocessChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; ctx.loadDataChunks(inputKeys); - doAddChunk(ctx, inputKeys, keyChunk, valuesChunk, 0); + doAddChunk(ctx, inputKeys, keyChunk, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } - @Override - public void reprocessChunkBucketed(@NotNull UpdateContext updateContext, - @NotNull final RowSequence chunkOk, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk runStartPositions, - @NotNull final IntChunk runLengths) { - addChunkBucketed(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); - ((Context)updateContext).getModifiedBuilder().appendRowSequence(chunkOk); - } - // endregion // region No-Op Operations @@ -422,15 +353,13 @@ final public void modifyChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk prevKeyChunk, @Nullable final LongChunk keyChunk, @NotNull final Chunk prevValuesChunk, - @NotNull final Chunk postValuesChunk, - long bucketPosition) { + @NotNull final Chunk postValuesChunk) { } @Override final public void removeChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk, - long bucketPosition) { + @NotNull final Chunk prevValuesChunk) { } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java index 64ac0a8ea30..461b1aaf263 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java @@ -29,8 +29,6 @@ public abstract class BaseWindowedLongUpdateByOperator extends UpdateByWindowedOperator { protected final ColumnSource valueSource; - protected boolean initialized = false; - // region extra-fields // endregion extra-fields @@ -243,15 +241,13 @@ public BaseWindowedLongUpdateByOperator(@NotNull final MatchPair pair, public abstract void reset(UpdateContext context); @Override - public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, boolean usingBuckets, boolean isUpstreamAppendOnly) { + public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, final long lastPrevKey, boolean isUpstreamAppendOnly) { final Context ctx = (Context) context; ctx.workingRowSet = resultSourceRowSet; - if(!usingBuckets) { - // we can only process directly from an update if the window is entire backward-looking. Since we - // allow negative values in fwd/rev ticks and timestamps, we need to check both - ctx.canProcessDirectly = isUpstreamAppendOnly && forwardTimeScaleUnits <= 0 && reverseTimeScaleUnits >= 0;; - } + // we can only process directly from an update if the window is entire backward-looking. Since we + // allow negative values in fwd/rev ticks and timestamps, we need to check both + ctx.canProcessDirectly = isUpstreamAppendOnly && forwardTimeScaleUnits <= 0 && reverseTimeScaleUnits >= 0;; } @Override @@ -307,66 +303,21 @@ final public boolean anyModified(@NotNull final UpdateContext ctx) { * @param ctx the context object * @param inputKeys the input keys for the chunk * @param workingChunk the chunk of values - * @param bucketPosition the bucket position that the values belong to. */ protected abstract void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, - @NotNull final Chunk workingChunk, - final long bucketPosition); + @NotNull final Chunk workingChunk); @Override public void addChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, - @NotNull final Chunk values, - long bucketPosition) { + @NotNull final Chunk values) { final Context ctx = (Context) updateContext; if (ctx.canProcessDirectly) { ctx.loadDataChunks(inputKeys); - doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); - } - } - - @Override - public void addChunkBucketed(final @NotNull UpdateByOperator.UpdateContext context, - final @NotNull Chunk values, - final @NotNull LongChunk keyChunk, - final @NotNull IntChunk bucketPositions, - final @NotNull IntChunk startPositions, - final @NotNull IntChunk runLengths) { - final Context ctx = (Context) context; - if (ctx.canProcessDirectly) { -// final ShortChunk asShorts = values.asShortChunk(); - for (int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int bucketPosition = bucketPositions.get(runStart); -// doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); - -// try (RowSequence rs = RowSequenceFactory.wrapRowKeysChunkAsRowSequence((LongChunk) keyChunk)_ - -// RowSetBuilderSequential builder = RowSetFactory.builderSequential(); -// for (int ii = runStart; ii < runStart + runLength; ii++) { -// builder.appendKey(keyChunk.get(ii)); -// } -// -// WritableRowSet bucketRs = bucketRowSet.get(bucketPosition); -// if (bucketRs == null) { -// bucketRs = builder.build(); -// bucketRowSet.set(bucketPosition, bucketRs); -// } else { -// try (final RowSet added = builder.build()) { -// bucketRs.insert(added); -// } -// } -// -// ctx.curVal = NULL_LONG; -// ctx.currentWindow.clear(); - -// accumulate(asShorts, (LongChunk) keyChunk, ctx, runStart, runLength, bucketRs); -// bucketLastVal.set(bucketPosition, ctx.curVal); - } + doAddChunk(ctx, inputKeys, keyChunk, values); } } @@ -381,14 +332,6 @@ public void resetForReprocess(@NotNull final UpdateContext context, ctx.workingRowSet = sourceRowSet; } - @Override - public void resetForReprocessBucketed(@NotNull final UpdateContext context, - @NotNull final RowSet bucketRowSet, - final long bucketPosition, - final long firstUnmodifiedKey) { - final Context ctx = (Context) context; - } - @Override public void reprocessChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @@ -397,22 +340,10 @@ public void reprocessChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; ctx.loadDataChunks(inputKeys); - doAddChunk(ctx, inputKeys, keyChunk, valuesChunk, 0); + doAddChunk(ctx, inputKeys, keyChunk, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } - @Override - public void reprocessChunkBucketed(@NotNull UpdateContext updateContext, - @NotNull final RowSequence chunkOk, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk runStartPositions, - @NotNull final IntChunk runLengths) { - addChunkBucketed(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); - ((Context)updateContext).getModifiedBuilder().appendRowSequence(chunkOk); - } - // endregion // region No-Op Operations @@ -422,15 +353,13 @@ final public void modifyChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk prevKeyChunk, @Nullable final LongChunk keyChunk, @NotNull final Chunk prevValuesChunk, - @NotNull final Chunk postValuesChunk, - long bucketPosition) { + @NotNull final Chunk postValuesChunk) { } @Override final public void removeChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk, - long bucketPosition) { + @NotNull final Chunk prevValuesChunk) { } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java index eee6c81c0a6..6119af4a8f2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java @@ -29,8 +29,6 @@ public abstract class BaseWindowedObjectUpdateByOperator extends UpdateByWindowedOperator { protected final ColumnSource valueSource; - protected boolean initialized = false; - // region extra-fields private final Class colType; // endregion extra-fields @@ -246,15 +244,13 @@ public BaseWindowedObjectUpdateByOperator(@NotNull final MatchPair pair, public abstract void reset(UpdateContext context); @Override - public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, boolean usingBuckets, boolean isUpstreamAppendOnly) { + public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, final long lastPrevKey, boolean isUpstreamAppendOnly) { final Context ctx = (Context) context; ctx.workingRowSet = resultSourceRowSet; - if(!usingBuckets) { - // we can only process directly from an update if the window is entire backward-looking. Since we - // allow negative values in fwd/rev ticks and timestamps, we need to check both - ctx.canProcessDirectly = isUpstreamAppendOnly && forwardTimeScaleUnits <= 0 && reverseTimeScaleUnits >= 0;; - } + // we can only process directly from an update if the window is entire backward-looking. Since we + // allow negative values in fwd/rev ticks and timestamps, we need to check both + ctx.canProcessDirectly = isUpstreamAppendOnly && forwardTimeScaleUnits <= 0 && reverseTimeScaleUnits >= 0;; } @Override @@ -310,66 +306,21 @@ final public boolean anyModified(@NotNull final UpdateContext ctx) { * @param ctx the context object * @param inputKeys the input keys for the chunk * @param workingChunk the chunk of values - * @param bucketPosition the bucket position that the values belong to. */ protected abstract void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, - @NotNull final Chunk workingChunk, - final long bucketPosition); + @NotNull final Chunk workingChunk); @Override public void addChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, - @NotNull final Chunk values, - long bucketPosition) { + @NotNull final Chunk values) { final Context ctx = (Context) updateContext; if (ctx.canProcessDirectly) { ctx.loadDataChunks(inputKeys); - doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); - } - } - - @Override - public void addChunkBucketed(final @NotNull UpdateByOperator.UpdateContext context, - final @NotNull Chunk values, - final @NotNull LongChunk keyChunk, - final @NotNull IntChunk bucketPositions, - final @NotNull IntChunk startPositions, - final @NotNull IntChunk runLengths) { - final Context ctx = (Context) context; - if (ctx.canProcessDirectly) { -// final ShortChunk asShorts = values.asShortChunk(); - for (int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int bucketPosition = bucketPositions.get(runStart); -// doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); - -// try (RowSequence rs = RowSequenceFactory.wrapRowKeysChunkAsRowSequence((LongChunk) keyChunk)_ - -// RowSetBuilderSequential builder = RowSetFactory.builderSequential(); -// for (int ii = runStart; ii < runStart + runLength; ii++) { -// builder.appendKey(keyChunk.get(ii)); -// } -// -// WritableRowSet bucketRs = bucketRowSet.get(bucketPosition); -// if (bucketRs == null) { -// bucketRs = builder.build(); -// bucketRowSet.set(bucketPosition, bucketRs); -// } else { -// try (final RowSet added = builder.build()) { -// bucketRs.insert(added); -// } -// } -// -// ctx.curVal = NULL_LONG; -// ctx.currentWindow.clear(); - -// accumulate(asShorts, (LongChunk) keyChunk, ctx, runStart, runLength, bucketRs); -// bucketLastVal.set(bucketPosition, ctx.curVal); - } + doAddChunk(ctx, inputKeys, keyChunk, values); } } @@ -384,14 +335,6 @@ public void resetForReprocess(@NotNull final UpdateContext context, ctx.workingRowSet = sourceRowSet; } - @Override - public void resetForReprocessBucketed(@NotNull final UpdateContext context, - @NotNull final RowSet bucketRowSet, - final long bucketPosition, - final long firstUnmodifiedKey) { - final Context ctx = (Context) context; - } - @Override public void reprocessChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @@ -400,22 +343,10 @@ public void reprocessChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; ctx.loadDataChunks(inputKeys); - doAddChunk(ctx, inputKeys, keyChunk, valuesChunk, 0); + doAddChunk(ctx, inputKeys, keyChunk, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } - @Override - public void reprocessChunkBucketed(@NotNull UpdateContext updateContext, - @NotNull final RowSequence chunkOk, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk runStartPositions, - @NotNull final IntChunk runLengths) { - addChunkBucketed(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); - ((Context)updateContext).getModifiedBuilder().appendRowSequence(chunkOk); - } - // endregion // region No-Op Operations @@ -425,15 +356,13 @@ final public void modifyChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk prevKeyChunk, @Nullable final LongChunk keyChunk, @NotNull final Chunk prevValuesChunk, - @NotNull final Chunk postValuesChunk, - long bucketPosition) { + @NotNull final Chunk postValuesChunk) { } @Override final public void removeChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk, - long bucketPosition) { + @NotNull final Chunk prevValuesChunk) { } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java index 7c1c05eacd4..a3ebf3b3670 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java @@ -29,8 +29,6 @@ public abstract class BaseWindowedShortUpdateByOperator extends UpdateByWindowedOperator { protected final ColumnSource valueSource; - protected boolean initialized = false; - // region extra-fields // endregion extra-fields @@ -243,15 +241,13 @@ public BaseWindowedShortUpdateByOperator(@NotNull final MatchPair pair, public abstract void reset(UpdateContext context); @Override - public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, boolean usingBuckets, boolean isUpstreamAppendOnly) { + public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, final long lastPrevKey, boolean isUpstreamAppendOnly) { final Context ctx = (Context) context; ctx.workingRowSet = resultSourceRowSet; - if(!usingBuckets) { - // we can only process directly from an update if the window is entire backward-looking. Since we - // allow negative values in fwd/rev ticks and timestamps, we need to check both - ctx.canProcessDirectly = isUpstreamAppendOnly && forwardTimeScaleUnits <= 0 && reverseTimeScaleUnits >= 0;; - } + // we can only process directly from an update if the window is entire backward-looking. Since we + // allow negative values in fwd/rev ticks and timestamps, we need to check both + ctx.canProcessDirectly = isUpstreamAppendOnly && forwardTimeScaleUnits <= 0 && reverseTimeScaleUnits >= 0;; } @Override @@ -307,66 +303,21 @@ final public boolean anyModified(@NotNull final UpdateContext ctx) { * @param ctx the context object * @param inputKeys the input keys for the chunk * @param workingChunk the chunk of values - * @param bucketPosition the bucket position that the values belong to. */ protected abstract void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, - @NotNull final Chunk workingChunk, - final long bucketPosition); + @NotNull final Chunk workingChunk); @Override public void addChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, - @NotNull final Chunk values, - long bucketPosition) { + @NotNull final Chunk values) { final Context ctx = (Context) updateContext; if (ctx.canProcessDirectly) { ctx.loadDataChunks(inputKeys); - doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); - } - } - - @Override - public void addChunkBucketed(final @NotNull UpdateByOperator.UpdateContext context, - final @NotNull Chunk values, - final @NotNull LongChunk keyChunk, - final @NotNull IntChunk bucketPositions, - final @NotNull IntChunk startPositions, - final @NotNull IntChunk runLengths) { - final Context ctx = (Context) context; - if (ctx.canProcessDirectly) { -// final ShortChunk asShorts = values.asShortChunk(); - for (int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int bucketPosition = bucketPositions.get(runStart); -// doAddChunk(ctx, inputKeys, keyChunk, values, bucketPosition); - -// try (RowSequence rs = RowSequenceFactory.wrapRowKeysChunkAsRowSequence((LongChunk) keyChunk)_ - -// RowSetBuilderSequential builder = RowSetFactory.builderSequential(); -// for (int ii = runStart; ii < runStart + runLength; ii++) { -// builder.appendKey(keyChunk.get(ii)); -// } -// -// WritableRowSet bucketRs = bucketRowSet.get(bucketPosition); -// if (bucketRs == null) { -// bucketRs = builder.build(); -// bucketRowSet.set(bucketPosition, bucketRs); -// } else { -// try (final RowSet added = builder.build()) { -// bucketRs.insert(added); -// } -// } -// -// ctx.curVal = NULL_LONG; -// ctx.currentWindow.clear(); - -// accumulate(asShorts, (LongChunk) keyChunk, ctx, runStart, runLength, bucketRs); -// bucketLastVal.set(bucketPosition, ctx.curVal); - } + doAddChunk(ctx, inputKeys, keyChunk, values); } } @@ -381,14 +332,6 @@ public void resetForReprocess(@NotNull final UpdateContext context, ctx.workingRowSet = sourceRowSet; } - @Override - public void resetForReprocessBucketed(@NotNull final UpdateContext context, - @NotNull final RowSet bucketRowSet, - final long bucketPosition, - final long firstUnmodifiedKey) { - final Context ctx = (Context) context; - } - @Override public void reprocessChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @@ -397,22 +340,10 @@ public void reprocessChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; ctx.loadDataChunks(inputKeys); - doAddChunk(ctx, inputKeys, keyChunk, valuesChunk, 0); + doAddChunk(ctx, inputKeys, keyChunk, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } - @Override - public void reprocessChunkBucketed(@NotNull UpdateContext updateContext, - @NotNull final RowSequence chunkOk, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk runStartPositions, - @NotNull final IntChunk runLengths) { - addChunkBucketed(updateContext, values, keyChunk, bucketPositions, runStartPositions, runLengths); - ((Context)updateContext).getModifiedBuilder().appendRowSequence(chunkOk); - } - // endregion // region No-Op Operations @@ -422,15 +353,13 @@ final public void modifyChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk prevKeyChunk, @Nullable final LongChunk keyChunk, @NotNull final Chunk prevValuesChunk, - @NotNull final Chunk postValuesChunk, - long bucketPosition) { + @NotNull final Chunk postValuesChunk) { } @Override final public void removeChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk, - long bucketPosition) { + @NotNull final Chunk prevValuesChunk) { } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/LongRecordingUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/LongRecordingUpdateByOperator.java index f8c26d7f6a9..9b652d01b1d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/LongRecordingUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/LongRecordingUpdateByOperator.java @@ -87,11 +87,6 @@ public void reprocessChunk(@NotNull UpdateContext updateContext, @NotNull RowSeq currentContext.addedChunk = valuesChunk.asLongChunk(); } - @Override - public void reprocessChunkBucketed(@NotNull UpdateContext updateContext, @NotNull RowSequence chunkOk, @NotNull Chunk values, @NotNull LongChunk keyChunk, @NotNull IntChunk bucketPositions, @NotNull IntChunk runStartPositions, @NotNull IntChunk runLengths) { - currentContext.addedChunk = values.asLongChunk(); - } - @NotNull @Override public UpdateContext makeUpdateContext(int chunkSize) { @@ -102,25 +97,11 @@ public UpdateContext makeUpdateContext(int chunkSize) { public void addChunk(@NotNull UpdateContext updateContext, @NotNull RowSequence inputKeys, @Nullable LongChunk keyChunk, - @NotNull Chunk values, - long bucketPosition) { - currentContext.addedChunk = values.asLongChunk(); - } - - @Override - public void addChunkBucketed(@NotNull UpdateContext context, - @NotNull Chunk values, - @NotNull LongChunk keyChunk, - @NotNull IntChunk bucketPositions, - @NotNull IntChunk startPositions, - @NotNull IntChunk runLengths) { + @NotNull Chunk values) { currentContext.addedChunk = values.asLongChunk(); } // region Unused methods - @Override - public void setBucketCapacity(int capacity) { - } @NotNull @Override @@ -147,7 +128,7 @@ public Map> getOutputColumns() { } @Override - public void initializeForUpdate(@NotNull UpdateContext ctx, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceIndex, boolean usingBuckets, boolean isUpstreamAppendOnly) { + public void initializeForUpdate(@NotNull UpdateContext ctx, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceIndex, final long key, boolean isUpstreamAppendOnly) { } @Override @@ -194,17 +175,12 @@ public void setChunkSize(@NotNull UpdateContext context, int chunkSize) { } @Override - public void onBucketsRemoved(@NotNull RowSet removedBuckets) { - - } - - @Override - public void modifyChunk(@NotNull UpdateContext updateContext, @Nullable LongChunk prevKeyChunk, @Nullable LongChunk keyChunk, @NotNull Chunk prevValuesChunk, @NotNull Chunk postValuesChunk, long bucketPosition) { + public void modifyChunk(@NotNull UpdateContext updateContext, @Nullable LongChunk prevKeyChunk, @Nullable LongChunk keyChunk, @NotNull Chunk prevValuesChunk, @NotNull Chunk postValuesChunk) { } @Override - public void removeChunk(@NotNull UpdateContext updateContext, @Nullable LongChunk keyChunk, @NotNull Chunk prevValuesChunk, long bucketPosition) { + public void removeChunk(@NotNull UpdateContext updateContext, @Nullable LongChunk keyChunk, @NotNull Chunk prevValuesChunk) { } @@ -223,10 +199,5 @@ public void resetForReprocess(@NotNull UpdateContext context, @NotNull RowSet so } - @Override - public void resetForReprocessBucketed(@NotNull UpdateContext context, @NotNull RowSet bucketIndex, long bucketPosition, long firstUnmodifiedKey) { - - } - // endregion } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java index 7da71c87883..e64580b614f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java @@ -40,38 +40,11 @@ public ByteCumMinMaxOperator(@NotNull final MatchPair inputPair, // region extra-methods // endregion extra-methods - @Override - public void addChunkBucketed(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { - final ByteChunk asBytes = values.asByteChunk(); - final Context ctx = (Context) context; - for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int bucketPosition = bucketPositions.get(runStart); - - ctx.curVal = bucketLastVal.getByte(bucketPosition); - accumulate(asBytes, ctx, runStart, runLength); - bucketLastVal.set(bucketPosition, ctx.curVal); - } - - //noinspection unchecked - outputSource.fillFromChunkUnordered(ctx.fillContext.get(), ctx.outputValues.get(), (LongChunk) keyChunk); - } - @Override protected void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - long groupPosition) { - ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_BYTE; + @NotNull final Chunk workingChunk) { accumulate(workingChunk.asByteChunk(), ctx, 0, workingChunk.size()); - singletonGroup = groupPosition; - singletonVal = ctx.curVal; outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java index 67a73e77466..739ece75eb8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java @@ -34,43 +34,10 @@ public DoubleCumMinMaxOperator(@NotNull final MatchPair inputPair, // endregion constructor } - @Override - public void addChunkBucketed(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { - final DoubleChunk asDoubles = values.asDoubleChunk(); - final Context ctx = (Context) context; - for (int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int bucketPosition = bucketPositions.get(runStart); - - ctx.curVal = bucketLastVal.getDouble(bucketPosition); - if (Double.isNaN(ctx.curVal) || Double.isInfinite(ctx.curVal)) { - ctx.outputValues.get().fillWithValue(runStart, runLength, ctx.curVal); - } else { - accumulateMinMax(asDoubles, ctx, runStart, runLength); - bucketLastVal.set(bucketPosition, ctx.curVal); - } - } - //noinspection unchecked - outputSource.fillFromChunkUnordered(ctx.fillContext.get(), ctx.outputValues.get(), (LongChunk) keyChunk); - } - @Override protected void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - long groupPosition) { - ctx.curVal = singletonGroup == groupPosition ? singletonVal : NULL_DOUBLE; - if(ctx.lastGroupPosition != groupPosition) { - ctx.lastGroupPosition = groupPosition; - ctx.filledWithPermanentValue = false; - } - + @NotNull final Chunk workingChunk) { if(Double.isNaN(ctx.curVal) || Double.isInfinite(ctx.curVal)) { if(!ctx.filledWithPermanentValue) { ctx.filledWithPermanentValue = true; @@ -79,9 +46,6 @@ protected void doAddChunk(@NotNull final Context ctx, } else { accumulateMinMax(workingChunk.asDoubleChunk(), ctx, 0, workingChunk.size()); } - - singletonGroup = groupPosition; - singletonVal = ctx.curVal; outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java index 46eebe3cf8d..43812d6c4ba 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java @@ -29,43 +29,10 @@ public FloatCumMinMaxOperator(@NotNull final MatchPair inputPair, // endregion constructor } - @Override - public void addChunkBucketed(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { - final FloatChunk asFloats = values.asFloatChunk(); - final Context ctx = (Context) context; - for (int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int bucketPosition = bucketPositions.get(runStart); - - ctx.curVal = bucketLastVal.getFloat(bucketPosition); - if (Float.isNaN(ctx.curVal) || Float.isInfinite(ctx.curVal)) { - ctx.outputValues.get().fillWithValue(runStart, runLength, ctx.curVal); - } else { - accumulateMinMax(asFloats, ctx, runStart, runLength); - bucketLastVal.set(bucketPosition, ctx.curVal); - } - } - //noinspection unchecked - outputSource.fillFromChunkUnordered(ctx.fillContext.get(), ctx.outputValues.get(), (LongChunk) keyChunk); - } - @Override protected void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - long groupPosition) { - ctx.curVal = singletonGroup == groupPosition ? singletonVal : NULL_FLOAT; - if(ctx.lastGroupPosition != groupPosition) { - ctx.lastGroupPosition = groupPosition; - ctx.filledWithPermanentValue = false; - } - + @NotNull final Chunk workingChunk) { if(Float.isNaN(ctx.curVal) || Float.isInfinite(ctx.curVal)) { if(!ctx.filledWithPermanentValue) { ctx.filledWithPermanentValue = true; @@ -74,9 +41,6 @@ protected void doAddChunk(@NotNull final Context ctx, } else { accumulateMinMax(workingChunk.asFloatChunk(), ctx, 0, workingChunk.size()); } - - singletonGroup = groupPosition; - singletonVal = ctx.curVal; outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java index 6bf964156ca..fe7b0d2dc6c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java @@ -40,38 +40,11 @@ public IntCumMinMaxOperator(@NotNull final MatchPair inputPair, // region extra-methods // endregion extra-methods - @Override - public void addChunkBucketed(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { - final IntChunk asIntegers = values.asIntChunk(); - final Context ctx = (Context) context; - for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int bucketPosition = bucketPositions.get(runStart); - - ctx.curVal = bucketLastVal.getInt(bucketPosition); - accumulate(asIntegers, ctx, runStart, runLength); - bucketLastVal.set(bucketPosition, ctx.curVal); - } - - //noinspection unchecked - outputSource.fillFromChunkUnordered(ctx.fillContext.get(), ctx.outputValues.get(), (LongChunk) keyChunk); - } - @Override protected void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - long groupPosition) { - ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_INT; + @NotNull final Chunk workingChunk) { accumulate(workingChunk.asIntChunk(), ctx, 0, workingChunk.size()); - singletonGroup = groupPosition; - singletonVal = ctx.curVal; outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java index 277ea9b8e21..ebfa116a903 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java @@ -61,38 +61,11 @@ public Map> getOutputColumns() { } // endregion extra-methods - @Override - public void addChunkBucketed(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { - final LongChunk asLongs = values.asLongChunk(); - final Context ctx = (Context) context; - for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int bucketPosition = bucketPositions.get(runStart); - - ctx.curVal = bucketLastVal.getLong(bucketPosition); - accumulate(asLongs, ctx, runStart, runLength); - bucketLastVal.set(bucketPosition, ctx.curVal); - } - - //noinspection unchecked - outputSource.fillFromChunkUnordered(ctx.fillContext.get(), ctx.outputValues.get(), (LongChunk) keyChunk); - } - @Override protected void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - long groupPosition) { - ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_LONG; + @NotNull final Chunk workingChunk) { accumulate(workingChunk.asLongChunk(), ctx, 0, workingChunk.size()); - singletonGroup = groupPosition; - singletonVal = ctx.curVal; outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java index 99fee944c04..65505be8359 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java @@ -35,38 +35,11 @@ public ShortCumMinMaxOperator(@NotNull final MatchPair inputPair, // region extra-methods // endregion extra-methods - @Override - public void addChunkBucketed(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { - final ShortChunk asShorts = values.asShortChunk(); - final Context ctx = (Context) context; - for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int bucketPosition = bucketPositions.get(runStart); - - ctx.curVal = bucketLastVal.getShort(bucketPosition); - accumulate(asShorts, ctx, runStart, runLength); - bucketLastVal.set(bucketPosition, ctx.curVal); - } - - //noinspection unchecked - outputSource.fillFromChunkUnordered(ctx.fillContext.get(), ctx.outputValues.get(), (LongChunk) keyChunk); - } - @Override protected void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - long groupPosition) { - ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_SHORT; + @NotNull final Chunk workingChunk) { accumulate(workingChunk.asShortChunk(), ctx, 0, workingChunk.size()); - singletonGroup = groupPosition; - singletonVal = ctx.curVal; outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java index 6f9373d80e6..348e7639fb6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java @@ -32,38 +32,14 @@ public ByteCumProdOperator(final @NotNull MatchPair inputPair, // endregion constructor } - @Override - public void addChunkBucketed(final @NotNull UpdateContext context, - final @NotNull Chunk values, - final @NotNull LongChunk keyChunk, - final @NotNull IntChunk bucketPositions, - final @NotNull IntChunk startPositions, - final @NotNull IntChunk runLengths) { - - final Context ctx = (Context) context; - final ByteChunk asBytes = values.asByteChunk(); - for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int bucketPosition = bucketPositions.get(runStart); - - ctx.curVal = bucketLastVal.getLong(bucketPosition); - accumulate(asBytes, ctx, runStart, runLength); - bucketLastVal.set(bucketPosition, ctx.curVal); - } - //noinspection unchecked - outputSource.fillFromChunkUnordered(ctx.fillContext.get(), ctx.outputValues.get(), (LongChunk) keyChunk); - } - @Override protected void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - final long groupPosition) { - ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_LONG; + @NotNull final Chunk workingChunk) { +// ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_LONG; accumulate(workingChunk.asByteChunk(), ctx, 0, workingChunk.size()); - singletonGroup = groupPosition; - singletonVal = ctx.curVal; +// singletonGroup = groupPosition; +// singletonVal = ctx.curVal; outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java index 6b30cf650e6..4571df8cd30 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java @@ -31,41 +31,9 @@ public DoubleCumProdOperator(@NotNull final MatchPair inputPair, // endregion constructor } - @Override - public void addChunkBucketed(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { - final Context ctx = (Context) context; - for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int bucketPosition = bucketPositions.get(runStart); - - ctx.curVal = bucketLastVal.getDouble(bucketPosition); - if(Double.isNaN(ctx.curVal) || Double.isInfinite(ctx.curVal)) { - ctx.outputValues.get().fillWithValue(runStart, runLength, ctx.curVal); - } else { - accumulate(values, ctx, runStart, runLength); - bucketLastVal.set(bucketPosition, ctx.curVal); - } - } - //noinspection unchecked - outputSource.fillFromChunkUnordered(ctx.fillContext.get(), ctx.outputValues.get(), (LongChunk) keyChunk); - } - protected void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - final long groupPosition) { - ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_DOUBLE; - if(ctx.lastGroupPosition != groupPosition) { - ctx.lastGroupPosition = groupPosition; - ctx.filledWithPermanentValue = false; - } - + @NotNull final Chunk workingChunk) { if(Double.isNaN(ctx.curVal)) { if(!ctx.filledWithPermanentValue) { ctx.filledWithPermanentValue = true; @@ -75,8 +43,6 @@ protected void doAddChunk(@NotNull final Context ctx, accumulate(workingChunk.asDoubleChunk(), ctx, 0, workingChunk.size()); } - singletonGroup = groupPosition; - singletonVal = ctx.curVal; outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java index 87292ea984e..5797c9b771e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java @@ -26,41 +26,9 @@ public FloatCumProdOperator(@NotNull final MatchPair inputPair, // endregion constructor } - @Override - public void addChunkBucketed(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { - final Context ctx = (Context) context; - for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int bucketPosition = bucketPositions.get(runStart); - - ctx.curVal = bucketLastVal.getFloat(bucketPosition); - if(Float.isNaN(ctx.curVal) || Float.isInfinite(ctx.curVal)) { - ctx.outputValues.get().fillWithValue(runStart, runLength, ctx.curVal); - } else { - accumulate(values, ctx, runStart, runLength); - bucketLastVal.set(bucketPosition, ctx.curVal); - } - } - //noinspection unchecked - outputSource.fillFromChunkUnordered(ctx.fillContext.get(), ctx.outputValues.get(), (LongChunk) keyChunk); - } - protected void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - final long groupPosition) { - ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_FLOAT; - if(ctx.lastGroupPosition != groupPosition) { - ctx.lastGroupPosition = groupPosition; - ctx.filledWithPermanentValue = false; - } - + @NotNull final Chunk workingChunk) { if(Float.isNaN(ctx.curVal)) { if(!ctx.filledWithPermanentValue) { ctx.filledWithPermanentValue = true; @@ -70,8 +38,6 @@ protected void doAddChunk(@NotNull final Context ctx, accumulate(workingChunk.asFloatChunk(), ctx, 0, workingChunk.size()); } - singletonGroup = groupPosition; - singletonVal = ctx.curVal; outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java index b0a2346f0fd..0846e1ab972 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java @@ -32,38 +32,14 @@ public IntCumProdOperator(final @NotNull MatchPair inputPair, // endregion constructor } - @Override - public void addChunkBucketed(final @NotNull UpdateContext context, - final @NotNull Chunk values, - final @NotNull LongChunk keyChunk, - final @NotNull IntChunk bucketPositions, - final @NotNull IntChunk startPositions, - final @NotNull IntChunk runLengths) { - - final Context ctx = (Context) context; - final IntChunk asIntegers = values.asIntChunk(); - for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int bucketPosition = bucketPositions.get(runStart); - - ctx.curVal = bucketLastVal.getLong(bucketPosition); - accumulate(asIntegers, ctx, runStart, runLength); - bucketLastVal.set(bucketPosition, ctx.curVal); - } - //noinspection unchecked - outputSource.fillFromChunkUnordered(ctx.fillContext.get(), ctx.outputValues.get(), (LongChunk) keyChunk); - } - @Override protected void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - final long groupPosition) { - ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_LONG; + @NotNull final Chunk workingChunk) { +// ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_LONG; accumulate(workingChunk.asIntChunk(), ctx, 0, workingChunk.size()); - singletonGroup = groupPosition; - singletonVal = ctx.curVal; +// singletonGroup = groupPosition; +// singletonVal = ctx.curVal; outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java index ce7931bc19f..ecc59d8e0d9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java @@ -32,38 +32,14 @@ public LongCumProdOperator(final @NotNull MatchPair inputPair, // endregion constructor } - @Override - public void addChunkBucketed(final @NotNull UpdateContext context, - final @NotNull Chunk values, - final @NotNull LongChunk keyChunk, - final @NotNull IntChunk bucketPositions, - final @NotNull IntChunk startPositions, - final @NotNull IntChunk runLengths) { - - final Context ctx = (Context) context; - final LongChunk asLongs = values.asLongChunk(); - for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int bucketPosition = bucketPositions.get(runStart); - - ctx.curVal = bucketLastVal.getLong(bucketPosition); - accumulate(asLongs, ctx, runStart, runLength); - bucketLastVal.set(bucketPosition, ctx.curVal); - } - //noinspection unchecked - outputSource.fillFromChunkUnordered(ctx.fillContext.get(), ctx.outputValues.get(), (LongChunk) keyChunk); - } - @Override protected void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - final long groupPosition) { - ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_LONG; + @NotNull final Chunk workingChunk) { +// ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_LONG; accumulate(workingChunk.asLongChunk(), ctx, 0, workingChunk.size()); - singletonGroup = groupPosition; - singletonVal = ctx.curVal; +// singletonGroup = groupPosition; +// singletonVal = ctx.curVal; outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java index ea90dd54c9d..a7e41777bf4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java @@ -27,38 +27,14 @@ public ShortCumProdOperator(final @NotNull MatchPair inputPair, // endregion constructor } - @Override - public void addChunkBucketed(final @NotNull UpdateContext context, - final @NotNull Chunk values, - final @NotNull LongChunk keyChunk, - final @NotNull IntChunk bucketPositions, - final @NotNull IntChunk startPositions, - final @NotNull IntChunk runLengths) { - - final Context ctx = (Context) context; - final ShortChunk asShorts = values.asShortChunk(); - for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int bucketPosition = bucketPositions.get(runStart); - - ctx.curVal = bucketLastVal.getLong(bucketPosition); - accumulate(asShorts, ctx, runStart, runLength); - bucketLastVal.set(bucketPosition, ctx.curVal); - } - //noinspection unchecked - outputSource.fillFromChunkUnordered(ctx.fillContext.get(), ctx.outputValues.get(), (LongChunk) keyChunk); - } - @Override protected void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - final long groupPosition) { - ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_LONG; + @NotNull final Chunk workingChunk) { +// ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_LONG; accumulate(workingChunk.asShortChunk(), ctx, 0, workingChunk.size()); - singletonGroup = groupPosition; - singletonVal = ctx.curVal; +// singletonGroup = groupPosition; +// singletonVal = ctx.curVal; outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java index 4ba0573ab15..d037f6857cc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java @@ -50,6 +50,8 @@ protected class Context extends BaseWindowedByteUpdateByOperator.Context { public LinkedList windowValues = new LinkedList<>(); + public long currentVal = NULL_LONG; + protected Context(final int chunkSize) { this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); this.fillContext.ensureCapacity(chunkSize); @@ -121,14 +123,14 @@ public void pop(UpdateContext context, long key) { @Override public void reset(UpdateContext context) { final Context ctx = (Context) context; + ctx.currentVal = NULL_LONG; } @Override public void doAddChunk(@NotNull final BaseWindowedByteUpdateByOperator.Context context, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, - @NotNull final Chunk workingChunk, - final long groupPosition) { + @NotNull final Chunk workingChunk) { final Context ctx = (Context) context; computeTicks(ctx, 0, inputKeys.intSize()); @@ -160,7 +162,7 @@ private void computeTicks(@NotNull final Context ctx, // this call generates the push/pop calls to satisfy the window // ctx.fillWindow(key, postUpdateSourceIndex); - localOutputValues.set(ii, sum.getValue()); + localOutputValues.set(ii, ctx.currentVal); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java index d8965f97a97..1e790bf6244 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java @@ -128,8 +128,7 @@ public void reset(UpdateContext context) { public void doAddChunk(@NotNull final BaseWindowedDoubleUpdateByOperator.Context context, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, - @NotNull final Chunk workingChunk, - final long groupPosition) { + @NotNull final Chunk workingChunk) { final Context ctx = (Context) context; computeTicks(ctx, 0, inputKeys.intSize()); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java index 271e8870a36..30cc3c16673 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java @@ -123,8 +123,7 @@ public void reset(UpdateContext context) { public void doAddChunk(@NotNull final BaseWindowedFloatUpdateByOperator.Context context, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, - @NotNull final Chunk workingChunk, - final long groupPosition) { + @NotNull final Chunk workingChunk) { final Context ctx = (Context) context; computeTicks(ctx, 0, inputKeys.intSize()); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java index 4df3f391ba5..1e754568627 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java @@ -49,6 +49,8 @@ protected class Context extends BaseWindowedIntUpdateByOperator.Context { public LinkedList windowValues = new LinkedList<>(); + public long currentVal = NULL_LONG; + protected Context(final int chunkSize) { this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); this.fillContext.ensureCapacity(chunkSize); @@ -118,14 +120,14 @@ public void pop(UpdateContext context, long key) { @Override public void reset(UpdateContext context) { final Context ctx = (Context) context; + ctx.currentVal = NULL_LONG; } @Override public void doAddChunk(@NotNull final BaseWindowedIntUpdateByOperator.Context context, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, - @NotNull final Chunk workingChunk, - final long groupPosition) { + @NotNull final Chunk workingChunk) { final Context ctx = (Context) context; computeTicks(ctx, 0, inputKeys.intSize()); @@ -157,7 +159,7 @@ private void computeTicks(@NotNull final Context ctx, // this call generates the push/pop calls to satisfy the window // ctx.fillWindow(key, postUpdateSourceIndex); - localOutputValues.set(ii, sum.getValue()); + localOutputValues.set(ii, ctx.currentVal); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java index aee9474b61e..2dd16aea186 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java @@ -49,6 +49,8 @@ protected class Context extends BaseWindowedLongUpdateByOperator.Context { public LinkedList windowValues = new LinkedList<>(); + public long currentVal = NULL_LONG; + protected Context(final int chunkSize) { this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); this.fillContext.ensureCapacity(chunkSize); @@ -118,14 +120,14 @@ public void pop(UpdateContext context, long key) { @Override public void reset(UpdateContext context) { final Context ctx = (Context) context; + ctx.currentVal = NULL_LONG; } @Override public void doAddChunk(@NotNull final BaseWindowedLongUpdateByOperator.Context context, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, - @NotNull final Chunk workingChunk, - final long groupPosition) { + @NotNull final Chunk workingChunk) { final Context ctx = (Context) context; computeTicks(ctx, 0, inputKeys.intSize()); @@ -157,7 +159,7 @@ private void computeTicks(@NotNull final Context ctx, // this call generates the push/pop calls to satisfy the window // ctx.fillWindow(key, postUpdateSourceIndex); - localOutputValues.set(ii, sum.getValue()); + localOutputValues.set(ii, ctx.currentVal); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java index fc13fdd56a3..7d9d96711f2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java @@ -44,6 +44,8 @@ protected class Context extends BaseWindowedShortUpdateByOperator.Context { public LinkedList windowValues = new LinkedList<>(); + public long currentVal = NULL_LONG; + protected Context(final int chunkSize) { this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); this.fillContext.ensureCapacity(chunkSize); @@ -113,14 +115,14 @@ public void pop(UpdateContext context, long key) { @Override public void reset(UpdateContext context) { final Context ctx = (Context) context; + ctx.currentVal = NULL_LONG; } @Override public void doAddChunk(@NotNull final BaseWindowedShortUpdateByOperator.Context context, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, - @NotNull final Chunk workingChunk, - final long groupPosition) { + @NotNull final Chunk workingChunk) { final Context ctx = (Context) context; computeTicks(ctx, 0, inputKeys.intSize()); @@ -152,7 +154,7 @@ private void computeTicks(@NotNull final Context ctx, // this call generates the push/pop calls to satisfy the window // ctx.fillWindow(key, postUpdateSourceIndex); - localOutputValues.set(ii, sum.getValue()); + localOutputValues.set(ii, ctx.currentVal); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java index 9429d708dee..557dfb2c71c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java @@ -37,38 +37,14 @@ public ByteCumSumOperator(@NotNull final MatchPair pair, // endregion constructor } - @Override - public void addChunkBucketed(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { - final ByteChunk asBytes = values.asByteChunk(); - final Context ctx = (Context) context; - for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int bucketPosition = bucketPositions.get(runStart); - - ctx.curVal = bucketLastVal.getLong(bucketPosition); - accumulate(asBytes, ctx, runStart, runLength); - bucketLastVal.set(bucketPosition, ctx.curVal); - } - - //noinspection unchecked - outputSource.fillFromChunkUnordered(ctx.fillContext.get(), ctx.outputValues.get(), (LongChunk) keyChunk); - } - @Override protected void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - final long groupPosition) { - ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_LONG; + @NotNull final Chunk workingChunk) { +// ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_LONG; accumulate(workingChunk.asByteChunk(), ctx, 0, workingChunk.size()); - singletonGroup = groupPosition; - singletonVal = ctx.curVal; +// singletonGroup = groupPosition; +// singletonVal = ctx.curVal; outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java index 6ea95acca2e..eb67d79a112 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java @@ -31,44 +31,9 @@ public DoubleCumSumOperator(@NotNull final MatchPair pair, // endregion constructor } - @Override - public void addChunkBucketed(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { - final DoubleChunk asDoubles = values.asDoubleChunk(); - final Context ctx = (Context) context; - final WritableDoubleChunk localOutputChunk = ctx.outputValues.get(); - for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int bucketPosition = bucketPositions.get(runStart); - - ctx.curVal = bucketLastVal.getDouble(bucketPosition); - if(Double.isNaN(ctx.curVal)) { - localOutputChunk.fillWithValue(runStart, runLength, ctx.curVal); - } else { - accumulate(asDoubles, ctx, runStart, runLength); - bucketLastVal.set(bucketPosition, ctx.curVal); - } - } - - //noinspection unchecked - outputSource.fillFromChunkUnordered(ctx.fillContext.get(), localOutputChunk, (LongChunk) keyChunk); - } - protected void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - final long groupPosition) { - ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_DOUBLE; - if(ctx.lastGroupPosition != groupPosition) { - ctx.lastGroupPosition = groupPosition; - ctx.filledWithPermanentValue = false; - } - + @NotNull final Chunk workingChunk) { if(Double.isNaN(ctx.curVal)) { if(!ctx.filledWithPermanentValue) { ctx.filledWithPermanentValue = true; @@ -77,9 +42,6 @@ protected void doAddChunk(@NotNull final Context ctx, } else { accumulate(workingChunk.asDoubleChunk(), ctx, 0, workingChunk.size()); } - - singletonGroup = groupPosition; - singletonVal = ctx.curVal; outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java index 225f67e2adb..da4c0802fa8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java @@ -26,44 +26,9 @@ public FloatCumSumOperator(@NotNull final MatchPair pair, // endregion constructor } - @Override - public void addChunkBucketed(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { - final FloatChunk asFloats = values.asFloatChunk(); - final Context ctx = (Context) context; - final WritableFloatChunk localOutputChunk = ctx.outputValues.get(); - for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int bucketPosition = bucketPositions.get(runStart); - - ctx.curVal = bucketLastVal.getFloat(bucketPosition); - if(Float.isNaN(ctx.curVal)) { - localOutputChunk.fillWithValue(runStart, runLength, ctx.curVal); - } else { - accumulate(asFloats, ctx, runStart, runLength); - bucketLastVal.set(bucketPosition, ctx.curVal); - } - } - - //noinspection unchecked - outputSource.fillFromChunkUnordered(ctx.fillContext.get(), localOutputChunk, (LongChunk) keyChunk); - } - protected void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - final long groupPosition) { - ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_FLOAT; - if(ctx.lastGroupPosition != groupPosition) { - ctx.lastGroupPosition = groupPosition; - ctx.filledWithPermanentValue = false; - } - + @NotNull final Chunk workingChunk) { if(Float.isNaN(ctx.curVal)) { if(!ctx.filledWithPermanentValue) { ctx.filledWithPermanentValue = true; @@ -72,9 +37,6 @@ protected void doAddChunk(@NotNull final Context ctx, } else { accumulate(workingChunk.asFloatChunk(), ctx, 0, workingChunk.size()); } - - singletonGroup = groupPosition; - singletonVal = ctx.curVal; outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java index 7c7045b20ae..2359d2d31e5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java @@ -34,38 +34,14 @@ public IntCumSumOperator(@NotNull final MatchPair pair, // endregion constructor } - @Override - public void addChunkBucketed(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { - final IntChunk asIntegers = values.asIntChunk(); - final Context ctx = (Context) context; - for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int bucketPosition = bucketPositions.get(runStart); - - ctx.curVal = bucketLastVal.getLong(bucketPosition); - accumulate(asIntegers, ctx, runStart, runLength); - bucketLastVal.set(bucketPosition, ctx.curVal); - } - - //noinspection unchecked - outputSource.fillFromChunkUnordered(ctx.fillContext.get(), ctx.outputValues.get(), (LongChunk) keyChunk); - } - @Override protected void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - final long groupPosition) { - ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_LONG; + @NotNull final Chunk workingChunk) { +// ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_LONG; accumulate(workingChunk.asIntChunk(), ctx, 0, workingChunk.size()); - singletonGroup = groupPosition; - singletonVal = ctx.curVal; +// singletonGroup = groupPosition; +// singletonVal = ctx.curVal; outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java index 0de21f4705d..6d871e4dc44 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java @@ -34,38 +34,14 @@ public LongCumSumOperator(@NotNull final MatchPair pair, // endregion constructor } - @Override - public void addChunkBucketed(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { - final LongChunk asLongs = values.asLongChunk(); - final Context ctx = (Context) context; - for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int bucketPosition = bucketPositions.get(runStart); - - ctx.curVal = bucketLastVal.getLong(bucketPosition); - accumulate(asLongs, ctx, runStart, runLength); - bucketLastVal.set(bucketPosition, ctx.curVal); - } - - //noinspection unchecked - outputSource.fillFromChunkUnordered(ctx.fillContext.get(), ctx.outputValues.get(), (LongChunk) keyChunk); - } - @Override protected void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - final long groupPosition) { - ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_LONG; + @NotNull final Chunk workingChunk) { +// ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_LONG; accumulate(workingChunk.asLongChunk(), ctx, 0, workingChunk.size()); - singletonGroup = groupPosition; - singletonVal = ctx.curVal; +// singletonGroup = groupPosition; +// singletonVal = ctx.curVal; outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java index 9fbf844290b..f63b10167ed 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java @@ -29,38 +29,14 @@ public ShortCumSumOperator(@NotNull final MatchPair pair, // endregion constructor } - @Override - public void addChunkBucketed(@NotNull final UpdateContext context, - @NotNull final Chunk values, - @NotNull final LongChunk keyChunk, - @NotNull final IntChunk bucketPositions, - @NotNull final IntChunk startPositions, - @NotNull final IntChunk runLengths) { - final ShortChunk asShorts = values.asShortChunk(); - final Context ctx = (Context) context; - for(int runIdx = 0; runIdx < startPositions.size(); runIdx++) { - final int runStart = startPositions.get(runIdx); - final int runLength = runLengths.get(runIdx); - final int bucketPosition = bucketPositions.get(runStart); - - ctx.curVal = bucketLastVal.getLong(bucketPosition); - accumulate(asShorts, ctx, runStart, runLength); - bucketLastVal.set(bucketPosition, ctx.curVal); - } - - //noinspection unchecked - outputSource.fillFromChunkUnordered(ctx.fillContext.get(), ctx.outputValues.get(), (LongChunk) keyChunk); - } - @Override protected void doAddChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk, - final long groupPosition) { - ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_LONG; + @NotNull final Chunk workingChunk) { +// ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_LONG; accumulate(workingChunk.asShortChunk(), ctx, 0, workingChunk.size()); - singletonGroup = groupPosition; - singletonVal = ctx.curVal; +// singletonGroup = groupPosition; +// singletonVal = ctx.curVal; outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } From 8036eaf38190fb9be2ed63adcc4f20db532759af Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 12 Aug 2022 08:39:14 -0700 Subject: [PATCH 009/123] added Partioned helper --- .../impl/BucketedPartitionedUpdateBy.java | 643 ++++++++++++++++++ 1 file changed, 643 insertions(+) create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java new file mode 100644 index 00000000000..af79375621d --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java @@ -0,0 +1,643 @@ +package io.deephaven.engine.table.impl; + +import io.deephaven.api.ColumnName; +import io.deephaven.api.updateby.UpdateByControl; +import io.deephaven.chunk.WritableChunk; +import io.deephaven.chunk.WritableLongChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.chunk.sized.SizedLongChunk; +import io.deephaven.engine.rowset.*; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.table.*; +import io.deephaven.engine.table.impl.util.SizedSafeCloseable; +import io.deephaven.engine.table.impl.util.UpdateSizeCalculator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.util.SafeCloseable; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Map; + +/** + * An implementation of {@link UpdateBy} dedicated to zero key computation. + */ +class BucketedPartitionedUpdateBy extends UpdateBy { + + /** + * Perform an updateBy without any key columns. + * + * @param description the operation description + * @param source the source table + * @param ops the operations to perform + * @param resultSources the result sources + * @param rowRedirection the {@link io.deephaven.engine.table.impl.util.RowRedirection}, if one is used. + * @param control the control object. + * @return the result table + */ + public static Table compute(@NotNull final String description, + @NotNull final QueryTable source, + @NotNull final UpdateByOperator[] ops, + @NotNull final Map> resultSources, + @NotNull final Collection byColumns, + @Nullable final WritableRowRedirection rowRedirection, + @NotNull final UpdateByControl control) { + + // this table will always have the rowset of the parent + final QueryTable result = new QueryTable(source.getRowSet(), resultSources); + +// final BucketedPartitionedUpdateBy updateBy = new BucketedPartitionedUpdateBy(ops, source, rowRedirection, control); + + final PartitionedTable pt = source.partitionBy(false, byColumns); + final PartitionedTable transformed = pt.transform(t -> { + // create the table + Table newTable = ZeroKeyUpdateBy.compute( + description, + (QueryTable)t, + ops, + resultSources, + rowRedirection, + control); + // add the listener + + // return the table + return newTable; + }); + result.addParentReference(transformed); + +// if (source.isRefreshing()) { +// if (rowRedirection != null) { +// rowRedirection.startTrackingPrevValues(); +// } +// Arrays.stream(ops).forEach(UpdateByOperator::startTrackingPrev); +// final ZeroKeyUpdateByListener listener = updateBy.newListener(description, result); +// source.listenForUpdates(listener); +// result.addParentReference(listener); +// } + + return result; + } + + protected BucketedPartitionedUpdateBy(@NotNull final UpdateByOperator[] operators, + @NotNull final QueryTable source, + @Nullable final WritableRowRedirection rowRedirection, + @NotNull final UpdateByControl control) { + super(operators, source, rowRedirection, control); + } + +// ZeroKeyUpdateByListener newListener(@NotNull final String description, @NotNull final QueryTable result) { +// return new ZeroKeyUpdateByListener(description, source, result); +// } +// +// void doInitialAdditions() { +// final TableUpdateImpl fakeUpdate = new TableUpdateImpl(source.getRowSet(), +// RowSetFactory.empty(), +// RowSetFactory.empty(), +// RowSetShiftData.EMPTY, +// ModifiedColumnSet.ALL); +// try (final UpdateContext ctx = new UpdateContext(fakeUpdate, null, true)) { +// ctx.setAllAffected(); +// if (rowRedirection != null && source.isRefreshing()) { +// processUpdateForRedirection(fakeUpdate); +// } +// // do an addition phase for all the operators that can add directly (i.e. backwards looking) +// ctx.doUpdate(source.getRowSet(), source.getRowSet(), UpdateType.Add); +// +// // do a reprocessing phase for operators that can't add directly +// ctx.reprocessRows(RowSetShiftData.EMPTY); +// } +// } +// +// /** +// * An object to hold the transient state during a single {@link ShiftAwareListener#onUpdate(TableUpdate)} update +// * cycle. +// */ +// private class UpdateContext implements SafeCloseable { +// /** The expected size of chunks to the various update stages */ +// int chunkSize; +// +// /** An indicator of if each slot has been populated with data or not for this phase. */ +// boolean[] inputChunkPopulated; +// +// /** An array of boolean denoting which operators are affected by the current update. */ +// final boolean[] opAffected; +// +// /** true if any operator requested keys */ +// boolean anyRequireKeys; +// +// /** An array of context objects for each underlying operator */ +// final UpdateByOperator.UpdateContext[] opContext; +// +// /** A {@link SharedContext} to be used while creating other contexts */ +// SharedContext sharedContext = SharedContext.makeSharedContext(); +// +// /** An array of {@link ChunkSource.FillContext}s for each input column */ +// final SizedSafeCloseable[] fillContexts; +// +// /** A set of chunks used to store post-shift working values */ +// final SizedSafeCloseable>[] postWorkingChunks; +// +// /** A Chunk of longs to store the keys being updated */ +// final SizedLongChunk keyChunk; +// +// /** A sharedContext to be used with previous values */ +// SharedContext prevSharedContext; +// +// /** An array of {@link ChunkSource.FillContext}s for previous values */ +// ChunkSource.FillContext[] prevFillContexts; +// +// /** An array of chunks for previous values */ +// WritableChunk[] prevWorkingChunks; +// +// /** A Long Chunk for previous keys */ +// WritableLongChunk prevKeyChunk; +// +// final RowSet affectedRows; +// +// @SuppressWarnings("resource") +// UpdateContext(@NotNull final TableUpdate upstream, +// @Nullable final ModifiedColumnSet[] inputModifiedColumnSets, +// final boolean isInitializeStep) { +// final int updateSize = UpdateSizeCalculator.chunkSize(upstream, control.chunkCapacityOrDefault()); +// +// this.chunkSize = +// UpdateSizeCalculator.chunkSize(updateSize, upstream.shifted(), control.chunkCapacityOrDefault()); +// this.opAffected = new boolean[operators.length]; +// // noinspection unchecked +// this.fillContexts = new SizedSafeCloseable[operators.length]; +// this.opContext = new UpdateByOperator.UpdateContext[operators.length]; +// this.keyChunk = new SizedLongChunk<>(); +// this.inputChunkPopulated = new boolean[operators.length]; +// +// if (upstream.shifted().nonempty()) { +// this.prevKeyChunk = WritableLongChunk.makeWritableChunk(chunkSize); +// } +// +// final boolean hasModifies = upstream.modified().isNonempty(); +// if (hasModifies) { +// // noinspection unchecked +// this.prevWorkingChunks = new WritableChunk[operators.length]; +// this.prevSharedContext = SharedContext.makeSharedContext(); +// this.prevFillContexts = new ChunkSource.FillContext[operators.length]; +// } +// +// final boolean upstreamAppendOnly = +// isInitializeStep || UpdateByOperator.isAppendOnly(upstream, source.getRowSet().lastRowKeyPrev()); +// +// // noinspection unchecked +// this.postWorkingChunks = new SizedSafeCloseable[operators.length]; +// for (int opIdx = 0; opIdx < operators.length; opIdx++) { +// opAffected[opIdx] = upstream.added().isNonempty() || +// upstream.removed().isNonempty() || +// upstream.shifted().nonempty() || +// (upstream.modifiedColumnSet().nonempty() && (inputModifiedColumnSets == null +// || upstream.modifiedColumnSet().containsAny(inputModifiedColumnSets[opIdx]))); +// if (!opAffected[opIdx]) { +// continue; +// } +// +// opContext[opIdx] = operators[opIdx].makeUpdateContext(chunkSize); +// +// final int slotPosition = inputSourceSlots[opIdx]; +// if (fillContexts[slotPosition] == null) { +// fillContexts[slotPosition] = new SizedSafeCloseable<>( +// sz -> inputSources[slotPosition].makeFillContext(sz, getSharedContext())); +// fillContexts[slotPosition].ensureCapacity(chunkSize); +// postWorkingChunks[slotPosition] = new SizedSafeCloseable<>( +// sz -> inputSources[slotPosition].getChunkType().makeWritableChunk(sz)); +// postWorkingChunks[slotPosition].ensureCapacity(chunkSize); +// +// // Note that these don't participate in setChunkSize() because nothing will use them. If that +// // changes then these must also become SizedSafeCloseables. +// if (hasModifies) { +// prevFillContexts[slotPosition] = +// inputSources[opIdx].makeFillContext(chunkSize, prevSharedContext); +// prevWorkingChunks[slotPosition] = +// inputSources[opIdx].getChunkType().makeWritableChunk(chunkSize); +// } +// } +// +// operators[opIdx].initializeForUpdate(opContext[opIdx], upstream, source.getRowSet(), false, +// upstreamAppendOnly); +// } +// +// // retrieve the affected rows from all operator update contexts +// WritableRowSet tmp = RowSetFactory.empty(); +// for (int opIdx = 0; opIdx < operators.length; opIdx++) { +// if (!opAffected[opIdx]) { +// continue; +// } +// // trigger the operator to determine its own set of affected rows (window-specific), do not close() +// // since this is managed by the operator context +// final RowSet rs = opContext[opIdx].determineAffectedRows(upstream, source.getRowSet(), upstreamAppendOnly); +// +// // union the operator rowsets together to get a global set +// tmp.insert(rs); +// } +// affectedRows = tmp; +// } +// +// public SharedContext getSharedContext() { +// return sharedContext; +// } +// +// void setChunkSize(int newChunkSize) { +// if (newChunkSize <= chunkSize) { +// return; +// } +// +// // We have to close and recreate the shared context because a .reset() is not enough to ensure that any +// // cached chunks that something stuffed into there are resized. +// this.sharedContext.close(); +// this.sharedContext = SharedContext.makeSharedContext(); +// +// if (prevSharedContext != null) { +// this.prevSharedContext.close(); +// this.prevSharedContext = null; +// } +// +// this.chunkSize = newChunkSize; +// this.keyChunk.ensureCapacity(newChunkSize); +// +// for (int opIdx = 0; opIdx < operators.length; opIdx++) { +// if (!opAffected[opIdx]) { +// continue; +// } +// +// operators[opIdx].setChunkSize(opContext[opIdx], newChunkSize); +// if (fillContexts[opIdx] != null) { +// fillContexts[opIdx].ensureCapacity(newChunkSize); +// postWorkingChunks[opIdx].ensureCapacity(newChunkSize); +// +// // Note that this doesn't include the prevFillContexts or prevWorkingChunks. If they become +// // needed for an op, they must be added here. +// } +// } +// } +// +// void initializeFor(@NotNull final RowSet updateRowSet, +// @NotNull final UpdateType type) { +// for (int opIdx = 0; opIdx < operators.length; opIdx++) { +// if (opAffected[opIdx]) { +// operators[opIdx].initializeFor(opContext[opIdx], updateRowSet, type); +// anyRequireKeys |= operators[opIdx].requiresKeys(); +// } +// } +// +// if (anyRequireKeys) { +// keyChunk.ensureCapacity(chunkSize); +// } +// } +// +// void finishFor(@NotNull final UpdateType type) { +// for (int opIdx = 0; opIdx < operators.length; opIdx++) { +// if (opAffected[opIdx]) { +// operators[opIdx].finishFor(opContext[opIdx], type); +// } +// } +// +// anyRequireKeys = false; +// Arrays.fill(inputChunkPopulated, false); +// } +// +// @Override +// public void close() { +// sharedContext.close(); +// keyChunk.close(); +// affectedRows.close(); +// +// if (prevKeyChunk != null) { +// prevKeyChunk.close(); +// } +// +// if (prevSharedContext != null) { +// prevSharedContext.close(); +// } +// +// for (int opIdx = 0; opIdx < operators.length; opIdx++) { +// if (opContext[opIdx] != null) { +// opContext[opIdx].close(); +// } +// +// if (fillContexts[opIdx] != null) { +// fillContexts[opIdx].close(); +// } +// +// if (postWorkingChunks[opIdx] != null) { +// postWorkingChunks[opIdx].close(); +// } +// +// if (prevFillContexts != null && prevFillContexts[opIdx] != null) { +// prevFillContexts[opIdx].close(); +// } +// +// if (prevWorkingChunks != null && prevWorkingChunks[opIdx] != null) { +// prevWorkingChunks[opIdx].close(); +// } +// } +// } +// +// /** +// * Mark all columns as affected +// */ +// public void setAllAffected() { +// Arrays.fill(opAffected, true); +// } +// +// /** +// * Check if any of the operators have produced additional modified rows. +// * +// * @return true if any operator produced more modified rows. +// */ +// boolean anyModified() { +// for (int opIdx = 0; opIdx < operators.length; opIdx++) { +// if (opAffected[opIdx] && operators[opIdx].anyModified(opContext[opIdx])) { +// return true; +// } +// } +// return false; +// } +// +// void doUpdate(@NotNull final RowSet updateRowSet, +// @NotNull final RowSet preShiftUpdateRowSet, +// @NotNull final UpdateType type) { +// if (updateRowSet.isEmpty()) { +// return; +// } +// +// try (final RowSequence.Iterator okIt = updateRowSet.getRowSequenceIterator(); +// final RowSequence.Iterator preShiftOkIt = preShiftUpdateRowSet == updateRowSet ? null +// : preShiftUpdateRowSet.getRowSequenceIterator()) { +// initializeFor(updateRowSet, type); +// +// while (okIt.hasMore()) { +// sharedContext.reset(); +// if (prevSharedContext != null) { +// prevSharedContext.reset(); +// } +// Arrays.fill(inputChunkPopulated, false); +// +// final RowSequence chunkOk = okIt.getNextRowSequenceWithLength(chunkSize); +// final RowSequence prevChunkOk = preShiftUpdateRowSet == updateRowSet ? chunkOk +// : preShiftOkIt.getNextRowSequenceWithLength(chunkSize); +// +// if (anyRequireKeys) { +// chunkOk.fillRowKeyChunk(keyChunk.get()); +// if (prevChunkOk != chunkOk) { +// prevChunkOk.fillRowKeyChunk(prevKeyChunk); +// } +// } +// +// for (int opIdx = 0; opIdx < operators.length; opIdx++) { +// if (!opAffected[opIdx]) { +// continue; +// } +// +// final UpdateByOperator currentOp = operators[opIdx]; +// final int slotPosition = inputSourceSlots[opIdx]; +// if (type == UpdateType.Add) { +// prepareValuesChunkFor(opIdx, slotPosition, false, true, chunkOk, prevChunkOk, +// null, postWorkingChunks[slotPosition].get(), +// null, fillContexts[slotPosition].get()); +// currentOp.addChunk(opContext[opIdx], chunkOk, keyChunk.get(), +// postWorkingChunks[slotPosition].get(), 0); +// } else if (type == UpdateType.Remove) { +// prepareValuesChunkFor(opIdx, slotPosition, true, false, chunkOk, prevChunkOk, +// postWorkingChunks[slotPosition].get(), null, +// fillContexts[slotPosition].get(), null); +// currentOp.removeChunk(opContext[opIdx], keyChunk.get(), +// postWorkingChunks[slotPosition].get(), 0); +// } else if (type == UpdateType.Modify) { +// prepareValuesChunkFor(opIdx, slotPosition, true, true, chunkOk, prevChunkOk, +// prevWorkingChunks[slotPosition], postWorkingChunks[slotPosition].get(), +// prevFillContexts[slotPosition], fillContexts[slotPosition].get()); +// currentOp.modifyChunk(opContext[opIdx], +// prevKeyChunk == null ? keyChunk.get() : prevKeyChunk, +// keyChunk.get(), +// prevWorkingChunks[slotPosition], +// postWorkingChunks[slotPosition].get(), +// 0); +// } else if (type == UpdateType.Reprocess) { +// // is this chunk relevant to this operator? If so, then intersect and process only the +// // relevant rows +// if (chunkOk.firstRowKey() <= opContext[opIdx].getAffectedRows().lastRowKey() +// && chunkOk.lastRowKey() >= opContext[opIdx].getAffectedRows().firstRowKey()) { +// try (final RowSet rs = chunkOk.asRowSet(); +// final RowSet intersect = rs.intersect(opContext[opIdx].getAffectedRows())) { +// +// prepareValuesChunkFor(opIdx, slotPosition, false, true, intersect, intersect, +// null, postWorkingChunks[slotPosition].get(), +// null, fillContexts[slotPosition].get()); +// currentOp.reprocessChunk(opContext[opIdx], +// intersect, +// keyChunk.get(), +// postWorkingChunks[slotPosition].get(), +// source.getRowSet()); +// } +// } +// } +// } +// } +// +// finishFor(type); +// } +// } +// +// /** +// * Locate the smallest key that requires reprocessing and then replay the table from that point +// */ +// private void reprocessRows(RowSetShiftData shifted) { +// // Get a sub-index of the source from that minimum reprocessing index and make sure we update our +// // contextual chunks and FillContexts to an appropriate size for this step. +// final RowSet sourceRowSet = source.getRowSet(); +// +// final int newChunkSize = (int) Math.min(control.chunkCapacityOrDefault(), affectedRows.size()); +// setChunkSize(newChunkSize); +// +// for (int opIndex = 0; opIndex < operators.length; opIndex++) { +// if (opAffected[opIndex]) { +// final long keyStart = opContext[opIndex].getAffectedRows().firstRowKey(); +// final long keyBefore; +// try (final RowSet.SearchIterator sit = sourceRowSet.searchIterator()) { +// keyBefore = sit.binarySearchValue( +// (compareTo, ignored) -> Long.compare(keyStart - 1, compareTo), 1); +// } +// operators[opIndex].resetForReprocess(opContext[opIndex], sourceRowSet, keyBefore); +// } +// } +// +// // We will not mess with shifts if we are using a redirection because we'll have applied the shift +// // to the redirection index already by now. +// if (rowRedirection == null && shifted.nonempty()) { +// try (final RowSet prevIdx = source.getRowSet().copyPrev()) { +// shifted.apply((begin, end, delta) -> { +// try (final RowSet subRowSet = prevIdx.subSetByKeyRange(begin, end)) { +// for (int opIdx = 0; opIdx < operators.length; opIdx++) { +// operators[opIdx].applyOutputShift(opContext[opIdx], subRowSet, delta); +// } +// } +// }); +// } +// } +// +// // Now iterate rowset to reprocess. +// doUpdate(affectedRows, affectedRows, UpdateType.Reprocess); +// } +// +// /** +// * Prepare the specified chunk for use. +// * +// * @param opIdx the operator index +// * @param usePrev if previous values should be fetched +// * @param chunkOk the {@link RowSequence} for current values +// * @param prevChunkOk the {@link RowSequence} for previous values. +// */ +// private void prepareValuesChunkFor(final int opIdx, +// final int inputSlot, +// final boolean usePrev, +// final boolean useCurrent, +// final RowSequence chunkOk, +// final RowSequence prevChunkOk, +// final WritableChunk prevWorkingChunk, +// final WritableChunk postWorkingChunk, +// final ChunkSource.FillContext prevFillContext, +// final ChunkSource.FillContext postFillContext) { +// if (!operators[opIdx].requiresValues(opContext[opIdx])) { +// return; +// } +// +// if (!inputChunkPopulated[inputSlot]) { +// // Using opIdx below is OK, because if we are sharing an input slot, we are referring to the same +// // input source. Instead of maintaining another array of sourced by slot, we'll just use the opIdx +// inputChunkPopulated[inputSlot] = true; +// if (usePrev) { +// inputSources[opIdx].fillPrevChunk(prevFillContext, prevWorkingChunk, prevChunkOk); +// } +// +// if (useCurrent) { +// inputSources[opIdx].fillChunk(postFillContext, postWorkingChunk, chunkOk); +// } +// } +// } +// +// private void onBucketsRemoved(@NotNull final RowSet removedBuckets) { +// for (final UpdateByOperator operator : operators) { +// operator.onBucketsRemoved(removedBuckets); +// } +// } +// +// public boolean canAnyProcessNormally() { +// for (int opIdx = 0; opIdx < operators.length; opIdx++) { +// if (opAffected[opIdx] && operators[opIdx].canProcessNormalUpdate(opContext[opIdx])) { +// return true; +// } +// } +// +// return false; +// } +// } +// +// /** +// * The Listener for apply an upstream {@link ShiftAwareListener#onUpdate(Update) update} +// */ +// class ZeroKeyUpdateByListener extends InstrumentedTableUpdateListenerAdapter { +// private final QueryTable result; +// private final ModifiedColumnSet[] inputModifiedColumnSets; +// private final ModifiedColumnSet[] outputModifiedColumnSets; +// private final ModifiedColumnSet.Transformer transformer; +// +// public ZeroKeyUpdateByListener(@Nullable String description, +// @NotNull final QueryTable source, +// @NotNull final QueryTable result) { +// super(description, source, false); +// this.result = result; +// this.inputModifiedColumnSets = new ModifiedColumnSet[operators.length]; +// this.outputModifiedColumnSets = new ModifiedColumnSet[operators.length]; +// +// for (int ii = 0; ii < operators.length; ii++) { +// final String[] outputColumnNames = operators[ii].getOutputColumnNames(); +// inputModifiedColumnSets[ii] = source.newModifiedColumnSet(operators[ii].getAffectingColumnNames()); +// outputModifiedColumnSets[ii] = result.newModifiedColumnSet(outputColumnNames); +// } +// +// this.transformer = +// source.newModifiedColumnSetTransformer(result, source.getDefinition().getColumnNamesArray()); +// } +// +// @Override +// public void onUpdate(TableUpdate upstream) { +// try (final UpdateContext ctx = new UpdateContext(upstream, inputModifiedColumnSets, false)) { +// if (rowRedirection != null) { +// processUpdateForRedirection(upstream); +// } +// +// // If anything can process normal operations we have to pass them down, otherwise we can skip this +// // entirely. +// if (ctx.canAnyProcessNormally()) { +// ctx.doUpdate(upstream.removed(), upstream.removed(), UpdateType.Remove); +// if (upstream.shifted().nonempty()) { +// try (final WritableRowSet prevRowSet = source.getRowSet().copyPrev(); +// final RowSet modPreShift = upstream.getModifiedPreShift()) { +// +// prevRowSet.remove(upstream.removed()); +// for (int ii = 0; ii < operators.length; ii++) { +// operators[ii].initializeFor(ctx.opContext[ii], prevRowSet, UpdateType.Shift); +// operators[ii].applyShift(ctx.opContext[ii], prevRowSet, upstream.shifted()); +// operators[ii].finishFor(ctx.opContext[ii], UpdateType.Shift); +// } +// ctx.doUpdate(upstream.modified(), modPreShift, UpdateType.Modify); +// } +// } else { +// ctx.doUpdate(upstream.modified(), upstream.modified(), UpdateType.Modify); +// } +// ctx.doUpdate(upstream.added(), upstream.added(), UpdateType.Add); +// } +// +// if (source.getRowSet().isEmpty()) { +// ctx.onBucketsRemoved(RowSetFactory.fromKeys(0)); +// } +// +// // Now do the reprocessing phase. +// ctx.reprocessRows(upstream.shifted()); +// +// final TableUpdateImpl downstream = new TableUpdateImpl(); +// // copy these rowSets since TableUpdateImpl#reset will close them with the upstream update +// downstream.added = upstream.added().copy(); +// downstream.removed = upstream.removed().copy(); +// downstream.shifted = upstream.shifted(); +// +// if (upstream.modified().isNonempty() || ctx.anyModified()) { +// downstream.modifiedColumnSet = result.getModifiedColumnSetForUpdates(); +// downstream.modifiedColumnSet.clear(); +// +// WritableRowSet modifiedRowSet = RowSetFactory.empty(); +// downstream.modified = modifiedRowSet; +// if (upstream.modified().isNonempty()) { +// // Transform any untouched modified columns to the output. +// transformer.clearAndTransform(upstream.modifiedColumnSet(), downstream.modifiedColumnSet); +// modifiedRowSet.insert(upstream.modified()); +// } +// +// for (int opIdx = 0; opIdx < operators.length; opIdx++) { +// if (ctx.opAffected[opIdx]) { +// downstream.modifiedColumnSet.setAll(outputModifiedColumnSets[opIdx]); +// if (operators[opIdx].anyModified(ctx.opContext[opIdx])) { +// modifiedRowSet +// .insert(operators[opIdx].getAdditionalModifications(ctx.opContext[opIdx])); +// } +// } +// } +// +// if (ctx.anyModified()) { +// modifiedRowSet.remove(upstream.added()); +// } +// } else { +// downstream.modified = RowSetFactory.empty(); +// downstream.modifiedColumnSet = ModifiedColumnSet.EMPTY; +// } +// result.notifyListeners(downstream); +// } +// } +// } +} From 493e1f3bd95516c9d9667e184050b3b296b5c9ad Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 12 Aug 2022 13:45:47 -0700 Subject: [PATCH 010/123] little hacky, but all cumulative static tests running --- .../java/io/deephaven/engine/table/Table.java | 23 +++++++ .../impl/BucketedPartitionedUpdateBy.java | 17 ++--- .../engine/table/impl/QueryTable.java | 28 +------- .../engine/table/impl/TableWithDefaults.java | 7 ++ .../engine/table/impl/UncoalescedTable.java | 13 +++- .../deephaven/engine/table/impl/UpdateBy.java | 46 ++++++------- .../engine/table/impl/ZeroKeyUpdateBy.java | 6 +- .../ema/BasePrimitiveEMAOperator.java | 37 ++++------- .../updateby/ema/BigNumberEMAOperator.java | 66 +++++++------------ .../internal/BaseByteUpdateByOperator.java | 10 +-- .../internal/BaseCharUpdateByOperator.java | 10 +-- .../internal/BaseDoubleUpdateByOperator.java | 10 +-- .../internal/BaseFloatUpdateByOperator.java | 10 +-- .../internal/BaseIntUpdateByOperator.java | 10 +-- .../internal/BaseLongUpdateByOperator.java | 10 +-- .../internal/BaseObjectUpdateByOperator.java | 10 +-- .../internal/BaseShortUpdateByOperator.java | 10 +-- 17 files changed, 134 insertions(+), 189 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 14a04da4a97..5e3636b0f81 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 @@ -1623,6 +1623,29 @@ Table aggBy(Collection aggregations, boolean preserveEmpt @ConcurrentMethod PartitionedTable partitionBy(String... keyColumnNames); + /** + * Convenience method that performs an {@link #aggBy(Collection, boolean, Table, Collection)} and wraps the result + * in a {@link PartitionedTable}. If {@code aggregations} does not include a {@link io.deephaven.api.agg.Partition + * partition}, one will be added automatically with the default constituent column name and behavior used in + * {@link #partitionBy(String...)}. + * + * @param aggregations The {@link Aggregation aggregations} to apply + * @param preserveEmpty Whether to keep result rows for groups that are initially empty or become empty as a result + * of updates. Each aggregation operator defines its own value for empty groups. + * @param initialGroups A table whose distinct combinations of values for the {@code groupByColumns} should be used + * to create an initial set of aggregation groups. All other columns are ignored. This is useful in + * combination with {@code preserveEmpty == true} to ensure that particular groups appear in the result + * table, or with {@code preserveEmpty == false} to control the encounter order for a collection of groups + * and thus their relative order in the result. Changes to {@code initialGroups} are not expected or handled; + * if {@code initialGroups} is a refreshing table, only its contents at instantiation time will be used. If + * {@code initialGroups == null}, the result will be the same as if a table with no rows was supplied. + * @param keyColumnNames The names of the key columns to aggregate by + * @return A {@link PartitionedTable} keyed by {@code keyColumnNames} + */ + @ConcurrentMethod + PartitionedTable partitionedAggBy(Collection aggregations, boolean preserveEmpty, + Table initialGroups, @NotNull Collection keyColumnNames); + /** * Convenience method that performs an {@link #aggBy(Collection, boolean, Table, Collection)} and wraps the result * in a {@link PartitionedTable}. If {@code aggregations} does not include a {@link io.deephaven.api.agg.Partition diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java index af79375621d..1004e3c1703 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java @@ -2,22 +2,15 @@ import io.deephaven.api.ColumnName; import io.deephaven.api.updateby.UpdateByControl; -import io.deephaven.chunk.WritableChunk; -import io.deephaven.chunk.WritableLongChunk; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.sized.SizedLongChunk; -import io.deephaven.engine.rowset.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.util.SizedSafeCloseable; -import io.deephaven.engine.table.impl.util.UpdateSizeCalculator; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.PartitionedTable; +import io.deephaven.engine.table.Table; import io.deephaven.engine.table.impl.util.WritableRowRedirection; -import io.deephaven.util.SafeCloseable; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.util.Arrays; import java.util.Collection; +import java.util.List; import java.util.Map; /** @@ -49,7 +42,7 @@ public static Table compute(@NotNull final String description, // final BucketedPartitionedUpdateBy updateBy = new BucketedPartitionedUpdateBy(ops, source, rowRedirection, control); - final PartitionedTable pt = source.partitionBy(false, byColumns); + final PartitionedTable pt = source.partitionedAggBy(List.of(), true, null, byColumns); final PartitionedTable transformed = pt.transform(t -> { // create the table Table newTable = ZeroKeyUpdateBy.compute( diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java index cbd50b1ca72..359bfbc3331 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java @@ -476,31 +476,9 @@ public PartitionedTable partitionBy(final boolean dropKeys, @NotNull final Colle }); } -// @Override -// public PartitionedTable partitionBy(final boolean dropKeys, final String... keyColumnNames) { -// if (isStream()) { -// throw streamUnsupported("partitionBy"); -// } -// final List columns = ColumnName.from(keyColumnNames); -// return memoizeResult(MemoizedOperationKey.partitionBy(dropKeys, columns), () -> { -// final Table partitioned = aggBy(Partition.of(CONSTITUENT, !dropKeys), columns); -// final Set keyColumnNamesSet = -// Arrays.stream(keyColumnNames).collect(Collectors.toCollection(LinkedHashSet::new)); -// final TableDefinition constituentDefinition; -// if (dropKeys) { -// constituentDefinition = TableDefinition.of(definition.getColumnStream() -// .filter(cd -> !keyColumnNamesSet.contains(cd.getName())).toArray(ColumnDefinition[]::new)); -// } else { -// constituentDefinition = definition; -// } -// return new PartitionedTableImpl(partitioned, keyColumnNamesSet, true, CONSTITUENT.name(), -// constituentDefinition, isRefreshing(), false); -// }); -// } - @Override public PartitionedTable partitionedAggBy(final Collection aggregations, - final boolean preserveEmpty, @Nullable final Table initialGroups, final String... keyColumnNames) { + final boolean preserveEmpty, @Nullable final Table initialGroups, @NotNull final Collection columns) { if (isStream()) { throw streamUnsupported("partitionedAggBy"); } @@ -513,9 +491,9 @@ public PartitionedTable partitionedAggBy(final Collection ? aggregations : Stream.concat(aggregations.stream(), Stream.of(partition)).collect(Collectors.toList()); final Table aggregated = - aggBy(aggregationsToUse, preserveEmpty, initialGroups, ColumnName.from(keyColumnNames)); + aggBy(aggregationsToUse, preserveEmpty, initialGroups, columns); final Set keyColumnNamesSet = - Arrays.stream(keyColumnNames).collect(Collectors.toCollection(LinkedHashSet::new)); + columns.stream().map(ColumnName::name).collect(Collectors.toCollection(LinkedHashSet::new)); final TableDefinition constituentDefinition; if (partition.includeGroupByColumns()) { constituentDefinition = definition; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/TableWithDefaults.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/TableWithDefaults.java index c2ad84c72a8..f6e83e89fc7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/TableWithDefaults.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/TableWithDefaults.java @@ -26,6 +26,7 @@ import io.deephaven.engine.liveness.LivenessScopeStack; import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.util.*; import java.util.function.Function; @@ -1089,6 +1090,12 @@ default PartitionedTable partitionBy(String... keyColumnNames) { return partitionBy(false, keyColumnNames); } + @Override + @ConcurrentMethod + default PartitionedTable partitionedAggBy(final Collection aggregations, + final boolean preserveEmpty, @Nullable final Table initialGroups, String... keyColumnNames) { + return partitionedAggBy(aggregations, preserveEmpty, initialGroups, ColumnName.from(keyColumnNames)); + } // ----------------------------------------------------------------------------------------------------------------- // Hierarchical table operations (rollup and treeTable). diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UncoalescedTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UncoalescedTable.java index 6b859a114f9..501ba1cea3a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UncoalescedTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UncoalescedTable.java @@ -379,17 +379,24 @@ public PartitionedTable partitionBy(boolean dropKeys, String... keyColumnNames) @Override @ConcurrentMethod - public PartitionedTable partitionBy(boolean dropKeys, @NotNull Collection keyColumnNames) { - return coalesce().partitionBy(dropKeys, keyColumnNames); + public PartitionedTable partitionBy(boolean dropKeys, @NotNull Collection columns) { + return coalesce().partitionBy(dropKeys, columns); } @Override @ConcurrentMethod public PartitionedTable partitionedAggBy(Collection aggregations, boolean preserveEmpty, - Table initialGroups, String... keyColumnNames) { + Table initialGroups, String... keyColumnNames) { return coalesce().partitionedAggBy(aggregations, preserveEmpty, initialGroups, keyColumnNames); } + @Override + @ConcurrentMethod + public PartitionedTable partitionedAggBy(Collection aggregations, boolean preserveEmpty, + Table initialGroups, @NotNull Collection columns) { + return coalesce().partitionedAggBy(aggregations, preserveEmpty, initialGroups, columns); + } + @Override @ConcurrentMethod public Table rollup(Collection aggregations, boolean includeConstituents, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java index 7091ee01227..046c44cf70d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java @@ -112,10 +112,6 @@ public static Table updateBy(@NotNull final QueryTable source, rowRedirection = null; } - // start tracking previous values - if (rowRedirection != null) { - rowRedirection.startTrackingPrevValues(); - } // TODO(deephaven-core#2693): Improve UpdateBy implementation for ColumnName // generate a MatchPair array for use by the existing algorithm @@ -150,13 +146,22 @@ public static Table updateBy(@NotNull final QueryTable source, final UpdateByOperator[] opArr = ops.toArray(UpdateByOperator.ZERO_LENGTH_OP_ARRAY); if (pairs.length == 0) { descriptionBuilder.append(")"); - return ZeroKeyUpdateBy.compute( + Table ret = ZeroKeyUpdateBy.compute( descriptionBuilder.toString(), source, opArr, resultSources, rowRedirection, control); + + if (source.isRefreshing()) { + // start tracking previous values + if (rowRedirection != null) { + rowRedirection.startTrackingPrevValues(); + } + ops.forEach(UpdateByOperator::startTrackingPrev); + } + return ret; } descriptionBuilder.append(", pairs={").append(MatchPair.matchString(pairs)).append("})"); @@ -173,7 +178,7 @@ public static Table updateBy(@NotNull final QueryTable source, String.join(", ", problems) + "}"); } - return BucketedPartitionedUpdateBy.compute( + Table ret = BucketedPartitionedUpdateBy.compute( descriptionBuilder.toString(), source, opArr, @@ -182,27 +187,14 @@ public static Table updateBy(@NotNull final QueryTable source, rowRedirection, control); -// return BucketedUpdateBy.compute(descriptionBuilder.toString(), -// source, -// opArr, -// resultSources, -// rowRedirection, -// keySources.toArray(ColumnSource.ZERO_LENGTH_COLUMN_SOURCE_ARRAY), -// originalKeySources.toArray(ColumnSource.ZERO_LENGTH_COLUMN_SOURCE_ARRAY), -// pairs, -// control); - -// -// -// return BucketedUpdateBy.compute(descriptionBuilder.toString(), -// source, -// opArr, -// resultSources, -// rowRedirection, -// keySources.toArray(ColumnSource.ZERO_LENGTH_COLUMN_SOURCE_ARRAY), -// originalKeySources.toArray(ColumnSource.ZERO_LENGTH_COLUMN_SOURCE_ARRAY), -// pairs, -// control); + if (source.isRefreshing()) { + // start tracking previous values + if (rowRedirection != null) { + rowRedirection.startTrackingPrevValues(); + } + ops.forEach(UpdateByOperator::startTrackingPrev); + } + return ret; } protected void processUpdateForRedirection(@NotNull final TableUpdate upstream) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java index a016e28166f..5b59f5f7b64 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java @@ -47,7 +47,11 @@ public static Table compute(@NotNull final String description, updateBy.doInitialAdditions(); if (source.isRefreshing()) { - Arrays.stream(ops).forEach(UpdateByOperator::startTrackingPrev); +// // start tracking previous values +// if (rowRedirection != null) { +// rowRedirection.startTrackingPrevValues(); +// } +// Arrays.stream(ops).forEach(UpdateByOperator::startTrackingPrev); final ZeroKeyUpdateByListener listener = updateBy.newListener(description, result); source.listenForUpdates(listener); result.addParentReference(listener); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java index 12d7ce4753d..8d851e4977c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java @@ -82,17 +82,11 @@ public void initializeForUpdate(@NotNull final UpdateByOperator.UpdateContext co @NotNull final RowSet resultSourceIndex, final long lastPrevKey , final boolean isAppendOnly) { - final EmaContext ctx = (EmaContext) context; - - // If we're redirected we have to make sure we tell the output source it's actual size, or we're going - // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not - // need to do anything with capacity. - if (isRedirected) { - outputSource.ensureCapacity(resultSourceIndex.size() + 1); - } + super.initializeForUpdate(context, upstream, resultSourceIndex, lastPrevKey, isAppendOnly); - // If we aren't bucketing, we'll just remember the appendyness. - ctx.canProcessDirectly = isAppendOnly; + final EmaContext ctx = (EmaContext) context; + // pre-load the context timestamp with the previous last value in the timestamp column (if possible) + ctx.lastStamp = (lastPrevKey == NULL_ROW_KEY || timeRecorder == null) ? NULL_LONG : locateFirstValidPreviousTimestamp(resultSourceIndex, lastPrevKey); } @Override @@ -100,7 +94,6 @@ public void initializeFor(@NotNull final UpdateByOperator.UpdateContext updateCo @NotNull final RowSet updateRowSet, @NotNull final UpdateBy.UpdateType type) { super.initializeFor(updateContext, updateRowSet, type); - ((EmaContext) updateContext).lastStamp = NULL_LONG; } @Override @@ -134,13 +127,8 @@ public void resetForReprocess(@NotNull final UpdateByOperator.UpdateContext cont ctx.lastStamp = NULL_LONG; } else { // If it hasn't been reset to null, then it's possible that the value at that position was null, in - // which case - // we must have ignored it, and so we have to actually keep looking backwards until we find something - // not null. - - - // Note that it's OK that we are not setting the singletonVal here, because if we had to go back more - // rows, then whatever the correct value was, was already set at the initial location. + // which case we must have ignored it, and so we have to actually keep looking backwards until we find + // something not null. ctx.lastStamp = locateFirstValidPreviousTimestamp(sourceIndex, firstUnmodifiedKey); } } @@ -154,12 +142,13 @@ private long locateFirstValidPreviousTimestamp(@NotNull final RowSet indexToSear } try (final RowSet.SearchIterator rIt = indexToSearch.reverseIterator()) { - rIt.advance(firstUnmodifiedKey); - while (rIt.hasNext()) { - final long nextKey = rIt.nextLong(); - potentialResetTimestamp = timeRecorder.getCurrentLong(nextKey); - if (potentialResetTimestamp != NULL_LONG && isValueValid(nextKey)) { - return potentialResetTimestamp; + if (rIt.advance(firstUnmodifiedKey)) { + while (rIt.hasNext()) { + final long nextKey = rIt.nextLong(); + potentialResetTimestamp = timeRecorder.getCurrentLong(nextKey); + if (potentialResetTimestamp != NULL_LONG && isValueValid(nextKey)) { + return potentialResetTimestamp; + } } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java index 44e6e9d1908..2e5d30409e3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java @@ -3,20 +3,15 @@ import io.deephaven.api.updateby.BadDataBehavior; import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; -import io.deephaven.chunk.IntChunk; -import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.ObjectChunk; -import io.deephaven.chunk.attributes.ChunkLengths; -import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; -import io.deephaven.engine.table.*; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.locations.TableDataException; -import io.deephaven.engine.table.impl.sources.LongArraySource; -import io.deephaven.engine.table.impl.sources.ObjectArraySource; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; @@ -26,6 +21,7 @@ import java.math.BigDecimal; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; +import static io.deephaven.util.QueryConstants.NULL_DOUBLE; import static io.deephaven.util.QueryConstants.NULL_LONG; public abstract class BigNumberEMAOperator extends BaseObjectUpdateByOperator { @@ -89,18 +85,11 @@ public void initializeForUpdate(@NotNull UpdateContext context, @NotNull RowSet resultSourceIndex, final long lastPrevKey, boolean isAppendOnly) { - // noinspection unchecked - final EmaContext ctx = (EmaContext) context; - - // If we're redirected we have to make sure we tell the output source it's actual size, or we're going - // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not - // need to do anything with capacity. - if (isRedirected) { - outputSource.ensureCapacity(resultSourceIndex.size() + 1); - } + super.initializeForUpdate(context, upstream, resultSourceIndex, lastPrevKey, isAppendOnly); - // If we aren't bucketing, we'll just remember the appendyness. - ctx.canProcessDirectly = isAppendOnly; + final EmaContext ctx = (EmaContext) context; + // pre-load the context timestamp with the previous last value in the timestamp column (if possible) + ctx.lastStamp = (lastPrevKey == NULL_ROW_KEY || timeRecorder == null) ? NULL_LONG : locateFirstValidPreviousTimestamp(resultSourceIndex, lastPrevKey); } @SuppressWarnings("unchecked") @@ -109,7 +98,6 @@ public void initializeFor(@NotNull final UpdateContext updateContext, @NotNull final RowSet updateRowSet, @NotNull final UpdateBy.UpdateType type) { super.initializeFor(updateContext, updateRowSet, type); - ((EmaContext) updateContext).lastStamp = NULL_LONG; } @Override @@ -140,21 +128,16 @@ public void resetForReprocess(@NotNull final UpdateContext context, final EmaContext ctx = (EmaContext) context; if (!ctx.canProcessDirectly) { -// // If we set the last state to null, then we know it was a reset state and the timestamp must also -// // have been reset. -// if (singletonVal == null || (firstUnmodifiedKey == NULL_ROW_KEY)) { -// singletonLastStamp = NULL_LONG; -// } else { -// // If it hasn't been reset to null, then it's possible that the value at that position was null, in -// // which case -// // we must have ignored it, and so we have to actually keep looking backwards until we find something -// // not null. -// -// -// // Note that it's OK that we are not setting the singletonVal here, because if we had to go back more -// // rows, then whatever the correct value was, was already set at the initial location. -// singletonLastStamp = locateFirstValidPreviousTimestamp(sourceIndex, firstUnmodifiedKey); -// } + // If we set the last state to null, then we know it was a reset state and the timestamp must also + // have been reset. + if (ctx.curVal == null || (firstUnmodifiedKey == NULL_ROW_KEY)) { + ctx.lastStamp = NULL_LONG; + } else { + // If it hasn't been reset to null, then it's possible that the value at that position was null, in + // which case we must have ignored it, and so we have to actually keep looking backwards until we find + // something not null. + ctx.lastStamp = locateFirstValidPreviousTimestamp(sourceIndex, firstUnmodifiedKey); + } } } @@ -166,12 +149,13 @@ private long locateFirstValidPreviousTimestamp(@NotNull final RowSet indexToSear } try (final RowSet.SearchIterator rIt = indexToSearch.reverseIterator()) { - rIt.advance(firstUnmodifiedKey); - while (rIt.hasNext()) { - final long nextKey = rIt.nextLong(); - potentialResetTimestamp = timeRecorder.getCurrentLong(nextKey); - if (potentialResetTimestamp != NULL_LONG && isValueValid(nextKey)) { - return potentialResetTimestamp; + if (rIt.advance(firstUnmodifiedKey)) { + while (rIt.hasNext()) { + final long nextKey = rIt.nextLong(); + potentialResetTimestamp = timeRecorder.getCurrentLong(nextKey); + if (potentialResetTimestamp != NULL_LONG && isValueValid(nextKey)) { + return potentialResetTimestamp; + } } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java index eca6d81a3cd..ee178bac861 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java @@ -38,7 +38,6 @@ public abstract class BaseByteUpdateByOperator extends UpdateByCumulativeOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; - protected boolean trackingPrev = false; protected final MatchPair pair; protected final String[] affectingColumns; @@ -217,12 +216,9 @@ public boolean requiresValues(@NotNull final UpdateContext context) { @Override public void startTrackingPrev() { - if (!trackingPrev) { - trackingPrev = true; - outputSource.startTrackingPrevValues(); - if (isRedirected) { - maybeInnerSource.startTrackingPrevValues(); - } + outputSource.startTrackingPrevValues(); + if (isRedirected) { + maybeInnerSource.startTrackingPrevValues(); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java index ad3468aa267..b6d34d55fbe 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java @@ -28,7 +28,6 @@ public abstract class BaseCharUpdateByOperator extends UpdateByCumulativeOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; - protected boolean trackingPrev = false; protected final MatchPair pair; protected final String[] affectingColumns; @@ -193,12 +192,9 @@ public boolean requiresValues(@NotNull final UpdateContext context) { @Override public void startTrackingPrev() { - if (!trackingPrev) { - trackingPrev = true; - outputSource.startTrackingPrevValues(); - if (isRedirected) { - maybeInnerSource.startTrackingPrevValues(); - } + outputSource.startTrackingPrevValues(); + if (isRedirected) { + maybeInnerSource.startTrackingPrevValues(); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java index 449dc701946..0f8290d523a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java @@ -37,7 +37,6 @@ public abstract class BaseDoubleUpdateByOperator extends UpdateByCumulativeOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; - protected boolean trackingPrev = false; private final MatchPair pair; private final String[] affectingColumns; @@ -187,12 +186,9 @@ public boolean requiresValues(@NotNull final UpdateContext context) { @Override public void startTrackingPrev() { - if (!trackingPrev) { - trackingPrev = true; - outputSource.startTrackingPrevValues(); - if (isRedirected) { - maybeInnerSource.startTrackingPrevValues(); - } + outputSource.startTrackingPrevValues(); + if (isRedirected) { + maybeInnerSource.startTrackingPrevValues(); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java index 6421d1cff26..5223921ccf3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java @@ -32,7 +32,6 @@ public abstract class BaseFloatUpdateByOperator extends UpdateByCumulativeOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; - protected boolean trackingPrev = false; private final MatchPair pair; private final String[] affectingColumns; @@ -182,12 +181,9 @@ public boolean requiresValues(@NotNull final UpdateContext context) { @Override public void startTrackingPrev() { - if (!trackingPrev) { - trackingPrev = true; - outputSource.startTrackingPrevValues(); - if (isRedirected) { - maybeInnerSource.startTrackingPrevValues(); - } + outputSource.startTrackingPrevValues(); + if (isRedirected) { + maybeInnerSource.startTrackingPrevValues(); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java index 5cb77f3316a..563a04abec3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java @@ -33,7 +33,6 @@ public abstract class BaseIntUpdateByOperator extends UpdateByCumulativeOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; - protected boolean trackingPrev = false; protected final MatchPair pair; protected final String[] affectingColumns; @@ -198,12 +197,9 @@ public boolean requiresValues(@NotNull final UpdateContext context) { @Override public void startTrackingPrev() { - if (!trackingPrev) { - trackingPrev = true; - outputSource.startTrackingPrevValues(); - if (isRedirected) { - maybeInnerSource.startTrackingPrevValues(); - } + outputSource.startTrackingPrevValues(); + if (isRedirected) { + maybeInnerSource.startTrackingPrevValues(); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java index 2be4bd95e5d..f4e78c76ab5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java @@ -33,7 +33,6 @@ public abstract class BaseLongUpdateByOperator extends UpdateByCumulativeOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; - protected boolean trackingPrev = false; protected final MatchPair pair; protected final String[] affectingColumns; @@ -198,12 +197,9 @@ public boolean requiresValues(@NotNull final UpdateContext context) { @Override public void startTrackingPrev() { - if (!trackingPrev) { - trackingPrev = true; - outputSource.startTrackingPrevValues(); - if (isRedirected) { - maybeInnerSource.startTrackingPrevValues(); - } + outputSource.startTrackingPrevValues(); + if (isRedirected) { + maybeInnerSource.startTrackingPrevValues(); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java index bf5f822c2da..a2cd5eb7b71 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java @@ -33,7 +33,6 @@ public abstract class BaseObjectUpdateByOperator extends UpdateByCumulativeOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; - protected boolean trackingPrev = false; protected final MatchPair pair; protected final String[] affectingColumns; @@ -201,12 +200,9 @@ public boolean requiresValues(@NotNull final UpdateContext context) { @Override public void startTrackingPrev() { - if (!trackingPrev) { - trackingPrev = true; - outputSource.startTrackingPrevValues(); - if (isRedirected) { - maybeInnerSource.startTrackingPrevValues(); - } + outputSource.startTrackingPrevValues(); + if (isRedirected) { + maybeInnerSource.startTrackingPrevValues(); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java index 5c407379eb4..5cf3f505ebc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java @@ -33,7 +33,6 @@ public abstract class BaseShortUpdateByOperator extends UpdateByCumulativeOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; - protected boolean trackingPrev = false; protected final MatchPair pair; protected final String[] affectingColumns; @@ -198,12 +197,9 @@ public boolean requiresValues(@NotNull final UpdateContext context) { @Override public void startTrackingPrev() { - if (!trackingPrev) { - trackingPrev = true; - outputSource.startTrackingPrevValues(); - if (isRedirected) { - maybeInnerSource.startTrackingPrevValues(); - } + outputSource.startTrackingPrevValues(); + if (isRedirected) { + maybeInnerSource.startTrackingPrevValues(); } } From 6cc64f52b9bc22fa3fa091ad7150b6dfbb7f3e46 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 12 Aug 2022 15:04:22 -0700 Subject: [PATCH 011/123] all static tests running (bucked TBD) --- .../table/impl/UpdateByWindowedOperator.java | 171 +++++++++++++++++- .../engine/table/impl/ZeroKeyUpdateBy.java | 5 - .../BaseWindowedByteUpdateByOperator.java | 155 +--------------- .../BaseWindowedCharUpdateByOperator.java | 155 +--------------- .../BaseWindowedDoubleUpdateByOperator.java | 155 +--------------- .../BaseWindowedFloatUpdateByOperator.java | 155 +--------------- .../BaseWindowedIntUpdateByOperator.java | 155 +--------------- .../BaseWindowedLongUpdateByOperator.java | 155 +--------------- .../BaseWindowedObjectUpdateByOperator.java | 155 +--------------- .../BaseWindowedShortUpdateByOperator.java | 155 +--------------- .../rollingsum/ByteRollingSumOperator.java | 38 ++-- .../rollingsum/DoubleRollingSumOperator.java | 14 +- .../rollingsum/FloatRollingSumOperator.java | 14 +- .../rollingsum/IntRollingSumOperator.java | 38 ++-- .../rollingsum/LongRollingSumOperator.java | 38 ++-- .../rollingsum/ShortRollingSumOperator.java | 38 ++-- 16 files changed, 289 insertions(+), 1307 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java index 85fe7800126..dd84b171eaa 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java @@ -1,10 +1,13 @@ package io.deephaven.engine.table.impl; import io.deephaven.api.updateby.OperationControl; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.rowset.RowSetBuilderRandom; -import io.deephaven.engine.rowset.RowSetFactory; -import io.deephaven.engine.rowset.WritableRowSet; +import io.deephaven.base.LongRingBuffer; +import io.deephaven.chunk.WritableCharChunk; +import io.deephaven.chunk.WritableLongChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.*; +import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; @@ -31,6 +34,29 @@ public abstract class UpdateWindowedContext implements UpdateContext { // store the current subset of rows that need computation protected RowSet affectedRows = RowSetFactory.empty(); + // candidate data for the window + public final int WINDOW_CHUNK_SIZE = 1024; + + // data that is actually in the current window + public LongRingBuffer windowRowKeys = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); + + // the selector that determines whether this value should be in the window, positions for tick-based and + // timestamps for time-based operators + public LongRingBuffer windowSelector = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); + + public RowSequence.Iterator windowIterator = null; + + public RowSet workingRowSet = null; + + public WritableLongChunk candidateRowKeysChunk; + public WritableLongChunk candidatePositionsChunk; + public WritableLongChunk candidateTimestampsChunk; + + // position data for the chunk being currently processed + public WritableLongChunk valuePositionChunk; + + public int candidateWindowIndex = 0; + /*** * This function is only correct if the proper {@code source} rowset is provided. If using buckets, then the * provided rowset must be limited to the rows in the current bucket @@ -94,8 +120,140 @@ public RowSet getAffectedRows() { return affectedRows; } + public abstract void loadCandidateValueChunk(RowSequence windowRowSequence); + + /*** + * Fill the working chunks with data for this key + * + * @param startKey the key for which we want to + */ + public void loadWindowChunks(final long startKey) { + // TODO: make sure this works for bucketed + if (windowIterator == null) { + windowIterator = workingRowSet.getRowSequenceIterator(); + } + windowIterator.advance(startKey); + + RowSequence windowRowSequence = windowIterator.getNextRowSequenceWithLength(WINDOW_CHUNK_SIZE); + + loadCandidateValueChunk(windowRowSequence); + + // fill the window keys chunk + if (candidateRowKeysChunk == null) { + candidateRowKeysChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + } + windowRowSequence.fillRowKeyChunk(candidateRowKeysChunk); + + if (recorder == null) { + // get position data for the window items (relative to the table or bucket rowset) + if (candidatePositionsChunk == null) { + candidatePositionsChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + } + + // TODO: gotta be a better way than creating two rowsets + try (final RowSet rs = windowRowSequence.asRowSet(); + final RowSet positions = workingRowSet.invert(rs)) { + positions.fillRowKeyChunk(candidatePositionsChunk); + } + } else { + // get timestamp values from the recorder column source + if (candidateTimestampsChunk == null) { + candidateTimestampsChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + } + try (final ChunkSource.FillContext fc = recorder.getColumnSource().makeFillContext(WINDOW_CHUNK_SIZE)) { + recorder.getColumnSource().fillChunk(fc, candidateTimestampsChunk, windowRowSequence); + } + } + + // reset the index to beginning of the chunks + candidateWindowIndex = 0; + } + + /*** + * Fill the working chunks with data for this key + * + * @param inputKeys the keys for which we want to get position or timestamp values + */ + public void loadDataChunks(final RowSequence inputKeys) { + if (recorder != null) { + // timestamp data will be available from the recorder + return; + } + + if (valuePositionChunk == null) { + valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); + } else if (valuePositionChunk.capacity() < inputKeys.size()) { + valuePositionChunk.close(); + valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); + } + + // produce position data for the window (will be timestamps for time-based) + // TODO: gotta be a better way than creating two rowsets + try (final RowSet rs = inputKeys.asRowSet(); + final RowSet positions = workingRowSet.invert(rs)) { + positions.fillRowKeyChunk(valuePositionChunk); + } + } + + public void fillWindowTicks(UpdateContext context, long currentPos) { + // compute the head and tail (inclusive) + final long tail = Math.max(0, currentPos - reverseTimeScaleUnits + 1); + final long head = Math.min(workingRowSet.size() - 1, currentPos + forwardTimeScaleUnits); + + while (windowSelector.peek(Long.MAX_VALUE) < tail) { + final long pos = windowSelector.remove(); + final long key = windowRowKeys.remove(); + + pop(context, key); + } + + + // look at the window data and push until satisfied or at the end of the rowset + while (candidatePositionsChunk.size() > 0 && candidatePositionsChunk.get(candidateWindowIndex) <= head) { + final long pos = candidatePositionsChunk.get(candidateWindowIndex); + final long key = candidateRowKeysChunk.get(candidateWindowIndex); + + push(context, key, candidateWindowIndex); + + windowSelector.add(pos); + windowRowKeys.add(key); + + if (++candidateWindowIndex >= candidatePositionsChunk.size()) { + // load the next chunk in order + loadWindowChunks(key + 1); + } + } + + if (windowSelector.isEmpty()) { + reset(context); + } + } + @Override public void close() { + if (windowIterator != null) { + windowIterator.close(); + windowIterator = null; + } + + if (candidateRowKeysChunk != null) { + candidateRowKeysChunk.close(); + candidateRowKeysChunk = null; + } + + if (candidatePositionsChunk != null) { + candidatePositionsChunk.close(); + candidatePositionsChunk = null; + } + + if (valuePositionChunk != null) { + valuePositionChunk.close(); + valuePositionChunk = null; + } + + // no need to close this, just release the reference + workingRowSet = null; + affectedRows.close(); } } @@ -126,6 +284,11 @@ public UpdateByWindowedOperator(@NotNull final MatchPair pair, this.isRedirected = rowRedirection != null; } + public abstract void push(UpdateContext context, long key, int index); + public abstract void pop(UpdateContext context, long key); + public abstract void reset(UpdateContext context); + + // return the first row that affects this key public long computeFirstAffectingKey(long key, @NotNull final RowSet source) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java index 5b59f5f7b64..7ad5c6b93d2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java @@ -47,11 +47,6 @@ public static Table compute(@NotNull final String description, updateBy.doInitialAdditions(); if (source.isRefreshing()) { -// // start tracking previous values -// if (rowRedirection != null) { -// rowRedirection.startTrackingPrevValues(); -// } -// Arrays.stream(ops).forEach(UpdateByOperator::startTrackingPrev); final ZeroKeyUpdateByListener listener = updateBy.newListener(description, result); source.listenForUpdates(listener); result.addParentReference(listener); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java index 6076d9288ad..10beb05d6e5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java @@ -40,28 +40,8 @@ public abstract class BaseWindowedByteUpdateByOperator extends UpdateByWindowedO protected class Context extends UpdateWindowedContext { public boolean canProcessDirectly; - public RowSet workingRowSet = null; - - // candidate data for the window - public final int WINDOW_CHUNK_SIZE = 1024; - - // data that is actually in the current window - public LongRingBuffer windowRowKeys = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); - // the selector that determines whether this value should be in the window, positions for tick-based and - // timestamps for time-based operators - public LongRingBuffer windowSelector = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); - - public RowSequence.Iterator windowIterator = null; public WritableByteChunk candidateValuesChunk; - public WritableLongChunk candidateRowKeysChunk; - public WritableLongChunk candidatePositionsChunk; - public WritableLongChunk candidateTimestampsChunk; - - public int candidateWindowIndex = 0; - - // position data for the chunk being currently processed - public WritableLongChunk valuePositionChunk; // other useful stuff public UpdateBy.UpdateType currentUpdateType; @@ -78,49 +58,15 @@ public RowSetBuilderSequential getModifiedBuilder() { @Override public void close() { - if (windowIterator != null) { - windowIterator.close(); - windowIterator = null; - } - + super.close(); if (candidateValuesChunk != null) { candidateValuesChunk.close(); candidateValuesChunk = null; } - - if (candidateRowKeysChunk != null) { - candidateRowKeysChunk.close(); - candidateRowKeysChunk = null; - } - - if (candidatePositionsChunk != null) { - candidatePositionsChunk.close(); - candidatePositionsChunk = null; - } - - if (valuePositionChunk != null) { - valuePositionChunk.close(); - valuePositionChunk = null; - } - - // no need to close, just release the reference - workingRowSet = null; } - /*** - * Fill the working chunks with data for this key - * - * @param startKey the key for which we want to - */ - public void loadWindowChunks(final long startKey) { - // TODO: make sure this works for bucketed - if (windowIterator == null) { - windowIterator = workingRowSet.getRowSequenceIterator(); - } - windowIterator.advance(startKey); - - RowSequence windowRowSequence = windowIterator.getNextRowSequenceWithLength(WINDOW_CHUNK_SIZE); - + @Override + public void loadCandidateValueChunk(RowSequence windowRowSequence) { // fill the window values chunk if (candidateValuesChunk == null) { candidateValuesChunk = WritableByteChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); @@ -128,97 +74,6 @@ public void loadWindowChunks(final long startKey) { try (ChunkSource.FillContext fc = valueSource.makeFillContext(WINDOW_CHUNK_SIZE)){ valueSource.fillChunk(fc, candidateValuesChunk, windowRowSequence); } - - // fill the window keys chunk - if (candidateRowKeysChunk == null) { - candidateRowKeysChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); - } - windowRowSequence.fillRowKeyChunk(candidateRowKeysChunk); - - if (recorder == null) { - // get position data for the window items (relative to the table or bucket rowset) - if (candidatePositionsChunk == null) { - candidatePositionsChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); - } - - // TODO: gotta be a better way than creating two rowsets - try (final RowSet rs = windowRowSequence.asRowSet(); - final RowSet positions = workingRowSet.invert(rs)) { - positions.fillRowKeyChunk(candidatePositionsChunk); - } - } else { - // get timestamp values from the recorder column source - if (candidateTimestampsChunk == null) { - candidateTimestampsChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); - } - try (final ChunkSource.FillContext fc = recorder.getColumnSource().makeFillContext(WINDOW_CHUNK_SIZE)) { - recorder.getColumnSource().fillChunk(fc, candidateTimestampsChunk, windowRowSequence); - } - } - - // reset the index to beginning of the chunks - candidateWindowIndex = 0; - } - - /*** - * Fill the working chunks with data for this key - * - * @param inputKeys the keys for which we want to get position or timestamp values - */ - public void loadDataChunks(final RowSequence inputKeys) { - if (recorder != null) { - // timestamp data will be available from the recorder - return; - } - - if (valuePositionChunk == null) { - valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); - } else if (valuePositionChunk.capacity() < inputKeys.size()) { - valuePositionChunk.close(); - valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); - } - - // produce position data for the window (will be timestamps for time-based) - // TODO: gotta be a better way than creating two rowsets - try (final RowSet rs = inputKeys.asRowSet(); - final RowSet positions = workingRowSet.invert(rs)) { - positions.fillRowKeyChunk(valuePositionChunk); - } - } - - public void fillWindowTicks(UpdateContext context, long currentPos) { - // compute the head and tail (inclusive) - final long tail = Math.max(0, currentPos - reverseTimeScaleUnits + 1); - final long head = Math.min(workingRowSet.size() - 1, currentPos + forwardTimeScaleUnits); - - while (windowSelector.peek(Long.MAX_VALUE) < tail) { - final long pos = windowSelector.remove(); - final long key = windowRowKeys.remove(); - - pop(context, key); - } - - - // look at the window data and push until satisfied or at the end of the rowset - while (candidatePositionsChunk.size() > 0 && candidatePositionsChunk.get(candidateWindowIndex) <= head) { - final long pos = candidatePositionsChunk.get(candidateWindowIndex); - final long key = candidateRowKeysChunk.get(candidateWindowIndex); - final byte val = candidateValuesChunk.get(candidateWindowIndex); - - push(context, key, val); - - windowSelector.add(pos); - windowRowKeys.add(key); - - if (++candidateWindowIndex >= candidatePositionsChunk.size()) { - // load the next chunk in order - loadWindowChunks(key + 1); - } - } - - if (windowSelector.isEmpty()) { - reset(context); - } } } @@ -255,10 +110,6 @@ protected WritableColumnSource makeDenseSource() { } // endregion extra-methods - public abstract void push(UpdateContext context, long key, byte val); - public abstract void pop(UpdateContext context, long key); - public abstract void reset(UpdateContext context); - @Override public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, final long lastPrevKey, boolean isUpstreamAppendOnly) { final Context ctx = (Context) context; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java index 9d18723f41a..75f354c5ddc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java @@ -29,28 +29,8 @@ public abstract class BaseWindowedCharUpdateByOperator extends UpdateByWindowedO protected class Context extends UpdateWindowedContext { public boolean canProcessDirectly; - public RowSet workingRowSet = null; - - // candidate data for the window - public final int WINDOW_CHUNK_SIZE = 1024; - - // data that is actually in the current window - public LongRingBuffer windowRowKeys = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); - // the selector that determines whether this value should be in the window, positions for tick-based and - // timestamps for time-based operators - public LongRingBuffer windowSelector = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); - - public RowSequence.Iterator windowIterator = null; public WritableCharChunk candidateValuesChunk; - public WritableLongChunk candidateRowKeysChunk; - public WritableLongChunk candidatePositionsChunk; - public WritableLongChunk candidateTimestampsChunk; - - public int candidateWindowIndex = 0; - - // position data for the chunk being currently processed - public WritableLongChunk valuePositionChunk; // other useful stuff public UpdateBy.UpdateType currentUpdateType; @@ -67,49 +47,15 @@ public RowSetBuilderSequential getModifiedBuilder() { @Override public void close() { - if (windowIterator != null) { - windowIterator.close(); - windowIterator = null; - } - + super.close(); if (candidateValuesChunk != null) { candidateValuesChunk.close(); candidateValuesChunk = null; } - - if (candidateRowKeysChunk != null) { - candidateRowKeysChunk.close(); - candidateRowKeysChunk = null; - } - - if (candidatePositionsChunk != null) { - candidatePositionsChunk.close(); - candidatePositionsChunk = null; - } - - if (valuePositionChunk != null) { - valuePositionChunk.close(); - valuePositionChunk = null; - } - - // no need to close, just release the reference - workingRowSet = null; } - /*** - * Fill the working chunks with data for this key - * - * @param startKey the key for which we want to - */ - public void loadWindowChunks(final long startKey) { - // TODO: make sure this works for bucketed - if (windowIterator == null) { - windowIterator = workingRowSet.getRowSequenceIterator(); - } - windowIterator.advance(startKey); - - RowSequence windowRowSequence = windowIterator.getNextRowSequenceWithLength(WINDOW_CHUNK_SIZE); - + @Override + public void loadCandidateValueChunk(RowSequence windowRowSequence) { // fill the window values chunk if (candidateValuesChunk == null) { candidateValuesChunk = WritableCharChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); @@ -117,97 +63,6 @@ public void loadWindowChunks(final long startKey) { try (ChunkSource.FillContext fc = valueSource.makeFillContext(WINDOW_CHUNK_SIZE)){ valueSource.fillChunk(fc, candidateValuesChunk, windowRowSequence); } - - // fill the window keys chunk - if (candidateRowKeysChunk == null) { - candidateRowKeysChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); - } - windowRowSequence.fillRowKeyChunk(candidateRowKeysChunk); - - if (recorder == null) { - // get position data for the window items (relative to the table or bucket rowset) - if (candidatePositionsChunk == null) { - candidatePositionsChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); - } - - // TODO: gotta be a better way than creating two rowsets - try (final RowSet rs = windowRowSequence.asRowSet(); - final RowSet positions = workingRowSet.invert(rs)) { - positions.fillRowKeyChunk(candidatePositionsChunk); - } - } else { - // get timestamp values from the recorder column source - if (candidateTimestampsChunk == null) { - candidateTimestampsChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); - } - try (final ChunkSource.FillContext fc = recorder.getColumnSource().makeFillContext(WINDOW_CHUNK_SIZE)) { - recorder.getColumnSource().fillChunk(fc, candidateTimestampsChunk, windowRowSequence); - } - } - - // reset the index to beginning of the chunks - candidateWindowIndex = 0; - } - - /*** - * Fill the working chunks with data for this key - * - * @param inputKeys the keys for which we want to get position or timestamp values - */ - public void loadDataChunks(final RowSequence inputKeys) { - if (recorder != null) { - // timestamp data will be available from the recorder - return; - } - - if (valuePositionChunk == null) { - valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); - } else if (valuePositionChunk.capacity() < inputKeys.size()) { - valuePositionChunk.close(); - valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); - } - - // produce position data for the window (will be timestamps for time-based) - // TODO: gotta be a better way than creating two rowsets - try (final RowSet rs = inputKeys.asRowSet(); - final RowSet positions = workingRowSet.invert(rs)) { - positions.fillRowKeyChunk(valuePositionChunk); - } - } - - public void fillWindowTicks(UpdateContext context, long currentPos) { - // compute the head and tail (inclusive) - final long tail = Math.max(0, currentPos - reverseTimeScaleUnits + 1); - final long head = Math.min(workingRowSet.size() - 1, currentPos + forwardTimeScaleUnits); - - while (windowSelector.peek(Long.MAX_VALUE) < tail) { - final long pos = windowSelector.remove(); - final long key = windowRowKeys.remove(); - - pop(context, key); - } - - - // look at the window data and push until satisfied or at the end of the rowset - while (candidatePositionsChunk.size() > 0 && candidatePositionsChunk.get(candidateWindowIndex) <= head) { - final long pos = candidatePositionsChunk.get(candidateWindowIndex); - final long key = candidateRowKeysChunk.get(candidateWindowIndex); - final char val = candidateValuesChunk.get(candidateWindowIndex); - - push(context, key, val); - - windowSelector.add(pos); - windowRowKeys.add(key); - - if (++candidateWindowIndex >= candidatePositionsChunk.size()) { - // load the next chunk in order - loadWindowChunks(key + 1); - } - } - - if (windowSelector.isEmpty()) { - reset(context); - } } } @@ -231,10 +86,6 @@ public BaseWindowedCharUpdateByOperator(@NotNull final MatchPair pair, // region extra-methods // endregion extra-methods - public abstract void push(UpdateContext context, long key, char val); - public abstract void pop(UpdateContext context, long key); - public abstract void reset(UpdateContext context); - @Override public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, final long lastPrevKey, boolean isUpstreamAppendOnly) { final Context ctx = (Context) context; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java index 722e5ab837c..0b3687f9061 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java @@ -32,28 +32,8 @@ public abstract class BaseWindowedDoubleUpdateByOperator extends UpdateByWindowe protected class Context extends UpdateWindowedContext { public boolean canProcessDirectly; - public RowSet workingRowSet = null; - - // candidate data for the window - public final int WINDOW_CHUNK_SIZE = 1024; - - // data that is actually in the current window - public LongRingBuffer windowRowKeys = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); - // the selector that determines whether this value should be in the window, positions for tick-based and - // timestamps for time-based operators - public LongRingBuffer windowSelector = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); - - public RowSequence.Iterator windowIterator = null; public WritableDoubleChunk candidateValuesChunk; - public WritableLongChunk candidateRowKeysChunk; - public WritableLongChunk candidatePositionsChunk; - public WritableLongChunk candidateTimestampsChunk; - - public int candidateWindowIndex = 0; - - // position data for the chunk being currently processed - public WritableLongChunk valuePositionChunk; // other useful stuff public UpdateBy.UpdateType currentUpdateType; @@ -70,49 +50,15 @@ public RowSetBuilderSequential getModifiedBuilder() { @Override public void close() { - if (windowIterator != null) { - windowIterator.close(); - windowIterator = null; - } - + super.close(); if (candidateValuesChunk != null) { candidateValuesChunk.close(); candidateValuesChunk = null; } - - if (candidateRowKeysChunk != null) { - candidateRowKeysChunk.close(); - candidateRowKeysChunk = null; - } - - if (candidatePositionsChunk != null) { - candidatePositionsChunk.close(); - candidatePositionsChunk = null; - } - - if (valuePositionChunk != null) { - valuePositionChunk.close(); - valuePositionChunk = null; - } - - // no need to close, just release the reference - workingRowSet = null; } - /*** - * Fill the working chunks with data for this key - * - * @param startKey the key for which we want to - */ - public void loadWindowChunks(final long startKey) { - // TODO: make sure this works for bucketed - if (windowIterator == null) { - windowIterator = workingRowSet.getRowSequenceIterator(); - } - windowIterator.advance(startKey); - - RowSequence windowRowSequence = windowIterator.getNextRowSequenceWithLength(WINDOW_CHUNK_SIZE); - + @Override + public void loadCandidateValueChunk(RowSequence windowRowSequence) { // fill the window values chunk if (candidateValuesChunk == null) { candidateValuesChunk = WritableDoubleChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); @@ -120,97 +66,6 @@ public void loadWindowChunks(final long startKey) { try (ChunkSource.FillContext fc = valueSource.makeFillContext(WINDOW_CHUNK_SIZE)){ valueSource.fillChunk(fc, candidateValuesChunk, windowRowSequence); } - - // fill the window keys chunk - if (candidateRowKeysChunk == null) { - candidateRowKeysChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); - } - windowRowSequence.fillRowKeyChunk(candidateRowKeysChunk); - - if (recorder == null) { - // get position data for the window items (relative to the table or bucket rowset) - if (candidatePositionsChunk == null) { - candidatePositionsChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); - } - - // TODO: gotta be a better way than creating two rowsets - try (final RowSet rs = windowRowSequence.asRowSet(); - final RowSet positions = workingRowSet.invert(rs)) { - positions.fillRowKeyChunk(candidatePositionsChunk); - } - } else { - // get timestamp values from the recorder column source - if (candidateTimestampsChunk == null) { - candidateTimestampsChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); - } - try (final ChunkSource.FillContext fc = recorder.getColumnSource().makeFillContext(WINDOW_CHUNK_SIZE)) { - recorder.getColumnSource().fillChunk(fc, candidateTimestampsChunk, windowRowSequence); - } - } - - // reset the index to beginning of the chunks - candidateWindowIndex = 0; - } - - /*** - * Fill the working chunks with data for this key - * - * @param inputKeys the keys for which we want to get position or timestamp values - */ - public void loadDataChunks(final RowSequence inputKeys) { - if (recorder != null) { - // timestamp data will be available from the recorder - return; - } - - if (valuePositionChunk == null) { - valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); - } else if (valuePositionChunk.capacity() < inputKeys.size()) { - valuePositionChunk.close(); - valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); - } - - // produce position data for the window (will be timestamps for time-based) - // TODO: gotta be a better way than creating two rowsets - try (final RowSet rs = inputKeys.asRowSet(); - final RowSet positions = workingRowSet.invert(rs)) { - positions.fillRowKeyChunk(valuePositionChunk); - } - } - - public void fillWindowTicks(UpdateContext context, long currentPos) { - // compute the head and tail (inclusive) - final long tail = Math.max(0, currentPos - reverseTimeScaleUnits + 1); - final long head = Math.min(workingRowSet.size() - 1, currentPos + forwardTimeScaleUnits); - - while (windowSelector.peek(Long.MAX_VALUE) < tail) { - final long pos = windowSelector.remove(); - final long key = windowRowKeys.remove(); - - pop(context, key); - } - - - // look at the window data and push until satisfied or at the end of the rowset - while (candidatePositionsChunk.size() > 0 && candidatePositionsChunk.get(candidateWindowIndex) <= head) { - final long pos = candidatePositionsChunk.get(candidateWindowIndex); - final long key = candidateRowKeysChunk.get(candidateWindowIndex); - final double val = candidateValuesChunk.get(candidateWindowIndex); - - push(context, key, val); - - windowSelector.add(pos); - windowRowKeys.add(key); - - if (++candidateWindowIndex >= candidatePositionsChunk.size()) { - // load the next chunk in order - loadWindowChunks(key + 1); - } - } - - if (windowSelector.isEmpty()) { - reset(context); - } } } @@ -234,10 +89,6 @@ public BaseWindowedDoubleUpdateByOperator(@NotNull final MatchPair pair, // region extra-methods // endregion extra-methods - public abstract void push(UpdateContext context, long key, double val); - public abstract void pop(UpdateContext context, long key); - public abstract void reset(UpdateContext context); - @Override public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, final long lastPrevKey, boolean isUpstreamAppendOnly) { final Context ctx = (Context) context; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java index 59db0b11f0f..44aaf1ebac1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java @@ -27,28 +27,8 @@ public abstract class BaseWindowedFloatUpdateByOperator extends UpdateByWindowed protected class Context extends UpdateWindowedContext { public boolean canProcessDirectly; - public RowSet workingRowSet = null; - - // candidate data for the window - public final int WINDOW_CHUNK_SIZE = 1024; - - // data that is actually in the current window - public LongRingBuffer windowRowKeys = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); - // the selector that determines whether this value should be in the window, positions for tick-based and - // timestamps for time-based operators - public LongRingBuffer windowSelector = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); - - public RowSequence.Iterator windowIterator = null; public WritableFloatChunk candidateValuesChunk; - public WritableLongChunk candidateRowKeysChunk; - public WritableLongChunk candidatePositionsChunk; - public WritableLongChunk candidateTimestampsChunk; - - public int candidateWindowIndex = 0; - - // position data for the chunk being currently processed - public WritableLongChunk valuePositionChunk; // other useful stuff public UpdateBy.UpdateType currentUpdateType; @@ -65,49 +45,15 @@ public RowSetBuilderSequential getModifiedBuilder() { @Override public void close() { - if (windowIterator != null) { - windowIterator.close(); - windowIterator = null; - } - + super.close(); if (candidateValuesChunk != null) { candidateValuesChunk.close(); candidateValuesChunk = null; } - - if (candidateRowKeysChunk != null) { - candidateRowKeysChunk.close(); - candidateRowKeysChunk = null; - } - - if (candidatePositionsChunk != null) { - candidatePositionsChunk.close(); - candidatePositionsChunk = null; - } - - if (valuePositionChunk != null) { - valuePositionChunk.close(); - valuePositionChunk = null; - } - - // no need to close, just release the reference - workingRowSet = null; } - /*** - * Fill the working chunks with data for this key - * - * @param startKey the key for which we want to - */ - public void loadWindowChunks(final long startKey) { - // TODO: make sure this works for bucketed - if (windowIterator == null) { - windowIterator = workingRowSet.getRowSequenceIterator(); - } - windowIterator.advance(startKey); - - RowSequence windowRowSequence = windowIterator.getNextRowSequenceWithLength(WINDOW_CHUNK_SIZE); - + @Override + public void loadCandidateValueChunk(RowSequence windowRowSequence) { // fill the window values chunk if (candidateValuesChunk == null) { candidateValuesChunk = WritableFloatChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); @@ -115,97 +61,6 @@ public void loadWindowChunks(final long startKey) { try (ChunkSource.FillContext fc = valueSource.makeFillContext(WINDOW_CHUNK_SIZE)){ valueSource.fillChunk(fc, candidateValuesChunk, windowRowSequence); } - - // fill the window keys chunk - if (candidateRowKeysChunk == null) { - candidateRowKeysChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); - } - windowRowSequence.fillRowKeyChunk(candidateRowKeysChunk); - - if (recorder == null) { - // get position data for the window items (relative to the table or bucket rowset) - if (candidatePositionsChunk == null) { - candidatePositionsChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); - } - - // TODO: gotta be a better way than creating two rowsets - try (final RowSet rs = windowRowSequence.asRowSet(); - final RowSet positions = workingRowSet.invert(rs)) { - positions.fillRowKeyChunk(candidatePositionsChunk); - } - } else { - // get timestamp values from the recorder column source - if (candidateTimestampsChunk == null) { - candidateTimestampsChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); - } - try (final ChunkSource.FillContext fc = recorder.getColumnSource().makeFillContext(WINDOW_CHUNK_SIZE)) { - recorder.getColumnSource().fillChunk(fc, candidateTimestampsChunk, windowRowSequence); - } - } - - // reset the index to beginning of the chunks - candidateWindowIndex = 0; - } - - /*** - * Fill the working chunks with data for this key - * - * @param inputKeys the keys for which we want to get position or timestamp values - */ - public void loadDataChunks(final RowSequence inputKeys) { - if (recorder != null) { - // timestamp data will be available from the recorder - return; - } - - if (valuePositionChunk == null) { - valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); - } else if (valuePositionChunk.capacity() < inputKeys.size()) { - valuePositionChunk.close(); - valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); - } - - // produce position data for the window (will be timestamps for time-based) - // TODO: gotta be a better way than creating two rowsets - try (final RowSet rs = inputKeys.asRowSet(); - final RowSet positions = workingRowSet.invert(rs)) { - positions.fillRowKeyChunk(valuePositionChunk); - } - } - - public void fillWindowTicks(UpdateContext context, long currentPos) { - // compute the head and tail (inclusive) - final long tail = Math.max(0, currentPos - reverseTimeScaleUnits + 1); - final long head = Math.min(workingRowSet.size() - 1, currentPos + forwardTimeScaleUnits); - - while (windowSelector.peek(Long.MAX_VALUE) < tail) { - final long pos = windowSelector.remove(); - final long key = windowRowKeys.remove(); - - pop(context, key); - } - - - // look at the window data and push until satisfied or at the end of the rowset - while (candidatePositionsChunk.size() > 0 && candidatePositionsChunk.get(candidateWindowIndex) <= head) { - final long pos = candidatePositionsChunk.get(candidateWindowIndex); - final long key = candidateRowKeysChunk.get(candidateWindowIndex); - final float val = candidateValuesChunk.get(candidateWindowIndex); - - push(context, key, val); - - windowSelector.add(pos); - windowRowKeys.add(key); - - if (++candidateWindowIndex >= candidatePositionsChunk.size()) { - // load the next chunk in order - loadWindowChunks(key + 1); - } - } - - if (windowSelector.isEmpty()) { - reset(context); - } } } @@ -229,10 +84,6 @@ public BaseWindowedFloatUpdateByOperator(@NotNull final MatchPair pair, // region extra-methods // endregion extra-methods - public abstract void push(UpdateContext context, long key, float val); - public abstract void pop(UpdateContext context, long key); - public abstract void reset(UpdateContext context); - @Override public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, final long lastPrevKey, boolean isUpstreamAppendOnly) { final Context ctx = (Context) context; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java index fa3bd2f9a85..591571bc2a3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java @@ -34,28 +34,8 @@ public abstract class BaseWindowedIntUpdateByOperator extends UpdateByWindowedOp protected class Context extends UpdateWindowedContext { public boolean canProcessDirectly; - public RowSet workingRowSet = null; - - // candidate data for the window - public final int WINDOW_CHUNK_SIZE = 1024; - - // data that is actually in the current window - public LongRingBuffer windowRowKeys = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); - // the selector that determines whether this value should be in the window, positions for tick-based and - // timestamps for time-based operators - public LongRingBuffer windowSelector = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); - - public RowSequence.Iterator windowIterator = null; public WritableIntChunk candidateValuesChunk; - public WritableLongChunk candidateRowKeysChunk; - public WritableLongChunk candidatePositionsChunk; - public WritableLongChunk candidateTimestampsChunk; - - public int candidateWindowIndex = 0; - - // position data for the chunk being currently processed - public WritableLongChunk valuePositionChunk; // other useful stuff public UpdateBy.UpdateType currentUpdateType; @@ -72,49 +52,15 @@ public RowSetBuilderSequential getModifiedBuilder() { @Override public void close() { - if (windowIterator != null) { - windowIterator.close(); - windowIterator = null; - } - + super.close(); if (candidateValuesChunk != null) { candidateValuesChunk.close(); candidateValuesChunk = null; } - - if (candidateRowKeysChunk != null) { - candidateRowKeysChunk.close(); - candidateRowKeysChunk = null; - } - - if (candidatePositionsChunk != null) { - candidatePositionsChunk.close(); - candidatePositionsChunk = null; - } - - if (valuePositionChunk != null) { - valuePositionChunk.close(); - valuePositionChunk = null; - } - - // no need to close, just release the reference - workingRowSet = null; } - /*** - * Fill the working chunks with data for this key - * - * @param startKey the key for which we want to - */ - public void loadWindowChunks(final long startKey) { - // TODO: make sure this works for bucketed - if (windowIterator == null) { - windowIterator = workingRowSet.getRowSequenceIterator(); - } - windowIterator.advance(startKey); - - RowSequence windowRowSequence = windowIterator.getNextRowSequenceWithLength(WINDOW_CHUNK_SIZE); - + @Override + public void loadCandidateValueChunk(RowSequence windowRowSequence) { // fill the window values chunk if (candidateValuesChunk == null) { candidateValuesChunk = WritableIntChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); @@ -122,97 +68,6 @@ public void loadWindowChunks(final long startKey) { try (ChunkSource.FillContext fc = valueSource.makeFillContext(WINDOW_CHUNK_SIZE)){ valueSource.fillChunk(fc, candidateValuesChunk, windowRowSequence); } - - // fill the window keys chunk - if (candidateRowKeysChunk == null) { - candidateRowKeysChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); - } - windowRowSequence.fillRowKeyChunk(candidateRowKeysChunk); - - if (recorder == null) { - // get position data for the window items (relative to the table or bucket rowset) - if (candidatePositionsChunk == null) { - candidatePositionsChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); - } - - // TODO: gotta be a better way than creating two rowsets - try (final RowSet rs = windowRowSequence.asRowSet(); - final RowSet positions = workingRowSet.invert(rs)) { - positions.fillRowKeyChunk(candidatePositionsChunk); - } - } else { - // get timestamp values from the recorder column source - if (candidateTimestampsChunk == null) { - candidateTimestampsChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); - } - try (final ChunkSource.FillContext fc = recorder.getColumnSource().makeFillContext(WINDOW_CHUNK_SIZE)) { - recorder.getColumnSource().fillChunk(fc, candidateTimestampsChunk, windowRowSequence); - } - } - - // reset the index to beginning of the chunks - candidateWindowIndex = 0; - } - - /*** - * Fill the working chunks with data for this key - * - * @param inputKeys the keys for which we want to get position or timestamp values - */ - public void loadDataChunks(final RowSequence inputKeys) { - if (recorder != null) { - // timestamp data will be available from the recorder - return; - } - - if (valuePositionChunk == null) { - valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); - } else if (valuePositionChunk.capacity() < inputKeys.size()) { - valuePositionChunk.close(); - valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); - } - - // produce position data for the window (will be timestamps for time-based) - // TODO: gotta be a better way than creating two rowsets - try (final RowSet rs = inputKeys.asRowSet(); - final RowSet positions = workingRowSet.invert(rs)) { - positions.fillRowKeyChunk(valuePositionChunk); - } - } - - public void fillWindowTicks(UpdateContext context, long currentPos) { - // compute the head and tail (inclusive) - final long tail = Math.max(0, currentPos - reverseTimeScaleUnits + 1); - final long head = Math.min(workingRowSet.size() - 1, currentPos + forwardTimeScaleUnits); - - while (windowSelector.peek(Long.MAX_VALUE) < tail) { - final long pos = windowSelector.remove(); - final long key = windowRowKeys.remove(); - - pop(context, key); - } - - - // look at the window data and push until satisfied or at the end of the rowset - while (candidatePositionsChunk.size() > 0 && candidatePositionsChunk.get(candidateWindowIndex) <= head) { - final long pos = candidatePositionsChunk.get(candidateWindowIndex); - final long key = candidateRowKeysChunk.get(candidateWindowIndex); - final int val = candidateValuesChunk.get(candidateWindowIndex); - - push(context, key, val); - - windowSelector.add(pos); - windowRowKeys.add(key); - - if (++candidateWindowIndex >= candidatePositionsChunk.size()) { - // load the next chunk in order - loadWindowChunks(key + 1); - } - } - - if (windowSelector.isEmpty()) { - reset(context); - } } } @@ -236,10 +91,6 @@ public BaseWindowedIntUpdateByOperator(@NotNull final MatchPair pair, // region extra-methods // endregion extra-methods - public abstract void push(UpdateContext context, long key, int val); - public abstract void pop(UpdateContext context, long key); - public abstract void reset(UpdateContext context); - @Override public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, final long lastPrevKey, boolean isUpstreamAppendOnly) { final Context ctx = (Context) context; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java index 461b1aaf263..b3ac6426dec 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java @@ -34,28 +34,8 @@ public abstract class BaseWindowedLongUpdateByOperator extends UpdateByWindowedO protected class Context extends UpdateWindowedContext { public boolean canProcessDirectly; - public RowSet workingRowSet = null; - - // candidate data for the window - public final int WINDOW_CHUNK_SIZE = 1024; - - // data that is actually in the current window - public LongRingBuffer windowRowKeys = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); - // the selector that determines whether this value should be in the window, positions for tick-based and - // timestamps for time-based operators - public LongRingBuffer windowSelector = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); - - public RowSequence.Iterator windowIterator = null; public WritableLongChunk candidateValuesChunk; - public WritableLongChunk candidateRowKeysChunk; - public WritableLongChunk candidatePositionsChunk; - public WritableLongChunk candidateTimestampsChunk; - - public int candidateWindowIndex = 0; - - // position data for the chunk being currently processed - public WritableLongChunk valuePositionChunk; // other useful stuff public UpdateBy.UpdateType currentUpdateType; @@ -72,49 +52,15 @@ public RowSetBuilderSequential getModifiedBuilder() { @Override public void close() { - if (windowIterator != null) { - windowIterator.close(); - windowIterator = null; - } - + super.close(); if (candidateValuesChunk != null) { candidateValuesChunk.close(); candidateValuesChunk = null; } - - if (candidateRowKeysChunk != null) { - candidateRowKeysChunk.close(); - candidateRowKeysChunk = null; - } - - if (candidatePositionsChunk != null) { - candidatePositionsChunk.close(); - candidatePositionsChunk = null; - } - - if (valuePositionChunk != null) { - valuePositionChunk.close(); - valuePositionChunk = null; - } - - // no need to close, just release the reference - workingRowSet = null; } - /*** - * Fill the working chunks with data for this key - * - * @param startKey the key for which we want to - */ - public void loadWindowChunks(final long startKey) { - // TODO: make sure this works for bucketed - if (windowIterator == null) { - windowIterator = workingRowSet.getRowSequenceIterator(); - } - windowIterator.advance(startKey); - - RowSequence windowRowSequence = windowIterator.getNextRowSequenceWithLength(WINDOW_CHUNK_SIZE); - + @Override + public void loadCandidateValueChunk(RowSequence windowRowSequence) { // fill the window values chunk if (candidateValuesChunk == null) { candidateValuesChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); @@ -122,97 +68,6 @@ public void loadWindowChunks(final long startKey) { try (ChunkSource.FillContext fc = valueSource.makeFillContext(WINDOW_CHUNK_SIZE)){ valueSource.fillChunk(fc, candidateValuesChunk, windowRowSequence); } - - // fill the window keys chunk - if (candidateRowKeysChunk == null) { - candidateRowKeysChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); - } - windowRowSequence.fillRowKeyChunk(candidateRowKeysChunk); - - if (recorder == null) { - // get position data for the window items (relative to the table or bucket rowset) - if (candidatePositionsChunk == null) { - candidatePositionsChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); - } - - // TODO: gotta be a better way than creating two rowsets - try (final RowSet rs = windowRowSequence.asRowSet(); - final RowSet positions = workingRowSet.invert(rs)) { - positions.fillRowKeyChunk(candidatePositionsChunk); - } - } else { - // get timestamp values from the recorder column source - if (candidateTimestampsChunk == null) { - candidateTimestampsChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); - } - try (final ChunkSource.FillContext fc = recorder.getColumnSource().makeFillContext(WINDOW_CHUNK_SIZE)) { - recorder.getColumnSource().fillChunk(fc, candidateTimestampsChunk, windowRowSequence); - } - } - - // reset the index to beginning of the chunks - candidateWindowIndex = 0; - } - - /*** - * Fill the working chunks with data for this key - * - * @param inputKeys the keys for which we want to get position or timestamp values - */ - public void loadDataChunks(final RowSequence inputKeys) { - if (recorder != null) { - // timestamp data will be available from the recorder - return; - } - - if (valuePositionChunk == null) { - valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); - } else if (valuePositionChunk.capacity() < inputKeys.size()) { - valuePositionChunk.close(); - valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); - } - - // produce position data for the window (will be timestamps for time-based) - // TODO: gotta be a better way than creating two rowsets - try (final RowSet rs = inputKeys.asRowSet(); - final RowSet positions = workingRowSet.invert(rs)) { - positions.fillRowKeyChunk(valuePositionChunk); - } - } - - public void fillWindowTicks(UpdateContext context, long currentPos) { - // compute the head and tail (inclusive) - final long tail = Math.max(0, currentPos - reverseTimeScaleUnits + 1); - final long head = Math.min(workingRowSet.size() - 1, currentPos + forwardTimeScaleUnits); - - while (windowSelector.peek(Long.MAX_VALUE) < tail) { - final long pos = windowSelector.remove(); - final long key = windowRowKeys.remove(); - - pop(context, key); - } - - - // look at the window data and push until satisfied or at the end of the rowset - while (candidatePositionsChunk.size() > 0 && candidatePositionsChunk.get(candidateWindowIndex) <= head) { - final long pos = candidatePositionsChunk.get(candidateWindowIndex); - final long key = candidateRowKeysChunk.get(candidateWindowIndex); - final long val = candidateValuesChunk.get(candidateWindowIndex); - - push(context, key, val); - - windowSelector.add(pos); - windowRowKeys.add(key); - - if (++candidateWindowIndex >= candidatePositionsChunk.size()) { - // load the next chunk in order - loadWindowChunks(key + 1); - } - } - - if (windowSelector.isEmpty()) { - reset(context); - } } } @@ -236,10 +91,6 @@ public BaseWindowedLongUpdateByOperator(@NotNull final MatchPair pair, // region extra-methods // endregion extra-methods - public abstract void push(UpdateContext context, long key, long val); - public abstract void pop(UpdateContext context, long key); - public abstract void reset(UpdateContext context); - @Override public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, final long lastPrevKey, boolean isUpstreamAppendOnly) { final Context ctx = (Context) context; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java index 6119af4a8f2..5640e52f8ab 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java @@ -35,28 +35,8 @@ public abstract class BaseWindowedObjectUpdateByOperator extends UpdateByWind protected class Context extends UpdateWindowedContext { public boolean canProcessDirectly; - public RowSet workingRowSet = null; - - // candidate data for the window - public final int WINDOW_CHUNK_SIZE = 1024; - - // data that is actually in the current window - public LongRingBuffer windowRowKeys = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); - // the selector that determines whether this value should be in the window, positions for tick-based and - // timestamps for time-based operators - public LongRingBuffer windowSelector = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); - - public RowSequence.Iterator windowIterator = null; public WritableObjectChunk candidateValuesChunk; - public WritableLongChunk candidateRowKeysChunk; - public WritableLongChunk candidatePositionsChunk; - public WritableLongChunk candidateTimestampsChunk; - - public int candidateWindowIndex = 0; - - // position data for the chunk being currently processed - public WritableLongChunk valuePositionChunk; // other useful stuff public UpdateBy.UpdateType currentUpdateType; @@ -73,49 +53,15 @@ public RowSetBuilderSequential getModifiedBuilder() { @Override public void close() { - if (windowIterator != null) { - windowIterator.close(); - windowIterator = null; - } - + super.close(); if (candidateValuesChunk != null) { candidateValuesChunk.close(); candidateValuesChunk = null; } - - if (candidateRowKeysChunk != null) { - candidateRowKeysChunk.close(); - candidateRowKeysChunk = null; - } - - if (candidatePositionsChunk != null) { - candidatePositionsChunk.close(); - candidatePositionsChunk = null; - } - - if (valuePositionChunk != null) { - valuePositionChunk.close(); - valuePositionChunk = null; - } - - // no need to close, just release the reference - workingRowSet = null; } - /*** - * Fill the working chunks with data for this key - * - * @param startKey the key for which we want to - */ - public void loadWindowChunks(final long startKey) { - // TODO: make sure this works for bucketed - if (windowIterator == null) { - windowIterator = workingRowSet.getRowSequenceIterator(); - } - windowIterator.advance(startKey); - - RowSequence windowRowSequence = windowIterator.getNextRowSequenceWithLength(WINDOW_CHUNK_SIZE); - + @Override + public void loadCandidateValueChunk(RowSequence windowRowSequence) { // fill the window values chunk if (candidateValuesChunk == null) { candidateValuesChunk = WritableObjectChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); @@ -123,97 +69,6 @@ public void loadWindowChunks(final long startKey) { try (ChunkSource.FillContext fc = valueSource.makeFillContext(WINDOW_CHUNK_SIZE)){ valueSource.fillChunk(fc, candidateValuesChunk, windowRowSequence); } - - // fill the window keys chunk - if (candidateRowKeysChunk == null) { - candidateRowKeysChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); - } - windowRowSequence.fillRowKeyChunk(candidateRowKeysChunk); - - if (recorder == null) { - // get position data for the window items (relative to the table or bucket rowset) - if (candidatePositionsChunk == null) { - candidatePositionsChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); - } - - // TODO: gotta be a better way than creating two rowsets - try (final RowSet rs = windowRowSequence.asRowSet(); - final RowSet positions = workingRowSet.invert(rs)) { - positions.fillRowKeyChunk(candidatePositionsChunk); - } - } else { - // get timestamp values from the recorder column source - if (candidateTimestampsChunk == null) { - candidateTimestampsChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); - } - try (final ChunkSource.FillContext fc = recorder.getColumnSource().makeFillContext(WINDOW_CHUNK_SIZE)) { - recorder.getColumnSource().fillChunk(fc, candidateTimestampsChunk, windowRowSequence); - } - } - - // reset the index to beginning of the chunks - candidateWindowIndex = 0; - } - - /*** - * Fill the working chunks with data for this key - * - * @param inputKeys the keys for which we want to get position or timestamp values - */ - public void loadDataChunks(final RowSequence inputKeys) { - if (recorder != null) { - // timestamp data will be available from the recorder - return; - } - - if (valuePositionChunk == null) { - valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); - } else if (valuePositionChunk.capacity() < inputKeys.size()) { - valuePositionChunk.close(); - valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); - } - - // produce position data for the window (will be timestamps for time-based) - // TODO: gotta be a better way than creating two rowsets - try (final RowSet rs = inputKeys.asRowSet(); - final RowSet positions = workingRowSet.invert(rs)) { - positions.fillRowKeyChunk(valuePositionChunk); - } - } - - public void fillWindowTicks(UpdateContext context, long currentPos) { - // compute the head and tail (inclusive) - final long tail = Math.max(0, currentPos - reverseTimeScaleUnits + 1); - final long head = Math.min(workingRowSet.size() - 1, currentPos + forwardTimeScaleUnits); - - while (windowSelector.peek(Long.MAX_VALUE) < tail) { - final long pos = windowSelector.remove(); - final long key = windowRowKeys.remove(); - - pop(context, key); - } - - - // look at the window data and push until satisfied or at the end of the rowset - while (candidatePositionsChunk.size() > 0 && candidatePositionsChunk.get(candidateWindowIndex) <= head) { - final long pos = candidatePositionsChunk.get(candidateWindowIndex); - final long key = candidateRowKeysChunk.get(candidateWindowIndex); - final Object val = candidateValuesChunk.get(candidateWindowIndex); - - push(context, key, val); - - windowSelector.add(pos); - windowRowKeys.add(key); - - if (++candidateWindowIndex >= candidatePositionsChunk.size()) { - // load the next chunk in order - loadWindowChunks(key + 1); - } - } - - if (windowSelector.isEmpty()) { - reset(context); - } } } @@ -239,10 +94,6 @@ public BaseWindowedObjectUpdateByOperator(@NotNull final MatchPair pair, // region extra-methods // endregion extra-methods - public abstract void push(UpdateContext context, long key, Object val); - public abstract void pop(UpdateContext context, long key); - public abstract void reset(UpdateContext context); - @Override public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, final long lastPrevKey, boolean isUpstreamAppendOnly) { final Context ctx = (Context) context; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java index a3ebf3b3670..feb76902e9f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java @@ -34,28 +34,8 @@ public abstract class BaseWindowedShortUpdateByOperator extends UpdateByWindowed protected class Context extends UpdateWindowedContext { public boolean canProcessDirectly; - public RowSet workingRowSet = null; - - // candidate data for the window - public final int WINDOW_CHUNK_SIZE = 1024; - - // data that is actually in the current window - public LongRingBuffer windowRowKeys = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); - // the selector that determines whether this value should be in the window, positions for tick-based and - // timestamps for time-based operators - public LongRingBuffer windowSelector = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); - - public RowSequence.Iterator windowIterator = null; public WritableShortChunk candidateValuesChunk; - public WritableLongChunk candidateRowKeysChunk; - public WritableLongChunk candidatePositionsChunk; - public WritableLongChunk candidateTimestampsChunk; - - public int candidateWindowIndex = 0; - - // position data for the chunk being currently processed - public WritableLongChunk valuePositionChunk; // other useful stuff public UpdateBy.UpdateType currentUpdateType; @@ -72,49 +52,15 @@ public RowSetBuilderSequential getModifiedBuilder() { @Override public void close() { - if (windowIterator != null) { - windowIterator.close(); - windowIterator = null; - } - + super.close(); if (candidateValuesChunk != null) { candidateValuesChunk.close(); candidateValuesChunk = null; } - - if (candidateRowKeysChunk != null) { - candidateRowKeysChunk.close(); - candidateRowKeysChunk = null; - } - - if (candidatePositionsChunk != null) { - candidatePositionsChunk.close(); - candidatePositionsChunk = null; - } - - if (valuePositionChunk != null) { - valuePositionChunk.close(); - valuePositionChunk = null; - } - - // no need to close, just release the reference - workingRowSet = null; } - /*** - * Fill the working chunks with data for this key - * - * @param startKey the key for which we want to - */ - public void loadWindowChunks(final long startKey) { - // TODO: make sure this works for bucketed - if (windowIterator == null) { - windowIterator = workingRowSet.getRowSequenceIterator(); - } - windowIterator.advance(startKey); - - RowSequence windowRowSequence = windowIterator.getNextRowSequenceWithLength(WINDOW_CHUNK_SIZE); - + @Override + public void loadCandidateValueChunk(RowSequence windowRowSequence) { // fill the window values chunk if (candidateValuesChunk == null) { candidateValuesChunk = WritableShortChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); @@ -122,97 +68,6 @@ public void loadWindowChunks(final long startKey) { try (ChunkSource.FillContext fc = valueSource.makeFillContext(WINDOW_CHUNK_SIZE)){ valueSource.fillChunk(fc, candidateValuesChunk, windowRowSequence); } - - // fill the window keys chunk - if (candidateRowKeysChunk == null) { - candidateRowKeysChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); - } - windowRowSequence.fillRowKeyChunk(candidateRowKeysChunk); - - if (recorder == null) { - // get position data for the window items (relative to the table or bucket rowset) - if (candidatePositionsChunk == null) { - candidatePositionsChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); - } - - // TODO: gotta be a better way than creating two rowsets - try (final RowSet rs = windowRowSequence.asRowSet(); - final RowSet positions = workingRowSet.invert(rs)) { - positions.fillRowKeyChunk(candidatePositionsChunk); - } - } else { - // get timestamp values from the recorder column source - if (candidateTimestampsChunk == null) { - candidateTimestampsChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); - } - try (final ChunkSource.FillContext fc = recorder.getColumnSource().makeFillContext(WINDOW_CHUNK_SIZE)) { - recorder.getColumnSource().fillChunk(fc, candidateTimestampsChunk, windowRowSequence); - } - } - - // reset the index to beginning of the chunks - candidateWindowIndex = 0; - } - - /*** - * Fill the working chunks with data for this key - * - * @param inputKeys the keys for which we want to get position or timestamp values - */ - public void loadDataChunks(final RowSequence inputKeys) { - if (recorder != null) { - // timestamp data will be available from the recorder - return; - } - - if (valuePositionChunk == null) { - valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); - } else if (valuePositionChunk.capacity() < inputKeys.size()) { - valuePositionChunk.close(); - valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); - } - - // produce position data for the window (will be timestamps for time-based) - // TODO: gotta be a better way than creating two rowsets - try (final RowSet rs = inputKeys.asRowSet(); - final RowSet positions = workingRowSet.invert(rs)) { - positions.fillRowKeyChunk(valuePositionChunk); - } - } - - public void fillWindowTicks(UpdateContext context, long currentPos) { - // compute the head and tail (inclusive) - final long tail = Math.max(0, currentPos - reverseTimeScaleUnits + 1); - final long head = Math.min(workingRowSet.size() - 1, currentPos + forwardTimeScaleUnits); - - while (windowSelector.peek(Long.MAX_VALUE) < tail) { - final long pos = windowSelector.remove(); - final long key = windowRowKeys.remove(); - - pop(context, key); - } - - - // look at the window data and push until satisfied or at the end of the rowset - while (candidatePositionsChunk.size() > 0 && candidatePositionsChunk.get(candidateWindowIndex) <= head) { - final long pos = candidatePositionsChunk.get(candidateWindowIndex); - final long key = candidateRowKeysChunk.get(candidateWindowIndex); - final short val = candidateValuesChunk.get(candidateWindowIndex); - - push(context, key, val); - - windowSelector.add(pos); - windowRowKeys.add(key); - - if (++candidateWindowIndex >= candidatePositionsChunk.size()) { - // load the next chunk in order - loadWindowChunks(key + 1); - } - } - - if (windowSelector.isEmpty()) { - reset(context); - } } } @@ -236,10 +91,6 @@ public BaseWindowedShortUpdateByOperator(@NotNull final MatchPair pair, // region extra-methods // endregion extra-methods - public abstract void push(UpdateContext context, long key, short val); - public abstract void pop(UpdateContext context, long key); - public abstract void reset(UpdateContext context); - @Override public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, final long lastPrevKey, boolean isUpstreamAppendOnly) { final Context ctx = (Context) context; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java index d037f6857cc..a93a6b9446f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java @@ -32,6 +32,7 @@ import java.util.Map; import static io.deephaven.util.QueryConstants.NULL_LONG; +import static io.deephaven.util.QueryConstants.NULL_BYTE; public class ByteRollingSumOperator extends BaseWindowedByteUpdateByOperator { @@ -109,15 +110,32 @@ public ByteRollingSumOperator(@NotNull final MatchPair pair, } @Override - public void push(UpdateContext context, long key, byte val) { + public void push(UpdateContext context, long key, int index) { final Context ctx = (Context) context; + Byte val = ctx.candidateValuesChunk.get(index); + + // add the value to the window buffer ctx.windowValues.addLast(val); + + // increase the running sum + if (val != NULL_BYTE) { + if (ctx.currentVal == NULL_LONG) { + ctx.currentVal = val; + } else { + ctx.currentVal += val; + } + } } @Override public void pop(UpdateContext context, long key) { final Context ctx = (Context) context; - ctx.windowValues.pop(); + Byte val = ctx.windowValues.pop(); + + // reduce the running sum + if (val != NULL_BYTE) { + ctx.currentVal -= val; + } } @Override @@ -147,21 +165,7 @@ private void computeTicks(@NotNull final Context ctx, if (recorder == null) { ctx.fillWindowTicks(ctx, ctx.valuePositionChunk.get(ii)); } - - MutableLong sum = new MutableLong(NULL_LONG); - ctx.windowValues.forEach(v-> { - if (v != null && v != QueryConstants.NULL_BYTE) { - if (sum.longValue() == NULL_LONG) { - sum.setValue(v); - } else { - sum.add(v); - } - } - }); - - // this call generates the push/pop calls to satisfy the window -// ctx.fillWindow(key, postUpdateSourceIndex); - + // the sum was computed by push/pop operations localOutputValues.set(ii, ctx.currentVal); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java index 1e790bf6244..929ccbdee40 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java @@ -27,6 +27,7 @@ import io.deephaven.engine.table.impl.util.RowRedirection; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; import io.deephaven.util.QueryConstants; +import org.apache.commons.lang3.mutable.MutableDouble; import org.apache.commons.lang3.mutable.MutableLong; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -35,6 +36,7 @@ import java.util.LinkedList; import java.util.Map; +import static io.deephaven.util.QueryConstants.NULL_DOUBLE; import static io.deephaven.util.QueryConstants.NULL_LONG; public class DoubleRollingSumOperator extends BaseWindowedDoubleUpdateByOperator { @@ -108,8 +110,9 @@ public DoubleRollingSumOperator(@NotNull final MatchPair pair, } @Override - public void push(UpdateContext context, long key, double val) { + public void push(UpdateContext context, long key, int index) { final Context ctx = (Context) context; + double val = ctx.candidateValuesChunk.get(index); ctx.windowValues.addLast(val); } @@ -146,10 +149,10 @@ private void computeTicks(@NotNull final Context ctx, ctx.fillWindowTicks(ctx, ctx.valuePositionChunk.get(ii)); } - MutableLong sum = new MutableLong(NULL_LONG); + MutableDouble sum = new MutableDouble(NULL_DOUBLE); ctx.windowValues.forEach(v-> { - if (v != null && v != QueryConstants.NULL_SHORT) { - if (sum.longValue() == NULL_LONG) { + if (v != QueryConstants.NULL_DOUBLE) { + if (sum.getValue() == NULL_DOUBLE) { sum.setValue(v); } else { sum.add(v); @@ -157,9 +160,6 @@ private void computeTicks(@NotNull final Context ctx, } }); - // this call generates the push/pop calls to satisfy the window -// ctx.fillWindow(key, postUpdateSourceIndex); - localOutputValues.set(ii, sum.getValue()); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java index 30cc3c16673..e9ef13dcf01 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java @@ -22,6 +22,7 @@ import io.deephaven.engine.table.impl.util.RowRedirection; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; import io.deephaven.util.QueryConstants; +import org.apache.commons.lang3.mutable.MutableFloat; import org.apache.commons.lang3.mutable.MutableLong; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -30,6 +31,7 @@ import java.util.LinkedList; import java.util.Map; +import static io.deephaven.util.QueryConstants.NULL_FLOAT; import static io.deephaven.util.QueryConstants.NULL_LONG; public class FloatRollingSumOperator extends BaseWindowedFloatUpdateByOperator { @@ -103,8 +105,9 @@ public FloatRollingSumOperator(@NotNull final MatchPair pair, } @Override - public void push(UpdateContext context, long key, float val) { + public void push(UpdateContext context, long key, int index) { final Context ctx = (Context) context; + float val = ctx.candidateValuesChunk.get(index); ctx.windowValues.addLast(val); } @@ -141,10 +144,10 @@ private void computeTicks(@NotNull final Context ctx, ctx.fillWindowTicks(ctx, ctx.valuePositionChunk.get(ii)); } - MutableLong sum = new MutableLong(NULL_LONG); + MutableFloat sum = new MutableFloat(NULL_FLOAT); ctx.windowValues.forEach(v-> { - if (v != null && v != QueryConstants.NULL_SHORT) { - if (sum.longValue() == NULL_LONG) { + if (v != QueryConstants.NULL_FLOAT) { + if (sum.getValue() == NULL_FLOAT) { sum.setValue(v); } else { sum.add(v); @@ -152,9 +155,6 @@ private void computeTicks(@NotNull final Context ctx, } }); - // this call generates the push/pop calls to satisfy the window -// ctx.fillWindow(key, postUpdateSourceIndex); - localOutputValues.set(ii, sum.getValue()); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java index 1e754568627..f7363227d22 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java @@ -32,6 +32,7 @@ import java.util.Map; import static io.deephaven.util.QueryConstants.NULL_LONG; +import static io.deephaven.util.QueryConstants.NULL_INT; public class IntRollingSumOperator extends BaseWindowedIntUpdateByOperator { @@ -106,15 +107,32 @@ public IntRollingSumOperator(@NotNull final MatchPair pair, } @Override - public void push(UpdateContext context, long key, int val) { + public void push(UpdateContext context, long key, int index) { final Context ctx = (Context) context; + Integer val = ctx.candidateValuesChunk.get(index); + + // add the value to the window buffer ctx.windowValues.addLast(val); + + // increase the running sum + if (val != NULL_INT) { + if (ctx.currentVal == NULL_LONG) { + ctx.currentVal = val; + } else { + ctx.currentVal += val; + } + } } @Override public void pop(UpdateContext context, long key) { final Context ctx = (Context) context; - ctx.windowValues.pop(); + Integer val = ctx.windowValues.pop(); + + // reduce the running sum + if (val != NULL_INT) { + ctx.currentVal -= val; + } } @Override @@ -144,21 +162,7 @@ private void computeTicks(@NotNull final Context ctx, if (recorder == null) { ctx.fillWindowTicks(ctx, ctx.valuePositionChunk.get(ii)); } - - MutableLong sum = new MutableLong(NULL_LONG); - ctx.windowValues.forEach(v-> { - if (v != null && v != QueryConstants.NULL_INT) { - if (sum.longValue() == NULL_LONG) { - sum.setValue(v); - } else { - sum.add(v); - } - } - }); - - // this call generates the push/pop calls to satisfy the window -// ctx.fillWindow(key, postUpdateSourceIndex); - + // the sum was computed by push/pop operations localOutputValues.set(ii, ctx.currentVal); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java index 2dd16aea186..181cf5cbf01 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java @@ -31,6 +31,7 @@ import java.util.LinkedList; import java.util.Map; +import static io.deephaven.util.QueryConstants.NULL_LONG; import static io.deephaven.util.QueryConstants.NULL_LONG; public class LongRollingSumOperator extends BaseWindowedLongUpdateByOperator { @@ -106,15 +107,32 @@ public LongRollingSumOperator(@NotNull final MatchPair pair, } @Override - public void push(UpdateContext context, long key, long val) { + public void push(UpdateContext context, long key, int index) { final Context ctx = (Context) context; + Long val = ctx.candidateValuesChunk.get(index); + + // add the value to the window buffer ctx.windowValues.addLast(val); + + // increase the running sum + if (val != NULL_LONG) { + if (ctx.currentVal == NULL_LONG) { + ctx.currentVal = val; + } else { + ctx.currentVal += val; + } + } } @Override public void pop(UpdateContext context, long key) { final Context ctx = (Context) context; - ctx.windowValues.pop(); + Long val = ctx.windowValues.pop(); + + // reduce the running sum + if (val != NULL_LONG) { + ctx.currentVal -= val; + } } @Override @@ -144,21 +162,7 @@ private void computeTicks(@NotNull final Context ctx, if (recorder == null) { ctx.fillWindowTicks(ctx, ctx.valuePositionChunk.get(ii)); } - - MutableLong sum = new MutableLong(NULL_LONG); - ctx.windowValues.forEach(v-> { - if (v != null && v != QueryConstants.NULL_LONG) { - if (sum.longValue() == NULL_LONG) { - sum.setValue(v); - } else { - sum.add(v); - } - } - }); - - // this call generates the push/pop calls to satisfy the window -// ctx.fillWindow(key, postUpdateSourceIndex); - + // the sum was computed by push/pop operations localOutputValues.set(ii, ctx.currentVal); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java index 7d9d96711f2..1804e9f3cd8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java @@ -27,6 +27,7 @@ import java.util.Map; import static io.deephaven.util.QueryConstants.NULL_LONG; +import static io.deephaven.util.QueryConstants.NULL_SHORT; public class ShortRollingSumOperator extends BaseWindowedShortUpdateByOperator { @@ -101,15 +102,32 @@ public ShortRollingSumOperator(@NotNull final MatchPair pair, } @Override - public void push(UpdateContext context, long key, short val) { + public void push(UpdateContext context, long key, int index) { final Context ctx = (Context) context; + Short val = ctx.candidateValuesChunk.get(index); + + // add the value to the window buffer ctx.windowValues.addLast(val); + + // increase the running sum + if (val != NULL_SHORT) { + if (ctx.currentVal == NULL_LONG) { + ctx.currentVal = val; + } else { + ctx.currentVal += val; + } + } } @Override public void pop(UpdateContext context, long key) { final Context ctx = (Context) context; - ctx.windowValues.pop(); + Short val = ctx.windowValues.pop(); + + // reduce the running sum + if (val != NULL_SHORT) { + ctx.currentVal -= val; + } } @Override @@ -139,21 +157,7 @@ private void computeTicks(@NotNull final Context ctx, if (recorder == null) { ctx.fillWindowTicks(ctx, ctx.valuePositionChunk.get(ii)); } - - MutableLong sum = new MutableLong(NULL_LONG); - ctx.windowValues.forEach(v-> { - if (v != null && v != QueryConstants.NULL_SHORT) { - if (sum.longValue() == NULL_LONG) { - sum.setValue(v); - } else { - sum.add(v); - } - } - }); - - // this call generates the push/pop calls to satisfy the window -// ctx.fillWindow(key, postUpdateSourceIndex); - + // the sum was computed by push/pop operations localOutputValues.set(ii, ctx.currentVal); } } From 1b2d25a7666656a9305522b0d4a6ba8c805c79e4 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 16 Aug 2022 17:31:49 -0700 Subject: [PATCH 012/123] very messy, but working --- .../impl/BucketedPartitionedUpdateBy.java | 743 ++++-------------- .../deephaven/engine/table/impl/UpdateBy.java | 171 ++-- .../engine/table/impl/UpdateByOperator.java | 1 - .../table/impl/UpdateByOperatorFactory.java | 108 +-- .../table/impl/UpdateByWindowedOperator.java | 9 +- .../engine/table/impl/ZeroKeyUpdateBy.java | 64 +- .../ema/BasePrimitiveEMAOperator.java | 6 +- .../updateby/ema/BigDecimalEMAOperator.java | 5 +- .../updateby/ema/BigIntegerEMAOperator.java | 5 +- .../updateby/ema/BigNumberEMAOperator.java | 4 +- .../impl/updateby/ema/ByteEMAOperator.java | 5 +- .../impl/updateby/ema/DoubleEMAOperator.java | 5 +- .../impl/updateby/ema/FloatEMAOperator.java | 5 +- .../impl/updateby/ema/IntEMAOperator.java | 5 +- .../impl/updateby/ema/LongEMAOperator.java | 5 +- .../impl/updateby/ema/ShortEMAOperator.java | 5 +- .../updateby/fill/BooleanFillByOperator.java | 5 +- .../updateby/fill/ByteFillByOperator.java | 5 +- .../updateby/fill/CharFillByOperator.java | 5 +- .../updateby/fill/DoubleFillByOperator.java | 5 +- .../updateby/fill/FloatFillByOperator.java | 5 +- .../impl/updateby/fill/IntFillByOperator.java | 5 +- .../updateby/fill/LongFillByOperator.java | 5 +- .../updateby/fill/ObjectFillByOperator.java | 5 +- .../updateby/fill/ShortFillByOperator.java | 5 +- .../internal/BaseByteUpdateByOperator.java | 23 +- .../internal/BaseCharUpdateByOperator.java | 23 +- .../internal/BaseDoubleUpdateByOperator.java | 18 +- .../internal/BaseFloatUpdateByOperator.java | 18 +- .../internal/BaseIntUpdateByOperator.java | 23 +- .../internal/BaseLongUpdateByOperator.java | 23 +- .../internal/BaseObjectBinaryOperator.java | 5 +- .../internal/BaseObjectUpdateByOperator.java | 23 +- .../internal/BaseShortUpdateByOperator.java | 23 +- .../BaseWindowedByteUpdateByOperator.java | 4 +- .../BaseWindowedCharUpdateByOperator.java | 4 +- .../BaseWindowedDoubleUpdateByOperator.java | 4 +- .../BaseWindowedFloatUpdateByOperator.java | 4 +- .../BaseWindowedIntUpdateByOperator.java | 4 +- .../BaseWindowedLongUpdateByOperator.java | 4 +- .../BaseWindowedObjectUpdateByOperator.java | 4 +- .../BaseWindowedShortUpdateByOperator.java | 4 +- .../LongRecordingUpdateByOperator.java | 10 + .../minmax/ByteCumMinMaxOperator.java | 5 +- .../minmax/ComparableCumMinMaxOperator.java | 5 +- .../minmax/DoubleCumMinMaxOperator.java | 5 +- .../minmax/FloatCumMinMaxOperator.java | 5 +- .../updateby/minmax/IntCumMinMaxOperator.java | 5 +- .../minmax/LongCumMinMaxOperator.java | 5 +- .../minmax/ShortCumMinMaxOperator.java | 5 +- .../prod/BigDecimalCumProdOperator.java | 5 +- .../prod/BigIntegerCumProdOperator.java | 5 +- .../updateby/prod/ByteCumProdOperator.java | 5 +- .../updateby/prod/DoubleCumProdOperator.java | 5 +- .../updateby/prod/FloatCumProdOperator.java | 5 +- .../updateby/prod/IntCumProdOperator.java | 5 +- .../updateby/prod/LongCumProdOperator.java | 5 +- .../updateby/prod/ShortCumProdOperator.java | 5 +- .../rollingsum/ByteRollingSumOperator.java | 10 +- .../rollingsum/DoubleRollingSumOperator.java | 10 +- .../rollingsum/FloatRollingSumOperator.java | 10 +- .../rollingsum/IntRollingSumOperator.java | 10 +- .../rollingsum/LongRollingSumOperator.java | 10 +- .../rollingsum/ShortRollingSumOperator.java | 10 +- .../sum/BigDecimalCumSumOperator.java | 5 +- .../sum/BigIntegerCumSumOperator.java | 5 +- .../impl/updateby/sum/ByteCumSumOperator.java | 5 +- .../updateby/sum/DoubleCumSumOperator.java | 5 +- .../updateby/sum/FloatCumSumOperator.java | 5 +- .../impl/updateby/sum/IntCumSumOperator.java | 5 +- .../impl/updateby/sum/LongCumSumOperator.java | 5 +- .../updateby/sum/ShortCumSumOperator.java | 5 +- .../replicators/ReplicateUpdateBy.java | 7 +- 73 files changed, 640 insertions(+), 954 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java index 1004e3c1703..661043e21b5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java @@ -2,21 +2,30 @@ import io.deephaven.api.ColumnName; import io.deephaven.api.updateby.UpdateByControl; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.PartitionedTable; -import io.deephaven.engine.table.Table; +import io.deephaven.engine.rowset.*; +import io.deephaven.engine.table.*; +import io.deephaven.engine.table.impl.sources.UnionSourceManager; import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.util.datastructures.linked.IntrusiveDoublyLinkedNode; +import org.apache.commons.lang3.mutable.MutableLong; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import java.util.Collection; +import java.util.LinkedList; import java.util.List; import java.util.Map; +import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; + /** * An implementation of {@link UpdateBy} dedicated to zero key computation. */ class BucketedPartitionedUpdateBy extends UpdateBy { + private final BucketedPartitionedUpdateByListener listener; + private final LinkedList recorders; + private final QueryTable resultTable; + /** * Perform an updateBy without any key columns. @@ -25,7 +34,7 @@ class BucketedPartitionedUpdateBy extends UpdateBy { * @param source the source table * @param ops the operations to perform * @param resultSources the result sources - * @param rowRedirection the {@link io.deephaven.engine.table.impl.util.RowRedirection}, if one is used. + * @param redirContext the row redirection shared context * @param control the control object. * @return the result table */ @@ -34,603 +43,167 @@ public static Table compute(@NotNull final String description, @NotNull final UpdateByOperator[] ops, @NotNull final Map> resultSources, @NotNull final Collection byColumns, - @Nullable final WritableRowRedirection rowRedirection, + @NotNull final UpdateByRedirectionContext redirContext, @NotNull final UpdateByControl control) { - // this table will always have the rowset of the parent - final QueryTable result = new QueryTable(source.getRowSet(), resultSources); + final BucketedPartitionedUpdateBy updateBy = new BucketedPartitionedUpdateBy(description, + ops, + source, + resultSources, + byColumns, + redirContext, + control); + + return updateBy.resultTable; + } + + protected BucketedPartitionedUpdateBy(@NotNull final String description, + @NotNull final UpdateByOperator[] operators, + @NotNull final QueryTable source, + @NotNull final Map> resultSources, + @NotNull final Collection byColumns, + @NotNull final UpdateByRedirectionContext redirContext, + @NotNull final UpdateByControl control) { + super(operators, source, redirContext, control); + + // create a source-listener that will listen to the source updates and apply the shifts to the output columns + final QueryTable sourceListenerTable = new QueryTable(source.getRowSet(), source.getColumnSourceMap()); + source.listenForUpdates(new BaseTable.ListenerImpl("", source, sourceListenerTable) { + @Override + public void onUpdate(@NotNull final TableUpdate upstream) { + if (!redirContext.isRedirected() && upstream.shifted().nonempty()) { + try (final RowSet prevIdx = source.getRowSet().copyPrev()) { + upstream.shifted().apply((begin, end, delta) -> { + try (final RowSet subRowSet = prevIdx.subSetByKeyRange(begin, end)) { + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + operators[opIdx].applyOutputShift(null, subRowSet, delta); + } + } + }); + } + } + super.onUpdate(upstream); + }}); + -// final BucketedPartitionedUpdateBy updateBy = new BucketedPartitionedUpdateBy(ops, source, rowRedirection, control); + // this table will always have the rowset of the source + resultTable = new QueryTable(source.getRowSet(), resultSources); - final PartitionedTable pt = source.partitionedAggBy(List.of(), true, null, byColumns); + if (source.isRefreshing()) { + recorders = new LinkedList<>(); + listener = newListener(description); + + // create a listener and recorder for the source table as first entry + BucketedPartitionedUpdateByListenerRecorder recorder = new BucketedPartitionedUpdateByListenerRecorder(description, source, resultTable); + recorder.setMergedListener(listener); + source.listenForUpdates(recorder); + + recorders.offerLast(recorder); + } else { + listener = null; + recorders = null; + } + + final PartitionedTable pt = sourceListenerTable.partitionedAggBy(List.of(), true, null, byColumns); final PartitionedTable transformed = pt.transform(t -> { // create the table Table newTable = ZeroKeyUpdateBy.compute( description, (QueryTable)t, - ops, + operators, resultSources, - rowRedirection, - control); - // add the listener + redirContext, + control, + false); + + if (listener != null) { + BucketedPartitionedUpdateByListenerRecorder recorder = new BucketedPartitionedUpdateByListenerRecorder(description, newTable, resultTable); + recorder.setMergedListener(listener); + newTable.listenForUpdates(recorder); + + // add the listener only while synchronized + synchronized (recorders) { + recorders.offerLast(recorder); + } + } // return the table return newTable; }); - result.addParentReference(transformed); - -// if (source.isRefreshing()) { -// if (rowRedirection != null) { -// rowRedirection.startTrackingPrevValues(); -// } -// Arrays.stream(ops).forEach(UpdateByOperator::startTrackingPrev); -// final ZeroKeyUpdateByListener listener = updateBy.newListener(description, result); -// source.listenForUpdates(listener); -// result.addParentReference(listener); -// } - - return result; + resultTable.addParentReference(transformed); } - protected BucketedPartitionedUpdateBy(@NotNull final UpdateByOperator[] operators, - @NotNull final QueryTable source, - @Nullable final WritableRowRedirection rowRedirection, - @NotNull final UpdateByControl control) { - super(operators, source, rowRedirection, control); + BucketedPartitionedUpdateByListener newListener(@NotNull final String description) { + return new BucketedPartitionedUpdateByListener(description); + } + + private final class BucketedPartitionedUpdateByListenerRecorder extends ListenerRecorder { + + private final ModifiedColumnSet.Transformer modifiedColumnsTransformer; + + BucketedPartitionedUpdateByListenerRecorder(@NotNull String description, @NotNull final Table constituent, @NotNull final Table dependent) { + super(description, constituent, dependent); + modifiedColumnsTransformer = ((QueryTable) constituent).newModifiedColumnSetTransformer((QueryTable)dependent, constituent.getDefinition().getColumnNamesArray()); + } } -// ZeroKeyUpdateByListener newListener(@NotNull final String description, @NotNull final QueryTable result) { -// return new ZeroKeyUpdateByListener(description, source, result); -// } -// -// void doInitialAdditions() { -// final TableUpdateImpl fakeUpdate = new TableUpdateImpl(source.getRowSet(), -// RowSetFactory.empty(), -// RowSetFactory.empty(), -// RowSetShiftData.EMPTY, -// ModifiedColumnSet.ALL); -// try (final UpdateContext ctx = new UpdateContext(fakeUpdate, null, true)) { -// ctx.setAllAffected(); -// if (rowRedirection != null && source.isRefreshing()) { -// processUpdateForRedirection(fakeUpdate); -// } -// // do an addition phase for all the operators that can add directly (i.e. backwards looking) -// ctx.doUpdate(source.getRowSet(), source.getRowSet(), UpdateType.Add); -// -// // do a reprocessing phase for operators that can't add directly -// ctx.reprocessRows(RowSetShiftData.EMPTY); -// } -// } -// -// /** -// * An object to hold the transient state during a single {@link ShiftAwareListener#onUpdate(TableUpdate)} update -// * cycle. -// */ -// private class UpdateContext implements SafeCloseable { -// /** The expected size of chunks to the various update stages */ -// int chunkSize; -// -// /** An indicator of if each slot has been populated with data or not for this phase. */ -// boolean[] inputChunkPopulated; -// -// /** An array of boolean denoting which operators are affected by the current update. */ -// final boolean[] opAffected; -// -// /** true if any operator requested keys */ -// boolean anyRequireKeys; -// -// /** An array of context objects for each underlying operator */ -// final UpdateByOperator.UpdateContext[] opContext; -// -// /** A {@link SharedContext} to be used while creating other contexts */ -// SharedContext sharedContext = SharedContext.makeSharedContext(); -// -// /** An array of {@link ChunkSource.FillContext}s for each input column */ -// final SizedSafeCloseable[] fillContexts; -// -// /** A set of chunks used to store post-shift working values */ -// final SizedSafeCloseable>[] postWorkingChunks; -// -// /** A Chunk of longs to store the keys being updated */ -// final SizedLongChunk keyChunk; -// -// /** A sharedContext to be used with previous values */ -// SharedContext prevSharedContext; -// -// /** An array of {@link ChunkSource.FillContext}s for previous values */ -// ChunkSource.FillContext[] prevFillContexts; -// -// /** An array of chunks for previous values */ -// WritableChunk[] prevWorkingChunks; -// -// /** A Long Chunk for previous keys */ -// WritableLongChunk prevKeyChunk; -// -// final RowSet affectedRows; -// -// @SuppressWarnings("resource") -// UpdateContext(@NotNull final TableUpdate upstream, -// @Nullable final ModifiedColumnSet[] inputModifiedColumnSets, -// final boolean isInitializeStep) { -// final int updateSize = UpdateSizeCalculator.chunkSize(upstream, control.chunkCapacityOrDefault()); -// -// this.chunkSize = -// UpdateSizeCalculator.chunkSize(updateSize, upstream.shifted(), control.chunkCapacityOrDefault()); -// this.opAffected = new boolean[operators.length]; -// // noinspection unchecked -// this.fillContexts = new SizedSafeCloseable[operators.length]; -// this.opContext = new UpdateByOperator.UpdateContext[operators.length]; -// this.keyChunk = new SizedLongChunk<>(); -// this.inputChunkPopulated = new boolean[operators.length]; -// -// if (upstream.shifted().nonempty()) { -// this.prevKeyChunk = WritableLongChunk.makeWritableChunk(chunkSize); -// } -// -// final boolean hasModifies = upstream.modified().isNonempty(); -// if (hasModifies) { -// // noinspection unchecked -// this.prevWorkingChunks = new WritableChunk[operators.length]; -// this.prevSharedContext = SharedContext.makeSharedContext(); -// this.prevFillContexts = new ChunkSource.FillContext[operators.length]; -// } -// -// final boolean upstreamAppendOnly = -// isInitializeStep || UpdateByOperator.isAppendOnly(upstream, source.getRowSet().lastRowKeyPrev()); -// -// // noinspection unchecked -// this.postWorkingChunks = new SizedSafeCloseable[operators.length]; -// for (int opIdx = 0; opIdx < operators.length; opIdx++) { -// opAffected[opIdx] = upstream.added().isNonempty() || -// upstream.removed().isNonempty() || -// upstream.shifted().nonempty() || -// (upstream.modifiedColumnSet().nonempty() && (inputModifiedColumnSets == null -// || upstream.modifiedColumnSet().containsAny(inputModifiedColumnSets[opIdx]))); -// if (!opAffected[opIdx]) { -// continue; -// } -// -// opContext[opIdx] = operators[opIdx].makeUpdateContext(chunkSize); -// -// final int slotPosition = inputSourceSlots[opIdx]; -// if (fillContexts[slotPosition] == null) { -// fillContexts[slotPosition] = new SizedSafeCloseable<>( -// sz -> inputSources[slotPosition].makeFillContext(sz, getSharedContext())); -// fillContexts[slotPosition].ensureCapacity(chunkSize); -// postWorkingChunks[slotPosition] = new SizedSafeCloseable<>( -// sz -> inputSources[slotPosition].getChunkType().makeWritableChunk(sz)); -// postWorkingChunks[slotPosition].ensureCapacity(chunkSize); -// -// // Note that these don't participate in setChunkSize() because nothing will use them. If that -// // changes then these must also become SizedSafeCloseables. -// if (hasModifies) { -// prevFillContexts[slotPosition] = -// inputSources[opIdx].makeFillContext(chunkSize, prevSharedContext); -// prevWorkingChunks[slotPosition] = -// inputSources[opIdx].getChunkType().makeWritableChunk(chunkSize); -// } -// } -// -// operators[opIdx].initializeForUpdate(opContext[opIdx], upstream, source.getRowSet(), false, -// upstreamAppendOnly); -// } -// -// // retrieve the affected rows from all operator update contexts -// WritableRowSet tmp = RowSetFactory.empty(); -// for (int opIdx = 0; opIdx < operators.length; opIdx++) { -// if (!opAffected[opIdx]) { -// continue; -// } -// // trigger the operator to determine its own set of affected rows (window-specific), do not close() -// // since this is managed by the operator context -// final RowSet rs = opContext[opIdx].determineAffectedRows(upstream, source.getRowSet(), upstreamAppendOnly); -// -// // union the operator rowsets together to get a global set -// tmp.insert(rs); -// } -// affectedRows = tmp; -// } -// -// public SharedContext getSharedContext() { -// return sharedContext; -// } -// -// void setChunkSize(int newChunkSize) { -// if (newChunkSize <= chunkSize) { -// return; -// } -// -// // We have to close and recreate the shared context because a .reset() is not enough to ensure that any -// // cached chunks that something stuffed into there are resized. -// this.sharedContext.close(); -// this.sharedContext = SharedContext.makeSharedContext(); -// -// if (prevSharedContext != null) { -// this.prevSharedContext.close(); -// this.prevSharedContext = null; -// } -// -// this.chunkSize = newChunkSize; -// this.keyChunk.ensureCapacity(newChunkSize); -// -// for (int opIdx = 0; opIdx < operators.length; opIdx++) { -// if (!opAffected[opIdx]) { -// continue; -// } -// -// operators[opIdx].setChunkSize(opContext[opIdx], newChunkSize); -// if (fillContexts[opIdx] != null) { -// fillContexts[opIdx].ensureCapacity(newChunkSize); -// postWorkingChunks[opIdx].ensureCapacity(newChunkSize); -// -// // Note that this doesn't include the prevFillContexts or prevWorkingChunks. If they become -// // needed for an op, they must be added here. -// } -// } -// } -// -// void initializeFor(@NotNull final RowSet updateRowSet, -// @NotNull final UpdateType type) { -// for (int opIdx = 0; opIdx < operators.length; opIdx++) { -// if (opAffected[opIdx]) { -// operators[opIdx].initializeFor(opContext[opIdx], updateRowSet, type); -// anyRequireKeys |= operators[opIdx].requiresKeys(); -// } -// } -// -// if (anyRequireKeys) { -// keyChunk.ensureCapacity(chunkSize); -// } -// } -// -// void finishFor(@NotNull final UpdateType type) { -// for (int opIdx = 0; opIdx < operators.length; opIdx++) { -// if (opAffected[opIdx]) { -// operators[opIdx].finishFor(opContext[opIdx], type); -// } -// } -// -// anyRequireKeys = false; -// Arrays.fill(inputChunkPopulated, false); -// } -// -// @Override -// public void close() { -// sharedContext.close(); -// keyChunk.close(); -// affectedRows.close(); -// -// if (prevKeyChunk != null) { -// prevKeyChunk.close(); -// } -// -// if (prevSharedContext != null) { -// prevSharedContext.close(); -// } -// -// for (int opIdx = 0; opIdx < operators.length; opIdx++) { -// if (opContext[opIdx] != null) { -// opContext[opIdx].close(); -// } -// -// if (fillContexts[opIdx] != null) { -// fillContexts[opIdx].close(); -// } -// -// if (postWorkingChunks[opIdx] != null) { -// postWorkingChunks[opIdx].close(); -// } -// -// if (prevFillContexts != null && prevFillContexts[opIdx] != null) { -// prevFillContexts[opIdx].close(); -// } -// -// if (prevWorkingChunks != null && prevWorkingChunks[opIdx] != null) { -// prevWorkingChunks[opIdx].close(); -// } -// } -// } -// -// /** -// * Mark all columns as affected -// */ -// public void setAllAffected() { -// Arrays.fill(opAffected, true); -// } -// -// /** -// * Check if any of the operators have produced additional modified rows. -// * -// * @return true if any operator produced more modified rows. -// */ -// boolean anyModified() { -// for (int opIdx = 0; opIdx < operators.length; opIdx++) { -// if (opAffected[opIdx] && operators[opIdx].anyModified(opContext[opIdx])) { -// return true; -// } -// } -// return false; -// } -// -// void doUpdate(@NotNull final RowSet updateRowSet, -// @NotNull final RowSet preShiftUpdateRowSet, -// @NotNull final UpdateType type) { -// if (updateRowSet.isEmpty()) { -// return; -// } -// -// try (final RowSequence.Iterator okIt = updateRowSet.getRowSequenceIterator(); -// final RowSequence.Iterator preShiftOkIt = preShiftUpdateRowSet == updateRowSet ? null -// : preShiftUpdateRowSet.getRowSequenceIterator()) { -// initializeFor(updateRowSet, type); -// -// while (okIt.hasMore()) { -// sharedContext.reset(); -// if (prevSharedContext != null) { -// prevSharedContext.reset(); -// } -// Arrays.fill(inputChunkPopulated, false); -// -// final RowSequence chunkOk = okIt.getNextRowSequenceWithLength(chunkSize); -// final RowSequence prevChunkOk = preShiftUpdateRowSet == updateRowSet ? chunkOk -// : preShiftOkIt.getNextRowSequenceWithLength(chunkSize); -// -// if (anyRequireKeys) { -// chunkOk.fillRowKeyChunk(keyChunk.get()); -// if (prevChunkOk != chunkOk) { -// prevChunkOk.fillRowKeyChunk(prevKeyChunk); -// } -// } -// -// for (int opIdx = 0; opIdx < operators.length; opIdx++) { -// if (!opAffected[opIdx]) { -// continue; -// } -// -// final UpdateByOperator currentOp = operators[opIdx]; -// final int slotPosition = inputSourceSlots[opIdx]; -// if (type == UpdateType.Add) { -// prepareValuesChunkFor(opIdx, slotPosition, false, true, chunkOk, prevChunkOk, -// null, postWorkingChunks[slotPosition].get(), -// null, fillContexts[slotPosition].get()); -// currentOp.addChunk(opContext[opIdx], chunkOk, keyChunk.get(), -// postWorkingChunks[slotPosition].get(), 0); -// } else if (type == UpdateType.Remove) { -// prepareValuesChunkFor(opIdx, slotPosition, true, false, chunkOk, prevChunkOk, -// postWorkingChunks[slotPosition].get(), null, -// fillContexts[slotPosition].get(), null); -// currentOp.removeChunk(opContext[opIdx], keyChunk.get(), -// postWorkingChunks[slotPosition].get(), 0); -// } else if (type == UpdateType.Modify) { -// prepareValuesChunkFor(opIdx, slotPosition, true, true, chunkOk, prevChunkOk, -// prevWorkingChunks[slotPosition], postWorkingChunks[slotPosition].get(), -// prevFillContexts[slotPosition], fillContexts[slotPosition].get()); -// currentOp.modifyChunk(opContext[opIdx], -// prevKeyChunk == null ? keyChunk.get() : prevKeyChunk, -// keyChunk.get(), -// prevWorkingChunks[slotPosition], -// postWorkingChunks[slotPosition].get(), -// 0); -// } else if (type == UpdateType.Reprocess) { -// // is this chunk relevant to this operator? If so, then intersect and process only the -// // relevant rows -// if (chunkOk.firstRowKey() <= opContext[opIdx].getAffectedRows().lastRowKey() -// && chunkOk.lastRowKey() >= opContext[opIdx].getAffectedRows().firstRowKey()) { -// try (final RowSet rs = chunkOk.asRowSet(); -// final RowSet intersect = rs.intersect(opContext[opIdx].getAffectedRows())) { -// -// prepareValuesChunkFor(opIdx, slotPosition, false, true, intersect, intersect, -// null, postWorkingChunks[slotPosition].get(), -// null, fillContexts[slotPosition].get()); -// currentOp.reprocessChunk(opContext[opIdx], -// intersect, -// keyChunk.get(), -// postWorkingChunks[slotPosition].get(), -// source.getRowSet()); -// } -// } -// } -// } -// } -// -// finishFor(type); -// } -// } -// -// /** -// * Locate the smallest key that requires reprocessing and then replay the table from that point -// */ -// private void reprocessRows(RowSetShiftData shifted) { -// // Get a sub-index of the source from that minimum reprocessing index and make sure we update our -// // contextual chunks and FillContexts to an appropriate size for this step. -// final RowSet sourceRowSet = source.getRowSet(); -// -// final int newChunkSize = (int) Math.min(control.chunkCapacityOrDefault(), affectedRows.size()); -// setChunkSize(newChunkSize); -// -// for (int opIndex = 0; opIndex < operators.length; opIndex++) { -// if (opAffected[opIndex]) { -// final long keyStart = opContext[opIndex].getAffectedRows().firstRowKey(); -// final long keyBefore; -// try (final RowSet.SearchIterator sit = sourceRowSet.searchIterator()) { -// keyBefore = sit.binarySearchValue( -// (compareTo, ignored) -> Long.compare(keyStart - 1, compareTo), 1); -// } -// operators[opIndex].resetForReprocess(opContext[opIndex], sourceRowSet, keyBefore); -// } -// } -// -// // We will not mess with shifts if we are using a redirection because we'll have applied the shift -// // to the redirection index already by now. -// if (rowRedirection == null && shifted.nonempty()) { -// try (final RowSet prevIdx = source.getRowSet().copyPrev()) { -// shifted.apply((begin, end, delta) -> { -// try (final RowSet subRowSet = prevIdx.subSetByKeyRange(begin, end)) { -// for (int opIdx = 0; opIdx < operators.length; opIdx++) { -// operators[opIdx].applyOutputShift(opContext[opIdx], subRowSet, delta); -// } -// } -// }); -// } -// } -// -// // Now iterate rowset to reprocess. -// doUpdate(affectedRows, affectedRows, UpdateType.Reprocess); -// } -// -// /** -// * Prepare the specified chunk for use. -// * -// * @param opIdx the operator index -// * @param usePrev if previous values should be fetched -// * @param chunkOk the {@link RowSequence} for current values -// * @param prevChunkOk the {@link RowSequence} for previous values. -// */ -// private void prepareValuesChunkFor(final int opIdx, -// final int inputSlot, -// final boolean usePrev, -// final boolean useCurrent, -// final RowSequence chunkOk, -// final RowSequence prevChunkOk, -// final WritableChunk prevWorkingChunk, -// final WritableChunk postWorkingChunk, -// final ChunkSource.FillContext prevFillContext, -// final ChunkSource.FillContext postFillContext) { -// if (!operators[opIdx].requiresValues(opContext[opIdx])) { -// return; -// } -// -// if (!inputChunkPopulated[inputSlot]) { -// // Using opIdx below is OK, because if we are sharing an input slot, we are referring to the same -// // input source. Instead of maintaining another array of sourced by slot, we'll just use the opIdx -// inputChunkPopulated[inputSlot] = true; -// if (usePrev) { -// inputSources[opIdx].fillPrevChunk(prevFillContext, prevWorkingChunk, prevChunkOk); -// } -// -// if (useCurrent) { -// inputSources[opIdx].fillChunk(postFillContext, postWorkingChunk, chunkOk); -// } -// } -// } -// -// private void onBucketsRemoved(@NotNull final RowSet removedBuckets) { -// for (final UpdateByOperator operator : operators) { -// operator.onBucketsRemoved(removedBuckets); -// } -// } -// -// public boolean canAnyProcessNormally() { -// for (int opIdx = 0; opIdx < operators.length; opIdx++) { -// if (opAffected[opIdx] && operators[opIdx].canProcessNormalUpdate(opContext[opIdx])) { -// return true; -// } -// } -// -// return false; -// } -// } -// -// /** -// * The Listener for apply an upstream {@link ShiftAwareListener#onUpdate(Update) update} -// */ -// class ZeroKeyUpdateByListener extends InstrumentedTableUpdateListenerAdapter { -// private final QueryTable result; -// private final ModifiedColumnSet[] inputModifiedColumnSets; -// private final ModifiedColumnSet[] outputModifiedColumnSets; -// private final ModifiedColumnSet.Transformer transformer; -// -// public ZeroKeyUpdateByListener(@Nullable String description, -// @NotNull final QueryTable source, -// @NotNull final QueryTable result) { -// super(description, source, false); -// this.result = result; -// this.inputModifiedColumnSets = new ModifiedColumnSet[operators.length]; -// this.outputModifiedColumnSets = new ModifiedColumnSet[operators.length]; -// -// for (int ii = 0; ii < operators.length; ii++) { -// final String[] outputColumnNames = operators[ii].getOutputColumnNames(); -// inputModifiedColumnSets[ii] = source.newModifiedColumnSet(operators[ii].getAffectingColumnNames()); -// outputModifiedColumnSets[ii] = result.newModifiedColumnSet(outputColumnNames); -// } -// -// this.transformer = -// source.newModifiedColumnSetTransformer(result, source.getDefinition().getColumnNamesArray()); -// } -// -// @Override -// public void onUpdate(TableUpdate upstream) { -// try (final UpdateContext ctx = new UpdateContext(upstream, inputModifiedColumnSets, false)) { -// if (rowRedirection != null) { -// processUpdateForRedirection(upstream); -// } -// -// // If anything can process normal operations we have to pass them down, otherwise we can skip this -// // entirely. -// if (ctx.canAnyProcessNormally()) { -// ctx.doUpdate(upstream.removed(), upstream.removed(), UpdateType.Remove); -// if (upstream.shifted().nonempty()) { -// try (final WritableRowSet prevRowSet = source.getRowSet().copyPrev(); -// final RowSet modPreShift = upstream.getModifiedPreShift()) { -// -// prevRowSet.remove(upstream.removed()); -// for (int ii = 0; ii < operators.length; ii++) { -// operators[ii].initializeFor(ctx.opContext[ii], prevRowSet, UpdateType.Shift); -// operators[ii].applyShift(ctx.opContext[ii], prevRowSet, upstream.shifted()); -// operators[ii].finishFor(ctx.opContext[ii], UpdateType.Shift); -// } -// ctx.doUpdate(upstream.modified(), modPreShift, UpdateType.Modify); -// } -// } else { -// ctx.doUpdate(upstream.modified(), upstream.modified(), UpdateType.Modify); -// } -// ctx.doUpdate(upstream.added(), upstream.added(), UpdateType.Add); -// } -// -// if (source.getRowSet().isEmpty()) { -// ctx.onBucketsRemoved(RowSetFactory.fromKeys(0)); -// } -// -// // Now do the reprocessing phase. -// ctx.reprocessRows(upstream.shifted()); -// -// final TableUpdateImpl downstream = new TableUpdateImpl(); -// // copy these rowSets since TableUpdateImpl#reset will close them with the upstream update -// downstream.added = upstream.added().copy(); -// downstream.removed = upstream.removed().copy(); -// downstream.shifted = upstream.shifted(); -// -// if (upstream.modified().isNonempty() || ctx.anyModified()) { -// downstream.modifiedColumnSet = result.getModifiedColumnSetForUpdates(); -// downstream.modifiedColumnSet.clear(); -// -// WritableRowSet modifiedRowSet = RowSetFactory.empty(); -// downstream.modified = modifiedRowSet; -// if (upstream.modified().isNonempty()) { -// // Transform any untouched modified columns to the output. -// transformer.clearAndTransform(upstream.modifiedColumnSet(), downstream.modifiedColumnSet); -// modifiedRowSet.insert(upstream.modified()); -// } -// -// for (int opIdx = 0; opIdx < operators.length; opIdx++) { -// if (ctx.opAffected[opIdx]) { -// downstream.modifiedColumnSet.setAll(outputModifiedColumnSets[opIdx]); -// if (operators[opIdx].anyModified(ctx.opContext[opIdx])) { -// modifiedRowSet -// .insert(operators[opIdx].getAdditionalModifications(ctx.opContext[opIdx])); -// } -// } -// } -// -// if (ctx.anyModified()) { -// modifiedRowSet.remove(upstream.added()); -// } -// } else { -// downstream.modified = RowSetFactory.empty(); -// downstream.modifiedColumnSet = ModifiedColumnSet.EMPTY; -// } -// result.notifyListeners(downstream); -// } -// } -// } + /** + * The Listener for apply to the constituent table updates + */ + class BucketedPartitionedUpdateByListener extends MergedListener { + private final ModifiedColumnSet[] inputModifiedColumnSets; + private final ModifiedColumnSet[] outputModifiedColumnSets; + + public BucketedPartitionedUpdateByListener(@Nullable String description) { + super(recorders, List.of(), description, resultTable); + + this.inputModifiedColumnSets = new ModifiedColumnSet[operators.length]; + this.outputModifiedColumnSets = new ModifiedColumnSet[operators.length]; + + for (int ii = 0; ii < operators.length; ii++) { + final String[] outputColumnNames = operators[ii].getOutputColumnNames(); + inputModifiedColumnSets[ii] = source.newModifiedColumnSet(operators[ii].getAffectingColumnNames()); + outputModifiedColumnSets[ii] = resultTable.newModifiedColumnSet(outputColumnNames); + } + } + + @Override + protected void process() { + final TableUpdateImpl downstream = new TableUpdateImpl(); + + // get the adds/removes/shifts from the first (source) entry, make a copy since TableUpdateImpl#reset will + // close them with the upstream update + ListenerRecorder sourceRecorder = recorders.peekFirst(); + downstream.added = sourceRecorder.getAdded().copy(); + downstream.removed = sourceRecorder.getRemoved().copy(); + downstream.shifted = sourceRecorder.getShifted(); + + // union the modifies from all the tables (including source) + downstream.modifiedColumnSet = resultTable.getModifiedColumnSetForUpdates(); + downstream.modifiedColumnSet.clear(); + + WritableRowSet modifiedRowSet = RowSetFactory.empty(); + downstream.modified = modifiedRowSet; + + recorders.forEach(lr -> { + if (lr.getModified().isNonempty()) { + // Transform any untouched modified columns to the output. + lr.modifiedColumnsTransformer.transform(lr.getModifiedColumnSet(), downstream.modifiedColumnSet); + modifiedRowSet.insert(lr.getModified()); + } + + }); + // should not include actual adds as modifies + modifiedRowSet.remove(downstream.added); + + result.notifyListeners(downstream); + } + + @Override + protected boolean canExecute(final long step) { + synchronized (recorders) { + return recorders.stream().allMatch(lr -> lr.satisfied(step)); + } + } + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java index 046c44cf70d..8ca3dd5464c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java @@ -32,17 +32,110 @@ public abstract class UpdateBy { protected final int[] inputSourceSlots; protected final UpdateByOperator[] operators; protected final QueryTable source; - @Nullable - protected final WritableRowRedirection rowRedirection; - protected final WritableRowSet freeRows; - protected long maxInnerIndex; + + protected final UpdateByRedirectionContext redirContext; protected final UpdateByControl control; + public static class UpdateByRedirectionContext implements Context { + @Nullable + protected final WritableRowRedirection rowRedirection; + protected final WritableRowSet freeRows; + protected long maxInnerIndex; + + public UpdateByRedirectionContext(@Nullable final WritableRowRedirection rowRedirection) { + this.rowRedirection = rowRedirection; + this.freeRows = rowRedirection == null ? null : RowSetFactory.empty(); + this.maxInnerIndex = 0; + } + + public boolean isRedirected() { + return rowRedirection != null; + } + + public long requiredCapacity() { + return maxInnerIndex + 1; + } + + @Nullable + public WritableRowRedirection getRowRedirection() { + return rowRedirection; + } + + public void processUpdateForRedirection(@NotNull final TableUpdate upstream, final TrackingRowSet prevRowSet) { + if (upstream.removed().isNonempty()) { + final RowSetBuilderRandom freeBuilder = RowSetFactory.builderRandom(); + synchronized (rowRedirection) { + upstream.removed().forAllRowKeys(key -> freeBuilder.addKey(rowRedirection.remove(key))); + freeRows.insert(freeBuilder.build()); + } + } + + if (upstream.shifted().nonempty()) { + try (final WritableRowSet prevIndexLessRemoves = prevRowSet.copyPrev()) { + prevIndexLessRemoves.remove(upstream.removed()); + final RowSet.SearchIterator fwdIt = prevIndexLessRemoves.searchIterator(); + + upstream.shifted().apply((start, end, delta) -> { + if (delta < 0 && fwdIt.advance(start)) { + for (long key = fwdIt.currentValue(); fwdIt.currentValue() <= end; key = fwdIt.nextLong()) { + if (shiftRedirectedKey(fwdIt, delta, key)) + break; + } + } else { + try (final RowSet.SearchIterator revIt = prevIndexLessRemoves.reverseIterator()) { + if (revIt.advance(end)) { + for (long key = revIt.currentValue(); revIt.currentValue() >= start; key = + revIt.nextLong()) { + if (shiftRedirectedKey(revIt, delta, key)) + break; + } + } + } + } + }); + } + } + + if (upstream.added().isNonempty()) { + final MutableLong lastAllocated = new MutableLong(0); + synchronized (rowRedirection) { + final WritableRowSet.Iterator freeIt = freeRows.iterator(); + upstream.added().forAllRowKeys(outerKey -> { + final long innerKey = freeIt.hasNext() ? freeIt.nextLong() : ++maxInnerIndex; + lastAllocated.setValue(innerKey); + rowRedirection.put(outerKey, innerKey); + }); + freeRows.removeRange(0, lastAllocated.longValue()); + } + } + } + + private boolean shiftRedirectedKey(@NotNull final RowSet.SearchIterator iterator, final long delta, + final long key) { + synchronized (rowRedirection) { + final long inner = rowRedirection.remove(key); + if (inner != NULL_ROW_KEY) { + rowRedirection.put(key + delta, inner); + } + return !iterator.hasNext(); + } + } + + @Override + public void close() { + try (final WritableRowSet ignored = freeRows) { + } + } + } + protected UpdateBy(@NotNull final UpdateByOperator[] operators, @NotNull final QueryTable source, - @Nullable final WritableRowRedirection rowRedirection, UpdateByControl control) { + @NotNull final UpdateByRedirectionContext redirContext, + UpdateByControl control) { this.control = control; + this.redirContext = redirContext; + if (operators.length == 0) { throw new IllegalArgumentException("At least one operator must be specified"); } @@ -65,8 +158,6 @@ protected UpdateBy(@NotNull final UpdateByOperator[] operators, inputSourceSlots[opIdx] = maybeExistingSlot; } } - this.rowRedirection = rowRedirection; - this.freeRows = rowRedirection == null ? null : RowSetFactory.empty(); } // region UpdateBy implementation @@ -85,6 +176,7 @@ public static Table updateBy(@NotNull final QueryTable source, @NotNull final Collection byColumns, @NotNull final UpdateByControl control) { + // create the rowRedirection if instructed final WritableRowRedirection rowRedirection; if (control.useRedirectionOrDefault()) { if (!source.isRefreshing()) { @@ -112,13 +204,15 @@ public static Table updateBy(@NotNull final QueryTable source, rowRedirection = null; } + // create an UpdateByRedirectionContext for use by the UpdateBy objects + UpdateByRedirectionContext ctx = new UpdateByRedirectionContext(rowRedirection); // TODO(deephaven-core#2693): Improve UpdateBy implementation for ColumnName // generate a MatchPair array for use by the existing algorithm MatchPair[] pairs = MatchPair.fromPairs(byColumns); final UpdateByOperatorFactory updateByOperatorFactory = - new UpdateByOperatorFactory(source, pairs, rowRedirection, control); + new UpdateByOperatorFactory(source, pairs, ctx, control); final Collection ops = updateByOperatorFactory.getOperators(clauses); final StringBuilder descriptionBuilder = new StringBuilder("updateBy(ops={") @@ -151,8 +245,9 @@ public static Table updateBy(@NotNull final QueryTable source, source, opArr, resultSources, - rowRedirection, - control); + ctx, + control, + true); if (source.isRefreshing()) { // start tracking previous values @@ -184,7 +279,7 @@ public static Table updateBy(@NotNull final QueryTable source, opArr, resultSources, byColumns, - rowRedirection, + ctx, control); if (source.isRefreshing()) { @@ -197,60 +292,6 @@ public static Table updateBy(@NotNull final QueryTable source, return ret; } - protected void processUpdateForRedirection(@NotNull final TableUpdate upstream) { - if (upstream.removed().isNonempty()) { - final RowSetBuilderRandom freeBuilder = RowSetFactory.builderRandom(); - upstream.removed().forAllRowKeys(key -> freeBuilder.addKey(rowRedirection.remove(key))); - freeRows.insert(freeBuilder.build()); - } - - if (upstream.shifted().nonempty()) { - try (final WritableRowSet prevIndexLessRemoves = source.getRowSet().copyPrev()) { - prevIndexLessRemoves.remove(upstream.removed()); - final RowSet.SearchIterator fwdIt = prevIndexLessRemoves.searchIterator(); - - upstream.shifted().apply((start, end, delta) -> { - if (delta < 0 && fwdIt.advance(start)) { - for (long key = fwdIt.currentValue(); fwdIt.currentValue() <= end; key = fwdIt.nextLong()) { - if (shiftRedirectedKey(fwdIt, delta, key)) - break; - } - } else { - try (final RowSet.SearchIterator revIt = prevIndexLessRemoves.reverseIterator()) { - if (revIt.advance(end)) { - for (long key = revIt.currentValue(); revIt.currentValue() >= start; key = - revIt.nextLong()) { - if (shiftRedirectedKey(revIt, delta, key)) - break; - } - } - } - } - }); - } - } - - if (upstream.added().isNonempty()) { - final MutableLong lastAllocated = new MutableLong(0); - final WritableRowSet.Iterator freeIt = freeRows.iterator(); - upstream.added().forAllRowKeys(outerKey -> { - final long innerKey = freeIt.hasNext() ? freeIt.nextLong() : ++maxInnerIndex; - lastAllocated.setValue(innerKey); - rowRedirection.put(outerKey, innerKey); - }); - freeRows.removeRange(0, lastAllocated.longValue()); - } - } - - private boolean shiftRedirectedKey(@NotNull final RowSet.SearchIterator iterator, final long delta, - final long key) { - final long inner = rowRedirection.remove(key); - if (inner != NULL_ROW_KEY) { - rowRedirection.put(key + delta, inner); - } - return !iterator.hasNext(); - } - /** * The type of update to be applied. For use with invocations of * {@link UpdateByOperator#initializeFor(UpdateByOperator.UpdateContext, RowSet, UpdateType)} and diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java index 27ba08f9ae8..35dce04341d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java @@ -376,7 +376,6 @@ void applyShift(@NotNull final UpdateContext context, /** * Apply a shift to the operation. * - * @param context the context object */ void applyOutputShift(@NotNull final UpdateContext context, @NotNull final RowSet subIndexToShift, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java index 4595db158f0..da05b63cc12 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java @@ -38,18 +38,18 @@ public class UpdateByOperatorFactory { private final TableWithDefaults source; private final MatchPair[] groupByColumns; - @Nullable - private final WritableRowRedirection rowRedirection; + @NotNull + private final UpdateBy.UpdateByRedirectionContext redirContext; @NotNull private final UpdateByControl control; public UpdateByOperatorFactory(@NotNull final TableWithDefaults source, @NotNull final MatchPair[] groupByColumns, - @Nullable final WritableRowRedirection rowRedirection, + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, @NotNull final UpdateByControl control) { this.source = source; this.groupByColumns = groupByColumns; - this.rowRedirection = rowRedirection; + this.redirContext = redirContext; this.control = control; } @@ -251,28 +251,28 @@ private UpdateByOperator makeEmaOperator(@NotNull final MatchPair pair, if (csType == byte.class || csType == Byte.class) { return new ByteEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, timeScaleUnits, - columnSource, rowRedirection); + columnSource, redirContext); } else if (csType == short.class || csType == Short.class) { return new ShortEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, timeScaleUnits, - columnSource, rowRedirection); + columnSource, redirContext); } else if (csType == int.class || csType == Integer.class) { return new IntEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, timeScaleUnits, - columnSource, rowRedirection); + columnSource, redirContext); } else if (csType == long.class || csType == Long.class) { return new LongEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, timeScaleUnits, - columnSource, rowRedirection); + columnSource, redirContext); } else if (csType == float.class || csType == Float.class) { return new FloatEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, timeScaleUnits, - columnSource, rowRedirection); + columnSource, redirContext); } else if (csType == double.class || csType == Double.class) { return new DoubleEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, - timeScaleUnits, columnSource, rowRedirection); + timeScaleUnits, columnSource, redirContext); } else if (csType == BigDecimal.class) { return new BigDecimalEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, - timeScaleUnits, columnSource, rowRedirection); + timeScaleUnits, columnSource, redirContext); } else if (csType == BigInteger.class) { return new BigIntegerEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, - timeScaleUnits, columnSource, rowRedirection); + timeScaleUnits, columnSource, redirContext); } throw new IllegalArgumentException("Can not perform EMA on type " + csType); @@ -298,21 +298,21 @@ private LongRecordingUpdateByOperator makeLongRecordingOperator(TableWithDefault private UpdateByOperator makeCumProdOperator(MatchPair fc, TableWithDefaults source) { final Class csType = source.getColumnSource(fc.rightColumn).getType(); if (csType == byte.class || csType == Byte.class) { - return new ByteCumProdOperator(fc, rowRedirection); + return new ByteCumProdOperator(fc, redirContext); } else if (csType == short.class || csType == Short.class) { - return new ShortCumProdOperator(fc, rowRedirection); + return new ShortCumProdOperator(fc, redirContext); } else if (csType == int.class || csType == Integer.class) { - return new IntCumProdOperator(fc, rowRedirection); + return new IntCumProdOperator(fc, redirContext); } else if (csType == long.class || csType == Long.class) { - return new LongCumProdOperator(fc, rowRedirection); + return new LongCumProdOperator(fc, redirContext); } else if (csType == float.class || csType == Float.class) { - return new FloatCumProdOperator(fc, rowRedirection); + return new FloatCumProdOperator(fc, redirContext); } else if (csType == double.class || csType == Double.class) { - return new DoubleCumProdOperator(fc, rowRedirection); + return new DoubleCumProdOperator(fc, redirContext); } else if (csType == BigDecimal.class) { - return new BigDecimalCumProdOperator(fc, rowRedirection, control.mathContextOrDefault()); + return new BigDecimalCumProdOperator(fc, redirContext, control.mathContextOrDefault()); } else if (csType == BigInteger.class) { - return new BigIntegerCumProdOperator(fc, rowRedirection); + return new BigIntegerCumProdOperator(fc, redirContext); } throw new IllegalArgumentException("Can not perform Cumulative Min/Max on type " + csType); @@ -322,20 +322,20 @@ private UpdateByOperator makeCumMinMaxOperator(MatchPair fc, TableWithDefaults s final ColumnSource columnSource = source.getColumnSource(fc.rightColumn); final Class csType = columnSource.getType(); if (csType == byte.class || csType == Byte.class) { - return new ByteCumMinMaxOperator(fc, isMax, rowRedirection); + return new ByteCumMinMaxOperator(fc, isMax, redirContext); } else if (csType == short.class || csType == Short.class) { - return new ShortCumMinMaxOperator(fc, isMax, rowRedirection); + return new ShortCumMinMaxOperator(fc, isMax, redirContext); } else if (csType == int.class || csType == Integer.class) { - return new IntCumMinMaxOperator(fc, isMax, rowRedirection); + return new IntCumMinMaxOperator(fc, isMax, redirContext); } else if (csType == long.class || csType == Long.class || isTimeType(csType)) { - return new LongCumMinMaxOperator(fc, isMax, rowRedirection, csType); + return new LongCumMinMaxOperator(fc, isMax, redirContext, csType); } else if (csType == float.class || csType == Float.class) { - return new FloatCumMinMaxOperator(fc, isMax, rowRedirection); + return new FloatCumMinMaxOperator(fc, isMax, redirContext); } else if (csType == double.class || csType == Double.class) { - return new DoubleCumMinMaxOperator(fc, isMax, rowRedirection); + return new DoubleCumMinMaxOperator(fc, isMax, redirContext); } else if (Comparable.class.isAssignableFrom(csType)) { // noinspection unchecked,rawtypes - return new ComparableCumMinMaxOperator(csType, fc, isMax, rowRedirection); + return new ComparableCumMinMaxOperator(csType, fc, isMax, redirContext); } throw new IllegalArgumentException("Can not perform Cumulative Min/Max on type " + csType); @@ -344,23 +344,23 @@ private UpdateByOperator makeCumMinMaxOperator(MatchPair fc, TableWithDefaults s private UpdateByOperator makeCumSumOperator(MatchPair fc, TableWithDefaults source) { final Class csType = source.getColumnSource(fc.rightColumn).getType(); if (csType == Boolean.class || csType == boolean.class) { - return new ByteCumSumOperator(fc, rowRedirection, NULL_BOOLEAN_AS_BYTE); + return new ByteCumSumOperator(fc, redirContext, NULL_BOOLEAN_AS_BYTE); } else if (csType == byte.class || csType == Byte.class) { - return new ByteCumSumOperator(fc, rowRedirection, NULL_BYTE); + return new ByteCumSumOperator(fc, redirContext, NULL_BYTE); } else if (csType == short.class || csType == Short.class) { - return new ShortCumSumOperator(fc, rowRedirection); + return new ShortCumSumOperator(fc, redirContext); } else if (csType == int.class || csType == Integer.class) { - return new IntCumSumOperator(fc, rowRedirection); + return new IntCumSumOperator(fc, redirContext); } else if (csType == long.class || csType == Long.class) { - return new LongCumSumOperator(fc, rowRedirection); + return new LongCumSumOperator(fc, redirContext); } else if (csType == float.class || csType == Float.class) { - return new FloatCumSumOperator(fc, rowRedirection); + return new FloatCumSumOperator(fc, redirContext); } else if (csType == double.class || csType == Double.class) { - return new DoubleCumSumOperator(fc, rowRedirection); + return new DoubleCumSumOperator(fc, redirContext); } else if (csType == BigDecimal.class) { - return new BigDecimalCumSumOperator(fc, rowRedirection, control.mathContextOrDefault()); + return new BigDecimalCumSumOperator(fc, redirContext, control.mathContextOrDefault()); } else if (csType == BigInteger.class) { - return new BigIntegerCumSumOperator(fc, rowRedirection); + return new BigIntegerCumSumOperator(fc, redirContext); } throw new IllegalArgumentException("Can not perform Cumulative Sum on type " + csType); @@ -370,23 +370,23 @@ private UpdateByOperator makeForwardFillOperator(MatchPair fc, TableWithDefaults final ColumnSource columnSource = source.getColumnSource(fc.rightColumn); final Class csType = columnSource.getType(); if (csType == char.class || csType == Character.class) { - return new CharFillByOperator(fc, rowRedirection); + return new CharFillByOperator(fc, redirContext); } else if (csType == byte.class || csType == Byte.class) { - return new ByteFillByOperator(fc, rowRedirection); + return new ByteFillByOperator(fc, redirContext); } else if (csType == short.class || csType == Short.class) { - return new ShortFillByOperator(fc, rowRedirection); + return new ShortFillByOperator(fc, redirContext); } else if (csType == int.class || csType == Integer.class) { - return new IntFillByOperator(fc, rowRedirection); + return new IntFillByOperator(fc, redirContext); } else if (csType == long.class || csType == Long.class || isTimeType(csType)) { - return new LongFillByOperator(fc, rowRedirection, csType); + return new LongFillByOperator(fc, redirContext, csType); } else if (csType == float.class || csType == Float.class) { - return new FloatFillByOperator(fc, rowRedirection); + return new FloatFillByOperator(fc, redirContext); } else if (csType == double.class || csType == Double.class) { - return new DoubleFillByOperator(fc, rowRedirection); + return new DoubleFillByOperator(fc, redirContext); } else if (csType == boolean.class || csType == Boolean.class) { - return new BooleanFillByOperator(fc, rowRedirection); + return new BooleanFillByOperator(fc, redirContext); } else { - return new ObjectFillByOperator<>(fc, rowRedirection, csType); + return new ObjectFillByOperator<>(fc, redirContext, csType); } } @@ -410,31 +410,31 @@ private UpdateByOperator makeRollingSumOperator(@NotNull final MatchPair pair, if (csType == Boolean.class || csType == boolean.class) { return new ByteRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, prevTimeScaleUnits, fwdTimeScaleUnits, - ReinterpretUtils.booleanToByteSource(columnSource), rowRedirection, NULL_BOOLEAN_AS_BYTE); + ReinterpretUtils.booleanToByteSource(columnSource), redirContext, NULL_BOOLEAN_AS_BYTE); } else if (csType == byte.class || csType == Byte.class) { return new ByteRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, prevTimeScaleUnits, fwdTimeScaleUnits, - columnSource, rowRedirection, NULL_BYTE); + columnSource, redirContext, NULL_BYTE); } else if (csType == short.class || csType == Short.class) { return new ShortRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, prevTimeScaleUnits, fwdTimeScaleUnits, - columnSource, rowRedirection); + columnSource, redirContext); } else if (csType == int.class || csType == Integer.class) { return new IntRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, prevTimeScaleUnits, fwdTimeScaleUnits, - columnSource, rowRedirection); + columnSource, redirContext); } else if (csType == long.class || csType == Long.class) { return new LongRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, prevTimeScaleUnits, fwdTimeScaleUnits, - columnSource, rowRedirection); + columnSource, redirContext); } else if (csType == float.class || csType == Float.class) { return new FloatRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, prevTimeScaleUnits, fwdTimeScaleUnits, - columnSource, rowRedirection); + columnSource, redirContext); } else if (csType == double.class || csType == Double.class) { return new DoubleRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, - prevTimeScaleUnits, fwdTimeScaleUnits, columnSource, rowRedirection); + prevTimeScaleUnits, fwdTimeScaleUnits, columnSource, redirContext); // } else if (csType == BigDecimal.class) { // return new BigDecimalRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, -// prevTimeScaleUnits, fwdTimeScaleUnits, columnSource, rowRedirection, control.mathContextOrDefault()); +// prevTimeScaleUnits, fwdTimeScaleUnits, columnSource, redirContext, control.mathContextOrDefault()); // } else if (csType == BigInteger.class) { // return new BigIntegerRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, -// prevTimeScaleUnits, fwdTimeScaleUnits, columnSource, rowRedirection); +// prevTimeScaleUnits, fwdTimeScaleUnits, columnSource, redirContext); } throw new IllegalArgumentException("Can not perform RollingSum on type " + csType); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java index dd84b171eaa..c266380c544 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java @@ -27,8 +27,7 @@ public abstract class UpdateByWindowedOperator implements UpdateByOperator { protected final MatchPair pair; protected final String[] affectingColumns; - protected final boolean isRedirected; - + protected UpdateBy.UpdateByRedirectionContext redirContext; public abstract class UpdateWindowedContext implements UpdateContext { // store the current subset of rows that need computation @@ -266,7 +265,7 @@ public void close() { * @param control the control parameters for operation * @param timeRecorder an optional recorder for a timestamp column. If this is null, it will be assumed time is * measured in integer ticks. - * @param rowRedirection the row redirection to use for the operation + * @param redirContext the row redirection context to use for the operation */ public UpdateByWindowedOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @@ -274,14 +273,14 @@ public UpdateByWindowedOperator(@NotNull final MatchPair pair, @Nullable final LongRecordingUpdateByOperator timeRecorder, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @Nullable final RowRedirection rowRedirection) { + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { this.pair = pair; this.affectingColumns = affectingColumns; this.control = control; this.recorder = timeRecorder; this.reverseTimeScaleUnits = reverseTimeScaleUnits; this.forwardTimeScaleUnits = forwardTimeScaleUnits; - this.isRedirected = rowRedirection != null; + this.redirContext = redirContext; } public abstract void push(UpdateContext context, long key, int index); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java index 7ad5c6b93d2..e0b9866247a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java @@ -24,6 +24,8 @@ * An implementation of {@link UpdateBy} dedicated to zero key computation. */ class ZeroKeyUpdateBy extends UpdateBy { + /** Apply shifts to operator outputs? */ + final boolean applyShifts; /** * Perform an updateBy without any key columns. @@ -32,7 +34,7 @@ class ZeroKeyUpdateBy extends UpdateBy { * @param source the source table * @param ops the operations to perform * @param resultSources the result sources - * @param rowRedirection the {@link io.deephaven.engine.table.impl.util.RowRedirection}, if one is used. + * @param redirContext the row redirection shared context * @param control the control object. * @return the result table */ @@ -40,10 +42,11 @@ public static Table compute(@NotNull final String description, @NotNull final QueryTable source, @NotNull final UpdateByOperator[] ops, @NotNull final Map> resultSources, - @Nullable final WritableRowRedirection rowRedirection, - @NotNull final UpdateByControl control) { + @NotNull final UpdateByRedirectionContext redirContext, + @NotNull final UpdateByControl control, + final boolean applyShifts) { final QueryTable result = new QueryTable(source.getRowSet(), resultSources); - final ZeroKeyUpdateBy updateBy = new ZeroKeyUpdateBy(ops, source, rowRedirection, control); + final ZeroKeyUpdateBy updateBy = new ZeroKeyUpdateBy(ops, source, redirContext, control, applyShifts); updateBy.doInitialAdditions(); if (source.isRefreshing()) { @@ -57,9 +60,11 @@ public static Table compute(@NotNull final String description, protected ZeroKeyUpdateBy(@NotNull final UpdateByOperator[] operators, @NotNull final QueryTable source, - @Nullable final WritableRowRedirection rowRedirection, - @NotNull final UpdateByControl control) { - super(operators, source, rowRedirection, control); + @NotNull final UpdateByRedirectionContext redirContext, + @NotNull final UpdateByControl control, + final boolean applyShifts) { + super(operators, source, redirContext, control); + this.applyShifts = applyShifts; } ZeroKeyUpdateByListener newListener(@NotNull final String description, @NotNull final QueryTable result) { @@ -72,11 +77,13 @@ void doInitialAdditions() { RowSetFactory.empty(), RowSetShiftData.EMPTY, ModifiedColumnSet.ALL); + if (redirContext.isRedirected()) { + redirContext.processUpdateForRedirection(fakeUpdate, source.getRowSet()); + } + try (final UpdateContext ctx = new UpdateContext(fakeUpdate, null, true)) { ctx.setAllAffected(); - if (rowRedirection != null && source.isRefreshing()) { - processUpdateForRedirection(fakeUpdate); - } + // do an addition phase for all the operators that can add directly (i.e. backwards looking) ctx.doUpdate(source.getRowSet(), source.getRowSet(), UpdateType.Add); @@ -129,6 +136,7 @@ private class UpdateContext implements SafeCloseable { /** A Long Chunk for previous keys */ WritableLongChunk prevKeyChunk; + final RowSet affectedRows; @SuppressWarnings("resource") @@ -438,21 +446,9 @@ private void reprocessRows(RowSetShiftData shifted) { keyBefore = sit.binarySearchValue( (compareTo, ignored) -> Long.compare(keyStart - 1, compareTo), 1); } - operators[opIndex].resetForReprocess(opContext[opIndex], sourceRowSet, keyBefore); - } - } + // apply a shift to keyBefore since the output column is still in prev key space - // We will not mess with shifts if we are using a redirection because we'll have applied the shift - // to the redirection index already by now. - if (rowRedirection == null && shifted.nonempty()) { - try (final RowSet prevIdx = source.getRowSet().copyPrev()) { - shifted.apply((begin, end, delta) -> { - try (final RowSet subRowSet = prevIdx.subSetByKeyRange(begin, end)) { - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - operators[opIdx].applyOutputShift(opContext[opIdx], subRowSet, delta); - } - } - }); + operators[opIndex].resetForReprocess(opContext[opIndex], sourceRowSet, keyBefore); } } @@ -536,8 +532,23 @@ public ZeroKeyUpdateByListener(@Nullable String description, @Override public void onUpdate(TableUpdate upstream) { try (final UpdateContext ctx = new UpdateContext(upstream, inputModifiedColumnSets, false)) { - if (rowRedirection != null) { - processUpdateForRedirection(upstream); + + if (redirContext.isRedirected()) { + redirContext.processUpdateForRedirection(upstream, source.getRowSet()); + } + + // We will not mess with shifts if we are using a redirection because we'll have applied the shift + // to the redirection index already by now. + if (applyShifts && !redirContext.isRedirected() && upstream.shifted().nonempty()) { + try (final RowSet prevIdx = source.getRowSet().copyPrev()) { + upstream.shifted().apply((begin, end, delta) -> { + try (final RowSet subRowSet = prevIdx.subSetByKeyRange(begin, end)) { + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + operators[opIdx].applyOutputShift(ctx.opContext[opIdx], subRowSet, delta); + } + } + }); + } } // If anything can process normal operations we have to pass them down, otherwise we can skip this @@ -600,6 +611,7 @@ public void onUpdate(TableUpdate upstream) { downstream.modified = RowSetFactory.empty(); downstream.modifiedColumnSet = ModifiedColumnSet.EMPTY; } + result.notifyListeners(downstream); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java index 8d851e4977c..8681ac495fe 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java @@ -56,15 +56,15 @@ class EmaContext extends Context { * measured in integer ticks. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timeRecorder} is provided, this is measured * in ticks, otherwise it is measured in nanoseconds. - * @param rowRedirection the row redirection to use for the EMA + * @param redirContext the row redirection context to use for the EMA */ public BasePrimitiveEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final LongRecordingUpdateByOperator timeRecorder, final long timeScaleUnits, - @Nullable final RowRedirection rowRedirection) { - super(pair, affectingColumns, rowRedirection); + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { + super(pair, affectingColumns, redirContext); this.control = control; this.timeRecorder = timeRecorder; this.timeScaleUnits = timeScaleUnits; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java index 3cb151c8a3e..1e7eae540b3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java @@ -6,6 +6,7 @@ import io.deephaven.engine.table.ColumnSource; import io.deephaven.api.updateby.OperationControl; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -35,11 +36,11 @@ public BigDecimalEMAOperator(@NotNull final MatchPair pair, @Nullable final LongRecordingUpdateByOperator timeRecorder, final long timeScaleUnits, @NotNull final ColumnSource valueSource, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, timeScaleUnits, valueSource, rowRedirection); + super(pair, affectingColumns, control, timeRecorder, timeScaleUnits, valueSource, redirContext); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java index 26a2676bc5e..f1cee43724f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java @@ -6,6 +6,7 @@ import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -35,11 +36,11 @@ public BigIntegerEMAOperator(@NotNull final MatchPair pair, @Nullable final LongRecordingUpdateByOperator timeRecorder, final long timeScaleUnits, @NotNull final ColumnSource valueSource, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, timeScaleUnits, valueSource, rowRedirection); + super(pair, affectingColumns, control, timeRecorder, timeScaleUnits, valueSource, redirContext); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java index 2e5d30409e3..980bbfa9f79 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java @@ -60,11 +60,11 @@ public BigNumberEMAOperator(@NotNull final MatchPair pair, @Nullable final LongRecordingUpdateByOperator timeRecorder, final long timeScaleUnits, @NotNull ColumnSource valueSource, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, rowRedirection, BigDecimal.class); + super(pair, affectingColumns, redirContext, BigDecimal.class); this.control = control; this.timeRecorder = timeRecorder; this.timeScaleUnits = timeScaleUnits; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java index f8ac5847f43..dab16c4b127 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java @@ -12,6 +12,7 @@ import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -40,11 +41,11 @@ public ByteEMAOperator(@NotNull final MatchPair pair, @Nullable final LongRecordingUpdateByOperator timeRecorder, final long timeScaleUnits, @NotNull final ColumnSource valueSource, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, timeScaleUnits, rowRedirection); + super(pair, affectingColumns, control, timeRecorder, timeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java index cd477823b0c..f294028308e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java @@ -13,6 +13,7 @@ import io.deephaven.api.updateby.BadDataBehavior; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -41,11 +42,11 @@ public DoubleEMAOperator(@NotNull final MatchPair pair, @Nullable final LongRecordingUpdateByOperator timeRecorder, final long timeScaleUnits, @NotNull final ColumnSource valueSource, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, timeScaleUnits, rowRedirection); + super(pair, affectingColumns, control, timeRecorder, timeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java index 5560ce36da0..e42b50cd133 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java @@ -8,6 +8,7 @@ import io.deephaven.api.updateby.BadDataBehavior; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -36,11 +37,11 @@ public FloatEMAOperator(@NotNull final MatchPair pair, @Nullable final LongRecordingUpdateByOperator timeRecorder, final long timeScaleUnits, @NotNull final ColumnSource valueSource, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, timeScaleUnits, rowRedirection); + super(pair, affectingColumns, control, timeRecorder, timeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java index 87620966e84..545ccc8e987 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java @@ -12,6 +12,7 @@ import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -40,11 +41,11 @@ public IntEMAOperator(@NotNull final MatchPair pair, @Nullable final LongRecordingUpdateByOperator timeRecorder, final long timeScaleUnits, @NotNull final ColumnSource valueSource, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, timeScaleUnits, rowRedirection); + super(pair, affectingColumns, control, timeRecorder, timeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java index c183adf706c..c07683048d9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java @@ -12,6 +12,7 @@ import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -40,11 +41,11 @@ public LongEMAOperator(@NotNull final MatchPair pair, @Nullable final LongRecordingUpdateByOperator timeRecorder, final long timeScaleUnits, @NotNull final ColumnSource valueSource, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, timeScaleUnits, rowRedirection); + super(pair, affectingColumns, control, timeRecorder, timeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java index 69877c4b1b8..4e701f7b42d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java @@ -7,6 +7,7 @@ import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -35,11 +36,11 @@ public ShortEMAOperator(@NotNull final MatchPair pair, @Nullable final LongRecordingUpdateByOperator timeRecorder, final long timeScaleUnits, @NotNull final ColumnSource valueSource, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, timeScaleUnits, rowRedirection); + super(pair, affectingColumns, control, timeRecorder, timeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java index c52f5623e18..db55d8e65aa 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java @@ -19,6 +19,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseByteUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -31,11 +32,11 @@ public class BooleanFillByOperator extends BaseByteUpdateByOperator { // endregion extra-fields public BooleanFillByOperator(@NotNull final MatchPair fillPair, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection); + super(fillPair, new String[] { fillPair.rightColumn }, redirContext); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java index 57af903fb06..2b10b12d892 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java @@ -12,6 +12,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseByteUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -24,11 +25,11 @@ public class ByteFillByOperator extends BaseByteUpdateByOperator { // endregion extra-fields public ByteFillByOperator(@NotNull final MatchPair fillPair, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection); + super(fillPair, new String[] { fillPair.rightColumn }, redirContext); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java index 857aa4de476..ab737b9ffe4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java @@ -7,6 +7,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseCharUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -19,11 +20,11 @@ public class CharFillByOperator extends BaseCharUpdateByOperator { // endregion extra-fields public CharFillByOperator(@NotNull final MatchPair fillPair, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection); + super(fillPair, new String[] { fillPair.rightColumn }, redirContext); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java index f2060dd5600..85179bb593e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java @@ -12,6 +12,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseDoubleUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -24,11 +25,11 @@ public class DoubleFillByOperator extends BaseDoubleUpdateByOperator { // endregion extra-fields public DoubleFillByOperator(@NotNull final MatchPair fillPair, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection); + super(fillPair, new String[] { fillPair.rightColumn }, redirContext); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java index c346341e72b..9d5cf19b05c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java @@ -12,6 +12,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseFloatUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -24,11 +25,11 @@ public class FloatFillByOperator extends BaseFloatUpdateByOperator { // endregion extra-fields public FloatFillByOperator(@NotNull final MatchPair fillPair, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection); + super(fillPair, new String[] { fillPair.rightColumn }, redirContext); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java index a19e9674db6..91bd6e2cdb2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java @@ -12,6 +12,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseIntUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -24,11 +25,11 @@ public class IntFillByOperator extends BaseIntUpdateByOperator { // endregion extra-fields public IntFillByOperator(@NotNull final MatchPair fillPair, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection); + super(fillPair, new String[] { fillPair.rightColumn }, redirContext); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java index 4183bae6887..c038b488941 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java @@ -19,6 +19,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -32,12 +33,12 @@ public class LongFillByOperator extends BaseLongUpdateByOperator { // endregion extra-fields public LongFillByOperator(@NotNull final MatchPair fillPair, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args ,@NotNull final Class type // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection); + super(fillPair, new String[] { fillPair.rightColumn }, redirContext); // region constructor this.type = type; // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java index 231fb2ce711..880c0e8baa0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java @@ -12,6 +12,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -23,12 +24,12 @@ public class ObjectFillByOperator extends BaseObjectUpdateByOperator { // endregion extra-fields public ObjectFillByOperator(@NotNull final MatchPair fillPair, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args , final Class colType // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection, colType); + super(fillPair, new String[] { fillPair.rightColumn }, redirContext, colType); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java index 77cdd1a1d22..57242733be4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java @@ -12,6 +12,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseShortUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -24,11 +25,11 @@ public class ShortFillByOperator extends BaseShortUpdateByOperator { // endregion extra-fields public ShortFillByOperator(@NotNull final MatchPair fillPair, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection); + super(fillPair, new String[] { fillPair.rightColumn }, redirContext); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java index ee178bac861..73ada869b75 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java @@ -41,7 +41,8 @@ public abstract class BaseByteUpdateByOperator extends UpdateByCumulativeOperato protected final MatchPair pair; protected final String[] affectingColumns; - protected final boolean isRedirected; + + private UpdateBy.UpdateByRedirectionContext redirContext; // region extra-fields final byte nullValue; @@ -84,22 +85,22 @@ public void close() { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param rowRedirection the {@link RowRedirection} if one is used + * @param redirContext the {@link UpdateBy.UpdateByRedirectionContext} for the overall update */ public BaseByteUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { this.pair = pair; this.affectingColumns = affectingColumns; - this.isRedirected = rowRedirection != null; - if(rowRedirection != null) { + this.redirContext = redirContext; + if(this.redirContext.isRedirected()) { // region create-dense this.maybeInnerSource = makeDenseSource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(this.redirContext.getRowRedirection(), maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -174,16 +175,16 @@ public void initializeForUpdate(@NotNull final UpdateContext context, // If we're redirected we have to make sure we tell the output source it's actual size, or we're going // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not // need to do anything with capacity. - if(isRedirected) { + if(redirContext.isRedirected()) { // The redirection index does not use the 0th index for some reason. - outputSource.ensureCapacity(resultSourceIndex.size() + 1); + outputSource.ensureCapacity(redirContext.requiredCapacity()); } // just remember the appendyness. ctx.canProcessDirectly = isUpstreamAppendOnly; - // pre-load the context with the previous last value in the table (if possibe) - ctx.curVal = lastPrevKey == NULL_ROW_KEY ? nullValue : outputSource.getByte(lastPrevKey); + // pre-load the context with the previous last value in the table (if possible) + ctx.curVal = lastPrevKey == NULL_ROW_KEY ? nullValue : outputSource.getPrevByte(lastPrevKey); } @Override @@ -217,7 +218,7 @@ public boolean requiresValues(@NotNull final UpdateContext context) { @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (isRedirected) { + if (redirContext.isRedirected()) { maybeInnerSource.startTrackingPrevValues(); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java index b6d34d55fbe..d31b7b092dd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java @@ -31,7 +31,8 @@ public abstract class BaseCharUpdateByOperator extends UpdateByCumulativeOperato protected final MatchPair pair; protected final String[] affectingColumns; - protected final boolean isRedirected; + + private UpdateBy.UpdateByRedirectionContext redirContext; // region extra-fields // endregion extra-fields @@ -73,22 +74,22 @@ public void close() { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param rowRedirection the {@link RowRedirection} if one is used + * @param redirContext the {@link UpdateBy.UpdateByRedirectionContext} for the overall update */ public BaseCharUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { this.pair = pair; this.affectingColumns = affectingColumns; - this.isRedirected = rowRedirection != null; - if(rowRedirection != null) { + this.redirContext = redirContext; + if(this.redirContext.isRedirected()) { // region create-dense this.maybeInnerSource = new CharacterArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(this.redirContext.getRowRedirection(), maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -150,16 +151,16 @@ public void initializeForUpdate(@NotNull final UpdateContext context, // If we're redirected we have to make sure we tell the output source it's actual size, or we're going // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not // need to do anything with capacity. - if(isRedirected) { + if(redirContext.isRedirected()) { // The redirection index does not use the 0th index for some reason. - outputSource.ensureCapacity(resultSourceIndex.size() + 1); + outputSource.ensureCapacity(redirContext.requiredCapacity()); } // just remember the appendyness. ctx.canProcessDirectly = isUpstreamAppendOnly; - // pre-load the context with the previous last value in the table (if possibe) - ctx.curVal = lastPrevKey == NULL_ROW_KEY ? QueryConstants.NULL_CHAR : outputSource.getChar(lastPrevKey); + // pre-load the context with the previous last value in the table (if possible) + ctx.curVal = lastPrevKey == NULL_ROW_KEY ? QueryConstants.NULL_CHAR : outputSource.getPrevChar(lastPrevKey); } @Override @@ -193,7 +194,7 @@ public boolean requiresValues(@NotNull final UpdateContext context) { @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (isRedirected) { + if (redirContext.isRedirected()) { maybeInnerSource.startTrackingPrevValues(); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java index 0f8290d523a..09272d7d247 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java @@ -40,7 +40,8 @@ public abstract class BaseDoubleUpdateByOperator extends UpdateByCumulativeOpera private final MatchPair pair; private final String[] affectingColumns; - protected final boolean isRedirected; + + private UpdateBy.UpdateByRedirectionContext redirContext; protected class Context extends UpdateCumulativeContext { public final SizedSafeCloseable fillContext; @@ -82,16 +83,17 @@ public void close() { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. + * @param redirContext the {@link UpdateBy.UpdateByRedirectionContext} for the overall update */ public BaseDoubleUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final RowRedirection rowRedirection) { + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { this.pair = pair; this.affectingColumns = affectingColumns; - this.isRedirected = rowRedirection != null; - if(rowRedirection != null) { + this.redirContext = redirContext; + if(this.redirContext.isRedirected()) { this.maybeInnerSource = new DoubleArraySource(); - this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(this.redirContext.getRowRedirection(), maybeInnerSource, 0); } else { this.maybeInnerSource = null; this.outputSource = new DoubleSparseArraySource(); @@ -145,8 +147,8 @@ public void initializeForUpdate(@NotNull final UpdateContext context, // If we're redirected we have to make sure we tell the output source it's actual size, or we're going // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not // need to do anything with capacity. - if(isRedirected) { - outputSource.ensureCapacity(resultSourceRowSet.size() + 1); + if(redirContext.isRedirected()) { + outputSource.ensureCapacity(redirContext.requiredCapacity()); } // If we aren't bucketing, we'll just remember the appendyness. @@ -187,7 +189,7 @@ public boolean requiresValues(@NotNull final UpdateContext context) { @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (isRedirected) { + if (redirContext.isRedirected()) { maybeInnerSource.startTrackingPrevValues(); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java index 5223921ccf3..9017e334cf4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java @@ -35,7 +35,8 @@ public abstract class BaseFloatUpdateByOperator extends UpdateByCumulativeOperat private final MatchPair pair; private final String[] affectingColumns; - protected final boolean isRedirected; + + private UpdateBy.UpdateByRedirectionContext redirContext; protected class Context extends UpdateCumulativeContext { public final SizedSafeCloseable fillContext; @@ -77,16 +78,17 @@ public void close() { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. + * @param redirContext the {@link UpdateBy.UpdateByRedirectionContext} for the overall update */ public BaseFloatUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final RowRedirection rowRedirection) { + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { this.pair = pair; this.affectingColumns = affectingColumns; - this.isRedirected = rowRedirection != null; - if(rowRedirection != null) { + this.redirContext = redirContext; + if(this.redirContext.isRedirected()) { this.maybeInnerSource = new FloatArraySource(); - this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(this.redirContext.getRowRedirection(), maybeInnerSource, 0); } else { this.maybeInnerSource = null; this.outputSource = new FloatSparseArraySource(); @@ -140,8 +142,8 @@ public void initializeForUpdate(@NotNull final UpdateContext context, // If we're redirected we have to make sure we tell the output source it's actual size, or we're going // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not // need to do anything with capacity. - if(isRedirected) { - outputSource.ensureCapacity(resultSourceRowSet.size() + 1); + if(redirContext.isRedirected()) { + outputSource.ensureCapacity(redirContext.requiredCapacity()); } // If we aren't bucketing, we'll just remember the appendyness. @@ -182,7 +184,7 @@ public boolean requiresValues(@NotNull final UpdateContext context) { @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (isRedirected) { + if (redirContext.isRedirected()) { maybeInnerSource.startTrackingPrevValues(); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java index 563a04abec3..a7c57e0c9c2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java @@ -36,7 +36,8 @@ public abstract class BaseIntUpdateByOperator extends UpdateByCumulativeOperator protected final MatchPair pair; protected final String[] affectingColumns; - protected final boolean isRedirected; + + private UpdateBy.UpdateByRedirectionContext redirContext; // region extra-fields // endregion extra-fields @@ -78,22 +79,22 @@ public void close() { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param rowRedirection the {@link RowRedirection} if one is used + * @param redirContext the {@link UpdateBy.UpdateByRedirectionContext} for the overall update */ public BaseIntUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { this.pair = pair; this.affectingColumns = affectingColumns; - this.isRedirected = rowRedirection != null; - if(rowRedirection != null) { + this.redirContext = redirContext; + if(this.redirContext.isRedirected()) { // region create-dense this.maybeInnerSource = new IntegerArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(this.redirContext.getRowRedirection(), maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -155,16 +156,16 @@ public void initializeForUpdate(@NotNull final UpdateContext context, // If we're redirected we have to make sure we tell the output source it's actual size, or we're going // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not // need to do anything with capacity. - if(isRedirected) { + if(redirContext.isRedirected()) { // The redirection index does not use the 0th index for some reason. - outputSource.ensureCapacity(resultSourceIndex.size() + 1); + outputSource.ensureCapacity(redirContext.requiredCapacity()); } // just remember the appendyness. ctx.canProcessDirectly = isUpstreamAppendOnly; - // pre-load the context with the previous last value in the table (if possibe) - ctx.curVal = lastPrevKey == NULL_ROW_KEY ? QueryConstants.NULL_INT : outputSource.getInt(lastPrevKey); + // pre-load the context with the previous last value in the table (if possible) + ctx.curVal = lastPrevKey == NULL_ROW_KEY ? QueryConstants.NULL_INT : outputSource.getPrevInt(lastPrevKey); } @Override @@ -198,7 +199,7 @@ public boolean requiresValues(@NotNull final UpdateContext context) { @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (isRedirected) { + if (redirContext.isRedirected()) { maybeInnerSource.startTrackingPrevValues(); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java index f4e78c76ab5..3b5ed756ca7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java @@ -36,7 +36,8 @@ public abstract class BaseLongUpdateByOperator extends UpdateByCumulativeOperato protected final MatchPair pair; protected final String[] affectingColumns; - protected final boolean isRedirected; + + private UpdateBy.UpdateByRedirectionContext redirContext; // region extra-fields // endregion extra-fields @@ -78,22 +79,22 @@ public void close() { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param rowRedirection the {@link RowRedirection} if one is used + * @param redirContext the {@link UpdateBy.UpdateByRedirectionContext} for the overall update */ public BaseLongUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { this.pair = pair; this.affectingColumns = affectingColumns; - this.isRedirected = rowRedirection != null; - if(rowRedirection != null) { + this.redirContext = redirContext; + if(this.redirContext.isRedirected()) { // region create-dense this.maybeInnerSource = new LongArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(this.redirContext.getRowRedirection(), maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -155,16 +156,16 @@ public void initializeForUpdate(@NotNull final UpdateContext context, // If we're redirected we have to make sure we tell the output source it's actual size, or we're going // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not // need to do anything with capacity. - if(isRedirected) { + if(redirContext.isRedirected()) { // The redirection index does not use the 0th index for some reason. - outputSource.ensureCapacity(resultSourceIndex.size() + 1); + outputSource.ensureCapacity(redirContext.requiredCapacity()); } // just remember the appendyness. ctx.canProcessDirectly = isUpstreamAppendOnly; - // pre-load the context with the previous last value in the table (if possibe) - ctx.curVal = lastPrevKey == NULL_ROW_KEY ? QueryConstants.NULL_LONG : outputSource.getLong(lastPrevKey); + // pre-load the context with the previous last value in the table (if possible) + ctx.curVal = lastPrevKey == NULL_ROW_KEY ? QueryConstants.NULL_LONG : outputSource.getPrevLong(lastPrevKey); } @Override @@ -198,7 +199,7 @@ public boolean requiresValues(@NotNull final UpdateContext context) { @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (isRedirected) { + if (redirContext.isRedirected()) { maybeInnerSource.startTrackingPrevValues(); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java index 9ae71229dc5..d89c2687244 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java @@ -7,6 +7,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -15,8 +16,8 @@ public abstract class BaseObjectBinaryOperator extends BaseObjectUpdateByOper public BaseObjectBinaryOperator(@NotNull final Class type, @NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final RowRedirection rowRedirection) { - super(pair, affectingColumns, rowRedirection, type); + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { + super(pair, affectingColumns, redirContext, type); } protected abstract T doOperation(T bucketCurVal, T chunkCurVal); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java index a2cd5eb7b71..575f6c8a537 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java @@ -36,7 +36,8 @@ public abstract class BaseObjectUpdateByOperator extends UpdateByCumulativeOp protected final MatchPair pair; protected final String[] affectingColumns; - protected final boolean isRedirected; + + private UpdateBy.UpdateByRedirectionContext redirContext; // region extra-fields private final Class colType; @@ -79,23 +80,23 @@ public void close() { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param rowRedirection the {@link RowRedirection} if one is used + * @param redirContext the {@link UpdateBy.UpdateByRedirectionContext} for the overall update */ public BaseObjectUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args , final Class colType // endregion extra-constructor-args ) { this.pair = pair; this.affectingColumns = affectingColumns; - this.isRedirected = rowRedirection != null; - if(rowRedirection != null) { + this.redirContext = redirContext; + if(this.redirContext.isRedirected()) { // region create-dense this.maybeInnerSource = new ObjectArraySource(colType); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(this.redirContext.getRowRedirection(), maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -158,16 +159,16 @@ public void initializeForUpdate(@NotNull final UpdateContext context, // If we're redirected we have to make sure we tell the output source it's actual size, or we're going // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not // need to do anything with capacity. - if(isRedirected) { + if(redirContext.isRedirected()) { // The redirection index does not use the 0th index for some reason. - outputSource.ensureCapacity(resultSourceIndex.size() + 1); + outputSource.ensureCapacity(redirContext.requiredCapacity()); } // just remember the appendyness. ctx.canProcessDirectly = isUpstreamAppendOnly; - // pre-load the context with the previous last value in the table (if possibe) - ctx.curVal = lastPrevKey == NULL_ROW_KEY ? null : outputSource.get(lastPrevKey); + // pre-load the context with the previous last value in the table (if possible) + ctx.curVal = lastPrevKey == NULL_ROW_KEY ? null : outputSource.getPrev(lastPrevKey); } @Override @@ -201,7 +202,7 @@ public boolean requiresValues(@NotNull final UpdateContext context) { @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (isRedirected) { + if (redirContext.isRedirected()) { maybeInnerSource.startTrackingPrevValues(); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java index 5cf3f505ebc..02cac6ce586 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java @@ -36,7 +36,8 @@ public abstract class BaseShortUpdateByOperator extends UpdateByCumulativeOperat protected final MatchPair pair; protected final String[] affectingColumns; - protected final boolean isRedirected; + + private UpdateBy.UpdateByRedirectionContext redirContext; // region extra-fields // endregion extra-fields @@ -78,22 +79,22 @@ public void close() { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param rowRedirection the {@link RowRedirection} if one is used + * @param redirContext the {@link UpdateBy.UpdateByRedirectionContext} for the overall update */ public BaseShortUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { this.pair = pair; this.affectingColumns = affectingColumns; - this.isRedirected = rowRedirection != null; - if(rowRedirection != null) { + this.redirContext = redirContext; + if(this.redirContext.isRedirected()) { // region create-dense this.maybeInnerSource = new ShortArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(this.redirContext.getRowRedirection(), maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -155,16 +156,16 @@ public void initializeForUpdate(@NotNull final UpdateContext context, // If we're redirected we have to make sure we tell the output source it's actual size, or we're going // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not // need to do anything with capacity. - if(isRedirected) { + if(redirContext.isRedirected()) { // The redirection index does not use the 0th index for some reason. - outputSource.ensureCapacity(resultSourceIndex.size() + 1); + outputSource.ensureCapacity(redirContext.requiredCapacity()); } // just remember the appendyness. ctx.canProcessDirectly = isUpstreamAppendOnly; - // pre-load the context with the previous last value in the table (if possibe) - ctx.curVal = lastPrevKey == NULL_ROW_KEY ? QueryConstants.NULL_SHORT : outputSource.getShort(lastPrevKey); + // pre-load the context with the previous last value in the table (if possible) + ctx.curVal = lastPrevKey == NULL_ROW_KEY ? QueryConstants.NULL_SHORT : outputSource.getPrevShort(lastPrevKey); } @Override @@ -198,7 +199,7 @@ public boolean requiresValues(@NotNull final UpdateContext context) { @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (isRedirected) { + if (redirContext.isRedirected()) { maybeInnerSource.startTrackingPrevValues(); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java index 10beb05d6e5..8b83e7afd7d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java @@ -83,12 +83,12 @@ public BaseWindowedByteUpdateByOperator(@NotNull final MatchPair pair, @Nullable final LongRecordingUpdateByOperator timeRecorder, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @Nullable final RowRedirection rowRedirection, + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, @NotNull final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); + super(pair, affectingColumns, control, timeRecorder, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor this.nullValue = getNullValue(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java index 75f354c5ddc..0bae1612dbe 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java @@ -72,12 +72,12 @@ public BaseWindowedCharUpdateByOperator(@NotNull final MatchPair pair, @Nullable final LongRecordingUpdateByOperator timeRecorder, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @Nullable final RowRedirection rowRedirection, + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, @NotNull final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); + super(pair, affectingColumns, control, timeRecorder, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java index 0b3687f9061..6b387573b4d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java @@ -75,12 +75,12 @@ public BaseWindowedDoubleUpdateByOperator(@NotNull final MatchPair pair, @Nullable final LongRecordingUpdateByOperator timeRecorder, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @Nullable final RowRedirection rowRedirection, + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, @NotNull final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); + super(pair, affectingColumns, control, timeRecorder, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java index 44aaf1ebac1..ed4246fe873 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java @@ -70,12 +70,12 @@ public BaseWindowedFloatUpdateByOperator(@NotNull final MatchPair pair, @Nullable final LongRecordingUpdateByOperator timeRecorder, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @Nullable final RowRedirection rowRedirection, + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, @NotNull final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); + super(pair, affectingColumns, control, timeRecorder, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java index 591571bc2a3..563a7442189 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java @@ -77,12 +77,12 @@ public BaseWindowedIntUpdateByOperator(@NotNull final MatchPair pair, @Nullable final LongRecordingUpdateByOperator timeRecorder, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @Nullable final RowRedirection rowRedirection, + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, @NotNull final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); + super(pair, affectingColumns, control, timeRecorder, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java index b3ac6426dec..c779cbf50ae 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java @@ -77,12 +77,12 @@ public BaseWindowedLongUpdateByOperator(@NotNull final MatchPair pair, @Nullable final LongRecordingUpdateByOperator timeRecorder, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @Nullable final RowRedirection rowRedirection, + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, @NotNull final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); + super(pair, affectingColumns, control, timeRecorder, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java index 5640e52f8ab..a19b4a7cc75 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java @@ -78,13 +78,13 @@ public BaseWindowedObjectUpdateByOperator(@NotNull final MatchPair pair, @Nullable final LongRecordingUpdateByOperator timeRecorder, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @Nullable final RowRedirection rowRedirection, + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, @NotNull final ColumnSource valueSource // region extra-constructor-args , final Class colType // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); + super(pair, affectingColumns, control, timeRecorder, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor this.colType = colType; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java index feb76902e9f..a86b769c2a6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java @@ -77,12 +77,12 @@ public BaseWindowedShortUpdateByOperator(@NotNull final MatchPair pair, @Nullable final LongRecordingUpdateByOperator timeRecorder, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @Nullable final RowRedirection rowRedirection, + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, @NotNull final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); + super(pair, affectingColumns, control, timeRecorder, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/LongRecordingUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/LongRecordingUpdateByOperator.java index 9b652d01b1d..7cf53ee9369 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/LongRecordingUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/LongRecordingUpdateByOperator.java @@ -73,6 +73,16 @@ public long getCurrentLong(final long key) { return columnSource.getLong(key); } + /** + * Get the pervious long value within the underlying {@link ColumnSource}. + * + * @param key the key + * @return the previous value at the key within the column source. + */ + public long getPrevLong(final long key) { + return columnSource.getPrevLong(key); + } + /** * Get the current underlying {@link ColumnSource}. * diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java index e64580b614f..0e3f2a3f4f3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java @@ -12,6 +12,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseByteUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -27,11 +28,11 @@ public class ByteCumMinMaxOperator extends BaseByteUpdateByOperator { public ByteCumMinMaxOperator(@NotNull final MatchPair inputPair, final boolean isMax, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(inputPair, new String[] { inputPair.rightColumn }, rowRedirection); + super(inputPair, new String[] { inputPair.rightColumn }, redirContext); this.isMax = isMax; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ComparableCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ComparableCumMinMaxOperator.java index 754d93a8509..d4ad7371e40 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ComparableCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ComparableCumMinMaxOperator.java @@ -1,6 +1,7 @@ package io.deephaven.engine.table.impl.updateby.minmax; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectBinaryOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -12,8 +13,8 @@ public final class ComparableCumMinMaxOperator> extends public ComparableCumMinMaxOperator(final Class colType, @NotNull final MatchPair inputPair, final boolean isMax, - @Nullable final RowRedirection rowRedirection) { - super(colType, inputPair, new String[] {inputPair.rightColumn}, rowRedirection); + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { + super(colType, inputPair, new String[] {inputPair.rightColumn}, redirContext); this.isMax = isMax; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java index 739ece75eb8..6d419ecbef1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java @@ -12,6 +12,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseDoubleUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -24,11 +25,11 @@ public class DoubleCumMinMaxOperator extends BaseDoubleUpdateByOperator { public DoubleCumMinMaxOperator(@NotNull final MatchPair inputPair, final boolean isMax, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(inputPair, new String[] { inputPair.rightColumn }, rowRedirection); + super(inputPair, new String[] { inputPair.rightColumn }, redirContext); this.isMax = isMax; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java index 43812d6c4ba..0bea3eaf1bf 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java @@ -7,6 +7,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseFloatUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -19,11 +20,11 @@ public class FloatCumMinMaxOperator extends BaseFloatUpdateByOperator { public FloatCumMinMaxOperator(@NotNull final MatchPair inputPair, final boolean isMax, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(inputPair, new String[] { inputPair.rightColumn }, rowRedirection); + super(inputPair, new String[] { inputPair.rightColumn }, redirContext); this.isMax = isMax; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java index fe7b0d2dc6c..68825f2ef43 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java @@ -12,6 +12,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseIntUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -27,11 +28,11 @@ public class IntCumMinMaxOperator extends BaseIntUpdateByOperator { public IntCumMinMaxOperator(@NotNull final MatchPair inputPair, final boolean isMax, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(inputPair, new String[] { inputPair.rightColumn }, rowRedirection); + super(inputPair, new String[] { inputPair.rightColumn }, redirContext); this.isMax = isMax; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java index ebfa116a903..1fa97c895b2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java @@ -19,6 +19,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -35,12 +36,12 @@ public class LongCumMinMaxOperator extends BaseLongUpdateByOperator { public LongCumMinMaxOperator(@NotNull final MatchPair inputPair, final boolean isMax, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args ,@NotNull final Class type // endregion extra-constructor-args ) { - super(inputPair, new String[] { inputPair.rightColumn }, rowRedirection); + super(inputPair, new String[] { inputPair.rightColumn }, redirContext); this.isMax = isMax; // region constructor this.type = type; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java index 65505be8359..5889301bf87 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java @@ -7,6 +7,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseShortUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -22,11 +23,11 @@ public class ShortCumMinMaxOperator extends BaseShortUpdateByOperator { public ShortCumMinMaxOperator(@NotNull final MatchPair inputPair, final boolean isMax, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(inputPair, new String[] { inputPair.rightColumn }, rowRedirection); + super(inputPair, new String[] { inputPair.rightColumn }, redirContext); this.isMax = isMax; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java index 28cb30ce2e3..bd04dc77270 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java @@ -1,6 +1,7 @@ package io.deephaven.engine.table.impl.updateby.prod; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectBinaryOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -14,9 +15,9 @@ public final class BigDecimalCumProdOperator extends BaseObjectBinaryOperator { public BigIntegerCumProdOperator(@NotNull final MatchPair inputPair, - @Nullable final RowRedirection rowRedirection) { - super(BigInteger.class, inputPair, new String[] { inputPair.rightColumn }, rowRedirection); + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { + super(BigInteger.class, inputPair, new String[] { inputPair.rightColumn }, redirContext); } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java index 348e7639fb6..776d3a8418b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java @@ -12,6 +12,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -23,11 +24,11 @@ public class ByteCumProdOperator extends BaseLongUpdateByOperator { public ByteCumProdOperator(final @NotNull MatchPair inputPair, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(inputPair, new String[]{ inputPair.rightColumn }, rowRedirection); + super(inputPair, new String[]{ inputPair.rightColumn }, redirContext); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java index 4571df8cd30..6b4d9005c6b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java @@ -12,6 +12,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseDoubleUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -22,11 +23,11 @@ public class DoubleCumProdOperator extends BaseDoubleUpdateByOperator { public DoubleCumProdOperator(@NotNull final MatchPair inputPair, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(inputPair, new String[]{inputPair.rightColumn}, rowRedirection); + super(inputPair, new String[]{inputPair.rightColumn}, redirContext); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java index 5797c9b771e..0038459136c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java @@ -7,6 +7,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseFloatUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -17,11 +18,11 @@ public class FloatCumProdOperator extends BaseFloatUpdateByOperator { public FloatCumProdOperator(@NotNull final MatchPair inputPair, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(inputPair, new String[]{inputPair.rightColumn}, rowRedirection); + super(inputPair, new String[]{inputPair.rightColumn}, redirContext); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java index 0846e1ab972..ad45109ccd7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java @@ -12,6 +12,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -23,11 +24,11 @@ public class IntCumProdOperator extends BaseLongUpdateByOperator { public IntCumProdOperator(final @NotNull MatchPair inputPair, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(inputPair, new String[]{ inputPair.rightColumn }, rowRedirection); + super(inputPair, new String[]{ inputPair.rightColumn }, redirContext); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java index ecc59d8e0d9..c34ac70609b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java @@ -12,6 +12,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -23,11 +24,11 @@ public class LongCumProdOperator extends BaseLongUpdateByOperator { public LongCumProdOperator(final @NotNull MatchPair inputPair, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(inputPair, new String[]{ inputPair.rightColumn }, rowRedirection); + super(inputPair, new String[]{ inputPair.rightColumn }, redirContext); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java index a7e41777bf4..6a8be7fb265 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java @@ -7,6 +7,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -18,11 +19,11 @@ public class ShortCumProdOperator extends BaseLongUpdateByOperator { public ShortCumProdOperator(final @NotNull MatchPair inputPair, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(inputPair, new String[]{ inputPair.rightColumn }, rowRedirection); + super(inputPair, new String[]{ inputPair.rightColumn }, redirContext); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java index a93a6b9446f..4281faa82bd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java @@ -86,17 +86,17 @@ public ByteRollingSumOperator(@NotNull final MatchPair pair, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final ColumnSource valueSource, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args ,final byte nullValue // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, recorder, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection, valueSource); - if(rowRedirection != null) { + super(pair, affectingColumns, control, recorder, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource); + if(redirContext.isRedirected()) { // region create-dense this.maybeInnerSource = new LongArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(redirContext.getRowRedirection(), maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -179,7 +179,7 @@ public Map> getOutputColumns() { @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if(isRedirected) { + if(redirContext.isRedirected()) { maybeInnerSource.startTrackingPrevValues(); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java index 929ccbdee40..d90995dc550 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java @@ -88,16 +88,16 @@ public DoubleRollingSumOperator(@NotNull final MatchPair pair, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final ColumnSource valueSource, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, recorder, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection, valueSource); - if(rowRedirection != null) { + super(pair, affectingColumns, control, recorder, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource); + if(redirContext.isRedirected()) { // region create-dense this.maybeInnerSource = new DoubleArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(redirContext.getRowRedirection(), maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -173,7 +173,7 @@ public Map> getOutputColumns() { @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if(isRedirected) { + if(redirContext.isRedirected()) { maybeInnerSource.startTrackingPrevValues(); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java index e9ef13dcf01..31381623dbf 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java @@ -83,16 +83,16 @@ public FloatRollingSumOperator(@NotNull final MatchPair pair, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final ColumnSource valueSource, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, recorder, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection, valueSource); - if(rowRedirection != null) { + super(pair, affectingColumns, control, recorder, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource); + if(redirContext.isRedirected()) { // region create-dense this.maybeInnerSource = new FloatArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(redirContext.getRowRedirection(), maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -168,7 +168,7 @@ public Map> getOutputColumns() { @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if(isRedirected) { + if(redirContext.isRedirected()) { maybeInnerSource.startTrackingPrevValues(); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java index f7363227d22..7b9922823ca 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java @@ -85,16 +85,16 @@ public IntRollingSumOperator(@NotNull final MatchPair pair, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final ColumnSource valueSource, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, recorder, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection, valueSource); - if(rowRedirection != null) { + super(pair, affectingColumns, control, recorder, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource); + if(redirContext.isRedirected()) { // region create-dense this.maybeInnerSource = new LongArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(redirContext.getRowRedirection(), maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -176,7 +176,7 @@ public Map> getOutputColumns() { @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if(isRedirected) { + if(redirContext.isRedirected()) { maybeInnerSource.startTrackingPrevValues(); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java index 181cf5cbf01..ce7e56f18b3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java @@ -85,16 +85,16 @@ public LongRollingSumOperator(@NotNull final MatchPair pair, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final ColumnSource valueSource, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, recorder, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection, valueSource); - if(rowRedirection != null) { + super(pair, affectingColumns, control, recorder, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource); + if(redirContext.isRedirected()) { // region create-dense this.maybeInnerSource = new LongArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(redirContext.getRowRedirection(), maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -176,7 +176,7 @@ public Map> getOutputColumns() { @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if(isRedirected) { + if(redirContext.isRedirected()) { maybeInnerSource.startTrackingPrevValues(); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java index 1804e9f3cd8..dc8e06624e8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java @@ -80,16 +80,16 @@ public ShortRollingSumOperator(@NotNull final MatchPair pair, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final ColumnSource valueSource, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, recorder, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection, valueSource); - if(rowRedirection != null) { + super(pair, affectingColumns, control, recorder, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource); + if(redirContext.isRedirected()) { // region create-dense this.maybeInnerSource = new LongArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(redirContext.getRowRedirection(), maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -171,7 +171,7 @@ public Map> getOutputColumns() { @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if(isRedirected) { + if(redirContext.isRedirected()) { maybeInnerSource.startTrackingPrevValues(); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java index 8cdf09f8d18..19e9951b72b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java @@ -1,6 +1,7 @@ package io.deephaven.engine.table.impl.updateby.sum; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectBinaryOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -14,9 +15,9 @@ public final class BigDecimalCumSumOperator extends BaseObjectBinaryOperator { public BigIntegerCumSumOperator(@NotNull final MatchPair inputPair, - @Nullable final RowRedirection rowRedirection) { - super(BigInteger.class, inputPair, new String[] { inputPair.rightColumn }, rowRedirection); + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { + super(BigInteger.class, inputPair, new String[] { inputPair.rightColumn }, redirContext); } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java index 557dfb2c71c..ac3ef78e1a1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java @@ -12,6 +12,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -26,12 +27,12 @@ public class ByteCumSumOperator extends BaseLongUpdateByOperator { // endregion extra-fields public ByteCumSumOperator(@NotNull final MatchPair pair, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args ,final byte nullValue // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection); + super(pair, new String[] { pair.rightColumn }, redirContext); // region constructor this.nullValue = nullValue; // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java index eb67d79a112..e3155959b0e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java @@ -12,6 +12,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseDoubleUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -22,11 +23,11 @@ public class DoubleCumSumOperator extends BaseDoubleUpdateByOperator { public DoubleCumSumOperator(@NotNull final MatchPair pair, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection); + super(pair, new String[] { pair.rightColumn }, redirContext); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java index da4c0802fa8..1ae2bd53126 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java @@ -7,6 +7,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseFloatUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -17,11 +18,11 @@ public class FloatCumSumOperator extends BaseFloatUpdateByOperator { public FloatCumSumOperator(@NotNull final MatchPair pair, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection); + super(pair, new String[] { pair.rightColumn }, redirContext); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java index 2359d2d31e5..4857bf2c000 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java @@ -12,6 +12,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -25,11 +26,11 @@ public class IntCumSumOperator extends BaseLongUpdateByOperator { // endregion extra-fields public IntCumSumOperator(@NotNull final MatchPair pair, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection); + super(pair, new String[] { pair.rightColumn }, redirContext); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java index 6d871e4dc44..5ece480b675 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java @@ -12,6 +12,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -25,11 +26,11 @@ public class LongCumSumOperator extends BaseLongUpdateByOperator { // endregion extra-fields public LongCumSumOperator(@NotNull final MatchPair pair, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection); + super(pair, new String[] { pair.rightColumn }, redirContext); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java index f63b10167ed..59134295ee7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java @@ -7,6 +7,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -20,11 +21,11 @@ public class ShortCumSumOperator extends BaseLongUpdateByOperator { // endregion extra-fields public ShortCumSumOperator(@NotNull final MatchPair pair, - @Nullable final RowRedirection rowRedirection + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection); + super(pair, new String[] { pair.rightColumn }, redirContext); // region constructor // endregion constructor } diff --git a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java index 6faaf3c1e9f..bed6f65e788 100644 --- a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java +++ b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java @@ -34,8 +34,8 @@ public static void main(String[] args) throws IOException { String objectResult = ReplicatePrimitiveCode.charToObject( "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java"); fixupStandardObject(objectResult, "ObjectFillByOperator", false, - "super\\(fillPair, new String\\[\\] \\{ fillPair.rightColumn \\}, rowRedirection\\);", - "super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection, colType);", + "super\\(fillPair, new String\\[\\] \\{ fillPair.rightColumn \\}, redirContext\\);", + "super(fillPair, new String[] { fillPair.rightColumn }, redirContext, colType);", " BaseObjectUpdateByOperator", " BaseObjectUpdateByOperator"); final String[] exemptions = new String[] { @@ -287,7 +287,8 @@ private static void fixupStandardObject(String objectResult, final String classN "Object curVal", "T curVal", "Object previousVal", "T previousVal", "Object singletonVal", "T singletonVal", - "getObject", "get"); + "getObject", "get", + "getPrevObject", "getPrev"); if (extraReplacements != null && extraReplacements.length > 0) { lines = globalReplacements(lines, extraReplacements); } From cc74901a3968740c47b875d82f81ef929519efa0 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Mon, 29 Aug 2022 15:36:05 -0700 Subject: [PATCH 013/123] Big refactor to simplify processing --- .../impl/BucketedPartitionedUpdateBy.java | 56 +-- .../deephaven/engine/table/impl/UpdateBy.java | 73 +--- .../impl/UpdateByCumulativeOperator.java | 68 +++- .../engine/table/impl/UpdateByOperator.java | 124 ++----- .../table/impl/UpdateByOperatorFactory.java | 32 +- .../table/impl/UpdateByWindowedOperator.java | 88 +++-- .../engine/table/impl/ZeroKeyUpdateBy.java | 175 ++++------ .../impl/by/PartitionByChunkedOperator.java | 322 ++++++++++-------- .../ema/BasePrimitiveEMAOperator.java | 70 ++-- .../updateby/ema/BigDecimalEMAOperator.java | 3 +- .../updateby/ema/BigIntegerEMAOperator.java | 3 +- .../updateby/ema/BigNumberEMAOperator.java | 57 +--- .../impl/updateby/ema/ByteEMAOperator.java | 3 +- .../impl/updateby/ema/DoubleEMAOperator.java | 3 +- .../impl/updateby/ema/FloatEMAOperator.java | 3 +- .../impl/updateby/ema/IntEMAOperator.java | 3 +- .../impl/updateby/ema/LongEMAOperator.java | 3 +- .../impl/updateby/ema/ShortEMAOperator.java | 3 +- .../updateby/fill/BooleanFillByOperator.java | 6 +- .../updateby/fill/ByteFillByOperator.java | 6 +- .../updateby/fill/CharFillByOperator.java | 6 +- .../updateby/fill/DoubleFillByOperator.java | 6 +- .../updateby/fill/FloatFillByOperator.java | 6 +- .../impl/updateby/fill/IntFillByOperator.java | 6 +- .../updateby/fill/LongFillByOperator.java | 6 +- .../updateby/fill/ObjectFillByOperator.java | 6 +- .../updateby/fill/ShortFillByOperator.java | 6 +- .../updateby/hashing/UpdateBySlotTracker.java | 2 +- .../internal/BaseByteUpdateByOperator.java | 161 ++------- .../internal/BaseCharUpdateByOperator.java | 161 ++------- .../internal/BaseDoubleUpdateByOperator.java | 134 +------- .../internal/BaseFloatUpdateByOperator.java | 134 +------- .../internal/BaseIntUpdateByOperator.java | 161 ++------- .../internal/BaseLongUpdateByOperator.java | 161 ++------- .../internal/BaseObjectBinaryOperator.java | 2 +- .../internal/BaseObjectUpdateByOperator.java | 161 ++------- .../internal/BaseShortUpdateByOperator.java | 161 ++------- .../BaseWindowedByteUpdateByOperator.java | 143 +------- .../BaseWindowedCharUpdateByOperator.java | 143 +------- .../BaseWindowedDoubleUpdateByOperator.java | 134 +------- .../BaseWindowedFloatUpdateByOperator.java | 134 +------- .../BaseWindowedIntUpdateByOperator.java | 143 +------- .../BaseWindowedLongUpdateByOperator.java | 143 +------- .../BaseWindowedObjectUpdateByOperator.java | 143 +------- .../BaseWindowedShortUpdateByOperator.java | 143 +------- .../LongRecordingUpdateByOperator.java | 59 +--- .../minmax/ByteCumMinMaxOperator.java | 4 +- .../minmax/DoubleCumMinMaxOperator.java | 2 +- .../minmax/FloatCumMinMaxOperator.java | 2 +- .../updateby/minmax/IntCumMinMaxOperator.java | 4 +- .../minmax/LongCumMinMaxOperator.java | 4 +- .../minmax/ShortCumMinMaxOperator.java | 4 +- .../updateby/prod/ByteCumProdOperator.java | 2 +- .../updateby/prod/DoubleCumProdOperator.java | 2 +- .../updateby/prod/FloatCumProdOperator.java | 2 +- .../updateby/prod/IntCumProdOperator.java | 2 +- .../updateby/prod/LongCumProdOperator.java | 2 +- .../updateby/prod/ShortCumProdOperator.java | 2 +- .../rollingsum/ByteRollingSumOperator.java | 6 +- .../rollingsum/DoubleRollingSumOperator.java | 12 +- .../rollingsum/FloatRollingSumOperator.java | 12 +- .../rollingsum/IntRollingSumOperator.java | 6 +- .../rollingsum/LongRollingSumOperator.java | 6 +- .../rollingsum/ShortRollingSumOperator.java | 6 +- .../impl/updateby/sum/ByteCumSumOperator.java | 2 +- .../updateby/sum/DoubleCumSumOperator.java | 2 +- .../updateby/sum/FloatCumSumOperator.java | 2 +- .../impl/updateby/sum/IntCumSumOperator.java | 2 +- .../impl/updateby/sum/LongCumSumOperator.java | 2 +- .../updateby/sum/ShortCumSumOperator.java | 2 +- .../table/impl/updateby/TestRollingSum.java | 4 +- 71 files changed, 894 insertions(+), 2768 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java index 661043e21b5..19b76c16d5a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java @@ -11,10 +11,7 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.util.Collection; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; +import java.util.*; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; @@ -26,7 +23,6 @@ class BucketedPartitionedUpdateBy extends UpdateBy { private final LinkedList recorders; private final QueryTable resultTable; - /** * Perform an updateBy without any key columns. * @@ -68,28 +64,30 @@ protected BucketedPartitionedUpdateBy(@NotNull final String description, // create a source-listener that will listen to the source updates and apply the shifts to the output columns final QueryTable sourceListenerTable = new QueryTable(source.getRowSet(), source.getColumnSourceMap()); - source.listenForUpdates(new BaseTable.ListenerImpl("", source, sourceListenerTable) { - @Override - public void onUpdate(@NotNull final TableUpdate upstream) { - if (!redirContext.isRedirected() && upstream.shifted().nonempty()) { - try (final RowSet prevIdx = source.getRowSet().copyPrev()) { - upstream.shifted().apply((begin, end, delta) -> { - try (final RowSet subRowSet = prevIdx.subSetByKeyRange(begin, end)) { - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - operators[opIdx].applyOutputShift(null, subRowSet, delta); - } - } - }); - } - } - super.onUpdate(upstream); - }}); - // this table will always have the rowset of the source resultTable = new QueryTable(source.getRowSet(), resultSources); if (source.isRefreshing()) { + source.listenForUpdates(new BaseTable.ListenerImpl("", source, sourceListenerTable) { + @Override + public void onUpdate(@NotNull final TableUpdate upstream) { + if (redirContext.isRedirected()) { + redirContext.processUpdateForRedirection(upstream, source.getRowSet()); + } else if (upstream.shifted().nonempty()) { + try (final RowSet prevIdx = source.getRowSet().copyPrev()) { + upstream.shifted().apply((begin, end, delta) -> { + try (final RowSet subRowSet = prevIdx.subSetByKeyRange(begin, end)) { + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + operators[opIdx].applyOutputShift(null, subRowSet, delta); + } + } + }); + } + } + super.onUpdate(upstream); + }}); + recorders = new LinkedList<>(); listener = newListener(description); @@ -104,6 +102,16 @@ public void onUpdate(@NotNull final TableUpdate upstream) { recorders = null; } + if (redirContext.isRedirected()) { + // make a dummy update to generate the initial row keys + final TableUpdateImpl fakeUpdate = new TableUpdateImpl(source.getRowSet(), + RowSetFactory.empty(), + RowSetFactory.empty(), + RowSetShiftData.EMPTY, + ModifiedColumnSet.ALL); + redirContext.processUpdateForRedirection(fakeUpdate, source.getRowSet()); + } + final PartitionedTable pt = sourceListenerTable.partitionedAggBy(List.of(), true, null, byColumns); final PartitionedTable transformed = pt.transform(t -> { // create the table @@ -187,10 +195,10 @@ protected void process() { recorders.forEach(lr -> { if (lr.getModified().isNonempty()) { - // Transform any untouched modified columns to the output. - lr.modifiedColumnsTransformer.transform(lr.getModifiedColumnSet(), downstream.modifiedColumnSet); modifiedRowSet.insert(lr.getModified()); } + // Transform any untouched modified columns to the output. + lr.modifiedColumnsTransformer.transform(lr.getModifiedColumnSet(), downstream.modifiedColumnSet); }); // should not include actual adds as modifies diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java index c3c68b4c13a..c044e50e395 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java @@ -65,10 +65,8 @@ public WritableRowRedirection getRowRedirection() { public void processUpdateForRedirection(@NotNull final TableUpdate upstream, final TrackingRowSet prevRowSet) { if (upstream.removed().isNonempty()) { final RowSetBuilderRandom freeBuilder = RowSetFactory.builderRandom(); - synchronized (rowRedirection) { - upstream.removed().forAllRowKeys(key -> freeBuilder.addKey(rowRedirection.remove(key))); - freeRows.insert(freeBuilder.build()); - } + upstream.removed().forAllRowKeys(key -> freeBuilder.addKey(rowRedirection.remove(key))); + freeRows.insert(freeBuilder.build()); } if (upstream.shifted().nonempty()) { @@ -99,27 +97,23 @@ public void processUpdateForRedirection(@NotNull final TableUpdate upstream, fin if (upstream.added().isNonempty()) { final MutableLong lastAllocated = new MutableLong(0); - synchronized (rowRedirection) { - final WritableRowSet.Iterator freeIt = freeRows.iterator(); - upstream.added().forAllRowKeys(outerKey -> { - final long innerKey = freeIt.hasNext() ? freeIt.nextLong() : ++maxInnerIndex; - lastAllocated.setValue(innerKey); - rowRedirection.put(outerKey, innerKey); - }); - freeRows.removeRange(0, lastAllocated.longValue()); - } + final WritableRowSet.Iterator freeIt = freeRows.iterator(); + upstream.added().forAllRowKeys(outerKey -> { + final long innerKey = freeIt.hasNext() ? freeIt.nextLong() : ++maxInnerIndex; + lastAllocated.setValue(innerKey); + rowRedirection.put(outerKey, innerKey); + }); + freeRows.removeRange(0, lastAllocated.longValue()); } } private boolean shiftRedirectedKey(@NotNull final RowSet.SearchIterator iterator, final long delta, final long key) { - synchronized (rowRedirection) { - final long inner = rowRedirection.remove(key); - if (inner != NULL_ROW_KEY) { - rowRedirection.put(key + delta, inner); - } - return !iterator.hasNext(); + final long inner = rowRedirection.remove(key); + if (inner != NULL_ROW_KEY) { + rowRedirection.put(key + delta, inner); } + return !iterator.hasNext(); } @Override @@ -291,46 +285,5 @@ public static Table updateBy(@NotNull final QueryTable source, } return ret; } - - /** - * The type of update to be applied. For use with invocations of - * {@link UpdateByOperator#initializeFor(UpdateByOperator.UpdateContext, RowSet, UpdateType)} and - * {@link UpdateByOperator#finishFor(UpdateByOperator.UpdateContext, UpdateType)} - */ - public enum UpdateType { - /** - * Indicates that rows are being - * {@link UpdateByOperator#addChunkBucketed(UpdateByOperator.UpdateContext, RowSequence, LongChunk, Chunk, long)} added} - * to the operator. - */ - Add, - - /** - * Indicates that rows are being - * {@link UpdateByOperator#removeChunk(UpdateByOperator.UpdateContext, LongChunk, Chunk, long) removed} from the - * operator. - */ - Remove, - - /** - * Indicates that rows are being - * {@link UpdateByOperator#modifyChunk(UpdateByOperator.UpdateContext, LongChunk, LongChunk, Chunk, Chunk, long) - * modified} within the operator - */ - Modify, - - /** - * Indicates that rows are being - * {@link UpdateByOperator#applyShift(UpdateByOperator.UpdateContext, RowSet, RowSetShiftData) shifted} within - * the operator. - */ - Shift, - - /** - * Indicates that the {@link TableUpdate} has been processed and rows are being revisited based upon the - * requests of individual operators for the purposes of doing recalculations on large portions of the table - */ - Reprocess - } // endregion } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java index 7e283e1a352..fb3f4cc5e85 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java @@ -1,24 +1,24 @@ package io.deephaven.engine.table.impl; import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.RowSetBuilderSequential; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.updateby.internal.BaseCharUpdateByOperator; +import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedByteUpdateByOperator; +import io.deephaven.tablelogger.Row; import org.jetbrains.annotations.NotNull; public abstract class UpdateByCumulativeOperator implements UpdateByOperator { public abstract class UpdateCumulativeContext implements UpdateContext { + protected RowSetBuilderSequential modifiedBuilder; + protected RowSet newModified; + // store the current subset of rows that need computation - protected RowSet affectedRows = RowSetFactory.empty(); + protected RowSet affectedRows; public RowSet determineAffectedRows(@NotNull final TableUpdate upstream, @NotNull final RowSet source, final boolean upstreamAppendOnly) { - if (upstreamAppendOnly) { - // cumulative operators do not need to reprocess any rows on append-only updates - try (final RowSet ignored = affectedRows) { - affectedRows = RowSetFactory.empty(); - } - return affectedRows; - } long smallestModifiedKey = UpdateByOperator.smallestAffectedKey(upstream.added(), upstream.modified(), upstream.removed(), upstream.shifted(), source); @@ -34,9 +34,59 @@ public RowSet getAffectedRows() { return affectedRows; } + public RowSetBuilderSequential getModifiedBuilder() { + if(modifiedBuilder == null) { + modifiedBuilder = RowSetFactory.builderSequential(); + } + return modifiedBuilder; + } + @Override public void close() { - affectedRows.close(); + try (final RowSet ignored = affectedRows; + final RowSet ignored2 = newModified) { + } } } + + @Override + public void initializeFor(@NotNull final UpdateContext context, + @NotNull final RowSet updateRowSet) { + } + + @Override + public void finishFor(@NotNull final UpdateContext context) { + UpdateCumulativeContext ctx = (UpdateCumulativeContext)context; + ctx.newModified = ctx.getModifiedBuilder().build(); + } + + @NotNull + final public RowSet getAdditionalModifications(@NotNull final UpdateContext context) { + UpdateCumulativeContext ctx = (UpdateCumulativeContext)context; + return ctx.newModified; + } + + @Override + final public boolean anyModified(@NotNull final UpdateContext context) { + UpdateCumulativeContext ctx = (UpdateCumulativeContext)context; + return ctx.newModified.isNonempty(); + } + + /*** nearly all cumulative operators will not reference a timestamp column, exceptions are Ema */ + @Override + public String getTimestampColumnName() { + return null; + } + + /*** cumulative operators do not need keys */ + @Override + public boolean requiresKeys() { + return false; + } + + /*** cumulative operators always need values */ + @Override + public boolean requiresValues(@NotNull final UpdateContext context) { + return true; + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java index 35dce04341d..606ca6f3449 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java @@ -4,17 +4,15 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.attributes.ChunkLengths; -import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetShiftData; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.updateby.internal.BaseCharUpdateByOperator; import io.deephaven.util.SafeCloseable; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -58,7 +56,7 @@ * *
  • Reprocess *
      - *
    • {@link #resetForReprocess(UpdateContext, RowSet, long)}
    • + *
    • {@link #resetForProcess(UpdateContext, RowSet, long)}
    • *
    • {@link #initializeFor(UpdateContext, RowSet, UpdateBy.UpdateType)}
    • *
    • {@link #reprocessChunkBucketed(UpdateContext, RowSequence, Chunk, LongChunk, IntChunk, IntChunk, IntChunk)}
    • *
    • {@link #finishFor(UpdateContext, UpdateBy.UpdateType)}
    • @@ -190,6 +188,14 @@ RowSet determineAffectedRows(@NotNull final TableUpdate upstream, @NotNull final @NotNull String getInputColumnName(); + /** + * Get the name of the timestamp column this operator depends on. + * + * @return the name of the input column + */ + @Nullable + String getTimestampColumnName(); + /** * Get an array of column names that, when modified, affect the result of this computation. * @@ -228,45 +234,27 @@ RowSet determineAffectedRows(@NotNull final TableUpdate upstream, @NotNull final @NotNull UpdateContext makeUpdateContext(final int chunkSize); - /** - * Initialize the operator for an update cycle. This is invoked before any other update processing occurs. - * - * @param context the context object - * @param upstream the upstream update to process - * @param resultSourceIndex the result index of the source table - * @param lastPrevKey the last key for this table before this update - * @param isUpstreamAppendOnly if the upstream update was detected to be append-only. - */ - void initializeForUpdate(@NotNull final UpdateContext context, - @NotNull final TableUpdate upstream, - @NotNull final RowSet resultSourceIndex, - final long lastPrevKey, - final boolean isUpstreamAppendOnly); - /** *

      * Initialize the context for the specified stage of the update process. This will always be followed by a call to - * {@link #finishFor(UpdateContext, UpdateBy.UpdateType)} at the end of each successful update. + * {@link #finishFor(UpdateContext)} at the end of each successful update. *

      * * @param context the context object * @param updateRowSet the index of rows associated with the update. - * @param type the type of update being applied */ void initializeFor(@NotNull final UpdateContext context, - @NotNull final RowSet updateRowSet, - @NotNull final UpdateBy.UpdateType type); + @NotNull final RowSet updateRowSet); /** * Perform and bookkeeping required at the end of a single part of the update. This is always preceded with a call * to * - * {@link #initializeFor(UpdateContext, RowSet, UpdateBy.UpdateType)} + * {@link #initializeFor(UpdateContext, RowSet)} * * @param context the context object - * @param type the type of update being applied */ - void finishFor(@NotNull final UpdateContext context, @NotNull final UpdateBy.UpdateType type); + void finishFor(@NotNull final UpdateContext context); /** * Get an index of rows that were modified beyond the input set of modifications from the upstream. This is invoked @@ -290,7 +278,7 @@ void initializeFor(@NotNull final UpdateContext context, /** * Query if the operator requires key values for the current stage. This method will always be invoked after an - * appropriate invocation of {@link #initializeFor(UpdateContext, RowSet, UpdateBy.UpdateType)} + * appropriate invocation of {@link #initializeFor(UpdateContext, RowSet)} * * @return true if the operator requires keys for this operation */ @@ -305,17 +293,7 @@ void initializeFor(@NotNull final UpdateContext context, boolean requiresValues(@NotNull final UpdateContext context); /** - * Query if this operator can process the update normally, or if it can only reprocess. This method is guaranteed to - * be invoked after {@link #initializeForUpdate(UpdateContext, TableUpdate, RowSet, long, boolean)} so the - * operator is aware of the upstream {@link TableUpdate}. - * - * @param context the context - * @return true if this operator can process the update normally. - */ - boolean canProcessNormalUpdate(@NotNull final UpdateContext context); - - /** - * Set the chunk size to be used for operations. This is used during the {@link UpdateBy.UpdateType#Reprocess} phase + * Set the chunk size to be used for operations. This is used during the processing phase * when the chunks allocated during the normal processing phase may not be large enough. * * @param context the context object @@ -323,56 +301,6 @@ void initializeFor(@NotNull final UpdateContext context, */ void setChunkSize(@NotNull final UpdateContext context, final int chunkSize); - /** - * Add a chunk of non-bucketed items to the operation. - * - * @param context the context object - * @param keyChunk a chunk of keys for the rows being added. If the operator returns {@code false} for - * {@link #requiresKeys()} this will be null. - * @param values the chunk of values for the rows being added - */ - void addChunk(@NotNull final UpdateContext context, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk values); - - /** - * Modify a chunk of values with the operation. - * - * @param context the context object - * @param prevKeyChunk a chunk of pre-shift keys. This will be equal to {@code keyChunk} if no shift is present - * @param keyChunk a chunk of post-shift space keys for the update. - * @param prevValuesChunk a chunk of previous values for the update - * @param postValuesChunk a chunk of current values for the update - */ - void modifyChunk(@NotNull final UpdateContext context, - @Nullable final LongChunk prevKeyChunk, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk, - @NotNull final Chunk postValuesChunk); - - /** - * Remove a chunk of values from the operation. - * - * @param context the context object - * @param keyChunk a chunk of keys being removed. - * @param prevValuesChunk the chunk of values being removed - */ - void removeChunk(@NotNull final UpdateContext context, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk); - - /** - * Apply a shift to the operation. - * - * @param context the context object - * @param prevIndex the pre-shifted index - * @param shifted the shifts being applied - */ - void applyShift(@NotNull final UpdateContext context, - @NotNull final RowSet prevIndex, - @NotNull final RowSetShiftData shifted); - /** * Apply a shift to the operation. * @@ -382,7 +310,7 @@ void applyOutputShift(@NotNull final UpdateContext context, final long delta); /** - * Reprocess a chunk of data for a non-bucketed table. + * Process a chunk of data for an updateBy table. * * @param context the context object * @param inputKeys the keys contained in the chunk @@ -390,15 +318,15 @@ void applyOutputShift(@NotNull final UpdateContext context, * @param valuesChunk the current chunk of working values. * @param postUpdateSourceIndex the resulting source index af */ - void reprocessChunk(@NotNull final UpdateContext context, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex); + void processChunk(@NotNull final UpdateContext context, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk valuesChunk, + @NotNull final RowSet postUpdateSourceIndex); /** * Reset the operator to the state at the `firstModifiedKey` for non-bucketed operation. This is invoked immediately - * prior to calls to {@link #resetForReprocess(UpdateContext, RowSet, long)}.
      + * prior to calls to {@link #resetForProcess(UpdateContext, RowSet, long)}.
      *
      * A `firstUnmodifiedKey` of {@link RowSet#NULL_ROW_KEY} indicates that the entire table needs to be recomputed. * @@ -406,7 +334,7 @@ void reprocessChunk(@NotNull final UpdateContext context, * @param sourceIndex the current index of the source table * @param firstUnmodifiedKey the first unmodified key after which we will reprocess rows. */ - void resetForReprocess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceIndex, - final long firstUnmodifiedKey); + void resetForProcess(@NotNull final UpdateContext context, + @NotNull final RowSet sourceIndex, + final long firstUnmodifiedKey); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java index 8d9e7685998..852316932b2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java @@ -250,28 +250,28 @@ private UpdateByOperator makeEmaOperator(@NotNull final MatchPair pair, final long timeScaleUnits = ema.timeScale().timescaleUnits(); if (csType == byte.class || csType == Byte.class) { - return new ByteEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, timeScaleUnits, + return new ByteEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, ema.timeScale().timestampCol(), timeScaleUnits, columnSource, redirContext); } else if (csType == short.class || csType == Short.class) { - return new ShortEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, timeScaleUnits, + return new ShortEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, ema.timeScale().timestampCol(), timeScaleUnits, columnSource, redirContext); } else if (csType == int.class || csType == Integer.class) { - return new IntEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, timeScaleUnits, + return new IntEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, ema.timeScale().timestampCol(), timeScaleUnits, columnSource, redirContext); } else if (csType == long.class || csType == Long.class) { - return new LongEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, timeScaleUnits, + return new LongEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, ema.timeScale().timestampCol(), timeScaleUnits, columnSource, redirContext); } else if (csType == float.class || csType == Float.class) { - return new FloatEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, timeScaleUnits, + return new FloatEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, ema.timeScale().timestampCol(), timeScaleUnits, columnSource, redirContext); } else if (csType == double.class || csType == Double.class) { - return new DoubleEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, + return new DoubleEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, ema.timeScale().timestampCol(), timeScaleUnits, columnSource, redirContext); } else if (csType == BigDecimal.class) { - return new BigDecimalEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, + return new BigDecimalEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, ema.timeScale().timestampCol(), timeScaleUnits, columnSource, redirContext); } else if (csType == BigInteger.class) { - return new BigIntegerEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, + return new BigIntegerEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, ema.timeScale().timestampCol(), timeScaleUnits, columnSource, redirContext); } @@ -391,7 +391,7 @@ private UpdateByOperator makeForwardFillOperator(MatchPair fc, TableDefaults sou } private UpdateByOperator makeRollingSumOperator(@NotNull final MatchPair pair, - @NotNull final TableWithDefaults source, + @NotNull final TableDefaults source, @Nullable final LongRecordingUpdateByOperator recorder, @NotNull final RollingSumSpec rs) { // noinspection rawtypes @@ -409,25 +409,25 @@ private UpdateByOperator makeRollingSumOperator(@NotNull final MatchPair pair, final long fwdTimeScaleUnits = rs.fwdTimeScale().timescaleUnits(); if (csType == Boolean.class || csType == boolean.class) { - return new ByteRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, prevTimeScaleUnits, fwdTimeScaleUnits, + return new ByteRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, ReinterpretUtils.booleanToByteSource(columnSource), redirContext, NULL_BOOLEAN_AS_BYTE); } else if (csType == byte.class || csType == Byte.class) { - return new ByteRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, prevTimeScaleUnits, fwdTimeScaleUnits, + return new ByteRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, columnSource, redirContext, NULL_BYTE); } else if (csType == short.class || csType == Short.class) { - return new ShortRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, prevTimeScaleUnits, fwdTimeScaleUnits, + return new ShortRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, columnSource, redirContext); } else if (csType == int.class || csType == Integer.class) { - return new IntRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, prevTimeScaleUnits, fwdTimeScaleUnits, + return new IntRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, columnSource, redirContext); } else if (csType == long.class || csType == Long.class) { - return new LongRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, prevTimeScaleUnits, fwdTimeScaleUnits, + return new LongRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, columnSource, redirContext); } else if (csType == float.class || csType == Float.class) { - return new FloatRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, prevTimeScaleUnits, fwdTimeScaleUnits, + return new FloatRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, columnSource, redirContext); } else if (csType == double.class || csType == Double.class) { - return new DoubleRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, + return new DoubleRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, columnSource, redirContext); // } else if (csType == BigDecimal.class) { // return new BigDecimalRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java index c266380c544..0aee493f745 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java @@ -2,7 +2,6 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.base.LongRingBuffer; -import io.deephaven.chunk.WritableCharChunk; import io.deephaven.chunk.WritableLongChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.*; @@ -10,17 +9,15 @@ import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedCharUpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.util.Arrays; -import java.util.List; - public abstract class UpdateByWindowedOperator implements UpdateByOperator { protected final OperationControl control; protected final LongRecordingUpdateByOperator recorder; + protected final String timestampColumnName; protected final long reverseTimeScaleUnits; protected final long forwardTimeScaleUnits; @@ -30,6 +27,16 @@ public abstract class UpdateByWindowedOperator implements UpdateByOperator { protected UpdateBy.UpdateByRedirectionContext redirContext; public abstract class UpdateWindowedContext implements UpdateContext { + public RowSetBuilderSequential modifiedBuilder; + public RowSet newModified; + + public RowSetBuilderSequential getModifiedBuilder() { + if(modifiedBuilder == null) { + modifiedBuilder = RowSetFactory.builderSequential(); + } + return modifiedBuilder; + } + // store the current subset of rows that need computation protected RowSet affectedRows = RowSetFactory.empty(); @@ -230,30 +237,14 @@ public void fillWindowTicks(UpdateContext context, long currentPos) { @Override public void close() { - if (windowIterator != null) { - windowIterator.close(); - windowIterator = null; - } - - if (candidateRowKeysChunk != null) { - candidateRowKeysChunk.close(); - candidateRowKeysChunk = null; - } - - if (candidatePositionsChunk != null) { - candidatePositionsChunk.close(); - candidatePositionsChunk = null; + try (final RowSequence.Iterator ignored1 = windowIterator; + final WritableLongChunk ignored2 = candidateRowKeysChunk; + final WritableLongChunk ignored3 = candidatePositionsChunk; + final WritableLongChunk ignored4 = candidateTimestampsChunk; + final WritableLongChunk ignored5 = valuePositionChunk; + final RowSet ignored6 = affectedRows; + final RowSet ignored7 = newModified) { } - - if (valuePositionChunk != null) { - valuePositionChunk.close(); - valuePositionChunk = null; - } - - // no need to close this, just release the reference - workingRowSet = null; - - affectedRows.close(); } } @@ -271,6 +262,7 @@ public UpdateByWindowedOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final LongRecordingUpdateByOperator timeRecorder, + @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { @@ -278,6 +270,7 @@ public UpdateByWindowedOperator(@NotNull final MatchPair pair, this.affectingColumns = affectingColumns; this.control = control; this.recorder = timeRecorder; + this.timestampColumnName = timestampColumnName; this.reverseTimeScaleUnits = reverseTimeScaleUnits; this.forwardTimeScaleUnits = forwardTimeScaleUnits; this.redirContext = redirContext; @@ -287,7 +280,6 @@ public UpdateByWindowedOperator(@NotNull final MatchPair pair, public abstract void pop(UpdateContext context, long key); public abstract void reset(UpdateContext context); - // return the first row that affects this key public long computeFirstAffectingKey(long key, @NotNull final RowSet source) { @@ -353,6 +345,32 @@ public long computeLastAffectedKey(long key, @NotNull final RowSet source) { return -1; } + @Override + public void initializeFor(@NotNull final UpdateContext context, + @NotNull final RowSet updateRowSet) { + final UpdateWindowedContext ctx = (UpdateWindowedContext) context; + long windowStartKey = computeFirstAffectingKey(updateRowSet.firstRowKey(), ctx.workingRowSet); + ctx.loadWindowChunks(windowStartKey); + } + + @Override + public void finishFor(@NotNull final UpdateContext context) { + UpdateWindowedContext ctx = (UpdateWindowedContext)context; + ctx.newModified = ctx.getModifiedBuilder().build(); + } + + @NotNull + final public RowSet getAdditionalModifications(@NotNull final UpdateContext context) { + UpdateWindowedContext ctx = (UpdateWindowedContext)context; + return ctx.newModified; + } + + @Override + final public boolean anyModified(@NotNull final UpdateContext context) { + UpdateWindowedContext ctx = (UpdateWindowedContext)context; + return ctx.newModified.isNonempty(); + } + @NotNull @Override public String getInputColumnName() { @@ -371,8 +389,20 @@ public String[] getOutputColumnNames() { return new String[] { pair.leftColumn }; } + @Override + public String getTimestampColumnName() { + return this.timestampColumnName; + } + @Override public boolean requiresKeys() { return true; } + + @Override + public boolean requiresValues(@NotNull final UpdateContext context) { + // windowed operators don't need current values supplied to them, they only care about windowed values which + // may or may not intersect with the column values + return false; + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java index e0b9866247a..f62dc51db28 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java @@ -8,9 +8,9 @@ import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.*; +import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; import io.deephaven.engine.table.impl.util.UpdateSizeCalculator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; import io.deephaven.util.SafeCloseable; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -27,6 +27,10 @@ class ZeroKeyUpdateBy extends UpdateBy { /** Apply shifts to operator outputs? */ final boolean applyShifts; + /** store timestamp data in an Ssa (if needed) */ + final LongSegmentedSortedArray tsSsa; + final String tsColumnName; + /** * Perform an updateBy without any key columns. * @@ -64,6 +68,18 @@ protected ZeroKeyUpdateBy(@NotNull final UpdateByOperator[] operators, @NotNull final UpdateByControl control, final boolean applyShifts) { super(operators, source, redirContext, control); + + // do we need a timestamp SSA? + this.tsColumnName = Arrays.stream(operators) + .filter(op -> op.getTimestampColumnName() != null) + .map(UpdateByOperator::getTimestampColumnName) + .findFirst().orElse(null); + + if (tsColumnName != null) { + this.tsSsa = new LongSegmentedSortedArray(1024); + } else { + this.tsSsa = null; + } this.applyShifts = applyShifts; } @@ -81,12 +97,11 @@ void doInitialAdditions() { redirContext.processUpdateForRedirection(fakeUpdate, source.getRowSet()); } + // add all the SSA data + try (final UpdateContext ctx = new UpdateContext(fakeUpdate, null, true)) { ctx.setAllAffected(); - // do an addition phase for all the operators that can add directly (i.e. backwards looking) - ctx.doUpdate(source.getRowSet(), source.getRowSet(), UpdateType.Add); - // do a reprocessing phase for operators that can't add directly ctx.reprocessRows(RowSetShiftData.EMPTY); } @@ -201,9 +216,6 @@ private class UpdateContext implements SafeCloseable { inputSources[opIdx].getChunkType().makeWritableChunk(chunkSize); } } - - operators[opIdx].initializeForUpdate(opContext[opIdx], upstream, source.getRowSet(), - isInitializeStep ? NULL_ROW_KEY : source.getRowSet().lastRowKeyPrev(), upstreamAppendOnly); } // retrieve the affected rows from all operator update contexts @@ -260,11 +272,10 @@ void setChunkSize(int newChunkSize) { } } - void initializeFor(@NotNull final RowSet updateRowSet, - @NotNull final UpdateType type) { + void initializeFor(@NotNull final RowSet updateRowSet) { for (int opIdx = 0; opIdx < operators.length; opIdx++) { if (opAffected[opIdx]) { - operators[opIdx].initializeFor(opContext[opIdx], updateRowSet, type); + operators[opIdx].initializeFor(opContext[opIdx], updateRowSet); anyRequireKeys |= operators[opIdx].requiresKeys(); } } @@ -274,10 +285,10 @@ void initializeFor(@NotNull final RowSet updateRowSet, } } - void finishFor(@NotNull final UpdateType type) { + void finishFor() { for (int opIdx = 0; opIdx < operators.length; opIdx++) { if (opAffected[opIdx]) { - operators[opIdx].finishFor(opContext[opIdx], type); + operators[opIdx].finishFor(opContext[opIdx]); } } @@ -344,8 +355,7 @@ boolean anyModified() { } void doUpdate(@NotNull final RowSet updateRowSet, - @NotNull final RowSet preShiftUpdateRowSet, - @NotNull final UpdateType type) { + @NotNull final RowSet preShiftUpdateRowSet) { if (updateRowSet.isEmpty()) { return; } @@ -353,7 +363,7 @@ void doUpdate(@NotNull final RowSet updateRowSet, try (final RowSequence.Iterator okIt = updateRowSet.getRowSequenceIterator(); final RowSequence.Iterator preShiftOkIt = preShiftUpdateRowSet == updateRowSet ? null : preShiftUpdateRowSet.getRowSequenceIterator()) { - initializeFor(updateRowSet, type); + initializeFor(updateRowSet); while (okIt.hasMore()) { sharedContext.reset(); @@ -380,50 +390,27 @@ void doUpdate(@NotNull final RowSet updateRowSet, final UpdateByOperator currentOp = operators[opIdx]; final int slotPosition = inputSourceSlots[opIdx]; - if (type == UpdateType.Add) { - prepareValuesChunkFor(opIdx, slotPosition, false, true, chunkOk, prevChunkOk, - null, postWorkingChunks[slotPosition].get(), - null, fillContexts[slotPosition].get()); - currentOp.addChunk(opContext[opIdx], chunkOk, keyChunk.get(), - postWorkingChunks[slotPosition].get()); - } else if (type == UpdateType.Remove) { - prepareValuesChunkFor(opIdx, slotPosition, true, false, chunkOk, prevChunkOk, - postWorkingChunks[slotPosition].get(), null, - fillContexts[slotPosition].get(), null); - currentOp.removeChunk(opContext[opIdx], keyChunk.get(), - postWorkingChunks[slotPosition].get()); - } else if (type == UpdateType.Modify) { - prepareValuesChunkFor(opIdx, slotPosition, true, true, chunkOk, prevChunkOk, - prevWorkingChunks[slotPosition], postWorkingChunks[slotPosition].get(), - prevFillContexts[slotPosition], fillContexts[slotPosition].get()); - currentOp.modifyChunk(opContext[opIdx], - prevKeyChunk == null ? keyChunk.get() : prevKeyChunk, - keyChunk.get(), - prevWorkingChunks[slotPosition], - postWorkingChunks[slotPosition].get()); - } else if (type == UpdateType.Reprocess) { - // is this chunk relevant to this operator? If so, then intersect and process only the - // relevant rows - if (chunkOk.firstRowKey() <= opContext[opIdx].getAffectedRows().lastRowKey() - && chunkOk.lastRowKey() >= opContext[opIdx].getAffectedRows().firstRowKey()) { - try (final RowSet rs = chunkOk.asRowSet(); - final RowSet intersect = rs.intersect(opContext[opIdx].getAffectedRows())) { - - prepareValuesChunkFor(opIdx, slotPosition, false, true, intersect, intersect, - null, postWorkingChunks[slotPosition].get(), - null, fillContexts[slotPosition].get()); - currentOp.reprocessChunk(opContext[opIdx], - intersect, - keyChunk.get(), - postWorkingChunks[slotPosition].get(), - source.getRowSet()); - } + // is this chunk relevant to this operator? If so, then intersect and process only the + // relevant rows + if (chunkOk.firstRowKey() <= opContext[opIdx].getAffectedRows().lastRowKey() + && chunkOk.lastRowKey() >= opContext[opIdx].getAffectedRows().firstRowKey()) { + try (final RowSet rs = chunkOk.asRowSet(); + final RowSet intersect = rs.intersect(opContext[opIdx].getAffectedRows())) { + + prepareValuesChunkFor(opIdx, slotPosition, false, true, intersect, intersect, + null, postWorkingChunks[slotPosition].get(), + null, fillContexts[slotPosition].get()); + currentOp.processChunk(opContext[opIdx], + intersect, + keyChunk.get(), + postWorkingChunks[slotPosition].get(), + source.getRowSet()); } } } } - finishFor(type); + finishFor(); } } @@ -446,14 +433,14 @@ private void reprocessRows(RowSetShiftData shifted) { keyBefore = sit.binarySearchValue( (compareTo, ignored) -> Long.compare(keyStart - 1, compareTo), 1); } - // apply a shift to keyBefore since the output column is still in prev key space + // apply a shift to keyBefore since the output column is still in prev key space- - operators[opIndex].resetForReprocess(opContext[opIndex], sourceRowSet, keyBefore); + operators[opIndex].resetForProcess(opContext[opIndex], sourceRowSet, keyBefore); } } // Now iterate rowset to reprocess. - doUpdate(affectedRows, affectedRows, UpdateType.Reprocess); + doUpdate(affectedRows, affectedRows); } /** @@ -491,15 +478,6 @@ private void prepareValuesChunkFor(final int opIdx, } } } - - public boolean canAnyProcessNormally() { - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx] && operators[opIdx].canProcessNormalUpdate(opContext[opIdx])) { - return true; - } - } - return false; - } } /** @@ -533,44 +511,24 @@ public ZeroKeyUpdateByListener(@Nullable String description, public void onUpdate(TableUpdate upstream) { try (final UpdateContext ctx = new UpdateContext(upstream, inputModifiedColumnSets, false)) { - if (redirContext.isRedirected()) { - redirContext.processUpdateForRedirection(upstream, source.getRowSet()); - } - - // We will not mess with shifts if we are using a redirection because we'll have applied the shift - // to the redirection index already by now. - if (applyShifts && !redirContext.isRedirected() && upstream.shifted().nonempty()) { - try (final RowSet prevIdx = source.getRowSet().copyPrev()) { - upstream.shifted().apply((begin, end, delta) -> { - try (final RowSet subRowSet = prevIdx.subSetByKeyRange(begin, end)) { - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - operators[opIdx].applyOutputShift(ctx.opContext[opIdx], subRowSet, delta); - } - } - }); - } - } - - // If anything can process normal operations we have to pass them down, otherwise we can skip this - // entirely. - if (ctx.canAnyProcessNormally()) { - ctx.doUpdate(upstream.removed(), upstream.removed(), UpdateType.Remove); - if (upstream.shifted().nonempty()) { - try (final WritableRowSet prevRowSet = source.getRowSet().copyPrev(); - final RowSet modPreShift = upstream.getModifiedPreShift()) { - - prevRowSet.remove(upstream.removed()); - for (int ii = 0; ii < operators.length; ii++) { - operators[ii].initializeFor(ctx.opContext[ii], prevRowSet, UpdateType.Shift); - operators[ii].applyShift(ctx.opContext[ii], prevRowSet, upstream.shifted()); - operators[ii].finishFor(ctx.opContext[ii], UpdateType.Shift); + if (applyShifts) { + if (redirContext.isRedirected()) { + redirContext.processUpdateForRedirection(upstream, source.getRowSet()); + } else { + // We will not mess with shifts if we are using a redirection because we'll have applied the shift + // to the redirection index already by now. + if (upstream.shifted().nonempty()) { + try (final RowSet prevIdx = source.getRowSet().copyPrev()) { + upstream.shifted().apply((begin, end, delta) -> { + try (final RowSet subRowSet = prevIdx.subSetByKeyRange(begin, end)) { + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + operators[opIdx].applyOutputShift(ctx.opContext[opIdx], subRowSet, delta); + } + } + }); } - ctx.doUpdate(upstream.modified(), modPreShift, UpdateType.Modify); } - } else { - ctx.doUpdate(upstream.modified(), upstream.modified(), UpdateType.Modify); } - ctx.doUpdate(upstream.added(), upstream.added(), UpdateType.Add); } // Now do the reprocessing phase. @@ -582,10 +540,10 @@ public void onUpdate(TableUpdate upstream) { downstream.removed = upstream.removed().copy(); downstream.shifted = upstream.shifted(); - if (upstream.modified().isNonempty() || ctx.anyModified()) { - downstream.modifiedColumnSet = result.getModifiedColumnSetForUpdates(); - downstream.modifiedColumnSet.clear(); + downstream.modifiedColumnSet = result.getModifiedColumnSetForUpdates(); + downstream.modifiedColumnSet.clear(); + if (upstream.modified().isNonempty() || ctx.anyModified()) { WritableRowSet modifiedRowSet = RowSetFactory.empty(); downstream.modified = modifiedRowSet; if (upstream.modified().isNonempty()) { @@ -596,7 +554,6 @@ public void onUpdate(TableUpdate upstream) { for (int opIdx = 0; opIdx < operators.length; opIdx++) { if (ctx.opAffected[opIdx]) { - downstream.modifiedColumnSet.setAll(outputModifiedColumnSets[opIdx]); if (operators[opIdx].anyModified(ctx.opContext[opIdx])) { modifiedRowSet .insert(operators[opIdx].getAdditionalModifications(ctx.opContext[opIdx])); @@ -609,7 +566,13 @@ public void onUpdate(TableUpdate upstream) { } } else { downstream.modified = RowSetFactory.empty(); - downstream.modifiedColumnSet = ModifiedColumnSet.EMPTY; + } + + // set the modified columns if any operators made changes (add/rem/modify) + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (ctx.opAffected[opIdx]) { + downstream.modifiedColumnSet.setAll(outputModifiedColumnSets[opIdx]); + } } result.notifyListeners(downstream); 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 892f1f5fd97..f7e5564cd1a 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 @@ -17,11 +17,7 @@ import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.liveness.LivenessReferent; import io.deephaven.engine.liveness.LivenessScopeStack; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.rowset.RowSetFactory; -import io.deephaven.engine.rowset.RowSetShiftData; -import io.deephaven.engine.rowset.WritableRowSet; +import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.rowset.impl.AdaptiveOrderedLongSetBuilderRandom; @@ -58,11 +54,17 @@ public final class PartitionByChunkedOperator implements IterativeChunkedAggrega // region nonexistent table sentinels /** - * Sentinel value for the row set belonging to a table that was never created because either the result table was no - * longer live or the aggregation update listener was no longer live. Should be used for assignment and reference - * equality tests, only. + * Sentinel value for the row set builder belonging to a table that was never created because either the result + * table was no longer live or the aggregation update listener was no longer live. Should be used for assignment and + * reference equality tests, only. + */ + private static final RowSetBuilderRandom NONEXISTENT_TABLE_ROW_SET_BUILDER = RowSetFactory.builderRandom(); + + /** + * Helper value for the following sentinel values but not used directly for comparison */ private static final WritableRowSet NONEXISTENT_TABLE_ROW_SET = RowSetFactory.empty(); + /** * Sentinel value for the shift builder belonging to a table that was never created because either the result table * was no longer live or the aggregation update listener was no longer live. Should be used for assignment and @@ -90,9 +92,10 @@ public interface AttributeCopier { private final String callSite; private final ObjectArraySource tables; - private final ObjectArraySource addedRowSets; - private final ObjectArraySource removedRowSets; - private final ObjectArraySource modifiedRowSets; + private final ObjectArraySource addedRowSetBuilders; + private final ObjectArraySource removedRowSetBuilders; + private final ObjectArraySource modifiedRowSetBuilders; + private final ObjectArraySource shiftDataBuilders; private final ModifiedColumnSet resultModifiedColumnSet; private final ModifiedColumnSet.Transformer upstreamToResultTransformer; @@ -133,17 +136,17 @@ public interface AttributeCopier { * @param keyColumnNames The key columns */ PartitionByChunkedOperator(@NotNull final QueryTable unadjustedParentTable, - @NotNull final QueryTable parentTable, - @NotNull final String resultName, - @NotNull final AttributeCopier attributeCopier, - @NotNull final String... keyColumnNames) { + @NotNull final QueryTable parentTable, + @NotNull final String resultName, + @NotNull final AttributeCopier attributeCopier, + @NotNull final String... keyColumnNames) { this.parentTable = parentTable; this.resultName = resultName; callSite = QueryPerformanceRecorder.getCallerLine(); tables = new ObjectArraySource<>(QueryTable.class); - addedRowSets = new ObjectArraySource<>(WritableRowSet.class); + addedRowSetBuilders = new ObjectArraySource<>(RowSetBuilderRandom.class); // Note: Sub-tables always share their ColumnSource map with the parent table, so they can all use this result // MCS. @@ -158,8 +161,8 @@ public interface AttributeCopier { } if (parentTable.isRefreshing()) { - removedRowSets = new ObjectArraySource<>(WritableRowSet.class); - modifiedRowSets = new ObjectArraySource<>(WritableRowSet.class); + removedRowSetBuilders = new ObjectArraySource<>(RowSetBuilderRandom.class); + modifiedRowSetBuilders = new ObjectArraySource<>(RowSetBuilderRandom.class); shiftDataBuilders = new ObjectArraySource<>(RowSetShiftData.SmartCoalescingBuilder.class); final Set keyColumnNameSet = Arrays.stream(keyColumnNames).collect(Collectors.toSet()); @@ -177,8 +180,8 @@ public interface AttributeCopier { retainedResultColumnNames, retainedResultModifiedColumnSets); } else { - removedRowSets = null; - modifiedRowSets = null; + removedRowSetBuilders = null; + modifiedRowSetBuilders = null; shiftDataBuilders = null; upstreamToResultTransformer = null; } @@ -186,9 +189,9 @@ public interface AttributeCopier { @Override public void addChunk(final BucketedContext bucketedContext, final Chunk values, - @NotNull final LongChunk inputRowKeys, - @NotNull final IntChunk destinations, @NotNull final IntChunk startPositions, - @NotNull final IntChunk length, @NotNull final WritableBooleanChunk stateModified) { + @NotNull final LongChunk inputRowKeys, + @NotNull final IntChunk destinations, @NotNull final IntChunk startPositions, + @NotNull final IntChunk length, @NotNull final WritableBooleanChunk stateModified) { Assert.eqNull(values, "values"); final AdaptiveOrderedLongSetBuilderRandom chunkDestinationsBuilder = stepUpdatedDestinations == null ? null : new AdaptiveOrderedLongSetBuilderRandom(); @@ -199,14 +202,14 @@ public void addChunk(final BucketedContext bucketedContext, final Chunk values, - @NotNull final LongChunk inputRowKeys, - @NotNull final IntChunk destinations, @NotNull final IntChunk startPositions, - @NotNull final IntChunk length, @NotNull final WritableBooleanChunk stateModified) { + @NotNull final LongChunk inputRowKeys, + @NotNull final IntChunk destinations, @NotNull final IntChunk startPositions, + @NotNull final IntChunk length, @NotNull final WritableBooleanChunk stateModified) { Assert.eqNull(values, "values"); Assert.neqNull(stepUpdatedDestinations, "stepUpdatedDestinations"); final AdaptiveOrderedLongSetBuilderRandom chunkDestinationsBuilder = new AdaptiveOrderedLongSetBuilderRandom(); @@ -227,32 +230,32 @@ public void removeChunk(final BucketedContext bucketedContext, final Chunk previousValues, - final Chunk newValues, - @NotNull final LongChunk postShiftRowKeys, - @NotNull final IntChunk destinations, @NotNull final IntChunk startPositions, - @NotNull final IntChunk length, @NotNull final WritableBooleanChunk stateModified) { + final Chunk newValues, + @NotNull final LongChunk postShiftRowKeys, + @NotNull final IntChunk destinations, @NotNull final IntChunk startPositions, + @NotNull final IntChunk length, @NotNull final WritableBooleanChunk stateModified) { // We have no inputs, so we should never get here. throw new IllegalStateException(); } @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) { + 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) { Assert.eqNull(previousValues, "previousValues"); Assert.eqNull(newValues, "newValues"); Assert.neqNull(stepUpdatedDestinations, "stepUpdatedDestinations"); @@ -268,16 +271,16 @@ public void shiftChunk(final BucketedContext bucketedContext, final Chunk inputRowKeys, - @NotNull final IntChunk destinations, @NotNull final IntChunk startPositions, - @NotNull final IntChunk length, @NotNull final WritableBooleanChunk stateModified) { + @NotNull final LongChunk inputRowKeys, + @NotNull final IntChunk destinations, @NotNull final IntChunk startPositions, + @NotNull final IntChunk length, @NotNull final WritableBooleanChunk stateModified) { if (!stepValuesModified) { return; } @@ -290,23 +293,23 @@ public void modifyRowKeys(final BucketedContext context, final int startPosition = startPositions.get(di); final int runLength = length.get(di); final long destination = destinations.get(startPosition); - accumulateToRowSet(modifiedRowSets, inputRowKeysAsOrdered, startPosition, runLength, destination); + accumulateToRowSet(modifiedRowSetBuilders, inputRowKeysAsOrdered, startPosition, runLength, destination); chunkDestinationsBuilder.addKey(destination); } try (final RowSet chunkUpdatedDestinations = - new WritableRowSetImpl(chunkDestinationsBuilder.getOrderedLongSet())) { + new WritableRowSetImpl(chunkDestinationsBuilder.getOrderedLongSet())) { stepUpdatedDestinations.insert(chunkUpdatedDestinations); } } @Override public boolean addChunk(final SingletonContext singletonContext, final int chunkSize, - final Chunk values, - @NotNull final LongChunk inputRowKeys, final long destination) { + final Chunk values, + @NotNull final LongChunk inputRowKeys, final long destination) { Assert.eqNull(values, "values"); // noinspection unchecked final LongChunk inputRowKeysAsOrdered = (LongChunk) inputRowKeys; - accumulateToRowSet(addedRowSets, inputRowKeysAsOrdered, 0, chunkSize, destination); + accumulateToRowSet(addedRowSetBuilders, inputRowKeysAsOrdered, 0, chunkSize, destination); if (stepUpdatedDestinations != null) { stepUpdatedDestinations.insert(destination); } @@ -315,7 +318,7 @@ public boolean addChunk(final SingletonContext singletonContext, final int chunk @Override public boolean addRowSet(SingletonContext context, RowSet rowSet, long destination) { - accumulateToRowSet(addedRowSets, rowSet, destination); + accumulateToRowSet(addedRowSetBuilders, rowSet, destination); if (stepUpdatedDestinations != null) { stepUpdatedDestinations.insert(destination); } @@ -324,32 +327,32 @@ public boolean addRowSet(SingletonContext context, RowSet rowSet, long destinati @Override public boolean removeChunk(final SingletonContext singletonContext, final int chunkSize, - final Chunk values, - @NotNull final LongChunk inputRowKeys, final long destination) { + final Chunk values, + @NotNull final LongChunk inputRowKeys, final long destination) { Assert.eqNull(values, "values"); Assert.neqNull(stepUpdatedDestinations, "stepUpdatedDestinations"); // noinspection unchecked final LongChunk inputRowKeysAsOrdered = (LongChunk) inputRowKeys; - accumulateToRowSet(removedRowSets, inputRowKeysAsOrdered, 0, chunkSize, destination); + accumulateToRowSet(removedRowSetBuilders, inputRowKeysAsOrdered, 0, chunkSize, destination); stepUpdatedDestinations.insert(destination); return false; } @Override public boolean modifyChunk(final SingletonContext singletonContext, final int chunkSize, - final Chunk previousValues, final Chunk newValues, - @NotNull final LongChunk postShiftRowKeys, - final long destination) { + final Chunk previousValues, final Chunk newValues, + @NotNull final LongChunk postShiftRowKeys, + final long destination) { // We have no inputs, so we should never get here. throw new IllegalStateException(); } @Override public boolean shiftChunk(final SingletonContext singletonContext, final Chunk previousValues, - final Chunk newValues, - @NotNull final LongChunk preShiftRowKeys, - @NotNull final LongChunk postShiftRowKeys, - final long destination) { + final Chunk newValues, + @NotNull final LongChunk preShiftRowKeys, + @NotNull final LongChunk postShiftRowKeys, + final long destination) { Assert.eqNull(previousValues, "previousValues"); Assert.eqNull(newValues, "newValues"); Assert.neqNull(stepUpdatedDestinations, "stepUpdatedDestinations"); @@ -361,62 +364,76 @@ public boolean shiftChunk(final SingletonContext singletonContext, final Chunk rowKeys, - final long destination) { + final long destination) { if (!stepValuesModified) { return false; } // noinspection unchecked final LongChunk rowKeysAsOrdered = (LongChunk) rowKeys; - accumulateToRowSet(modifiedRowSets, rowKeysAsOrdered, 0, rowKeys.size(), destination); + accumulateToRowSet(modifiedRowSetBuilders, rowKeysAsOrdered, 0, rowKeys.size(), destination); stepUpdatedDestinations.insert(destination); return false; } - private static void accumulateToRowSet(@NotNull final ObjectArraySource rowSetColumn, - @NotNull final LongChunk rowKeysToAdd, - final int start, final int length, final long destination) { - final WritableRowSet rowSet = rowSetColumn.getUnsafe(destination); - if (rowSet == NONEXISTENT_TABLE_ROW_SET) { + private static void accumulateToRowSet(@NotNull final ObjectArraySource rowSetColumn, + @NotNull final LongChunk rowKeysToAdd, + final int start, final int length, final long destination) { + final RowSetBuilderRandom builder = rowSetColumn.getUnsafe(destination); + if (builder == NONEXISTENT_TABLE_ROW_SET_BUILDER) { return; } - if (rowSet == null) { - rowSetColumn.set(destination, - new WritableRowSetImpl(OrderedLongSet.fromChunk(rowKeysToAdd, start, length, false))); + // slice the chunk to the start and length + LongChunk sliced = rowKeysToAdd.slice(start, length); + if (builder == null) { + // create (and store) a new builder, fill with these keys + final RowSetBuilderRandom newBuilder = RowSetFactory.builderRandom(); + newBuilder.addRowKeysChunk(sliced); + rowSetColumn.set(destination, newBuilder); return; } - rowSet.insert(rowKeysToAdd, start, length); + // add the keys to the stored builder + builder.addRowKeysChunk(sliced); } - private static void accumulateToRowSet(@NotNull final ObjectArraySource rowSetColumn, - @NotNull final RowSet rowSetToAdd, final long destination) { - final WritableRowSet rowSet = rowSetColumn.getUnsafe(destination); - if (rowSet == NONEXISTENT_TABLE_ROW_SET) { + private static void accumulateToRowSet(@NotNull final ObjectArraySource rowSetColumn, + @NotNull final RowSet rowSetToAdd, final long destination) { + final RowSetBuilderRandom builder = rowSetColumn.getUnsafe(destination); + if (builder == NONEXISTENT_TABLE_ROW_SET_BUILDER) { return; } - if (rowSet == null) { - final WritableRowSet currentRowSet = RowSetFactory.empty(); - currentRowSet.insert(rowSetToAdd); - rowSetColumn.set(destination, currentRowSet); + if (builder == null) { + // create (and store) a new builder, fill with this rowset + final RowSetBuilderRandom newBuilder = RowSetFactory.builderRandom(); + newBuilder.addRowSet(rowSetToAdd); + rowSetColumn.set(destination, newBuilder); return; } - rowSet.insert(rowSetToAdd); + // add the rowset to the stored builder + builder.addRowSet(rowSetToAdd); } private boolean appendShifts(@NotNull final LongChunk preShiftRowKeys, - @NotNull final LongChunk postShiftRowKeys, - final int startPosition, final int runLength, final long destination) { + @NotNull final LongChunk postShiftRowKeys, + final int startPosition, final int runLength, final long destination) { RowSetShiftData.SmartCoalescingBuilder builder = shiftDataBuilders.getUnsafe(destination); if (builder == NONEXISTENT_TABLE_SHIFT_BUILDER) { return false; } if (builder == null) { final RowSet tableRowSet = tables.getUnsafe(destination).getRowSet(); - final RowSet removedRowSet = removedRowSets.getUnsafe(destination); + final RowSetBuilderRandom removedRowSetBuilder = removedRowSetBuilders.getUnsafe(destination); final RowSet preShiftKeys; - if (removedRowSet == null) { + if (removedRowSetBuilder == null) { preShiftKeys = tableRowSet.copy(); } else { - preShiftKeys = tableRowSet.minus(removedRowSet); + // we want to keep the builder in `removedRowSetBuilders` ready for a call to `extractAndClearRowSet` so + // we remake the builder after this call + try (final RowSet rs = removedRowSetBuilder.build()) { + preShiftKeys = tableRowSet.minus(rs); + RowSetBuilderRandom newBuilder = RowSetFactory.builderRandom(); + newBuilder.addRowSet(rs); + removedRowSetBuilders.set(destination, newBuilder); + } } shiftDataBuilders.set(destination, builder = new RowSetShiftData.SmartCoalescingBuilder(preShiftKeys)); } @@ -446,10 +463,10 @@ private boolean appendShifts(@NotNull final LongChunk preShif @Override public void ensureCapacity(final long tableSize) { tables.ensureCapacity(tableSize); - addedRowSets.ensureCapacity(tableSize); + addedRowSetBuilders.ensureCapacity(tableSize); if (parentTable.isRefreshing()) { - removedRowSets.ensureCapacity(tableSize); - modifiedRowSets.ensureCapacity(tableSize); + removedRowSetBuilders.ensureCapacity(tableSize); + modifiedRowSetBuilders.ensureCapacity(tableSize); shiftDataBuilders.ensureCapacity(tableSize); } } @@ -470,24 +487,25 @@ public void propagateInitialState(@NotNull final QueryTable resultTable) { // initialization scope, and don't need to do anything special to ensure liveness. final boolean setCallSite = QueryPerformanceRecorder.setCallsite(callSite); try (final ResettableWritableObjectChunk tablesResettableChunk = - ResettableWritableObjectChunk.makeResettableChunk(); - final ResettableWritableObjectChunk addedRowSetsResettableChunk = - ResettableWritableObjectChunk.makeResettableChunk(); - final RowSequence.Iterator initialDestinationsIterator = - initialDestinations.getRowSequenceIterator()) { + ResettableWritableObjectChunk.makeResettableChunk(); + final ResettableWritableObjectChunk addedRowSetsResettableChunk = + ResettableWritableObjectChunk.makeResettableChunk(); + final RowSequence.Iterator initialDestinationsIterator = + initialDestinations.getRowSequenceIterator()) { // noinspection unchecked final WritableObjectChunk tablesBackingChunk = tablesResettableChunk.asWritableObjectChunk(); // noinspection unchecked - final WritableObjectChunk addedRowSetsBackingChunk = + final WritableObjectChunk addedRowSetsBackingChunk = addedRowSetsResettableChunk.asWritableObjectChunk(); while (initialDestinationsIterator.hasMore()) { final long firstSliceDestination = initialDestinationsIterator.peekNextKey(); final long firstBackingChunkDestination = tables.resetWritableChunkToBackingStore(tablesResettableChunk, firstSliceDestination); - addedRowSets.resetWritableChunkToBackingStore(addedRowSetsResettableChunk, firstSliceDestination); + addedRowSetBuilders.resetWritableChunkToBackingStore(addedRowSetsResettableChunk, + firstSliceDestination); final long lastBackingChunkDestination = firstBackingChunkDestination + tablesBackingChunk.size() - 1; final RowSequence initialDestinationsSlice = @@ -581,24 +599,25 @@ private void propagateResurrectedDestinations(@NotNull final RowSequence resurre return; } try (final ResettableWritableObjectChunk tablesResettableChunk = - ResettableWritableObjectChunk.makeResettableChunk(); - final ResettableWritableObjectChunk addedRowSetsResettableChunk = - ResettableWritableObjectChunk.makeResettableChunk(); - final RowSequence.Iterator resurrectedDestinationsIterator = - resurrectedDestinations.getRowSequenceIterator()) { + ResettableWritableObjectChunk.makeResettableChunk(); + final ResettableWritableObjectChunk addedRowSetsResettableChunk = + ResettableWritableObjectChunk.makeResettableChunk(); + final RowSequence.Iterator resurrectedDestinationsIterator = + resurrectedDestinations.getRowSequenceIterator()) { // Destinations that were added can't have any removals, modifications, or shifts. // noinspection unchecked final ObjectChunk tablesBackingChunk = tablesResettableChunk.asObjectChunk(); // noinspection unchecked - final WritableObjectChunk addedRowSetsBackingChunk = + final WritableObjectChunk addedRowSetsBackingChunk = addedRowSetsResettableChunk.asWritableObjectChunk(); while (resurrectedDestinationsIterator.hasMore()) { final long firstSliceDestination = resurrectedDestinationsIterator.peekNextKey(); final long firstBackingChunkDestination = tables.resetWritableChunkToBackingStore(tablesResettableChunk, firstSliceDestination); - addedRowSets.resetWritableChunkToBackingStore(addedRowSetsResettableChunk, firstSliceDestination); + addedRowSetBuilders.resetWritableChunkToBackingStore(addedRowSetsResettableChunk, + firstSliceDestination); final long lastBackingChunkDestination = firstBackingChunkDestination + tablesBackingChunk.size() - 1; final RowSequence resurrectedDestinationsSlice = resurrectedDestinationsIterator.getNextRowSequenceThrough(lastBackingChunkDestination); @@ -645,28 +664,28 @@ private void propagateNewDestinations(@NotNull final RowSequence newDestinations final boolean allowCreation = retainedResultTable && retainedAggregationUpdateListener; final boolean setCallSite = QueryPerformanceRecorder.setCallsite(callSite); try (final ResettableWritableObjectChunk tablesResettableChunk = - ResettableWritableObjectChunk.makeResettableChunk(); - final ResettableWritableObjectChunk addedRowSetsResettableChunk = - ResettableWritableObjectChunk.makeResettableChunk(); - final ResettableWritableObjectChunk removedRowSetsResettableChunk = - allowCreation ? null : ResettableWritableObjectChunk.makeResettableChunk(); - final ResettableWritableObjectChunk modifiedRowSetsResettableChunk = - allowCreation ? null : ResettableWritableObjectChunk.makeResettableChunk(); - final ResettableWritableObjectChunk shiftDataBuildersResettableChunk = - allowCreation ? null : ResettableWritableObjectChunk.makeResettableChunk(); - final RowSequence.Iterator newDestinationsIterator = newDestinations.getRowSequenceIterator()) { + ResettableWritableObjectChunk.makeResettableChunk(); + final ResettableWritableObjectChunk addedRowSetsResettableChunk = + ResettableWritableObjectChunk.makeResettableChunk(); + final ResettableWritableObjectChunk removedRowSetsResettableChunk = + allowCreation ? null : ResettableWritableObjectChunk.makeResettableChunk(); + final ResettableWritableObjectChunk modifiedRowSetsResettableChunk = + allowCreation ? null : ResettableWritableObjectChunk.makeResettableChunk(); + final ResettableWritableObjectChunk shiftDataBuildersResettableChunk = + allowCreation ? null : ResettableWritableObjectChunk.makeResettableChunk(); + final RowSequence.Iterator newDestinationsIterator = newDestinations.getRowSequenceIterator()) { // noinspection unchecked final WritableObjectChunk tablesBackingChunk = tablesResettableChunk.asWritableObjectChunk(); // noinspection unchecked - final WritableObjectChunk addedRowSetsBackingChunk = + final WritableObjectChunk addedRowSetsBackingChunk = addedRowSetsResettableChunk.asWritableObjectChunk(); // noinspection unchecked - final WritableObjectChunk removedRowSetsBackingChunk = + final WritableObjectChunk removedRowSetsBackingChunk = allowCreation ? null : removedRowSetsResettableChunk.asWritableObjectChunk(); // noinspection unchecked - final WritableObjectChunk modifiedRowSetsBackingChunk = + final WritableObjectChunk modifiedRowSetsBackingChunk = allowCreation ? null : modifiedRowSetsResettableChunk.asWritableObjectChunk(); // noinspection unchecked final WritableObjectChunk shiftDataBuildersBackingChunk = @@ -676,11 +695,12 @@ private void propagateNewDestinations(@NotNull final RowSequence newDestinations final long firstSliceDestination = newDestinationsIterator.peekNextKey(); final long firstBackingChunkDestination = tables.resetWritableChunkToBackingStore(tablesResettableChunk, firstSliceDestination); - addedRowSets.resetWritableChunkToBackingStore(addedRowSetsResettableChunk, firstSliceDestination); + addedRowSetBuilders.resetWritableChunkToBackingStore(addedRowSetsResettableChunk, + firstSliceDestination); if (!allowCreation) { - removedRowSets.resetWritableChunkToBackingStore(removedRowSetsResettableChunk, + removedRowSetBuilders.resetWritableChunkToBackingStore(removedRowSetsResettableChunk, firstSliceDestination); - modifiedRowSets.resetWritableChunkToBackingStore(modifiedRowSetsResettableChunk, + modifiedRowSetBuilders.resetWritableChunkToBackingStore(modifiedRowSetsResettableChunk, firstSliceDestination); shiftDataBuilders.resetWritableChunkToBackingStore(shiftDataBuildersResettableChunk, firstSliceDestination); @@ -700,9 +720,9 @@ private void propagateNewDestinations(@NotNull final RowSequence newDestinations } else { // We will never try to create this table again, or accumulate further state for it. tablesBackingChunk.set(backingChunkOffset, NONEXISTENT_TABLE); - addedRowSetsBackingChunk.set(backingChunkOffset, NONEXISTENT_TABLE_ROW_SET); - removedRowSetsBackingChunk.set(backingChunkOffset, NONEXISTENT_TABLE_ROW_SET); - modifiedRowSetsBackingChunk.set(backingChunkOffset, NONEXISTENT_TABLE_ROW_SET); + addedRowSetsBackingChunk.set(backingChunkOffset, NONEXISTENT_TABLE_ROW_SET_BUILDER); + removedRowSetsBackingChunk.set(backingChunkOffset, NONEXISTENT_TABLE_ROW_SET_BUILDER); + modifiedRowSetsBackingChunk.set(backingChunkOffset, NONEXISTENT_TABLE_ROW_SET_BUILDER); shiftDataBuildersBackingChunk.set(backingChunkOffset, NONEXISTENT_TABLE_SHIFT_BUILDER); } }); @@ -733,23 +753,24 @@ private void propagateUpdatesToRemovedDestinations(@NotNull final RowSequence re return; } try (final ResettableWritableObjectChunk tablesResettableChunk = - ResettableWritableObjectChunk.makeResettableChunk(); - final ResettableWritableObjectChunk removedRowSetsResettableChunk = - ResettableWritableObjectChunk.makeResettableChunk(); - final RowSequence.Iterator removedDestinationsIterator = removedDestinations.getRowSequenceIterator()) { + ResettableWritableObjectChunk.makeResettableChunk(); + final ResettableWritableObjectChunk removedRowSetsResettableChunk = + ResettableWritableObjectChunk.makeResettableChunk(); + final RowSequence.Iterator removedDestinationsIterator = removedDestinations.getRowSequenceIterator()) { // Destinations that were completely removed can't have any additions, modifications, or shifts. // noinspection unchecked final ObjectChunk tablesBackingChunk = tablesResettableChunk.asObjectChunk(); // noinspection unchecked - final WritableObjectChunk removedRowSetsBackingChunk = + final WritableObjectChunk removedRowSetsBackingChunk = removedRowSetsResettableChunk.asWritableObjectChunk(); while (removedDestinationsIterator.hasMore()) { final long firstSliceDestination = removedDestinationsIterator.peekNextKey(); final long firstBackingChunkDestination = tables.resetWritableChunkToBackingStore(tablesResettableChunk, firstSliceDestination); - removedRowSets.resetWritableChunkToBackingStore(removedRowSetsResettableChunk, firstSliceDestination); + removedRowSetBuilders.resetWritableChunkToBackingStore(removedRowSetsResettableChunk, + firstSliceDestination); final long lastBackingChunkDestination = firstBackingChunkDestination + tablesBackingChunk.size() - 1; final RowSequence removedDestinationsSlice = removedDestinationsIterator.getNextRowSequenceThrough(lastBackingChunkDestination); @@ -788,28 +809,28 @@ private void propagateUpdatesToModifiedDestinations(@NotNull final RowSequence m return; } try (final ResettableWritableObjectChunk tablesResettableChunk = - ResettableWritableObjectChunk.makeResettableChunk(); - final ResettableWritableObjectChunk addedRowSetsResettableChunk = - ResettableWritableObjectChunk.makeResettableChunk(); - final ResettableWritableObjectChunk removedRowSetsResettableChunk = - ResettableWritableObjectChunk.makeResettableChunk(); - final ResettableWritableObjectChunk modifiedRowSetsResettableChunk = - ResettableWritableObjectChunk.makeResettableChunk(); - final ResettableWritableObjectChunk shiftDataBuildersResettableChunk = - ResettableWritableObjectChunk.makeResettableChunk(); - final RowSequence.Iterator modifiedDestinationsIterator = - modifiedDestinations.getRowSequenceIterator()) { + ResettableWritableObjectChunk.makeResettableChunk(); + final ResettableWritableObjectChunk addedRowSetsResettableChunk = + ResettableWritableObjectChunk.makeResettableChunk(); + final ResettableWritableObjectChunk removedRowSetsResettableChunk = + ResettableWritableObjectChunk.makeResettableChunk(); + final ResettableWritableObjectChunk modifiedRowSetsResettableChunk = + ResettableWritableObjectChunk.makeResettableChunk(); + final ResettableWritableObjectChunk shiftDataBuildersResettableChunk = + ResettableWritableObjectChunk.makeResettableChunk(); + final RowSequence.Iterator modifiedDestinationsIterator = + modifiedDestinations.getRowSequenceIterator()) { // noinspection unchecked final ObjectChunk tablesBackingChunk = tablesResettableChunk.asObjectChunk(); // noinspection unchecked - final WritableObjectChunk addedRowSetsBackingChunk = + final WritableObjectChunk addedRowSetsBackingChunk = addedRowSetsResettableChunk.asWritableObjectChunk(); // noinspection unchecked - final WritableObjectChunk removedRowSetsBackingChunk = + final WritableObjectChunk removedRowSetsBackingChunk = removedRowSetsResettableChunk.asWritableObjectChunk(); // noinspection unchecked - final WritableObjectChunk modifiedRowSetsBackingChunk = + final WritableObjectChunk modifiedRowSetsBackingChunk = modifiedRowSetsResettableChunk.asWritableObjectChunk(); // noinspection unchecked final WritableObjectChunk shiftDataBuildersBackingChunk = @@ -820,9 +841,12 @@ private void propagateUpdatesToModifiedDestinations(@NotNull final RowSequence m final long firstBackingChunkDestination = tables.resetWritableChunkToBackingStore(tablesResettableChunk, firstSliceDestination); // The (valid) assumption is that the other write-through resets will address the same range. - addedRowSets.resetWritableChunkToBackingStore(addedRowSetsResettableChunk, firstSliceDestination); - removedRowSets.resetWritableChunkToBackingStore(removedRowSetsResettableChunk, firstSliceDestination); - modifiedRowSets.resetWritableChunkToBackingStore(modifiedRowSetsResettableChunk, firstSliceDestination); + addedRowSetBuilders.resetWritableChunkToBackingStore(addedRowSetsResettableChunk, + firstSliceDestination); + removedRowSetBuilders.resetWritableChunkToBackingStore(removedRowSetsResettableChunk, + firstSliceDestination); + modifiedRowSetBuilders.resetWritableChunkToBackingStore(modifiedRowSetsResettableChunk, + firstSliceDestination); shiftDataBuilders.resetWritableChunkToBackingStore(shiftDataBuildersResettableChunk, firstSliceDestination); final long lastBackingChunkDestination = firstBackingChunkDestination + tablesBackingChunk.size() - 1; @@ -872,14 +896,16 @@ private void propagateUpdatesToModifiedDestinations(@NotNull final RowSequence m } private static WritableRowSet extractAndClearRowSet( - @NotNull final WritableObjectChunk rowSetChunk, + @NotNull final WritableObjectChunk rowSetChunk, final int offset) { - final WritableRowSet rowSet = rowSetChunk.get(offset); - Assert.neq(rowSet, "rowSet", NONEXISTENT_TABLE_ROW_SET, "NONEXISTENT_TABLE_ROW_SET"); - if (rowSet != null) { + final RowSetBuilderRandom builder = rowSetChunk.get(offset); + Assert.neq(builder, "builder", NONEXISTENT_TABLE_ROW_SET_BUILDER, "NONEXISTENT_TABLE_ROW_SET_BUILDER"); + if (builder != null) { + final WritableRowSet rowSet = builder.build(); rowSetChunk.set(offset, null); + return rowSet; } - return rowSet; + return null; } private static RowSet nullToEmpty(@Nullable final RowSet rowSet) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java index 8681ac495fe..b227b0e633e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java @@ -2,26 +2,16 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; -import io.deephaven.chunk.IntChunk; -import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.attributes.ChunkLengths; -import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.api.updateby.BadDataBehavior; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.locations.TableDataException; -import io.deephaven.engine.table.impl.sources.DoubleArraySource; -import io.deephaven.engine.table.impl.sources.LongArraySource; import io.deephaven.engine.table.impl.updateby.internal.BaseDoubleUpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; -import io.deephaven.util.QueryConstants; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -32,6 +22,7 @@ public abstract class BasePrimitiveEMAOperator extends BaseDoubleUpdateByOperator { protected final OperationControl control; protected final LongRecordingUpdateByOperator timeRecorder; + protected final String timestampColumnName; protected final double timeScaleUnits; class EmaContext extends Context { @@ -62,11 +53,13 @@ public BasePrimitiveEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final LongRecordingUpdateByOperator timeRecorder, + @Nullable final String timestampColumnName, final long timeScaleUnits, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { super(pair, affectingColumns, redirContext); this.control = control; this.timeRecorder = timeRecorder; + this.timestampColumnName = timestampColumnName; this.timeScaleUnits = timeScaleUnits; } @@ -76,30 +69,16 @@ public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize) { return new EmaContext(timeScaleUnits, chunkSize); } - @Override - public void initializeForUpdate(@NotNull final UpdateByOperator.UpdateContext context, - @NotNull final TableUpdate upstream, - @NotNull final RowSet resultSourceIndex, - final long lastPrevKey , - final boolean isAppendOnly) { - super.initializeForUpdate(context, upstream, resultSourceIndex, lastPrevKey, isAppendOnly); - - final EmaContext ctx = (EmaContext) context; - // pre-load the context timestamp with the previous last value in the timestamp column (if possible) - ctx.lastStamp = (lastPrevKey == NULL_ROW_KEY || timeRecorder == null) ? NULL_LONG : locateFirstValidPreviousTimestamp(resultSourceIndex, lastPrevKey); - } - @Override public void initializeFor(@NotNull final UpdateByOperator.UpdateContext updateContext, - @NotNull final RowSet updateRowSet, - @NotNull final UpdateBy.UpdateType type) { - super.initializeFor(updateContext, updateRowSet, type); + @NotNull final RowSet updateRowSet) { + super.initializeFor(updateContext, updateRowSet); } @Override - protected void doAddChunk(@NotNull final Context context, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { + protected void doProcessChunk(@NotNull final Context context, + @NotNull final RowSequence inputKeys, + @NotNull final Chunk workingChunk) { final EmaContext ctx = (EmaContext) context; if (timeRecorder == null) { computeWithTicks(ctx, workingChunk, 0, inputKeys.intSize()); @@ -110,27 +89,25 @@ protected void doAddChunk(@NotNull final Context context, } @Override - public void resetForReprocess(@NotNull final UpdateByOperator.UpdateContext context, - @NotNull final RowSet sourceIndex, - final long firstUnmodifiedKey) { - super.resetForReprocess(context, sourceIndex, firstUnmodifiedKey); + public void resetForProcess(@NotNull final UpdateByOperator.UpdateContext context, + @NotNull final RowSet sourceIndex, + final long firstUnmodifiedKey) { + super.resetForProcess(context, sourceIndex, firstUnmodifiedKey); if (timeRecorder == null) { return; } final EmaContext ctx = (EmaContext) context; - if (!ctx.canProcessDirectly) { - // If we set the last state to null, then we know it was a reset state and the timestamp must also - // have been reset. - if (ctx.curVal == NULL_DOUBLE || (firstUnmodifiedKey == NULL_ROW_KEY)) { - ctx.lastStamp = NULL_LONG; - } else { - // If it hasn't been reset to null, then it's possible that the value at that position was null, in - // which case we must have ignored it, and so we have to actually keep looking backwards until we find - // something not null. - ctx.lastStamp = locateFirstValidPreviousTimestamp(sourceIndex, firstUnmodifiedKey); - } + // If we set the last state to null, then we know it was a reset state and the timestamp must also + // have been reset. + if (ctx.curVal == NULL_DOUBLE || (firstUnmodifiedKey == NULL_ROW_KEY)) { + ctx.lastStamp = NULL_LONG; + } else { + // If it hasn't been reset to null, then it's possible that the value at that position was null, in + // which case we must have ignored it, and so we have to actually keep looking backwards until we find + // something not null. + ctx.lastStamp = locateFirstValidPreviousTimestamp(sourceIndex, firstUnmodifiedKey); } } @@ -156,6 +133,11 @@ private long locateFirstValidPreviousTimestamp(@NotNull final RowSet indexToSear return NULL_LONG; } + @Override + public String getTimestampColumnName() { + return this.timestampColumnName; + } + abstract boolean isValueValid(final long atKey); abstract void computeWithTicks(final EmaContext ctx, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java index 1e7eae540b3..263036794b4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java @@ -34,13 +34,14 @@ public BigDecimalEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final LongRecordingUpdateByOperator timeRecorder, + @Nullable final String timestampColumnName, final long timeScaleUnits, @NotNull final ColumnSource valueSource, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, timeScaleUnits, valueSource, redirContext); + super(pair, affectingColumns, control, timeRecorder, timestampColumnName, timeScaleUnits, valueSource, redirContext); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java index f1cee43724f..b73a646de41 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java @@ -34,13 +34,14 @@ public BigIntegerEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final LongRecordingUpdateByOperator timeRecorder, + @Nullable final String timestampColumnName, final long timeScaleUnits, @NotNull final ColumnSource valueSource, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, timeScaleUnits, valueSource, redirContext); + super(pair, affectingColumns, control, timeRecorder, timestampColumnName, timeScaleUnits, valueSource, redirContext); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java index 980bbfa9f79..1b1859377b6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java @@ -14,14 +14,12 @@ import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import java.math.BigDecimal; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; -import static io.deephaven.util.QueryConstants.NULL_DOUBLE; import static io.deephaven.util.QueryConstants.NULL_LONG; public abstract class BigNumberEMAOperator extends BaseObjectUpdateByOperator { @@ -29,6 +27,7 @@ public abstract class BigNumberEMAOperator extends BaseObjectUpdateByOperator protected final OperationControl control; protected final LongRecordingUpdateByOperator timeRecorder; + protected final String timestampColumnName; protected final double timeScaleUnits; class EmaContext extends Context { @@ -58,6 +57,7 @@ public BigNumberEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final LongRecordingUpdateByOperator timeRecorder, + @Nullable final String timestampColumnName, final long timeScaleUnits, @NotNull ColumnSource valueSource, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext @@ -67,6 +67,7 @@ public BigNumberEMAOperator(@NotNull final MatchPair pair, super(pair, affectingColumns, redirContext, BigDecimal.class); this.control = control; this.timeRecorder = timeRecorder; + this.timestampColumnName = timestampColumnName; this.timeScaleUnits = timeScaleUnits; this.valueSource = valueSource; // region constructor @@ -80,28 +81,7 @@ public UpdateContext makeUpdateContext(final int chunkSize) { } @Override - public void initializeForUpdate(@NotNull UpdateContext context, - @NotNull TableUpdate upstream, - @NotNull RowSet resultSourceIndex, - final long lastPrevKey, - boolean isAppendOnly) { - super.initializeForUpdate(context, upstream, resultSourceIndex, lastPrevKey, isAppendOnly); - - final EmaContext ctx = (EmaContext) context; - // pre-load the context timestamp with the previous last value in the timestamp column (if possible) - ctx.lastStamp = (lastPrevKey == NULL_ROW_KEY || timeRecorder == null) ? NULL_LONG : locateFirstValidPreviousTimestamp(resultSourceIndex, lastPrevKey); - } - - @SuppressWarnings("unchecked") - @Override - public void initializeFor(@NotNull final UpdateContext updateContext, - @NotNull final RowSet updateRowSet, - @NotNull final UpdateBy.UpdateType type) { - super.initializeFor(updateContext, updateRowSet, type); - } - - @Override - protected void doAddChunk(@NotNull final Context updateContext, + protected void doProcessChunk(@NotNull final Context updateContext, @NotNull final RowSequence inputKeys, @NotNull final Chunk workingChunk) { final ObjectChunk asObjects = workingChunk.asObjectChunk(); @@ -117,27 +97,26 @@ protected void doAddChunk(@NotNull final Context updateContext, @Override - public void resetForReprocess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceIndex, - final long firstUnmodifiedKey) { - super.resetForReprocess(context, sourceIndex, firstUnmodifiedKey); + public void resetForProcess(@NotNull final UpdateContext context, + @NotNull final RowSet sourceIndex, + final long firstUnmodifiedKey) { + super.resetForProcess(context, sourceIndex, firstUnmodifiedKey); if (timeRecorder == null) { return; } final EmaContext ctx = (EmaContext) context; - if (!ctx.canProcessDirectly) { - // If we set the last state to null, then we know it was a reset state and the timestamp must also - // have been reset. - if (ctx.curVal == null || (firstUnmodifiedKey == NULL_ROW_KEY)) { - ctx.lastStamp = NULL_LONG; - } else { - // If it hasn't been reset to null, then it's possible that the value at that position was null, in - // which case we must have ignored it, and so we have to actually keep looking backwards until we find - // something not null. - ctx.lastStamp = locateFirstValidPreviousTimestamp(sourceIndex, firstUnmodifiedKey); - } + + // If we set the last state to null, then we know it was a reset state and the timestamp must also + // have been reset. + if (ctx.curVal == null || (firstUnmodifiedKey == NULL_ROW_KEY)) { + ctx.lastStamp = NULL_LONG; + } else { + // If it hasn't been reset to null, then it's possible that the value at that position was null, in + // which case we must have ignored it, and so we have to actually keep looking backwards until we find + // something not null. + ctx.lastStamp = locateFirstValidPreviousTimestamp(sourceIndex, firstUnmodifiedKey); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java index dab16c4b127..5775a07d9f6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java @@ -39,13 +39,14 @@ public ByteEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final LongRecordingUpdateByOperator timeRecorder, + @Nullable final String timestampColumnName, final long timeScaleUnits, @NotNull final ColumnSource valueSource, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, timeScaleUnits, redirContext); + super(pair, affectingColumns, control, timeRecorder, timestampColumnName, timeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java index f294028308e..33ddb59351b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java @@ -40,13 +40,14 @@ public DoubleEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final LongRecordingUpdateByOperator timeRecorder, + @Nullable final String timestampColumnName, final long timeScaleUnits, @NotNull final ColumnSource valueSource, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, timeScaleUnits, redirContext); + super(pair, affectingColumns, control, timeRecorder, timestampColumnName, timeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java index e42b50cd133..a68e61fcd96 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java @@ -35,13 +35,14 @@ public FloatEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final LongRecordingUpdateByOperator timeRecorder, + @Nullable final String timestampColumnName, final long timeScaleUnits, @NotNull final ColumnSource valueSource, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, timeScaleUnits, redirContext); + super(pair, affectingColumns, control, timeRecorder, timestampColumnName, timeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java index 545ccc8e987..ab584fd87ca 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java @@ -39,13 +39,14 @@ public IntEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final LongRecordingUpdateByOperator timeRecorder, + @Nullable final String timestampColumnName, final long timeScaleUnits, @NotNull final ColumnSource valueSource, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, timeScaleUnits, redirContext); + super(pair, affectingColumns, control, timeRecorder, timestampColumnName, timeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java index c07683048d9..97794db3111 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java @@ -39,13 +39,14 @@ public LongEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final LongRecordingUpdateByOperator timeRecorder, + @Nullable final String timestampColumnName, final long timeScaleUnits, @NotNull final ColumnSource valueSource, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, timeScaleUnits, redirContext); + super(pair, affectingColumns, control, timeRecorder, timestampColumnName, timeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java index 4e701f7b42d..d964c80c348 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java @@ -34,13 +34,14 @@ public ShortEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final LongRecordingUpdateByOperator timeRecorder, + @Nullable final String timestampColumnName, final long timeScaleUnits, @NotNull final ColumnSource valueSource, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, timeScaleUnits, redirContext); + super(pair, affectingColumns, control, timeRecorder, timestampColumnName, timeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java index db55d8e65aa..71fac368a3b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java @@ -64,9 +64,9 @@ public Map> getOutputColumns() { // endregion extra-methods @Override - protected void doAddChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { + protected void doProcessChunk(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys, + @NotNull final Chunk workingChunk) { accumulate(workingChunk.asByteChunk(), ctx, 0, workingChunk.size()); outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java index 2b10b12d892..7ac31e69e93 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java @@ -38,9 +38,9 @@ public ByteFillByOperator(@NotNull final MatchPair fillPair, // endregion extra-methods @Override - protected void doAddChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { + protected void doProcessChunk(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys, + @NotNull final Chunk workingChunk) { accumulate(workingChunk.asByteChunk(), ctx, 0, workingChunk.size()); outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java index ab737b9ffe4..887b6a2ed8e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java @@ -33,9 +33,9 @@ public CharFillByOperator(@NotNull final MatchPair fillPair, // endregion extra-methods @Override - protected void doAddChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { + protected void doProcessChunk(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys, + @NotNull final Chunk workingChunk) { accumulate(workingChunk.asCharChunk(), ctx, 0, workingChunk.size()); outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java index 85179bb593e..760c784a396 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java @@ -38,9 +38,9 @@ public DoubleFillByOperator(@NotNull final MatchPair fillPair, // endregion extra-methods @Override - protected void doAddChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { + protected void doProcessChunk(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys, + @NotNull final Chunk workingChunk) { accumulate(workingChunk.asDoubleChunk(), ctx, 0, workingChunk.size()); outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java index 9d5cf19b05c..bdd0d9dd040 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java @@ -38,9 +38,9 @@ public FloatFillByOperator(@NotNull final MatchPair fillPair, // endregion extra-methods @Override - protected void doAddChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { + protected void doProcessChunk(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys, + @NotNull final Chunk workingChunk) { accumulate(workingChunk.asFloatChunk(), ctx, 0, workingChunk.size()); outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java index 91bd6e2cdb2..0f50f86223e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java @@ -38,9 +38,9 @@ public IntFillByOperator(@NotNull final MatchPair fillPair, // endregion extra-methods @Override - protected void doAddChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { + protected void doProcessChunk(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys, + @NotNull final Chunk workingChunk) { accumulate(workingChunk.asIntChunk(), ctx, 0, workingChunk.size()); outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java index c038b488941..4a6e60616fe 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java @@ -59,9 +59,9 @@ public Map> getOutputColumns() { // endregion extra-methods @Override - protected void doAddChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { + protected void doProcessChunk(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys, + @NotNull final Chunk workingChunk) { accumulate(workingChunk.asLongChunk(), ctx, 0, workingChunk.size()); outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java index 880c0e8baa0..b35dd7e9a13 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java @@ -38,9 +38,9 @@ public ObjectFillByOperator(@NotNull final MatchPair fillPair, // endregion extra-methods @Override - protected void doAddChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { + protected void doProcessChunk(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys, + @NotNull final Chunk workingChunk) { accumulate(workingChunk.asObjectChunk(), ctx, 0, workingChunk.size()); outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java index 57242733be4..94b47dfe27d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java @@ -38,9 +38,9 @@ public ShortFillByOperator(@NotNull final MatchPair fillPair, // endregion extra-methods @Override - protected void doAddChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { + protected void doProcessChunk(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys, + @NotNull final Chunk workingChunk) { accumulate(workingChunk.asShortChunk(), ctx, 0, workingChunk.size()); outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/hashing/UpdateBySlotTracker.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/hashing/UpdateBySlotTracker.java index eca00aca813..e9c13ae5ded 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/hashing/UpdateBySlotTracker.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/hashing/UpdateBySlotTracker.java @@ -357,7 +357,7 @@ public UpdateTracker getTracker(final int slot) { } private long createUpdateForSlot(final int slot) { - if (pointer == allocated) { + if (pointer == allocated || slot == allocated) { allocated += chunkSize; slotUpdates.ensureCapacity(allocated); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java index 73ada869b75..8f982b928da 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java @@ -11,20 +11,15 @@ import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.chunk.Chunk; -import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.attributes.ChunkLengths; -import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.sized.SizedByteChunk; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.util.RowRedirection; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; import io.deephaven.util.QueryConstants; import org.jetbrains.annotations.NotNull; @@ -51,11 +46,6 @@ public abstract class BaseByteUpdateByOperator extends UpdateByCumulativeOperato protected class Context extends UpdateCumulativeContext { public final SizedSafeCloseable fillContext; public final SizedByteChunk outputValues; - public boolean canProcessDirectly; - public UpdateBy.UpdateType currentUpdateType; - - public RowSetBuilderSequential modifiedBuilder; - public RowSet newModified; public byte curVal = nullValue; @@ -65,15 +55,10 @@ protected Context(final int chunkSize) { this.outputValues = new SizedByteChunk<>(chunkSize); } - public RowSetBuilderSequential getModifiedBuilder() { - if(modifiedBuilder == null) { - modifiedBuilder = RowSetFactory.builderSequential(); - } - return modifiedBuilder; - } - @Override public void close() { + super.close(); + outputValues.close(); fillContext.close(); } @@ -164,57 +149,6 @@ public UpdateContext makeUpdateContext(final int chunkSize) { return new Context(chunkSize); } - @Override - public void initializeForUpdate(@NotNull final UpdateContext context, - @NotNull final TableUpdate upstream, - @NotNull final RowSet resultSourceIndex, - final long lastPrevKey, - final boolean isUpstreamAppendOnly) { - final Context ctx = (Context) context; - - // If we're redirected we have to make sure we tell the output source it's actual size, or we're going - // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not - // need to do anything with capacity. - if(redirContext.isRedirected()) { - // The redirection index does not use the 0th index for some reason. - outputSource.ensureCapacity(redirContext.requiredCapacity()); - } - - // just remember the appendyness. - ctx.canProcessDirectly = isUpstreamAppendOnly; - - // pre-load the context with the previous last value in the table (if possible) - ctx.curVal = lastPrevKey == NULL_ROW_KEY ? nullValue : outputSource.getPrevByte(lastPrevKey); - } - - @Override - public void initializeFor(@NotNull final UpdateContext updateContext, - @NotNull final RowSet updateRowSet, - @NotNull final UpdateBy.UpdateType type) { - ((Context)updateContext).currentUpdateType = type; - } - - @Override - public void finishFor(@NotNull final UpdateContext updateContext, @NotNull final UpdateBy.UpdateType type) { - final Context ctx = (Context) updateContext; - if(type == UpdateBy.UpdateType.Reprocess) { - ctx.newModified = ctx.modifiedBuilder.build(); - } - } - - @Override - public boolean requiresKeys() { - return false; - } - - @Override - public boolean requiresValues(@NotNull final UpdateContext context) { - final Context ctx = (Context) context; - // We only need to read actual values if we can process them at this moment, so that means if we are - // either part of an append-only update, or within the reprocess cycle - return (ctx.currentUpdateType == UpdateBy.UpdateType.Add && ctx.canProcessDirectly) || ctx.currentUpdateType == UpdateBy.UpdateType.Reprocess; - } - @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); @@ -223,44 +157,16 @@ public void startTrackingPrev() { } } - @NotNull - @Override - final public RowSet getAdditionalModifications(@NotNull final UpdateContext ctx) { - return ((Context)ctx).newModified == null ? RowSetFactory.empty() : ((Context)ctx).newModified; - } - - @Override - final public boolean anyModified(@NotNull final UpdateContext ctx) { - return ((Context)ctx).newModified != null; - } - - @Override - public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { - return ((Context)context).canProcessDirectly; - } - - // region Addition - @Override - public void addChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk values) { - final Context ctx = (Context) updateContext; - if (ctx.canProcessDirectly) { - doAddChunk(ctx, inputKeys, values); - } - } - /** - * Add a chunk of values to the operator. + * Perform the processing for a chunk of values by the operator. * * @param ctx the context object * @param inputKeys the input keys for the chunk * @param workingChunk the chunk of values */ - protected abstract void doAddChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk); + protected abstract void doProcessChunk(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys, + @NotNull final Chunk workingChunk); // endregion @@ -279,49 +185,32 @@ public void applyOutputShift(@NotNull final UpdateContext context, // region Reprocessing - public void resetForReprocess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceIndex, - long firstUnmodifiedKey) { - final Context ctx = (Context) context; - if(!ctx.canProcessDirectly) { - ctx.curVal = firstUnmodifiedKey == NULL_ROW_KEY ? nullValue : outputSource.getByte(firstUnmodifiedKey); + public void resetForProcess(@NotNull final UpdateContext context, + @NotNull final RowSet sourceIndex, + long firstUnmodifiedKey) { + + // If we're redirected we have to make sure we tell the output source it's actual size, or we're going + // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not + // need to do anything with capacity. + if(redirContext.isRedirected()) { + // The redirection index does not use the 0th index for some reason. + outputSource.ensureCapacity(redirContext.requiredCapacity()); } + + final Context ctx = (Context) context; + ctx.curVal = firstUnmodifiedKey == NULL_ROW_KEY ? nullValue : outputSource.getByte(firstUnmodifiedKey); } @Override - public void reprocessChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex) { + public void processChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk valuesChunk, + @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; - doAddChunk(ctx, inputKeys, valuesChunk); + doProcessChunk(ctx, inputKeys, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } // endregion - - // region No-Op Operations - - @Override - final public void modifyChunk(@NotNull final UpdateContext updateContext, - @Nullable final LongChunk prevKeyChunk, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk, - @NotNull final Chunk postValuesChunk) { - } - - @Override - final public void removeChunk(@NotNull final UpdateContext updateContext, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk) { - } - - @Override - final public void applyShift(@NotNull final UpdateContext updateContext, - @NotNull final RowSet prevIndex, - @NotNull final RowSetShiftData shifted) { - - } - // endregion } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java index d31b7b092dd..94fd4f673e4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java @@ -1,20 +1,15 @@ package io.deephaven.engine.table.impl.updateby.internal; import io.deephaven.chunk.Chunk; -import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.attributes.ChunkLengths; -import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.sized.SizedCharChunk; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.util.RowRedirection; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; import io.deephaven.util.QueryConstants; import org.jetbrains.annotations.NotNull; @@ -40,11 +35,6 @@ public abstract class BaseCharUpdateByOperator extends UpdateByCumulativeOperato protected class Context extends UpdateCumulativeContext { public final SizedSafeCloseable fillContext; public final SizedCharChunk outputValues; - public boolean canProcessDirectly; - public UpdateBy.UpdateType currentUpdateType; - - public RowSetBuilderSequential modifiedBuilder; - public RowSet newModified; public char curVal = QueryConstants.NULL_CHAR; @@ -54,15 +44,10 @@ protected Context(final int chunkSize) { this.outputValues = new SizedCharChunk<>(chunkSize); } - public RowSetBuilderSequential getModifiedBuilder() { - if(modifiedBuilder == null) { - modifiedBuilder = RowSetFactory.builderSequential(); - } - return modifiedBuilder; - } - @Override public void close() { + super.close(); + outputValues.close(); fillContext.close(); } @@ -140,57 +125,6 @@ public UpdateContext makeUpdateContext(final int chunkSize) { return new Context(chunkSize); } - @Override - public void initializeForUpdate(@NotNull final UpdateContext context, - @NotNull final TableUpdate upstream, - @NotNull final RowSet resultSourceIndex, - final long lastPrevKey, - final boolean isUpstreamAppendOnly) { - final Context ctx = (Context) context; - - // If we're redirected we have to make sure we tell the output source it's actual size, or we're going - // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not - // need to do anything with capacity. - if(redirContext.isRedirected()) { - // The redirection index does not use the 0th index for some reason. - outputSource.ensureCapacity(redirContext.requiredCapacity()); - } - - // just remember the appendyness. - ctx.canProcessDirectly = isUpstreamAppendOnly; - - // pre-load the context with the previous last value in the table (if possible) - ctx.curVal = lastPrevKey == NULL_ROW_KEY ? QueryConstants.NULL_CHAR : outputSource.getPrevChar(lastPrevKey); - } - - @Override - public void initializeFor(@NotNull final UpdateContext updateContext, - @NotNull final RowSet updateRowSet, - @NotNull final UpdateBy.UpdateType type) { - ((Context)updateContext).currentUpdateType = type; - } - - @Override - public void finishFor(@NotNull final UpdateContext updateContext, @NotNull final UpdateBy.UpdateType type) { - final Context ctx = (Context) updateContext; - if(type == UpdateBy.UpdateType.Reprocess) { - ctx.newModified = ctx.modifiedBuilder.build(); - } - } - - @Override - public boolean requiresKeys() { - return false; - } - - @Override - public boolean requiresValues(@NotNull final UpdateContext context) { - final Context ctx = (Context) context; - // We only need to read actual values if we can process them at this moment, so that means if we are - // either part of an append-only update, or within the reprocess cycle - return (ctx.currentUpdateType == UpdateBy.UpdateType.Add && ctx.canProcessDirectly) || ctx.currentUpdateType == UpdateBy.UpdateType.Reprocess; - } - @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); @@ -199,44 +133,16 @@ public void startTrackingPrev() { } } - @NotNull - @Override - final public RowSet getAdditionalModifications(@NotNull final UpdateContext ctx) { - return ((Context)ctx).newModified == null ? RowSetFactory.empty() : ((Context)ctx).newModified; - } - - @Override - final public boolean anyModified(@NotNull final UpdateContext ctx) { - return ((Context)ctx).newModified != null; - } - - @Override - public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { - return ((Context)context).canProcessDirectly; - } - - // region Addition - @Override - public void addChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk values) { - final Context ctx = (Context) updateContext; - if (ctx.canProcessDirectly) { - doAddChunk(ctx, inputKeys, values); - } - } - /** - * Add a chunk of values to the operator. + * Perform the processing for a chunk of values by the operator. * * @param ctx the context object * @param inputKeys the input keys for the chunk * @param workingChunk the chunk of values */ - protected abstract void doAddChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk); + protected abstract void doProcessChunk(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys, + @NotNull final Chunk workingChunk); // endregion @@ -253,49 +159,32 @@ public void applyOutputShift(@NotNull final UpdateContext context, // region Reprocessing - public void resetForReprocess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceIndex, - long firstUnmodifiedKey) { - final Context ctx = (Context) context; - if(!ctx.canProcessDirectly) { - ctx.curVal = firstUnmodifiedKey == NULL_ROW_KEY ? QueryConstants.NULL_CHAR : outputSource.getChar(firstUnmodifiedKey); + public void resetForProcess(@NotNull final UpdateContext context, + @NotNull final RowSet sourceIndex, + long firstUnmodifiedKey) { + + // If we're redirected we have to make sure we tell the output source it's actual size, or we're going + // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not + // need to do anything with capacity. + if(redirContext.isRedirected()) { + // The redirection index does not use the 0th index for some reason. + outputSource.ensureCapacity(redirContext.requiredCapacity()); } + + final Context ctx = (Context) context; + ctx.curVal = firstUnmodifiedKey == NULL_ROW_KEY ? QueryConstants.NULL_CHAR : outputSource.getChar(firstUnmodifiedKey); } @Override - public void reprocessChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex) { + public void processChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk valuesChunk, + @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; - doAddChunk(ctx, inputKeys, valuesChunk); + doProcessChunk(ctx, inputKeys, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } // endregion - - // region No-Op Operations - - @Override - final public void modifyChunk(@NotNull final UpdateContext updateContext, - @Nullable final LongChunk prevKeyChunk, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk, - @NotNull final Chunk postValuesChunk) { - } - - @Override - final public void removeChunk(@NotNull final UpdateContext updateContext, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk) { - } - - @Override - final public void applyShift(@NotNull final UpdateContext updateContext, - @NotNull final RowSet prevIndex, - @NotNull final RowSetShiftData shifted) { - - } - // endregion } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java index 09272d7d247..006272b01b8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java @@ -46,8 +46,6 @@ public abstract class BaseDoubleUpdateByOperator extends UpdateByCumulativeOpera protected class Context extends UpdateCumulativeContext { public final SizedSafeCloseable fillContext; public final SizedDoubleChunk outputValues; - public boolean canProcessDirectly; - public UpdateBy.UpdateType currentUpdateType; public RowSetBuilderSequential modifiedBuilder; public RowSet newModified; @@ -55,7 +53,6 @@ protected class Context extends UpdateCumulativeContext { public double curVal = NULL_DOUBLE; public boolean filledWithPermanentValue = false; - public long lastGroupPosition = NULL_LONG; protected Context(final int chunkSize) { this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); @@ -136,56 +133,6 @@ public UpdateContext makeUpdateContext(final int chunkSize) { return new Context(chunkSize); } - @Override - public void initializeForUpdate(@NotNull final UpdateContext context, - @NotNull final TableUpdate upstream, - @NotNull final RowSet resultSourceRowSet, - final long lastPrevKey, - final boolean isUpstreamAppendOnly) { - final Context ctx = (Context) context; - - // If we're redirected we have to make sure we tell the output source it's actual size, or we're going - // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not - // need to do anything with capacity. - if(redirContext.isRedirected()) { - outputSource.ensureCapacity(redirContext.requiredCapacity()); - } - - // If we aren't bucketing, we'll just remember the appendyness. - ctx.canProcessDirectly = isUpstreamAppendOnly; - - // pre-load the context with the previous last value in the table (if possible) - ctx.curVal = lastPrevKey == NULL_ROW_KEY ? QueryConstants.NULL_DOUBLE : outputSource.getDouble(lastPrevKey); - } - - @Override - public void initializeFor(@NotNull final UpdateContext updateContext, - @NotNull final RowSet updateRowSet, - @NotNull final UpdateBy.UpdateType type) { - ((Context)updateContext).currentUpdateType = type; - } - - @Override - public void finishFor(@NotNull final UpdateContext updateContext, @NotNull final UpdateBy.UpdateType type) { - final Context ctx = (Context) updateContext; - if(type == UpdateBy.UpdateType.Reprocess) { - ctx.newModified = ctx.modifiedBuilder.build(); - } - } - - @Override - public boolean requiresKeys() { - return false; - } - - @Override - public boolean requiresValues(@NotNull final UpdateContext context) { - final Context ctx = (Context) context; - // We only need to read actual values if we can process them at this moment, so that means if we are - // either part of an append-only update, or within the reprocess cycle - return (ctx.currentUpdateType == UpdateBy.UpdateType.Add && ctx.canProcessDirectly) || ctx.currentUpdateType == UpdateBy.UpdateType.Reprocess; - } - @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); @@ -194,35 +141,7 @@ public void startTrackingPrev() { } } - @NotNull - @Override - final public RowSet getAdditionalModifications(@NotNull final UpdateContext ctx) { - return ((Context)ctx).newModified == null ? RowSetFactory.empty() : ((Context)ctx).newModified; - } - - @Override - final public boolean anyModified(@NotNull final UpdateContext ctx) { - return ((Context)ctx).newModified != null; - } - - @Override - public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { - return ((Context)context).canProcessDirectly; - } - - // region Addition - @Override - public void addChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk values) { - final Context ctx = (Context) updateContext; - if (ctx.canProcessDirectly) { - doAddChunk(ctx, inputKeys, values); - } - } - - protected abstract void doAddChunk(@NotNull final Context ctx, + protected abstract void doProcessChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, @NotNull final Chunk workingChunk); // endregion @@ -240,47 +159,30 @@ public void applyOutputShift(@NotNull final UpdateContext context, // region Reprocessing - public void resetForReprocess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceRowSet, - final long firstUnmodifiedKey) { + public void resetForProcess(@NotNull final UpdateContext context, + @NotNull final RowSet sourceRowSet, + final long firstUnmodifiedKey) { final Context ctx = (Context) context; - if(!ctx.canProcessDirectly) { - ctx.curVal = firstUnmodifiedKey == NULL_ROW_KEY ? NULL_DOUBLE : outputSource.getDouble(firstUnmodifiedKey); + + // If we're redirected we have to make sure we tell the output source it's actual size, or we're going + // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not + // need to do anything with capacity. + if(redirContext.isRedirected()) { + outputSource.ensureCapacity(redirContext.requiredCapacity()); } + + ctx.curVal = firstUnmodifiedKey == NULL_ROW_KEY ? NULL_DOUBLE : outputSource.getDouble(firstUnmodifiedKey); } @Override - public void reprocessChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceRowSet) { + public void processChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk valuesChunk, + @NotNull final RowSet postUpdateSourceRowSet) { final Context ctx = (Context) updateContext; - doAddChunk(ctx, inputKeys, valuesChunk); + doProcessChunk(ctx, inputKeys, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } // endregion - - // region No-Op Operations - - @Override - final public void modifyChunk(@NotNull final UpdateContext updateContext, - @Nullable final LongChunk prevKeyChunk, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk, - @NotNull final Chunk postValuesChunk) { - } - - @Override - final public void removeChunk(@NotNull final UpdateContext updateContext, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk) { - } - - @Override - final public void applyShift(@NotNull final UpdateContext updateContext, - @NotNull final RowSet prevRowSet, - @NotNull final RowSetShiftData shifted) { - } - // endregion } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java index 9017e334cf4..75fec93e14e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java @@ -41,8 +41,6 @@ public abstract class BaseFloatUpdateByOperator extends UpdateByCumulativeOperat protected class Context extends UpdateCumulativeContext { public final SizedSafeCloseable fillContext; public final SizedFloatChunk outputValues; - public boolean canProcessDirectly; - public UpdateBy.UpdateType currentUpdateType; public RowSetBuilderSequential modifiedBuilder; public RowSet newModified; @@ -50,7 +48,6 @@ protected class Context extends UpdateCumulativeContext { public float curVal = NULL_FLOAT; public boolean filledWithPermanentValue = false; - public long lastGroupPosition = NULL_LONG; protected Context(final int chunkSize) { this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); @@ -131,56 +128,6 @@ public UpdateContext makeUpdateContext(final int chunkSize) { return new Context(chunkSize); } - @Override - public void initializeForUpdate(@NotNull final UpdateContext context, - @NotNull final TableUpdate upstream, - @NotNull final RowSet resultSourceRowSet, - final long lastPrevKey, - final boolean isUpstreamAppendOnly) { - final Context ctx = (Context) context; - - // If we're redirected we have to make sure we tell the output source it's actual size, or we're going - // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not - // need to do anything with capacity. - if(redirContext.isRedirected()) { - outputSource.ensureCapacity(redirContext.requiredCapacity()); - } - - // If we aren't bucketing, we'll just remember the appendyness. - ctx.canProcessDirectly = isUpstreamAppendOnly; - - // pre-load the context with the previous last value in the table (if possible) - ctx.curVal = lastPrevKey == NULL_ROW_KEY ? QueryConstants.NULL_FLOAT : outputSource.getFloat(lastPrevKey); - } - - @Override - public void initializeFor(@NotNull final UpdateContext updateContext, - @NotNull final RowSet updateRowSet, - @NotNull final UpdateBy.UpdateType type) { - ((Context)updateContext).currentUpdateType = type; - } - - @Override - public void finishFor(@NotNull final UpdateContext updateContext, @NotNull final UpdateBy.UpdateType type) { - final Context ctx = (Context) updateContext; - if(type == UpdateBy.UpdateType.Reprocess) { - ctx.newModified = ctx.modifiedBuilder.build(); - } - } - - @Override - public boolean requiresKeys() { - return false; - } - - @Override - public boolean requiresValues(@NotNull final UpdateContext context) { - final Context ctx = (Context) context; - // We only need to read actual values if we can process them at this moment, so that means if we are - // either part of an append-only update, or within the reprocess cycle - return (ctx.currentUpdateType == UpdateBy.UpdateType.Add && ctx.canProcessDirectly) || ctx.currentUpdateType == UpdateBy.UpdateType.Reprocess; - } - @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); @@ -189,35 +136,7 @@ public void startTrackingPrev() { } } - @NotNull - @Override - final public RowSet getAdditionalModifications(@NotNull final UpdateContext ctx) { - return ((Context)ctx).newModified == null ? RowSetFactory.empty() : ((Context)ctx).newModified; - } - - @Override - final public boolean anyModified(@NotNull final UpdateContext ctx) { - return ((Context)ctx).newModified != null; - } - - @Override - public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { - return ((Context)context).canProcessDirectly; - } - - // region Addition - @Override - public void addChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk values) { - final Context ctx = (Context) updateContext; - if (ctx.canProcessDirectly) { - doAddChunk(ctx, inputKeys, values); - } - } - - protected abstract void doAddChunk(@NotNull final Context ctx, + protected abstract void doProcessChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, @NotNull final Chunk workingChunk); // endregion @@ -235,47 +154,30 @@ public void applyOutputShift(@NotNull final UpdateContext context, // region Reprocessing - public void resetForReprocess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceRowSet, - final long firstUnmodifiedKey) { + public void resetForProcess(@NotNull final UpdateContext context, + @NotNull final RowSet sourceRowSet, + final long firstUnmodifiedKey) { final Context ctx = (Context) context; - if(!ctx.canProcessDirectly) { - ctx.curVal = firstUnmodifiedKey == NULL_ROW_KEY ? NULL_FLOAT : outputSource.getFloat(firstUnmodifiedKey); + + // If we're redirected we have to make sure we tell the output source it's actual size, or we're going + // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not + // need to do anything with capacity. + if(redirContext.isRedirected()) { + outputSource.ensureCapacity(redirContext.requiredCapacity()); } + + ctx.curVal = firstUnmodifiedKey == NULL_ROW_KEY ? NULL_FLOAT : outputSource.getFloat(firstUnmodifiedKey); } @Override - public void reprocessChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceRowSet) { + public void processChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk valuesChunk, + @NotNull final RowSet postUpdateSourceRowSet) { final Context ctx = (Context) updateContext; - doAddChunk(ctx, inputKeys, valuesChunk); + doProcessChunk(ctx, inputKeys, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } // endregion - - // region No-Op Operations - - @Override - final public void modifyChunk(@NotNull final UpdateContext updateContext, - @Nullable final LongChunk prevKeyChunk, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk, - @NotNull final Chunk postValuesChunk) { - } - - @Override - final public void removeChunk(@NotNull final UpdateContext updateContext, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk) { - } - - @Override - final public void applyShift(@NotNull final UpdateContext updateContext, - @NotNull final RowSet prevRowSet, - @NotNull final RowSetShiftData shifted) { - } - // endregion } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java index a7c57e0c9c2..f869d8cb81e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java @@ -6,20 +6,15 @@ package io.deephaven.engine.table.impl.updateby.internal; import io.deephaven.chunk.Chunk; -import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.attributes.ChunkLengths; -import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.sized.SizedIntChunk; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.util.RowRedirection; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; import io.deephaven.util.QueryConstants; import org.jetbrains.annotations.NotNull; @@ -45,11 +40,6 @@ public abstract class BaseIntUpdateByOperator extends UpdateByCumulativeOperator protected class Context extends UpdateCumulativeContext { public final SizedSafeCloseable fillContext; public final SizedIntChunk outputValues; - public boolean canProcessDirectly; - public UpdateBy.UpdateType currentUpdateType; - - public RowSetBuilderSequential modifiedBuilder; - public RowSet newModified; public int curVal = QueryConstants.NULL_INT; @@ -59,15 +49,10 @@ protected Context(final int chunkSize) { this.outputValues = new SizedIntChunk<>(chunkSize); } - public RowSetBuilderSequential getModifiedBuilder() { - if(modifiedBuilder == null) { - modifiedBuilder = RowSetFactory.builderSequential(); - } - return modifiedBuilder; - } - @Override public void close() { + super.close(); + outputValues.close(); fillContext.close(); } @@ -145,57 +130,6 @@ public UpdateContext makeUpdateContext(final int chunkSize) { return new Context(chunkSize); } - @Override - public void initializeForUpdate(@NotNull final UpdateContext context, - @NotNull final TableUpdate upstream, - @NotNull final RowSet resultSourceIndex, - final long lastPrevKey, - final boolean isUpstreamAppendOnly) { - final Context ctx = (Context) context; - - // If we're redirected we have to make sure we tell the output source it's actual size, or we're going - // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not - // need to do anything with capacity. - if(redirContext.isRedirected()) { - // The redirection index does not use the 0th index for some reason. - outputSource.ensureCapacity(redirContext.requiredCapacity()); - } - - // just remember the appendyness. - ctx.canProcessDirectly = isUpstreamAppendOnly; - - // pre-load the context with the previous last value in the table (if possible) - ctx.curVal = lastPrevKey == NULL_ROW_KEY ? QueryConstants.NULL_INT : outputSource.getPrevInt(lastPrevKey); - } - - @Override - public void initializeFor(@NotNull final UpdateContext updateContext, - @NotNull final RowSet updateRowSet, - @NotNull final UpdateBy.UpdateType type) { - ((Context)updateContext).currentUpdateType = type; - } - - @Override - public void finishFor(@NotNull final UpdateContext updateContext, @NotNull final UpdateBy.UpdateType type) { - final Context ctx = (Context) updateContext; - if(type == UpdateBy.UpdateType.Reprocess) { - ctx.newModified = ctx.modifiedBuilder.build(); - } - } - - @Override - public boolean requiresKeys() { - return false; - } - - @Override - public boolean requiresValues(@NotNull final UpdateContext context) { - final Context ctx = (Context) context; - // We only need to read actual values if we can process them at this moment, so that means if we are - // either part of an append-only update, or within the reprocess cycle - return (ctx.currentUpdateType == UpdateBy.UpdateType.Add && ctx.canProcessDirectly) || ctx.currentUpdateType == UpdateBy.UpdateType.Reprocess; - } - @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); @@ -204,44 +138,16 @@ public void startTrackingPrev() { } } - @NotNull - @Override - final public RowSet getAdditionalModifications(@NotNull final UpdateContext ctx) { - return ((Context)ctx).newModified == null ? RowSetFactory.empty() : ((Context)ctx).newModified; - } - - @Override - final public boolean anyModified(@NotNull final UpdateContext ctx) { - return ((Context)ctx).newModified != null; - } - - @Override - public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { - return ((Context)context).canProcessDirectly; - } - - // region Addition - @Override - public void addChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk values) { - final Context ctx = (Context) updateContext; - if (ctx.canProcessDirectly) { - doAddChunk(ctx, inputKeys, values); - } - } - /** - * Add a chunk of values to the operator. + * Perform the processing for a chunk of values by the operator. * * @param ctx the context object * @param inputKeys the input keys for the chunk * @param workingChunk the chunk of values */ - protected abstract void doAddChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk); + protected abstract void doProcessChunk(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys, + @NotNull final Chunk workingChunk); // endregion @@ -258,49 +164,32 @@ public void applyOutputShift(@NotNull final UpdateContext context, // region Reprocessing - public void resetForReprocess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceIndex, - long firstUnmodifiedKey) { - final Context ctx = (Context) context; - if(!ctx.canProcessDirectly) { - ctx.curVal = firstUnmodifiedKey == NULL_ROW_KEY ? QueryConstants.NULL_INT : outputSource.getInt(firstUnmodifiedKey); + public void resetForProcess(@NotNull final UpdateContext context, + @NotNull final RowSet sourceIndex, + long firstUnmodifiedKey) { + + // If we're redirected we have to make sure we tell the output source it's actual size, or we're going + // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not + // need to do anything with capacity. + if(redirContext.isRedirected()) { + // The redirection index does not use the 0th index for some reason. + outputSource.ensureCapacity(redirContext.requiredCapacity()); } + + final Context ctx = (Context) context; + ctx.curVal = firstUnmodifiedKey == NULL_ROW_KEY ? QueryConstants.NULL_INT : outputSource.getInt(firstUnmodifiedKey); } @Override - public void reprocessChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex) { + public void processChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk valuesChunk, + @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; - doAddChunk(ctx, inputKeys, valuesChunk); + doProcessChunk(ctx, inputKeys, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } // endregion - - // region No-Op Operations - - @Override - final public void modifyChunk(@NotNull final UpdateContext updateContext, - @Nullable final LongChunk prevKeyChunk, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk, - @NotNull final Chunk postValuesChunk) { - } - - @Override - final public void removeChunk(@NotNull final UpdateContext updateContext, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk) { - } - - @Override - final public void applyShift(@NotNull final UpdateContext updateContext, - @NotNull final RowSet prevIndex, - @NotNull final RowSetShiftData shifted) { - - } - // endregion } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java index 3b5ed756ca7..c08ca39f3ad 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java @@ -6,20 +6,15 @@ package io.deephaven.engine.table.impl.updateby.internal; import io.deephaven.chunk.Chunk; -import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.attributes.ChunkLengths; -import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.sized.SizedLongChunk; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.util.RowRedirection; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; import io.deephaven.util.QueryConstants; import org.jetbrains.annotations.NotNull; @@ -45,11 +40,6 @@ public abstract class BaseLongUpdateByOperator extends UpdateByCumulativeOperato protected class Context extends UpdateCumulativeContext { public final SizedSafeCloseable fillContext; public final SizedLongChunk outputValues; - public boolean canProcessDirectly; - public UpdateBy.UpdateType currentUpdateType; - - public RowSetBuilderSequential modifiedBuilder; - public RowSet newModified; public long curVal = QueryConstants.NULL_LONG; @@ -59,15 +49,10 @@ protected Context(final int chunkSize) { this.outputValues = new SizedLongChunk<>(chunkSize); } - public RowSetBuilderSequential getModifiedBuilder() { - if(modifiedBuilder == null) { - modifiedBuilder = RowSetFactory.builderSequential(); - } - return modifiedBuilder; - } - @Override public void close() { + super.close(); + outputValues.close(); fillContext.close(); } @@ -145,57 +130,6 @@ public UpdateContext makeUpdateContext(final int chunkSize) { return new Context(chunkSize); } - @Override - public void initializeForUpdate(@NotNull final UpdateContext context, - @NotNull final TableUpdate upstream, - @NotNull final RowSet resultSourceIndex, - final long lastPrevKey, - final boolean isUpstreamAppendOnly) { - final Context ctx = (Context) context; - - // If we're redirected we have to make sure we tell the output source it's actual size, or we're going - // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not - // need to do anything with capacity. - if(redirContext.isRedirected()) { - // The redirection index does not use the 0th index for some reason. - outputSource.ensureCapacity(redirContext.requiredCapacity()); - } - - // just remember the appendyness. - ctx.canProcessDirectly = isUpstreamAppendOnly; - - // pre-load the context with the previous last value in the table (if possible) - ctx.curVal = lastPrevKey == NULL_ROW_KEY ? QueryConstants.NULL_LONG : outputSource.getPrevLong(lastPrevKey); - } - - @Override - public void initializeFor(@NotNull final UpdateContext updateContext, - @NotNull final RowSet updateRowSet, - @NotNull final UpdateBy.UpdateType type) { - ((Context)updateContext).currentUpdateType = type; - } - - @Override - public void finishFor(@NotNull final UpdateContext updateContext, @NotNull final UpdateBy.UpdateType type) { - final Context ctx = (Context) updateContext; - if(type == UpdateBy.UpdateType.Reprocess) { - ctx.newModified = ctx.modifiedBuilder.build(); - } - } - - @Override - public boolean requiresKeys() { - return false; - } - - @Override - public boolean requiresValues(@NotNull final UpdateContext context) { - final Context ctx = (Context) context; - // We only need to read actual values if we can process them at this moment, so that means if we are - // either part of an append-only update, or within the reprocess cycle - return (ctx.currentUpdateType == UpdateBy.UpdateType.Add && ctx.canProcessDirectly) || ctx.currentUpdateType == UpdateBy.UpdateType.Reprocess; - } - @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); @@ -204,44 +138,16 @@ public void startTrackingPrev() { } } - @NotNull - @Override - final public RowSet getAdditionalModifications(@NotNull final UpdateContext ctx) { - return ((Context)ctx).newModified == null ? RowSetFactory.empty() : ((Context)ctx).newModified; - } - - @Override - final public boolean anyModified(@NotNull final UpdateContext ctx) { - return ((Context)ctx).newModified != null; - } - - @Override - public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { - return ((Context)context).canProcessDirectly; - } - - // region Addition - @Override - public void addChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk values) { - final Context ctx = (Context) updateContext; - if (ctx.canProcessDirectly) { - doAddChunk(ctx, inputKeys, values); - } - } - /** - * Add a chunk of values to the operator. + * Perform the processing for a chunk of values by the operator. * * @param ctx the context object * @param inputKeys the input keys for the chunk * @param workingChunk the chunk of values */ - protected abstract void doAddChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk); + protected abstract void doProcessChunk(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys, + @NotNull final Chunk workingChunk); // endregion @@ -258,49 +164,32 @@ public void applyOutputShift(@NotNull final UpdateContext context, // region Reprocessing - public void resetForReprocess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceIndex, - long firstUnmodifiedKey) { - final Context ctx = (Context) context; - if(!ctx.canProcessDirectly) { - ctx.curVal = firstUnmodifiedKey == NULL_ROW_KEY ? QueryConstants.NULL_LONG : outputSource.getLong(firstUnmodifiedKey); + public void resetForProcess(@NotNull final UpdateContext context, + @NotNull final RowSet sourceIndex, + long firstUnmodifiedKey) { + + // If we're redirected we have to make sure we tell the output source it's actual size, or we're going + // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not + // need to do anything with capacity. + if(redirContext.isRedirected()) { + // The redirection index does not use the 0th index for some reason. + outputSource.ensureCapacity(redirContext.requiredCapacity()); } + + final Context ctx = (Context) context; + ctx.curVal = firstUnmodifiedKey == NULL_ROW_KEY ? QueryConstants.NULL_LONG : outputSource.getLong(firstUnmodifiedKey); } @Override - public void reprocessChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex) { + public void processChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk valuesChunk, + @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; - doAddChunk(ctx, inputKeys, valuesChunk); + doProcessChunk(ctx, inputKeys, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } // endregion - - // region No-Op Operations - - @Override - final public void modifyChunk(@NotNull final UpdateContext updateContext, - @Nullable final LongChunk prevKeyChunk, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk, - @NotNull final Chunk postValuesChunk) { - } - - @Override - final public void removeChunk(@NotNull final UpdateContext updateContext, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk) { - } - - @Override - final public void applyShift(@NotNull final UpdateContext updateContext, - @NotNull final RowSet prevIndex, - @NotNull final RowSetShiftData shifted) { - - } - // endregion } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java index d89c2687244..d346892a370 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java @@ -26,7 +26,7 @@ public BaseObjectBinaryOperator(@NotNull final Class type, @Override - protected void doAddChunk(@NotNull final Context ctx, + protected void doProcessChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, @NotNull final Chunk workingChunk) { accumulate(workingChunk.asObjectChunk(), ctx, 0, inputKeys.intSize()); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java index 575f6c8a537..cfb20c79123 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java @@ -6,20 +6,15 @@ package io.deephaven.engine.table.impl.updateby.internal; import io.deephaven.chunk.Chunk; -import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.attributes.ChunkLengths; -import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.sized.SizedObjectChunk; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.util.RowRedirection; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; import io.deephaven.util.QueryConstants; import org.jetbrains.annotations.NotNull; @@ -46,11 +41,6 @@ public abstract class BaseObjectUpdateByOperator extends UpdateByCumulativeOp protected class Context extends UpdateCumulativeContext { public final SizedSafeCloseable fillContext; public final SizedObjectChunk outputValues; - public boolean canProcessDirectly; - public UpdateBy.UpdateType currentUpdateType; - - public RowSetBuilderSequential modifiedBuilder; - public RowSet newModified; public T curVal = null; @@ -60,15 +50,10 @@ protected Context(final int chunkSize) { this.outputValues = new SizedObjectChunk<>(chunkSize); } - public RowSetBuilderSequential getModifiedBuilder() { - if(modifiedBuilder == null) { - modifiedBuilder = RowSetFactory.builderSequential(); - } - return modifiedBuilder; - } - @Override public void close() { + super.close(); + outputValues.close(); fillContext.close(); } @@ -148,57 +133,6 @@ public UpdateContext makeUpdateContext(final int chunkSize) { return new Context(chunkSize); } - @Override - public void initializeForUpdate(@NotNull final UpdateContext context, - @NotNull final TableUpdate upstream, - @NotNull final RowSet resultSourceIndex, - final long lastPrevKey, - final boolean isUpstreamAppendOnly) { - final Context ctx = (Context) context; - - // If we're redirected we have to make sure we tell the output source it's actual size, or we're going - // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not - // need to do anything with capacity. - if(redirContext.isRedirected()) { - // The redirection index does not use the 0th index for some reason. - outputSource.ensureCapacity(redirContext.requiredCapacity()); - } - - // just remember the appendyness. - ctx.canProcessDirectly = isUpstreamAppendOnly; - - // pre-load the context with the previous last value in the table (if possible) - ctx.curVal = lastPrevKey == NULL_ROW_KEY ? null : outputSource.getPrev(lastPrevKey); - } - - @Override - public void initializeFor(@NotNull final UpdateContext updateContext, - @NotNull final RowSet updateRowSet, - @NotNull final UpdateBy.UpdateType type) { - ((Context)updateContext).currentUpdateType = type; - } - - @Override - public void finishFor(@NotNull final UpdateContext updateContext, @NotNull final UpdateBy.UpdateType type) { - final Context ctx = (Context) updateContext; - if(type == UpdateBy.UpdateType.Reprocess) { - ctx.newModified = ctx.modifiedBuilder.build(); - } - } - - @Override - public boolean requiresKeys() { - return false; - } - - @Override - public boolean requiresValues(@NotNull final UpdateContext context) { - final Context ctx = (Context) context; - // We only need to read actual values if we can process them at this moment, so that means if we are - // either part of an append-only update, or within the reprocess cycle - return (ctx.currentUpdateType == UpdateBy.UpdateType.Add && ctx.canProcessDirectly) || ctx.currentUpdateType == UpdateBy.UpdateType.Reprocess; - } - @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); @@ -207,44 +141,16 @@ public void startTrackingPrev() { } } - @NotNull - @Override - final public RowSet getAdditionalModifications(@NotNull final UpdateContext ctx) { - return ((Context)ctx).newModified == null ? RowSetFactory.empty() : ((Context)ctx).newModified; - } - - @Override - final public boolean anyModified(@NotNull final UpdateContext ctx) { - return ((Context)ctx).newModified != null; - } - - @Override - public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { - return ((Context)context).canProcessDirectly; - } - - // region Addition - @Override - public void addChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk values) { - final Context ctx = (Context) updateContext; - if (ctx.canProcessDirectly) { - doAddChunk(ctx, inputKeys, values); - } - } - /** - * Add a chunk of values to the operator. + * Perform the processing for a chunk of values by the operator. * * @param ctx the context object * @param inputKeys the input keys for the chunk * @param workingChunk the chunk of values */ - protected abstract void doAddChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk); + protected abstract void doProcessChunk(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys, + @NotNull final Chunk workingChunk); // endregion @@ -261,49 +167,32 @@ public void applyOutputShift(@NotNull final UpdateContext context, // region Reprocessing - public void resetForReprocess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceIndex, - long firstUnmodifiedKey) { - final Context ctx = (Context) context; - if(!ctx.canProcessDirectly) { - ctx.curVal = firstUnmodifiedKey == NULL_ROW_KEY ? null : outputSource.get(firstUnmodifiedKey); + public void resetForProcess(@NotNull final UpdateContext context, + @NotNull final RowSet sourceIndex, + long firstUnmodifiedKey) { + + // If we're redirected we have to make sure we tell the output source it's actual size, or we're going + // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not + // need to do anything with capacity. + if(redirContext.isRedirected()) { + // The redirection index does not use the 0th index for some reason. + outputSource.ensureCapacity(redirContext.requiredCapacity()); } + + final Context ctx = (Context) context; + ctx.curVal = firstUnmodifiedKey == NULL_ROW_KEY ? null : outputSource.get(firstUnmodifiedKey); } @Override - public void reprocessChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex) { + public void processChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk valuesChunk, + @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; - doAddChunk(ctx, inputKeys, valuesChunk); + doProcessChunk(ctx, inputKeys, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } // endregion - - // region No-Op Operations - - @Override - final public void modifyChunk(@NotNull final UpdateContext updateContext, - @Nullable final LongChunk prevKeyChunk, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk, - @NotNull final Chunk postValuesChunk) { - } - - @Override - final public void removeChunk(@NotNull final UpdateContext updateContext, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk) { - } - - @Override - final public void applyShift(@NotNull final UpdateContext updateContext, - @NotNull final RowSet prevIndex, - @NotNull final RowSetShiftData shifted) { - - } - // endregion } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java index 02cac6ce586..06d773c29dd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java @@ -6,20 +6,15 @@ package io.deephaven.engine.table.impl.updateby.internal; import io.deephaven.chunk.Chunk; -import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.attributes.ChunkLengths; -import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.sized.SizedShortChunk; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.util.RowRedirection; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; import io.deephaven.util.QueryConstants; import org.jetbrains.annotations.NotNull; @@ -45,11 +40,6 @@ public abstract class BaseShortUpdateByOperator extends UpdateByCumulativeOperat protected class Context extends UpdateCumulativeContext { public final SizedSafeCloseable fillContext; public final SizedShortChunk outputValues; - public boolean canProcessDirectly; - public UpdateBy.UpdateType currentUpdateType; - - public RowSetBuilderSequential modifiedBuilder; - public RowSet newModified; public short curVal = QueryConstants.NULL_SHORT; @@ -59,15 +49,10 @@ protected Context(final int chunkSize) { this.outputValues = new SizedShortChunk<>(chunkSize); } - public RowSetBuilderSequential getModifiedBuilder() { - if(modifiedBuilder == null) { - modifiedBuilder = RowSetFactory.builderSequential(); - } - return modifiedBuilder; - } - @Override public void close() { + super.close(); + outputValues.close(); fillContext.close(); } @@ -145,57 +130,6 @@ public UpdateContext makeUpdateContext(final int chunkSize) { return new Context(chunkSize); } - @Override - public void initializeForUpdate(@NotNull final UpdateContext context, - @NotNull final TableUpdate upstream, - @NotNull final RowSet resultSourceIndex, - final long lastPrevKey, - final boolean isUpstreamAppendOnly) { - final Context ctx = (Context) context; - - // If we're redirected we have to make sure we tell the output source it's actual size, or we're going - // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not - // need to do anything with capacity. - if(redirContext.isRedirected()) { - // The redirection index does not use the 0th index for some reason. - outputSource.ensureCapacity(redirContext.requiredCapacity()); - } - - // just remember the appendyness. - ctx.canProcessDirectly = isUpstreamAppendOnly; - - // pre-load the context with the previous last value in the table (if possible) - ctx.curVal = lastPrevKey == NULL_ROW_KEY ? QueryConstants.NULL_SHORT : outputSource.getPrevShort(lastPrevKey); - } - - @Override - public void initializeFor(@NotNull final UpdateContext updateContext, - @NotNull final RowSet updateRowSet, - @NotNull final UpdateBy.UpdateType type) { - ((Context)updateContext).currentUpdateType = type; - } - - @Override - public void finishFor(@NotNull final UpdateContext updateContext, @NotNull final UpdateBy.UpdateType type) { - final Context ctx = (Context) updateContext; - if(type == UpdateBy.UpdateType.Reprocess) { - ctx.newModified = ctx.modifiedBuilder.build(); - } - } - - @Override - public boolean requiresKeys() { - return false; - } - - @Override - public boolean requiresValues(@NotNull final UpdateContext context) { - final Context ctx = (Context) context; - // We only need to read actual values if we can process them at this moment, so that means if we are - // either part of an append-only update, or within the reprocess cycle - return (ctx.currentUpdateType == UpdateBy.UpdateType.Add && ctx.canProcessDirectly) || ctx.currentUpdateType == UpdateBy.UpdateType.Reprocess; - } - @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); @@ -204,44 +138,16 @@ public void startTrackingPrev() { } } - @NotNull - @Override - final public RowSet getAdditionalModifications(@NotNull final UpdateContext ctx) { - return ((Context)ctx).newModified == null ? RowSetFactory.empty() : ((Context)ctx).newModified; - } - - @Override - final public boolean anyModified(@NotNull final UpdateContext ctx) { - return ((Context)ctx).newModified != null; - } - - @Override - public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { - return ((Context)context).canProcessDirectly; - } - - // region Addition - @Override - public void addChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk values) { - final Context ctx = (Context) updateContext; - if (ctx.canProcessDirectly) { - doAddChunk(ctx, inputKeys, values); - } - } - /** - * Add a chunk of values to the operator. + * Perform the processing for a chunk of values by the operator. * * @param ctx the context object * @param inputKeys the input keys for the chunk * @param workingChunk the chunk of values */ - protected abstract void doAddChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk); + protected abstract void doProcessChunk(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys, + @NotNull final Chunk workingChunk); // endregion @@ -258,49 +164,32 @@ public void applyOutputShift(@NotNull final UpdateContext context, // region Reprocessing - public void resetForReprocess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceIndex, - long firstUnmodifiedKey) { - final Context ctx = (Context) context; - if(!ctx.canProcessDirectly) { - ctx.curVal = firstUnmodifiedKey == NULL_ROW_KEY ? QueryConstants.NULL_SHORT : outputSource.getShort(firstUnmodifiedKey); + public void resetForProcess(@NotNull final UpdateContext context, + @NotNull final RowSet sourceIndex, + long firstUnmodifiedKey) { + + // If we're redirected we have to make sure we tell the output source it's actual size, or we're going + // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not + // need to do anything with capacity. + if(redirContext.isRedirected()) { + // The redirection index does not use the 0th index for some reason. + outputSource.ensureCapacity(redirContext.requiredCapacity()); } + + final Context ctx = (Context) context; + ctx.curVal = firstUnmodifiedKey == NULL_ROW_KEY ? QueryConstants.NULL_SHORT : outputSource.getShort(firstUnmodifiedKey); } @Override - public void reprocessChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex) { + public void processChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk valuesChunk, + @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; - doAddChunk(ctx, inputKeys, valuesChunk); + doProcessChunk(ctx, inputKeys, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } // endregion - - // region No-Op Operations - - @Override - final public void modifyChunk(@NotNull final UpdateContext updateContext, - @Nullable final LongChunk prevKeyChunk, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk, - @NotNull final Chunk postValuesChunk) { - } - - @Override - final public void removeChunk(@NotNull final UpdateContext updateContext, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk) { - } - - @Override - final public void applyShift(@NotNull final UpdateContext updateContext, - @NotNull final RowSet prevIndex, - @NotNull final RowSetShiftData shifted) { - - } - // endregion } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java index 8b83e7afd7d..2fcb6311be5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java @@ -11,23 +11,15 @@ import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.api.updateby.OperationControl; -import io.deephaven.base.LongRingBuffer; 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.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; -import io.deephaven.engine.table.impl.updateby.rollingsum.ShortRollingSumOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -39,23 +31,8 @@ public abstract class BaseWindowedByteUpdateByOperator extends UpdateByWindowedO // endregion extra-fields protected class Context extends UpdateWindowedContext { - public boolean canProcessDirectly; - public WritableByteChunk candidateValuesChunk; - // other useful stuff - public UpdateBy.UpdateType currentUpdateType; - - public RowSetBuilderSequential modifiedBuilder; - public RowSet newModified; - - public RowSetBuilderSequential getModifiedBuilder() { - if(modifiedBuilder == null) { - modifiedBuilder = RowSetFactory.builderSequential(); - } - return modifiedBuilder; - } - @Override public void close() { super.close(); @@ -81,6 +58,7 @@ public BaseWindowedByteUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final LongRecordingUpdateByOperator timeRecorder, + @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, @@ -88,7 +66,7 @@ public BaseWindowedByteUpdateByOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); + super(pair, affectingColumns, control, timeRecorder, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor this.nullValue = getNullValue(); @@ -110,62 +88,6 @@ protected WritableColumnSource makeDenseSource() { } // endregion extra-methods - @Override - public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, final long lastPrevKey, boolean isUpstreamAppendOnly) { - final Context ctx = (Context) context; - ctx.workingRowSet = resultSourceRowSet; - - // we can only process directly from an update if the window is entire backward-looking. Since we - // allow negative values in fwd/rev ticks and timestamps, we need to check both - ctx.canProcessDirectly = isUpstreamAppendOnly && forwardTimeScaleUnits <= 0 && reverseTimeScaleUnits >= 0;; - } - - @Override - public void initializeFor(@NotNull final UpdateContext context, - @NotNull final RowSet updateRowSet, - @NotNull final UpdateBy.UpdateType type) { - final Context ctx = (Context) context; - ctx.currentUpdateType = type; - - if ((type == UpdateBy.UpdateType.Add && ctx.canProcessDirectly) - || type == UpdateBy.UpdateType.Reprocess) { - long windowStartKey = computeFirstAffectingKey(updateRowSet.firstRowKey(), ctx.workingRowSet); - ctx.loadWindowChunks(windowStartKey); - } - } - - @Override - public void finishFor(@NotNull final UpdateContext updateContext, @NotNull final UpdateBy.UpdateType type) { - final Context ctx = (Context) updateContext; - if(type == UpdateBy.UpdateType.Reprocess && ctx.modifiedBuilder != null) { - ctx.newModified = ctx.modifiedBuilder.build(); - } - } - - @Override - public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { - final Context ctx = (Context) context; - return ctx.canProcessDirectly; - } - - @Override - public boolean requiresValues(@NotNull final UpdateContext context) { - // windowed operators don't need current values supplied to them, they only care about windowed values which - // may or may not intersect with the column values - return false; - } - - @NotNull - @Override - final public RowSet getAdditionalModifications(@NotNull final UpdateContext ctx) { - return ((Context)ctx).newModified == null ? RowSetFactory.empty() : ((Context)ctx).newModified; - } - - @Override - final public boolean anyModified(@NotNull final UpdateContext ctx) { - return ((Context)ctx).newModified != null; - } - // region Addition /** * Add a chunk of values to the operator. @@ -174,68 +96,33 @@ final public boolean anyModified(@NotNull final UpdateContext ctx) { * @param inputKeys the input keys for the chunk * @param workingChunk the chunk of values */ - protected abstract void doAddChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk workingChunk); - - @Override - public void addChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk values) { - final Context ctx = (Context) updateContext; - if (ctx.canProcessDirectly) { - ctx.loadDataChunks(inputKeys); - doAddChunk(ctx, inputKeys, keyChunk, values); - } - } + protected abstract void doProcessChunk(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk workingChunk); // endregion // region Reprocessing - public void resetForReprocess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceRowSet, - long firstUnmodifiedKey) { + public void resetForProcess(@NotNull final UpdateContext context, + @NotNull final RowSet sourceRowSet, + long firstUnmodifiedKey) { final Context ctx = (Context) context; ctx.workingRowSet = sourceRowSet; } @Override - public void reprocessChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex) { + public void processChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk valuesChunk, + @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; ctx.loadDataChunks(inputKeys); - doAddChunk(ctx, inputKeys, keyChunk, valuesChunk); + doProcessChunk(ctx, inputKeys, keyChunk, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } // endregion - - // region No-Op Operations - - @Override - final public void modifyChunk(@NotNull final UpdateContext updateContext, - @Nullable final LongChunk prevKeyChunk, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk, - @NotNull final Chunk postValuesChunk) { - } - - @Override - final public void removeChunk(@NotNull final UpdateContext updateContext, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk) { - } - - @Override - final public void applyShift(@NotNull final UpdateContext updateContext, - @NotNull final RowSet prevIndex, - @NotNull final RowSetShiftData shifted) { - } - // endregion } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java index 0bae1612dbe..ba254497d2a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java @@ -1,23 +1,15 @@ package io.deephaven.engine.table.impl.updateby.internal; import io.deephaven.api.updateby.OperationControl; -import io.deephaven.base.LongRingBuffer; 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.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; -import io.deephaven.engine.table.impl.updateby.rollingsum.ShortRollingSumOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -28,23 +20,8 @@ public abstract class BaseWindowedCharUpdateByOperator extends UpdateByWindowedO // endregion extra-fields protected class Context extends UpdateWindowedContext { - public boolean canProcessDirectly; - public WritableCharChunk candidateValuesChunk; - // other useful stuff - public UpdateBy.UpdateType currentUpdateType; - - public RowSetBuilderSequential modifiedBuilder; - public RowSet newModified; - - public RowSetBuilderSequential getModifiedBuilder() { - if(modifiedBuilder == null) { - modifiedBuilder = RowSetFactory.builderSequential(); - } - return modifiedBuilder; - } - @Override public void close() { super.close(); @@ -70,6 +47,7 @@ public BaseWindowedCharUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final LongRecordingUpdateByOperator timeRecorder, + @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, @@ -77,7 +55,7 @@ public BaseWindowedCharUpdateByOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); + super(pair, affectingColumns, control, timeRecorder, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor // endregion constructor @@ -86,62 +64,6 @@ public BaseWindowedCharUpdateByOperator(@NotNull final MatchPair pair, // region extra-methods // endregion extra-methods - @Override - public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, final long lastPrevKey, boolean isUpstreamAppendOnly) { - final Context ctx = (Context) context; - ctx.workingRowSet = resultSourceRowSet; - - // we can only process directly from an update if the window is entire backward-looking. Since we - // allow negative values in fwd/rev ticks and timestamps, we need to check both - ctx.canProcessDirectly = isUpstreamAppendOnly && forwardTimeScaleUnits <= 0 && reverseTimeScaleUnits >= 0;; - } - - @Override - public void initializeFor(@NotNull final UpdateContext context, - @NotNull final RowSet updateRowSet, - @NotNull final UpdateBy.UpdateType type) { - final Context ctx = (Context) context; - ctx.currentUpdateType = type; - - if ((type == UpdateBy.UpdateType.Add && ctx.canProcessDirectly) - || type == UpdateBy.UpdateType.Reprocess) { - long windowStartKey = computeFirstAffectingKey(updateRowSet.firstRowKey(), ctx.workingRowSet); - ctx.loadWindowChunks(windowStartKey); - } - } - - @Override - public void finishFor(@NotNull final UpdateContext updateContext, @NotNull final UpdateBy.UpdateType type) { - final Context ctx = (Context) updateContext; - if(type == UpdateBy.UpdateType.Reprocess && ctx.modifiedBuilder != null) { - ctx.newModified = ctx.modifiedBuilder.build(); - } - } - - @Override - public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { - final Context ctx = (Context) context; - return ctx.canProcessDirectly; - } - - @Override - public boolean requiresValues(@NotNull final UpdateContext context) { - // windowed operators don't need current values supplied to them, they only care about windowed values which - // may or may not intersect with the column values - return false; - } - - @NotNull - @Override - final public RowSet getAdditionalModifications(@NotNull final UpdateContext ctx) { - return ((Context)ctx).newModified == null ? RowSetFactory.empty() : ((Context)ctx).newModified; - } - - @Override - final public boolean anyModified(@NotNull final UpdateContext ctx) { - return ((Context)ctx).newModified != null; - } - // region Addition /** * Add a chunk of values to the operator. @@ -150,68 +72,33 @@ final public boolean anyModified(@NotNull final UpdateContext ctx) { * @param inputKeys the input keys for the chunk * @param workingChunk the chunk of values */ - protected abstract void doAddChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk workingChunk); - - @Override - public void addChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk values) { - final Context ctx = (Context) updateContext; - if (ctx.canProcessDirectly) { - ctx.loadDataChunks(inputKeys); - doAddChunk(ctx, inputKeys, keyChunk, values); - } - } + protected abstract void doProcessChunk(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk workingChunk); // endregion // region Reprocessing - public void resetForReprocess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceRowSet, - long firstUnmodifiedKey) { + public void resetForProcess(@NotNull final UpdateContext context, + @NotNull final RowSet sourceRowSet, + long firstUnmodifiedKey) { final Context ctx = (Context) context; ctx.workingRowSet = sourceRowSet; } @Override - public void reprocessChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex) { + public void processChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk valuesChunk, + @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; ctx.loadDataChunks(inputKeys); - doAddChunk(ctx, inputKeys, keyChunk, valuesChunk); + doProcessChunk(ctx, inputKeys, keyChunk, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } // endregion - - // region No-Op Operations - - @Override - final public void modifyChunk(@NotNull final UpdateContext updateContext, - @Nullable final LongChunk prevKeyChunk, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk, - @NotNull final Chunk postValuesChunk) { - } - - @Override - final public void removeChunk(@NotNull final UpdateContext updateContext, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk) { - } - - @Override - final public void applyShift(@NotNull final UpdateContext updateContext, - @NotNull final RowSet prevIndex, - @NotNull final RowSetShiftData shifted) { - } - // endregion } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java index 6b387573b4d..e79efbf2719 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java @@ -6,21 +6,15 @@ package io.deephaven.engine.table.impl.updateby.internal; import io.deephaven.api.updateby.OperationControl; -import io.deephaven.base.LongRingBuffer; 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.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -31,23 +25,8 @@ public abstract class BaseWindowedDoubleUpdateByOperator extends UpdateByWindowe // endregion extra-fields protected class Context extends UpdateWindowedContext { - public boolean canProcessDirectly; - public WritableDoubleChunk candidateValuesChunk; - // other useful stuff - public UpdateBy.UpdateType currentUpdateType; - - public RowSetBuilderSequential modifiedBuilder; - public RowSet newModified; - - public RowSetBuilderSequential getModifiedBuilder() { - if(modifiedBuilder == null) { - modifiedBuilder = RowSetFactory.builderSequential(); - } - return modifiedBuilder; - } - @Override public void close() { super.close(); @@ -73,6 +52,7 @@ public BaseWindowedDoubleUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final LongRecordingUpdateByOperator timeRecorder, + @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, @@ -80,7 +60,7 @@ public BaseWindowedDoubleUpdateByOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); + super(pair, affectingColumns, control, timeRecorder, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor // endregion constructor @@ -89,44 +69,6 @@ public BaseWindowedDoubleUpdateByOperator(@NotNull final MatchPair pair, // region extra-methods // endregion extra-methods - @Override - public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, final long lastPrevKey, boolean isUpstreamAppendOnly) { - final Context ctx = (Context) context; - ctx.workingRowSet = resultSourceRowSet; - - // we can only process directly from an update if the window is entire backward-looking. Since we - // allow negative values in fwd/rev ticks and timestamps, we need to check both - ctx.canProcessDirectly = isUpstreamAppendOnly && forwardTimeScaleUnits <= 0 && reverseTimeScaleUnits >= 0;; - } - - @Override - public void initializeFor(@NotNull final UpdateContext context, - @NotNull final RowSet updateRowSet, - @NotNull final UpdateBy.UpdateType type) { - final Context ctx = (Context) context; - ctx.currentUpdateType = type; - - if ((type == UpdateBy.UpdateType.Add && ctx.canProcessDirectly) - || type == UpdateBy.UpdateType.Reprocess) { - long windowStartKey = computeFirstAffectingKey(updateRowSet.firstRowKey(), ctx.workingRowSet); - ctx.loadWindowChunks(windowStartKey); - } - } - - @Override - public void finishFor(@NotNull final UpdateContext updateContext, @NotNull final UpdateBy.UpdateType type) { - final Context ctx = (Context) updateContext; - if(type == UpdateBy.UpdateType.Reprocess && ctx.modifiedBuilder != null) { - ctx.newModified = ctx.modifiedBuilder.build(); - } - } - - @Override - public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { - final Context ctx = (Context) context; - return ctx.canProcessDirectly; - } - @Override public boolean requiresValues(@NotNull final UpdateContext context) { // windowed operators don't need current values supplied to them, they only care about windowed values which @@ -134,17 +76,6 @@ public boolean requiresValues(@NotNull final UpdateContext context) { return false; } - @NotNull - @Override - final public RowSet getAdditionalModifications(@NotNull final UpdateContext ctx) { - return ((Context)ctx).newModified == null ? RowSetFactory.empty() : ((Context)ctx).newModified; - } - - @Override - final public boolean anyModified(@NotNull final UpdateContext ctx) { - return ((Context)ctx).newModified != null; - } - // region Addition /** * Add a chunk of values to the operator. @@ -153,68 +84,33 @@ final public boolean anyModified(@NotNull final UpdateContext ctx) { * @param inputKeys the input keys for the chunk * @param workingChunk the chunk of values */ - protected abstract void doAddChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk workingChunk); - - @Override - public void addChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk values) { - final Context ctx = (Context) updateContext; - if (ctx.canProcessDirectly) { - ctx.loadDataChunks(inputKeys); - doAddChunk(ctx, inputKeys, keyChunk, values); - } - } + protected abstract void doProcessChunk(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk workingChunk); // endregion // region Reprocessing - public void resetForReprocess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceRowSet, - long firstUnmodifiedKey) { + public void resetForProcess(@NotNull final UpdateContext context, + @NotNull final RowSet sourceRowSet, + long firstUnmodifiedKey) { final Context ctx = (Context) context; ctx.workingRowSet = sourceRowSet; } @Override - public void reprocessChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex) { + public void processChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk valuesChunk, + @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; ctx.loadDataChunks(inputKeys); - doAddChunk(ctx, inputKeys, keyChunk, valuesChunk); + doProcessChunk(ctx, inputKeys, keyChunk, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } // endregion - - // region No-Op Operations - - @Override - final public void modifyChunk(@NotNull final UpdateContext updateContext, - @Nullable final LongChunk prevKeyChunk, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk, - @NotNull final Chunk postValuesChunk) { - } - - @Override - final public void removeChunk(@NotNull final UpdateContext updateContext, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk) { - } - - @Override - final public void applyShift(@NotNull final UpdateContext updateContext, - @NotNull final RowSet prevIndex, - @NotNull final RowSetShiftData shifted) { - } - // endregion } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java index ed4246fe873..c148984e0ce 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java @@ -1,21 +1,15 @@ package io.deephaven.engine.table.impl.updateby.internal; import io.deephaven.api.updateby.OperationControl; -import io.deephaven.base.LongRingBuffer; 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.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -26,23 +20,8 @@ public abstract class BaseWindowedFloatUpdateByOperator extends UpdateByWindowed // endregion extra-fields protected class Context extends UpdateWindowedContext { - public boolean canProcessDirectly; - public WritableFloatChunk candidateValuesChunk; - // other useful stuff - public UpdateBy.UpdateType currentUpdateType; - - public RowSetBuilderSequential modifiedBuilder; - public RowSet newModified; - - public RowSetBuilderSequential getModifiedBuilder() { - if(modifiedBuilder == null) { - modifiedBuilder = RowSetFactory.builderSequential(); - } - return modifiedBuilder; - } - @Override public void close() { super.close(); @@ -68,6 +47,7 @@ public BaseWindowedFloatUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final LongRecordingUpdateByOperator timeRecorder, + @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, @@ -75,7 +55,7 @@ public BaseWindowedFloatUpdateByOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); + super(pair, affectingColumns, control, timeRecorder, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor // endregion constructor @@ -84,44 +64,6 @@ public BaseWindowedFloatUpdateByOperator(@NotNull final MatchPair pair, // region extra-methods // endregion extra-methods - @Override - public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, final long lastPrevKey, boolean isUpstreamAppendOnly) { - final Context ctx = (Context) context; - ctx.workingRowSet = resultSourceRowSet; - - // we can only process directly from an update if the window is entire backward-looking. Since we - // allow negative values in fwd/rev ticks and timestamps, we need to check both - ctx.canProcessDirectly = isUpstreamAppendOnly && forwardTimeScaleUnits <= 0 && reverseTimeScaleUnits >= 0;; - } - - @Override - public void initializeFor(@NotNull final UpdateContext context, - @NotNull final RowSet updateRowSet, - @NotNull final UpdateBy.UpdateType type) { - final Context ctx = (Context) context; - ctx.currentUpdateType = type; - - if ((type == UpdateBy.UpdateType.Add && ctx.canProcessDirectly) - || type == UpdateBy.UpdateType.Reprocess) { - long windowStartKey = computeFirstAffectingKey(updateRowSet.firstRowKey(), ctx.workingRowSet); - ctx.loadWindowChunks(windowStartKey); - } - } - - @Override - public void finishFor(@NotNull final UpdateContext updateContext, @NotNull final UpdateBy.UpdateType type) { - final Context ctx = (Context) updateContext; - if(type == UpdateBy.UpdateType.Reprocess && ctx.modifiedBuilder != null) { - ctx.newModified = ctx.modifiedBuilder.build(); - } - } - - @Override - public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { - final Context ctx = (Context) context; - return ctx.canProcessDirectly; - } - @Override public boolean requiresValues(@NotNull final UpdateContext context) { // windowed operators don't need current values supplied to them, they only care about windowed values which @@ -129,17 +71,6 @@ public boolean requiresValues(@NotNull final UpdateContext context) { return false; } - @NotNull - @Override - final public RowSet getAdditionalModifications(@NotNull final UpdateContext ctx) { - return ((Context)ctx).newModified == null ? RowSetFactory.empty() : ((Context)ctx).newModified; - } - - @Override - final public boolean anyModified(@NotNull final UpdateContext ctx) { - return ((Context)ctx).newModified != null; - } - // region Addition /** * Add a chunk of values to the operator. @@ -148,68 +79,33 @@ final public boolean anyModified(@NotNull final UpdateContext ctx) { * @param inputKeys the input keys for the chunk * @param workingChunk the chunk of values */ - protected abstract void doAddChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk workingChunk); - - @Override - public void addChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk values) { - final Context ctx = (Context) updateContext; - if (ctx.canProcessDirectly) { - ctx.loadDataChunks(inputKeys); - doAddChunk(ctx, inputKeys, keyChunk, values); - } - } + protected abstract void doProcessChunk(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk workingChunk); // endregion // region Reprocessing - public void resetForReprocess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceRowSet, - long firstUnmodifiedKey) { + public void resetForProcess(@NotNull final UpdateContext context, + @NotNull final RowSet sourceRowSet, + long firstUnmodifiedKey) { final Context ctx = (Context) context; ctx.workingRowSet = sourceRowSet; } @Override - public void reprocessChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex) { + public void processChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk valuesChunk, + @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; ctx.loadDataChunks(inputKeys); - doAddChunk(ctx, inputKeys, keyChunk, valuesChunk); + doProcessChunk(ctx, inputKeys, keyChunk, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } // endregion - - // region No-Op Operations - - @Override - final public void modifyChunk(@NotNull final UpdateContext updateContext, - @Nullable final LongChunk prevKeyChunk, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk, - @NotNull final Chunk postValuesChunk) { - } - - @Override - final public void removeChunk(@NotNull final UpdateContext updateContext, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk) { - } - - @Override - final public void applyShift(@NotNull final UpdateContext updateContext, - @NotNull final RowSet prevIndex, - @NotNull final RowSetShiftData shifted) { - } - // endregion } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java index 563a7442189..110a35a53ca 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java @@ -6,23 +6,15 @@ package io.deephaven.engine.table.impl.updateby.internal; import io.deephaven.api.updateby.OperationControl; -import io.deephaven.base.LongRingBuffer; 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.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; -import io.deephaven.engine.table.impl.updateby.rollingsum.ShortRollingSumOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -33,23 +25,8 @@ public abstract class BaseWindowedIntUpdateByOperator extends UpdateByWindowedOp // endregion extra-fields protected class Context extends UpdateWindowedContext { - public boolean canProcessDirectly; - public WritableIntChunk candidateValuesChunk; - // other useful stuff - public UpdateBy.UpdateType currentUpdateType; - - public RowSetBuilderSequential modifiedBuilder; - public RowSet newModified; - - public RowSetBuilderSequential getModifiedBuilder() { - if(modifiedBuilder == null) { - modifiedBuilder = RowSetFactory.builderSequential(); - } - return modifiedBuilder; - } - @Override public void close() { super.close(); @@ -75,6 +52,7 @@ public BaseWindowedIntUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final LongRecordingUpdateByOperator timeRecorder, + @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, @@ -82,7 +60,7 @@ public BaseWindowedIntUpdateByOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); + super(pair, affectingColumns, control, timeRecorder, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor // endregion constructor @@ -91,62 +69,6 @@ public BaseWindowedIntUpdateByOperator(@NotNull final MatchPair pair, // region extra-methods // endregion extra-methods - @Override - public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, final long lastPrevKey, boolean isUpstreamAppendOnly) { - final Context ctx = (Context) context; - ctx.workingRowSet = resultSourceRowSet; - - // we can only process directly from an update if the window is entire backward-looking. Since we - // allow negative values in fwd/rev ticks and timestamps, we need to check both - ctx.canProcessDirectly = isUpstreamAppendOnly && forwardTimeScaleUnits <= 0 && reverseTimeScaleUnits >= 0;; - } - - @Override - public void initializeFor(@NotNull final UpdateContext context, - @NotNull final RowSet updateRowSet, - @NotNull final UpdateBy.UpdateType type) { - final Context ctx = (Context) context; - ctx.currentUpdateType = type; - - if ((type == UpdateBy.UpdateType.Add && ctx.canProcessDirectly) - || type == UpdateBy.UpdateType.Reprocess) { - long windowStartKey = computeFirstAffectingKey(updateRowSet.firstRowKey(), ctx.workingRowSet); - ctx.loadWindowChunks(windowStartKey); - } - } - - @Override - public void finishFor(@NotNull final UpdateContext updateContext, @NotNull final UpdateBy.UpdateType type) { - final Context ctx = (Context) updateContext; - if(type == UpdateBy.UpdateType.Reprocess && ctx.modifiedBuilder != null) { - ctx.newModified = ctx.modifiedBuilder.build(); - } - } - - @Override - public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { - final Context ctx = (Context) context; - return ctx.canProcessDirectly; - } - - @Override - public boolean requiresValues(@NotNull final UpdateContext context) { - // windowed operators don't need current values supplied to them, they only care about windowed values which - // may or may not intersect with the column values - return false; - } - - @NotNull - @Override - final public RowSet getAdditionalModifications(@NotNull final UpdateContext ctx) { - return ((Context)ctx).newModified == null ? RowSetFactory.empty() : ((Context)ctx).newModified; - } - - @Override - final public boolean anyModified(@NotNull final UpdateContext ctx) { - return ((Context)ctx).newModified != null; - } - // region Addition /** * Add a chunk of values to the operator. @@ -155,68 +77,33 @@ final public boolean anyModified(@NotNull final UpdateContext ctx) { * @param inputKeys the input keys for the chunk * @param workingChunk the chunk of values */ - protected abstract void doAddChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk workingChunk); - - @Override - public void addChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk values) { - final Context ctx = (Context) updateContext; - if (ctx.canProcessDirectly) { - ctx.loadDataChunks(inputKeys); - doAddChunk(ctx, inputKeys, keyChunk, values); - } - } + protected abstract void doProcessChunk(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk workingChunk); // endregion // region Reprocessing - public void resetForReprocess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceRowSet, - long firstUnmodifiedKey) { + public void resetForProcess(@NotNull final UpdateContext context, + @NotNull final RowSet sourceRowSet, + long firstUnmodifiedKey) { final Context ctx = (Context) context; ctx.workingRowSet = sourceRowSet; } @Override - public void reprocessChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex) { + public void processChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk valuesChunk, + @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; ctx.loadDataChunks(inputKeys); - doAddChunk(ctx, inputKeys, keyChunk, valuesChunk); + doProcessChunk(ctx, inputKeys, keyChunk, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } // endregion - - // region No-Op Operations - - @Override - final public void modifyChunk(@NotNull final UpdateContext updateContext, - @Nullable final LongChunk prevKeyChunk, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk, - @NotNull final Chunk postValuesChunk) { - } - - @Override - final public void removeChunk(@NotNull final UpdateContext updateContext, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk) { - } - - @Override - final public void applyShift(@NotNull final UpdateContext updateContext, - @NotNull final RowSet prevIndex, - @NotNull final RowSetShiftData shifted) { - } - // endregion } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java index c779cbf50ae..6317af36f92 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java @@ -6,23 +6,15 @@ package io.deephaven.engine.table.impl.updateby.internal; import io.deephaven.api.updateby.OperationControl; -import io.deephaven.base.LongRingBuffer; 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.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; -import io.deephaven.engine.table.impl.updateby.rollingsum.ShortRollingSumOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -33,23 +25,8 @@ public abstract class BaseWindowedLongUpdateByOperator extends UpdateByWindowedO // endregion extra-fields protected class Context extends UpdateWindowedContext { - public boolean canProcessDirectly; - public WritableLongChunk candidateValuesChunk; - // other useful stuff - public UpdateBy.UpdateType currentUpdateType; - - public RowSetBuilderSequential modifiedBuilder; - public RowSet newModified; - - public RowSetBuilderSequential getModifiedBuilder() { - if(modifiedBuilder == null) { - modifiedBuilder = RowSetFactory.builderSequential(); - } - return modifiedBuilder; - } - @Override public void close() { super.close(); @@ -75,6 +52,7 @@ public BaseWindowedLongUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final LongRecordingUpdateByOperator timeRecorder, + @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, @@ -82,7 +60,7 @@ public BaseWindowedLongUpdateByOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); + super(pair, affectingColumns, control, timeRecorder, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor // endregion constructor @@ -91,62 +69,6 @@ public BaseWindowedLongUpdateByOperator(@NotNull final MatchPair pair, // region extra-methods // endregion extra-methods - @Override - public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, final long lastPrevKey, boolean isUpstreamAppendOnly) { - final Context ctx = (Context) context; - ctx.workingRowSet = resultSourceRowSet; - - // we can only process directly from an update if the window is entire backward-looking. Since we - // allow negative values in fwd/rev ticks and timestamps, we need to check both - ctx.canProcessDirectly = isUpstreamAppendOnly && forwardTimeScaleUnits <= 0 && reverseTimeScaleUnits >= 0;; - } - - @Override - public void initializeFor(@NotNull final UpdateContext context, - @NotNull final RowSet updateRowSet, - @NotNull final UpdateBy.UpdateType type) { - final Context ctx = (Context) context; - ctx.currentUpdateType = type; - - if ((type == UpdateBy.UpdateType.Add && ctx.canProcessDirectly) - || type == UpdateBy.UpdateType.Reprocess) { - long windowStartKey = computeFirstAffectingKey(updateRowSet.firstRowKey(), ctx.workingRowSet); - ctx.loadWindowChunks(windowStartKey); - } - } - - @Override - public void finishFor(@NotNull final UpdateContext updateContext, @NotNull final UpdateBy.UpdateType type) { - final Context ctx = (Context) updateContext; - if(type == UpdateBy.UpdateType.Reprocess && ctx.modifiedBuilder != null) { - ctx.newModified = ctx.modifiedBuilder.build(); - } - } - - @Override - public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { - final Context ctx = (Context) context; - return ctx.canProcessDirectly; - } - - @Override - public boolean requiresValues(@NotNull final UpdateContext context) { - // windowed operators don't need current values supplied to them, they only care about windowed values which - // may or may not intersect with the column values - return false; - } - - @NotNull - @Override - final public RowSet getAdditionalModifications(@NotNull final UpdateContext ctx) { - return ((Context)ctx).newModified == null ? RowSetFactory.empty() : ((Context)ctx).newModified; - } - - @Override - final public boolean anyModified(@NotNull final UpdateContext ctx) { - return ((Context)ctx).newModified != null; - } - // region Addition /** * Add a chunk of values to the operator. @@ -155,68 +77,33 @@ final public boolean anyModified(@NotNull final UpdateContext ctx) { * @param inputKeys the input keys for the chunk * @param workingChunk the chunk of values */ - protected abstract void doAddChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk workingChunk); - - @Override - public void addChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk values) { - final Context ctx = (Context) updateContext; - if (ctx.canProcessDirectly) { - ctx.loadDataChunks(inputKeys); - doAddChunk(ctx, inputKeys, keyChunk, values); - } - } + protected abstract void doProcessChunk(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk workingChunk); // endregion // region Reprocessing - public void resetForReprocess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceRowSet, - long firstUnmodifiedKey) { + public void resetForProcess(@NotNull final UpdateContext context, + @NotNull final RowSet sourceRowSet, + long firstUnmodifiedKey) { final Context ctx = (Context) context; ctx.workingRowSet = sourceRowSet; } @Override - public void reprocessChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex) { + public void processChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk valuesChunk, + @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; ctx.loadDataChunks(inputKeys); - doAddChunk(ctx, inputKeys, keyChunk, valuesChunk); + doProcessChunk(ctx, inputKeys, keyChunk, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } // endregion - - // region No-Op Operations - - @Override - final public void modifyChunk(@NotNull final UpdateContext updateContext, - @Nullable final LongChunk prevKeyChunk, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk, - @NotNull final Chunk postValuesChunk) { - } - - @Override - final public void removeChunk(@NotNull final UpdateContext updateContext, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk) { - } - - @Override - final public void applyShift(@NotNull final UpdateContext updateContext, - @NotNull final RowSet prevIndex, - @NotNull final RowSetShiftData shifted) { - } - // endregion } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java index a19b4a7cc75..fc5d2699cbc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java @@ -6,23 +6,15 @@ package io.deephaven.engine.table.impl.updateby.internal; import io.deephaven.api.updateby.OperationControl; -import io.deephaven.base.LongRingBuffer; 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.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; -import io.deephaven.engine.table.impl.updateby.rollingsum.ShortRollingSumOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -34,23 +26,8 @@ public abstract class BaseWindowedObjectUpdateByOperator extends UpdateByWind // endregion extra-fields protected class Context extends UpdateWindowedContext { - public boolean canProcessDirectly; - public WritableObjectChunk candidateValuesChunk; - // other useful stuff - public UpdateBy.UpdateType currentUpdateType; - - public RowSetBuilderSequential modifiedBuilder; - public RowSet newModified; - - public RowSetBuilderSequential getModifiedBuilder() { - if(modifiedBuilder == null) { - modifiedBuilder = RowSetFactory.builderSequential(); - } - return modifiedBuilder; - } - @Override public void close() { super.close(); @@ -76,6 +53,7 @@ public BaseWindowedObjectUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final LongRecordingUpdateByOperator timeRecorder, + @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, @@ -84,7 +62,7 @@ public BaseWindowedObjectUpdateByOperator(@NotNull final MatchPair pair, , final Class colType // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); + super(pair, affectingColumns, control, timeRecorder, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor this.colType = colType; @@ -94,62 +72,6 @@ public BaseWindowedObjectUpdateByOperator(@NotNull final MatchPair pair, // region extra-methods // endregion extra-methods - @Override - public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, final long lastPrevKey, boolean isUpstreamAppendOnly) { - final Context ctx = (Context) context; - ctx.workingRowSet = resultSourceRowSet; - - // we can only process directly from an update if the window is entire backward-looking. Since we - // allow negative values in fwd/rev ticks and timestamps, we need to check both - ctx.canProcessDirectly = isUpstreamAppendOnly && forwardTimeScaleUnits <= 0 && reverseTimeScaleUnits >= 0;; - } - - @Override - public void initializeFor(@NotNull final UpdateContext context, - @NotNull final RowSet updateRowSet, - @NotNull final UpdateBy.UpdateType type) { - final Context ctx = (Context) context; - ctx.currentUpdateType = type; - - if ((type == UpdateBy.UpdateType.Add && ctx.canProcessDirectly) - || type == UpdateBy.UpdateType.Reprocess) { - long windowStartKey = computeFirstAffectingKey(updateRowSet.firstRowKey(), ctx.workingRowSet); - ctx.loadWindowChunks(windowStartKey); - } - } - - @Override - public void finishFor(@NotNull final UpdateContext updateContext, @NotNull final UpdateBy.UpdateType type) { - final Context ctx = (Context) updateContext; - if(type == UpdateBy.UpdateType.Reprocess && ctx.modifiedBuilder != null) { - ctx.newModified = ctx.modifiedBuilder.build(); - } - } - - @Override - public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { - final Context ctx = (Context) context; - return ctx.canProcessDirectly; - } - - @Override - public boolean requiresValues(@NotNull final UpdateContext context) { - // windowed operators don't need current values supplied to them, they only care about windowed values which - // may or may not intersect with the column values - return false; - } - - @NotNull - @Override - final public RowSet getAdditionalModifications(@NotNull final UpdateContext ctx) { - return ((Context)ctx).newModified == null ? RowSetFactory.empty() : ((Context)ctx).newModified; - } - - @Override - final public boolean anyModified(@NotNull final UpdateContext ctx) { - return ((Context)ctx).newModified != null; - } - // region Addition /** * Add a chunk of values to the operator. @@ -158,68 +80,33 @@ final public boolean anyModified(@NotNull final UpdateContext ctx) { * @param inputKeys the input keys for the chunk * @param workingChunk the chunk of values */ - protected abstract void doAddChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk workingChunk); - - @Override - public void addChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk values) { - final Context ctx = (Context) updateContext; - if (ctx.canProcessDirectly) { - ctx.loadDataChunks(inputKeys); - doAddChunk(ctx, inputKeys, keyChunk, values); - } - } + protected abstract void doProcessChunk(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk workingChunk); // endregion // region Reprocessing - public void resetForReprocess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceRowSet, - long firstUnmodifiedKey) { + public void resetForProcess(@NotNull final UpdateContext context, + @NotNull final RowSet sourceRowSet, + long firstUnmodifiedKey) { final Context ctx = (Context) context; ctx.workingRowSet = sourceRowSet; } @Override - public void reprocessChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex) { + public void processChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk valuesChunk, + @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; ctx.loadDataChunks(inputKeys); - doAddChunk(ctx, inputKeys, keyChunk, valuesChunk); + doProcessChunk(ctx, inputKeys, keyChunk, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } // endregion - - // region No-Op Operations - - @Override - final public void modifyChunk(@NotNull final UpdateContext updateContext, - @Nullable final LongChunk prevKeyChunk, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk, - @NotNull final Chunk postValuesChunk) { - } - - @Override - final public void removeChunk(@NotNull final UpdateContext updateContext, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk) { - } - - @Override - final public void applyShift(@NotNull final UpdateContext updateContext, - @NotNull final RowSet prevIndex, - @NotNull final RowSetShiftData shifted) { - } - // endregion } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java index a86b769c2a6..3cbee310841 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java @@ -6,23 +6,15 @@ package io.deephaven.engine.table.impl.updateby.internal; import io.deephaven.api.updateby.OperationControl; -import io.deephaven.base.LongRingBuffer; 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.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; -import io.deephaven.engine.table.impl.updateby.rollingsum.ShortRollingSumOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -33,23 +25,8 @@ public abstract class BaseWindowedShortUpdateByOperator extends UpdateByWindowed // endregion extra-fields protected class Context extends UpdateWindowedContext { - public boolean canProcessDirectly; - public WritableShortChunk candidateValuesChunk; - // other useful stuff - public UpdateBy.UpdateType currentUpdateType; - - public RowSetBuilderSequential modifiedBuilder; - public RowSet newModified; - - public RowSetBuilderSequential getModifiedBuilder() { - if(modifiedBuilder == null) { - modifiedBuilder = RowSetFactory.builderSequential(); - } - return modifiedBuilder; - } - @Override public void close() { super.close(); @@ -75,6 +52,7 @@ public BaseWindowedShortUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final LongRecordingUpdateByOperator timeRecorder, + @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, @@ -82,7 +60,7 @@ public BaseWindowedShortUpdateByOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); + super(pair, affectingColumns, control, timeRecorder, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor // endregion constructor @@ -91,62 +69,6 @@ public BaseWindowedShortUpdateByOperator(@NotNull final MatchPair pair, // region extra-methods // endregion extra-methods - @Override - public void initializeForUpdate(@NotNull UpdateContext context, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceRowSet, final long lastPrevKey, boolean isUpstreamAppendOnly) { - final Context ctx = (Context) context; - ctx.workingRowSet = resultSourceRowSet; - - // we can only process directly from an update if the window is entire backward-looking. Since we - // allow negative values in fwd/rev ticks and timestamps, we need to check both - ctx.canProcessDirectly = isUpstreamAppendOnly && forwardTimeScaleUnits <= 0 && reverseTimeScaleUnits >= 0;; - } - - @Override - public void initializeFor(@NotNull final UpdateContext context, - @NotNull final RowSet updateRowSet, - @NotNull final UpdateBy.UpdateType type) { - final Context ctx = (Context) context; - ctx.currentUpdateType = type; - - if ((type == UpdateBy.UpdateType.Add && ctx.canProcessDirectly) - || type == UpdateBy.UpdateType.Reprocess) { - long windowStartKey = computeFirstAffectingKey(updateRowSet.firstRowKey(), ctx.workingRowSet); - ctx.loadWindowChunks(windowStartKey); - } - } - - @Override - public void finishFor(@NotNull final UpdateContext updateContext, @NotNull final UpdateBy.UpdateType type) { - final Context ctx = (Context) updateContext; - if(type == UpdateBy.UpdateType.Reprocess && ctx.modifiedBuilder != null) { - ctx.newModified = ctx.modifiedBuilder.build(); - } - } - - @Override - public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { - final Context ctx = (Context) context; - return ctx.canProcessDirectly; - } - - @Override - public boolean requiresValues(@NotNull final UpdateContext context) { - // windowed operators don't need current values supplied to them, they only care about windowed values which - // may or may not intersect with the column values - return false; - } - - @NotNull - @Override - final public RowSet getAdditionalModifications(@NotNull final UpdateContext ctx) { - return ((Context)ctx).newModified == null ? RowSetFactory.empty() : ((Context)ctx).newModified; - } - - @Override - final public boolean anyModified(@NotNull final UpdateContext ctx) { - return ((Context)ctx).newModified != null; - } - // region Addition /** * Add a chunk of values to the operator. @@ -155,68 +77,33 @@ final public boolean anyModified(@NotNull final UpdateContext ctx) { * @param inputKeys the input keys for the chunk * @param workingChunk the chunk of values */ - protected abstract void doAddChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk workingChunk); - - @Override - public void addChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk values) { - final Context ctx = (Context) updateContext; - if (ctx.canProcessDirectly) { - ctx.loadDataChunks(inputKeys); - doAddChunk(ctx, inputKeys, keyChunk, values); - } - } + protected abstract void doProcessChunk(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk workingChunk); // endregion // region Reprocessing - public void resetForReprocess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceRowSet, - long firstUnmodifiedKey) { + public void resetForProcess(@NotNull final UpdateContext context, + @NotNull final RowSet sourceRowSet, + long firstUnmodifiedKey) { final Context ctx = (Context) context; ctx.workingRowSet = sourceRowSet; } @Override - public void reprocessChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex) { + public void processChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk valuesChunk, + @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; ctx.loadDataChunks(inputKeys); - doAddChunk(ctx, inputKeys, keyChunk, valuesChunk); + doProcessChunk(ctx, inputKeys, keyChunk, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } // endregion - - // region No-Op Operations - - @Override - final public void modifyChunk(@NotNull final UpdateContext updateContext, - @Nullable final LongChunk prevKeyChunk, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk, - @NotNull final Chunk postValuesChunk) { - } - - @Override - final public void removeChunk(@NotNull final UpdateContext updateContext, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk prevValuesChunk) { - } - - @Override - final public void applyShift(@NotNull final UpdateContext updateContext, - @NotNull final RowSet prevIndex, - @NotNull final RowSetShiftData shifted) { - } - // endregion } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/LongRecordingUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/LongRecordingUpdateByOperator.java index 7cf53ee9369..fab250c7cea 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/LongRecordingUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/LongRecordingUpdateByOperator.java @@ -19,6 +19,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.UpdateByOperator; +import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.sources.ReinterpretUtils; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -93,7 +94,7 @@ public ColumnSource getColumnSource() { } @Override - public void reprocessChunk(@NotNull UpdateContext updateContext, @NotNull RowSequence inputKeys, @Nullable LongChunk keyChunk, @NotNull Chunk valuesChunk, @NotNull RowSet postUpdateSourceIndex) { + public void processChunk(@NotNull UpdateContext updateContext, @NotNull RowSequence inputKeys, @Nullable LongChunk keyChunk, @NotNull Chunk valuesChunk, @NotNull RowSet postUpdateSourceIndex) { currentContext.addedChunk = valuesChunk.asLongChunk(); } @@ -103,14 +104,6 @@ public UpdateContext makeUpdateContext(int chunkSize) { return this.currentContext = new RecordingContext(); } - @Override - public void addChunk(@NotNull UpdateContext updateContext, - @NotNull RowSequence inputKeys, - @Nullable LongChunk keyChunk, - @NotNull Chunk values) { - currentContext.addedChunk = values.asLongChunk(); - } - // region Unused methods @NotNull @@ -137,29 +130,6 @@ public Map> getOutputColumns() { return Collections.emptyMap(); } - @Override - public void initializeForUpdate(@NotNull UpdateContext ctx, @NotNull TableUpdate upstream, @NotNull RowSet resultSourceIndex, final long key, boolean isUpstreamAppendOnly) { - } - - @Override - public void initializeFor(@NotNull UpdateContext updateContext, @NotNull RowSet updateRowSet, @NotNull UpdateBy.UpdateType type) { - } - - @Override - public void finishFor(@NotNull UpdateContext updateContext, @NotNull UpdateBy.UpdateType type) { - } - - @NotNull - @Override - public RowSet getAdditionalModifications(@NotNull UpdateContext ctx) { - return RowSetFactory.empty(); - } - - @Override - public boolean anyModified(@NotNull UpdateContext ctx) { - return false; - } - @Override public void startTrackingPrev() { } @@ -174,39 +144,16 @@ public boolean requiresValues(@NotNull UpdateContext ctx) { return true; } - @Override - public boolean canProcessNormalUpdate(@NotNull UpdateContext context) { - return false; - } - @Override public void setChunkSize(@NotNull UpdateContext context, int chunkSize) { - - } - - @Override - public void modifyChunk(@NotNull UpdateContext updateContext, @Nullable LongChunk prevKeyChunk, @Nullable LongChunk keyChunk, @NotNull Chunk prevValuesChunk, @NotNull Chunk postValuesChunk) { - - } - - @Override - public void removeChunk(@NotNull UpdateContext updateContext, @Nullable LongChunk keyChunk, @NotNull Chunk prevValuesChunk) { - - } - - @Override - public void applyShift(@NotNull UpdateContext updateContext, @NotNull RowSet prevIndex, @NotNull RowSetShiftData shifted) { - } @Override public void applyOutputShift(@NotNull UpdateContext context, @NotNull RowSet subIndexToShift, long delta) { - } @Override - public void resetForReprocess(@NotNull UpdateContext context, @NotNull RowSet sourceIndex, long firstUnmodifiedKey) { - + public void resetForProcess(@NotNull UpdateContext context, @NotNull RowSet sourceIndex, long firstUnmodifiedKey) { } // endregion diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java index 0e3f2a3f4f3..5fc850e5bf9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java @@ -34,7 +34,7 @@ public ByteCumMinMaxOperator(@NotNull final MatchPair inputPair, ) { super(inputPair, new String[] { inputPair.rightColumn }, redirContext); this.isMax = isMax; - // region constructor + // region constructorÏ // endregion constructor } @@ -42,7 +42,7 @@ public ByteCumMinMaxOperator(@NotNull final MatchPair inputPair, // endregion extra-methods @Override - protected void doAddChunk(@NotNull final Context ctx, + protected void doProcessChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, @NotNull final Chunk workingChunk) { accumulate(workingChunk.asByteChunk(), ctx, 0, workingChunk.size()); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java index 6d419ecbef1..ceaeeb1e338 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java @@ -36,7 +36,7 @@ public DoubleCumMinMaxOperator(@NotNull final MatchPair inputPair, } @Override - protected void doAddChunk(@NotNull final Context ctx, + protected void doProcessChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, @NotNull final Chunk workingChunk) { if(Double.isNaN(ctx.curVal) || Double.isInfinite(ctx.curVal)) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java index 0bea3eaf1bf..6c1d5dbf2f7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java @@ -31,7 +31,7 @@ public FloatCumMinMaxOperator(@NotNull final MatchPair inputPair, } @Override - protected void doAddChunk(@NotNull final Context ctx, + protected void doProcessChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, @NotNull final Chunk workingChunk) { if(Float.isNaN(ctx.curVal) || Float.isInfinite(ctx.curVal)) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java index 68825f2ef43..2ae3b10136b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java @@ -34,7 +34,7 @@ public IntCumMinMaxOperator(@NotNull final MatchPair inputPair, ) { super(inputPair, new String[] { inputPair.rightColumn }, redirContext); this.isMax = isMax; - // region constructor + // region constructorÏ // endregion constructor } @@ -42,7 +42,7 @@ public IntCumMinMaxOperator(@NotNull final MatchPair inputPair, // endregion extra-methods @Override - protected void doAddChunk(@NotNull final Context ctx, + protected void doProcessChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, @NotNull final Chunk workingChunk) { accumulate(workingChunk.asIntChunk(), ctx, 0, workingChunk.size()); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java index 1fa97c895b2..05c98ff630c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java @@ -43,7 +43,7 @@ public LongCumMinMaxOperator(@NotNull final MatchPair inputPair, ) { super(inputPair, new String[] { inputPair.rightColumn }, redirContext); this.isMax = isMax; - // region constructor + // region constructorÏ this.type = type; // endregion constructor } @@ -63,7 +63,7 @@ public Map> getOutputColumns() { // endregion extra-methods @Override - protected void doAddChunk(@NotNull final Context ctx, + protected void doProcessChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, @NotNull final Chunk workingChunk) { accumulate(workingChunk.asLongChunk(), ctx, 0, workingChunk.size()); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java index 5889301bf87..83b1a9f1fe3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java @@ -29,7 +29,7 @@ public ShortCumMinMaxOperator(@NotNull final MatchPair inputPair, ) { super(inputPair, new String[] { inputPair.rightColumn }, redirContext); this.isMax = isMax; - // region constructor + // region constructorÏ // endregion constructor } @@ -37,7 +37,7 @@ public ShortCumMinMaxOperator(@NotNull final MatchPair inputPair, // endregion extra-methods @Override - protected void doAddChunk(@NotNull final Context ctx, + protected void doProcessChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, @NotNull final Chunk workingChunk) { accumulate(workingChunk.asShortChunk(), ctx, 0, workingChunk.size()); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java index 776d3a8418b..d202ee5741e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java @@ -34,7 +34,7 @@ public ByteCumProdOperator(final @NotNull MatchPair inputPair, } @Override - protected void doAddChunk(@NotNull final Context ctx, + protected void doProcessChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, @NotNull final Chunk workingChunk) { // ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_LONG; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java index 6b4d9005c6b..3892429924d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java @@ -32,7 +32,7 @@ public DoubleCumProdOperator(@NotNull final MatchPair inputPair, // endregion constructor } - protected void doAddChunk(@NotNull final Context ctx, + protected void doProcessChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, @NotNull final Chunk workingChunk) { if(Double.isNaN(ctx.curVal)) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java index 0038459136c..6d21b12c9aa 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java @@ -27,7 +27,7 @@ public FloatCumProdOperator(@NotNull final MatchPair inputPair, // endregion constructor } - protected void doAddChunk(@NotNull final Context ctx, + protected void doProcessChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, @NotNull final Chunk workingChunk) { if(Float.isNaN(ctx.curVal)) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java index ad45109ccd7..6fd417057f1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java @@ -34,7 +34,7 @@ public IntCumProdOperator(final @NotNull MatchPair inputPair, } @Override - protected void doAddChunk(@NotNull final Context ctx, + protected void doProcessChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, @NotNull final Chunk workingChunk) { // ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_LONG; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java index c34ac70609b..ad00bde1bc1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java @@ -34,7 +34,7 @@ public LongCumProdOperator(final @NotNull MatchPair inputPair, } @Override - protected void doAddChunk(@NotNull final Context ctx, + protected void doProcessChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, @NotNull final Chunk workingChunk) { // ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_LONG; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java index 6a8be7fb265..f18852c3ab4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java @@ -29,7 +29,7 @@ public ShortCumProdOperator(final @NotNull MatchPair inputPair, } @Override - protected void doAddChunk(@NotNull final Context ctx, + protected void doProcessChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, @NotNull final Chunk workingChunk) { // ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_LONG; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java index 4281faa82bd..3fb6cf89d7a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java @@ -47,7 +47,6 @@ public class ByteRollingSumOperator extends BaseWindowedByteUpdateByOperator { protected class Context extends BaseWindowedByteUpdateByOperator.Context { public final SizedSafeCloseable fillContext; public final SizedLongChunk outputValues; - public UpdateBy.UpdateType currentUpdateType; public LinkedList windowValues = new LinkedList<>(); @@ -83,6 +82,7 @@ public ByteRollingSumOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final LongRecordingUpdateByOperator recorder, + @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final ColumnSource valueSource, @@ -91,7 +91,7 @@ public ByteRollingSumOperator(@NotNull final MatchPair pair, ,final byte nullValue // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, recorder, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource); + super(pair, affectingColumns, control, recorder, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource); if(redirContext.isRedirected()) { // region create-dense this.maybeInnerSource = new LongArraySource(); @@ -145,7 +145,7 @@ public void reset(UpdateContext context) { } @Override - public void doAddChunk(@NotNull final BaseWindowedByteUpdateByOperator.Context context, + public void doProcessChunk(@NotNull final BaseWindowedByteUpdateByOperator.Context context, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, @NotNull final Chunk workingChunk) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java index d90995dc550..0890f8aac50 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java @@ -51,7 +51,6 @@ public class DoubleRollingSumOperator extends BaseWindowedDoubleUpdateByOperator protected class Context extends BaseWindowedDoubleUpdateByOperator.Context { public final SizedSafeCloseable fillContext; public final SizedDoubleChunk outputValues; - public UpdateBy.UpdateType currentUpdateType; public LinkedList windowValues = new LinkedList<>(); @@ -85,6 +84,7 @@ public DoubleRollingSumOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final LongRecordingUpdateByOperator recorder, + @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final ColumnSource valueSource, @@ -92,7 +92,7 @@ public DoubleRollingSumOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, recorder, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource); + super(pair, affectingColumns, control, recorder, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource); if(redirContext.isRedirected()) { // region create-dense this.maybeInnerSource = new DoubleArraySource(); @@ -128,10 +128,10 @@ public void reset(UpdateContext context) { } @Override - public void doAddChunk(@NotNull final BaseWindowedDoubleUpdateByOperator.Context context, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk workingChunk) { + public void doProcessChunk(@NotNull final BaseWindowedDoubleUpdateByOperator.Context context, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk workingChunk) { final Context ctx = (Context) context; computeTicks(ctx, 0, inputKeys.intSize()); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java index 31381623dbf..faf477cfd82 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java @@ -46,7 +46,6 @@ public class FloatRollingSumOperator extends BaseWindowedFloatUpdateByOperator { protected class Context extends BaseWindowedFloatUpdateByOperator.Context { public final SizedSafeCloseable fillContext; public final SizedFloatChunk outputValues; - public UpdateBy.UpdateType currentUpdateType; public LinkedList windowValues = new LinkedList<>(); @@ -80,6 +79,7 @@ public FloatRollingSumOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final LongRecordingUpdateByOperator recorder, + @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final ColumnSource valueSource, @@ -87,7 +87,7 @@ public FloatRollingSumOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, recorder, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource); + super(pair, affectingColumns, control, recorder, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource); if(redirContext.isRedirected()) { // region create-dense this.maybeInnerSource = new FloatArraySource(); @@ -123,10 +123,10 @@ public void reset(UpdateContext context) { } @Override - public void doAddChunk(@NotNull final BaseWindowedFloatUpdateByOperator.Context context, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @NotNull final Chunk workingChunk) { + public void doProcessChunk(@NotNull final BaseWindowedFloatUpdateByOperator.Context context, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @NotNull final Chunk workingChunk) { final Context ctx = (Context) context; computeTicks(ctx, 0, inputKeys.intSize()); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java index 7b9922823ca..4ba2014db94 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java @@ -46,7 +46,6 @@ public class IntRollingSumOperator extends BaseWindowedIntUpdateByOperator { protected class Context extends BaseWindowedIntUpdateByOperator.Context { public final SizedSafeCloseable fillContext; public final SizedLongChunk outputValues; - public UpdateBy.UpdateType currentUpdateType; public LinkedList windowValues = new LinkedList<>(); @@ -82,6 +81,7 @@ public IntRollingSumOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final LongRecordingUpdateByOperator recorder, + @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final ColumnSource valueSource, @@ -89,7 +89,7 @@ public IntRollingSumOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, recorder, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource); + super(pair, affectingColumns, control, recorder, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource); if(redirContext.isRedirected()) { // region create-dense this.maybeInnerSource = new LongArraySource(); @@ -142,7 +142,7 @@ public void reset(UpdateContext context) { } @Override - public void doAddChunk(@NotNull final BaseWindowedIntUpdateByOperator.Context context, + public void doProcessChunk(@NotNull final BaseWindowedIntUpdateByOperator.Context context, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, @NotNull final Chunk workingChunk) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java index ce7e56f18b3..ecbe2fb28f9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java @@ -46,7 +46,6 @@ public class LongRollingSumOperator extends BaseWindowedLongUpdateByOperator { protected class Context extends BaseWindowedLongUpdateByOperator.Context { public final SizedSafeCloseable fillContext; public final SizedLongChunk outputValues; - public UpdateBy.UpdateType currentUpdateType; public LinkedList windowValues = new LinkedList<>(); @@ -82,6 +81,7 @@ public LongRollingSumOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final LongRecordingUpdateByOperator recorder, + @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final ColumnSource valueSource, @@ -89,7 +89,7 @@ public LongRollingSumOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, recorder, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource); + super(pair, affectingColumns, control, recorder, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource); if(redirContext.isRedirected()) { // region create-dense this.maybeInnerSource = new LongArraySource(); @@ -142,7 +142,7 @@ public void reset(UpdateContext context) { } @Override - public void doAddChunk(@NotNull final BaseWindowedLongUpdateByOperator.Context context, + public void doProcessChunk(@NotNull final BaseWindowedLongUpdateByOperator.Context context, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, @NotNull final Chunk workingChunk) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java index dc8e06624e8..0deb7e2057e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java @@ -41,7 +41,6 @@ public class ShortRollingSumOperator extends BaseWindowedShortUpdateByOperator { protected class Context extends BaseWindowedShortUpdateByOperator.Context { public final SizedSafeCloseable fillContext; public final SizedLongChunk outputValues; - public UpdateBy.UpdateType currentUpdateType; public LinkedList windowValues = new LinkedList<>(); @@ -77,6 +76,7 @@ public ShortRollingSumOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final LongRecordingUpdateByOperator recorder, + @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final ColumnSource valueSource, @@ -84,7 +84,7 @@ public ShortRollingSumOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, recorder, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource); + super(pair, affectingColumns, control, recorder, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource); if(redirContext.isRedirected()) { // region create-dense this.maybeInnerSource = new LongArraySource(); @@ -137,7 +137,7 @@ public void reset(UpdateContext context) { } @Override - public void doAddChunk(@NotNull final BaseWindowedShortUpdateByOperator.Context context, + public void doProcessChunk(@NotNull final BaseWindowedShortUpdateByOperator.Context context, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, @NotNull final Chunk workingChunk) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java index ac3ef78e1a1..96ad3af7571 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java @@ -39,7 +39,7 @@ public ByteCumSumOperator(@NotNull final MatchPair pair, } @Override - protected void doAddChunk(@NotNull final Context ctx, + protected void doProcessChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, @NotNull final Chunk workingChunk) { // ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_LONG; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java index e3155959b0e..23a5a104981 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java @@ -32,7 +32,7 @@ public DoubleCumSumOperator(@NotNull final MatchPair pair, // endregion constructor } - protected void doAddChunk(@NotNull final Context ctx, + protected void doProcessChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, @NotNull final Chunk workingChunk) { if(Double.isNaN(ctx.curVal)) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java index 1ae2bd53126..321ab2c1805 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java @@ -27,7 +27,7 @@ public FloatCumSumOperator(@NotNull final MatchPair pair, // endregion constructor } - protected void doAddChunk(@NotNull final Context ctx, + protected void doProcessChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, @NotNull final Chunk workingChunk) { if(Float.isNaN(ctx.curVal)) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java index 4857bf2c000..f7512763054 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java @@ -36,7 +36,7 @@ public IntCumSumOperator(@NotNull final MatchPair pair, } @Override - protected void doAddChunk(@NotNull final Context ctx, + protected void doProcessChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, @NotNull final Chunk workingChunk) { // ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_LONG; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java index 5ece480b675..a279b4edea2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java @@ -36,7 +36,7 @@ public LongCumSumOperator(@NotNull final MatchPair pair, } @Override - protected void doAddChunk(@NotNull final Context ctx, + protected void doProcessChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, @NotNull final Chunk workingChunk) { // ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_LONG; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java index 59134295ee7..542bc776f1d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java @@ -31,7 +31,7 @@ public ShortCumSumOperator(@NotNull final MatchPair pair, } @Override - protected void doAddChunk(@NotNull final Context ctx, + protected void doProcessChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, @NotNull final Chunk workingChunk) { // ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_LONG; diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java index e83694e5e0c..933dabe9a04 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java @@ -84,12 +84,12 @@ public void testStaticZeroKeyFwdRevWindow() { @Test public void testNullOnBucketChange() throws IOException { - final TableWithDefaults t = testTable(stringCol("Sym", "A", "A", "B", "B"), + final TableDefaults t = testTable(stringCol("Sym", "A", "A", "B", "B"), byteCol("ByteVal", (byte) 1, (byte) 2, NULL_BYTE, (byte) 3), shortCol("ShortVal", (short) 1, (short) 2, NULL_SHORT, (short) 3), intCol("IntVal", 1, 2, NULL_INT, 3)); - final TableWithDefaults expected = testTable(stringCol("Sym", "A", "A", "B", "B"), + final TableDefaults expected = testTable(stringCol("Sym", "A", "A", "B", "B"), byteCol("ByteVal", (byte) 1, (byte) 2, NULL_BYTE, (byte) 3), shortCol("ShortVal", (short) 1, (short) 2, NULL_SHORT, (short) 3), intCol("IntVal", 1, 2, NULL_INT, 3), From d434d2168292fb96c0737a1ae583f6f1ed5bf503 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Mon, 29 Aug 2022 15:45:13 -0700 Subject: [PATCH 014/123] bugfix --- .../deephaven/engine/table/impl/UpdateByCumulativeOperator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java index fb3f4cc5e85..64697faec11 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java @@ -69,7 +69,7 @@ final public RowSet getAdditionalModifications(@NotNull final UpdateContext cont @Override final public boolean anyModified(@NotNull final UpdateContext context) { UpdateCumulativeContext ctx = (UpdateCumulativeContext)context; - return ctx.newModified.isNonempty(); + return ctx.newModified != null && ctx.newModified.isNonempty(); } /*** nearly all cumulative operators will not reference a timestamp column, exceptions are Ema */ From d075b50e3ce6c43d43003eedaa209e41411688de Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Mon, 29 Aug 2022 16:01:31 -0700 Subject: [PATCH 015/123] WIP, COB --- .../engine/table/impl/UpdateByWindowedOperator.java | 10 +--------- 1 file changed, 1 insertion(+), 9 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java index 0aee493f745..3ef61f9c8e1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java @@ -75,14 +75,6 @@ public RowSetBuilderSequential getModifiedBuilder() { public RowSet determineAffectedRows(@NotNull final TableUpdate upstream, @NotNull final RowSet source, final boolean upstreamAppendOnly) { - // under certain circumstances, we can process directly and do not need to reprocess - if (upstreamAppendOnly && forwardTimeScaleUnits <= 0 && reverseTimeScaleUnits >= 0) { - try (final RowSet ignored = affectedRows) { - affectedRows = RowSetFactory.empty(); - } - return affectedRows; - } - // NOTE: this is fast rather than bounding to the smallest set possible. Will result in computing more than // actually necessary @@ -368,7 +360,7 @@ final public RowSet getAdditionalModifications(@NotNull final UpdateContext cont @Override final public boolean anyModified(@NotNull final UpdateContext context) { UpdateWindowedContext ctx = (UpdateWindowedContext)context; - return ctx.newModified.isNonempty(); + return ctx.newModified != null && ctx.newModified.isNonempty(); } @NotNull From 97212860a6a597336864e72eaeba79d24a019872 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 30 Aug 2022 12:27:32 -0700 Subject: [PATCH 016/123] Tests passing, time to optimize --- .../impl/UpdateByCumulativeOperator.java | 3 ++- .../engine/table/impl/UpdateByOperator.java | 3 ++- .../table/impl/UpdateByWindowedOperator.java | 20 +++++++++++++------ .../table/impl/updateby/TestRollingSum.java | 4 +++- 4 files changed, 21 insertions(+), 9 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java index 64697faec11..abb3dda2ccb 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java @@ -3,6 +3,7 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetBuilderSequential; import io.deephaven.engine.rowset.RowSetFactory; +import io.deephaven.engine.rowset.TrackingRowSet; import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.impl.updateby.internal.BaseCharUpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedByteUpdateByOperator; @@ -18,7 +19,7 @@ public abstract class UpdateCumulativeContext implements UpdateContext { // store the current subset of rows that need computation protected RowSet affectedRows; - public RowSet determineAffectedRows(@NotNull final TableUpdate upstream, @NotNull final RowSet source, final boolean upstreamAppendOnly) { + public RowSet determineAffectedRows(@NotNull final TableUpdate upstream, @NotNull final TrackingRowSet source, final boolean upstreamAppendOnly) { long smallestModifiedKey = UpdateByOperator.smallestAffectedKey(upstream.added(), upstream.modified(), upstream.removed(), upstream.shifted(), source); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java index 606ca6f3449..35e31b984d1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java @@ -8,6 +8,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetShiftData; +import io.deephaven.engine.rowset.TrackingRowSet; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.Table; @@ -171,7 +172,7 @@ interface UpdateContext extends SafeCloseable { * @param upstream the update * @param source the rowset of the parent table (affected rows will be a subset) */ - RowSet determineAffectedRows(@NotNull final TableUpdate upstream, @NotNull final RowSet source, + RowSet determineAffectedRows(@NotNull final TableUpdate upstream, @NotNull final TrackingRowSet source, final boolean upstreamAppendOnly); /** diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java index 3ef61f9c8e1..8af86e62120 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java @@ -6,6 +6,8 @@ import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.rowset.impl.singlerange.IntStartIntDeltaSingleRange; +import io.deephaven.engine.rowset.impl.singlerange.SingleRange; import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.TableUpdate; @@ -72,7 +74,7 @@ public RowSetBuilderSequential getModifiedBuilder() { * @param source the rowset of the parent table (affected rows will be a subset) * @param upstreamAppendOnly */ - public RowSet determineAffectedRows(@NotNull final TableUpdate upstream, @NotNull final RowSet source, + public RowSet determineAffectedRows(@NotNull final TableUpdate upstream, @NotNull final TrackingRowSet source, final boolean upstreamAppendOnly) { // NOTE: this is fast rather than bounding to the smallest set possible. Will result in computing more than @@ -83,12 +85,18 @@ public RowSet determineAffectedRows(@NotNull final TableUpdate upstream, @NotNul RowSetBuilderRandom builder = RowSetFactory.builderRandom(); if (upstream.removed().isNonempty()) { - // need removes in post-shift space to determine rows to recompute - try (final WritableRowSet shiftedRemoves = upstream.removed().copy()) { - upstream.shifted().apply(shiftedRemoves); + // need rows affected by removes in pre-shift space to determine rows to recompute + try (final RowSet prevSource = source.copyPrev()) { - builder.addRange(computeFirstAffectedKey(shiftedRemoves.firstRowKey(), source), - computeLastAffectedKey(shiftedRemoves.lastRowKey(), source)); + long s = computeFirstAffectedKey(upstream.removed().firstRowKey(), prevSource); + long e = computeLastAffectedKey(upstream.removed().lastRowKey(), prevSource); + + try (final WritableRowSet tmp = RowSetFactory.fromRange(s,e)) { + // apply shifts to get this back to post-shift + upstream.shifted().apply(tmp); + + builder.addRowSet(tmp); + } } } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java index 933dabe9a04..99ef2d656c3 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java @@ -226,7 +226,9 @@ protected Table e() { @Test public void testBucketedGeneralTicking() { - final CreateResult result = createTestTable(10000, true, false, true, 0x31313131); + final CreateResult result = createTestTable(100, true, false, true, 0x31313131); + +// final CreateResult result = createTestTable(10000, true, false, true, 0x31313131); final QueryTable t = result.t; final EvalNugget[] nuggets = new EvalNugget[] { From 7e15bc47ed71a8b277bb45bbe2bcfddf90f00542 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 31 Aug 2022 13:51:39 -0700 Subject: [PATCH 017/123] WIP --- .../table/impl/UpdateByWindowedOperator.java | 220 +++++++----------- .../BaseWindowedByteUpdateByOperator.java | 12 +- .../BaseWindowedCharUpdateByOperator.java | 12 +- .../BaseWindowedDoubleUpdateByOperator.java | 12 +- .../BaseWindowedFloatUpdateByOperator.java | 12 +- .../BaseWindowedIntUpdateByOperator.java | 12 +- .../BaseWindowedLongUpdateByOperator.java | 12 +- .../BaseWindowedObjectUpdateByOperator.java | 12 +- .../BaseWindowedShortUpdateByOperator.java | 12 +- .../rollingsum/ByteRollingSumOperator.java | 36 ++- .../rollingsum/DoubleRollingSumOperator.java | 30 ++- .../rollingsum/FloatRollingSumOperator.java | 30 ++- .../rollingsum/IntRollingSumOperator.java | 36 ++- .../rollingsum/LongRollingSumOperator.java | 36 ++- .../rollingsum/ShortRollingSumOperator.java | 36 ++- .../table/impl/updateby/TestRollingSum.java | 1 + .../deephaven/engine/liveness/Liveness.java | 2 +- 17 files changed, 280 insertions(+), 243 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java index 8af86e62120..3b9d9a2c688 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java @@ -3,19 +3,16 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.base.LongRingBuffer; import io.deephaven.chunk.WritableLongChunk; -import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.RowKeys; -import io.deephaven.engine.rowset.impl.singlerange.IntStartIntDeltaSingleRange; -import io.deephaven.engine.rowset.impl.singlerange.SingleRange; -import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.TableUpdate; -import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedCharUpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; + public abstract class UpdateByWindowedOperator implements UpdateByOperator { protected final OperationControl control; protected final LongRecordingUpdateByOperator recorder; @@ -39,31 +36,33 @@ public RowSetBuilderSequential getModifiedBuilder() { return modifiedBuilder; } - // store the current subset of rows that need computation - protected RowSet affectedRows = RowSetFactory.empty(); - - // candidate data for the window - public final int WINDOW_CHUNK_SIZE = 1024; - - // data that is actually in the current window - public LongRingBuffer windowRowKeys = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); - - // the selector that determines whether this value should be in the window, positions for tick-based and - // timestamps for time-based operators - public LongRingBuffer windowSelector = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); + // store a local copy of the source rowset (may not be needed) + public RowSet sourceRowSet = null; - public RowSequence.Iterator windowIterator = null; + // there are two sets of rows we will be tracking. `affected` rows need to be recomputed because of this + // update and `influencer` rows contain the data that will be used to compute the new values for the `affected` + // items. Because the windows are user-configurable, there may be no overlap between these two sets and we + // don't need values for the `affected` rows at all + protected RowSet affectedRows; + protected RowSet influencerKeys; + protected long currentInfluencerKey; - public RowSet workingRowSet = null; + // candidate data for the window + public final int WINDOW_CHUNK_SIZE = 4096; - public WritableLongChunk candidateRowKeysChunk; - public WritableLongChunk candidatePositionsChunk; - public WritableLongChunk candidateTimestampsChunk; + // persist two iterators, for the head and the tail of the current window + protected RowSet.Iterator influencerPosIterator; + protected RowSet.Iterator influencerKeyIterator; - // position data for the chunk being currently processed - public WritableLongChunk valuePositionChunk; + // for use with a ticking window + protected RowSet affectedRowPositions; + protected RowSet influencerPositions; + protected long currentInfluencerPos; + protected int currentInfluencerIndex; - public int candidateWindowIndex = 0; + protected LongRingBuffer windowKeys = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); + protected LongRingBuffer windowPos = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); + protected LongRingBuffer windowIndices = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); /*** * This function is only correct if the proper {@code source} rowset is provided. If using buckets, then the @@ -116,9 +115,27 @@ public RowSet determineAffectedRows(@NotNull final TableUpdate upstream, @NotNul } try (final RowSet ignored = affectedRows; + final RowSet ignored2 = influencerKeys; + final RowSet ignored3 = affectedRowPositions; + final RowSet ignored4 = influencerPositions; final RowSet brs = builder.build()) { + affectedRows = source.intersect(brs); + + WritableRowSet tmp = RowSetFactory.fromRange( + computeFirstAffectingKey(affectedRows.firstRowKey(), source), + computeLastAffectingKey(affectedRows.lastRowKey(), source) + ); + tmp.retain(source); + influencerKeys = tmp; + + // generate position data rowsets for efficiently computed position offsets + if (timestampColumnName == null) { + affectedRowPositions = source.invert(affectedRows); + influencerPositions = source.invert(influencerKeys); + } } + return affectedRows; } @@ -126,124 +143,50 @@ public RowSet getAffectedRows() { return affectedRows; } - public abstract void loadCandidateValueChunk(RowSequence windowRowSequence); - - /*** - * Fill the working chunks with data for this key - * - * @param startKey the key for which we want to - */ - public void loadWindowChunks(final long startKey) { - // TODO: make sure this works for bucketed - if (windowIterator == null) { - windowIterator = workingRowSet.getRowSequenceIterator(); - } - windowIterator.advance(startKey); - - RowSequence windowRowSequence = windowIterator.getNextRowSequenceWithLength(WINDOW_CHUNK_SIZE); - - loadCandidateValueChunk(windowRowSequence); - - // fill the window keys chunk - if (candidateRowKeysChunk == null) { - candidateRowKeysChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); - } - windowRowSequence.fillRowKeyChunk(candidateRowKeysChunk); - - if (recorder == null) { - // get position data for the window items (relative to the table or bucket rowset) - if (candidatePositionsChunk == null) { - candidatePositionsChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); - } - - // TODO: gotta be a better way than creating two rowsets - try (final RowSet rs = windowRowSequence.asRowSet(); - final RowSet positions = workingRowSet.invert(rs)) { - positions.fillRowKeyChunk(candidatePositionsChunk); - } - } else { - // get timestamp values from the recorder column source - if (candidateTimestampsChunk == null) { - candidateTimestampsChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); - } - try (final ChunkSource.FillContext fc = recorder.getColumnSource().makeFillContext(WINDOW_CHUNK_SIZE)) { - recorder.getColumnSource().fillChunk(fc, candidateTimestampsChunk, windowRowSequence); - } - } - - // reset the index to beginning of the chunks - candidateWindowIndex = 0; - } - - /*** - * Fill the working chunks with data for this key - * - * @param inputKeys the keys for which we want to get position or timestamp values - */ - public void loadDataChunks(final RowSequence inputKeys) { - if (recorder != null) { - // timestamp data will be available from the recorder - return; - } - - if (valuePositionChunk == null) { - valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); - } else if (valuePositionChunk.capacity() < inputKeys.size()) { - valuePositionChunk.close(); - valuePositionChunk = WritableLongChunk.makeWritableChunk(inputKeys.intSize()); - } - - // produce position data for the window (will be timestamps for time-based) - // TODO: gotta be a better way than creating two rowsets - try (final RowSet rs = inputKeys.asRowSet(); - final RowSet positions = workingRowSet.invert(rs)) { - positions.fillRowKeyChunk(valuePositionChunk); - } - } + public abstract void loadInfluencerValueChunk(); public void fillWindowTicks(UpdateContext context, long currentPos) { - // compute the head and tail (inclusive) - final long tail = Math.max(0, currentPos - reverseTimeScaleUnits + 1); - final long head = Math.min(workingRowSet.size() - 1, currentPos + forwardTimeScaleUnits); - - while (windowSelector.peek(Long.MAX_VALUE) < tail) { - final long pos = windowSelector.remove(); - final long key = windowRowKeys.remove(); - - pop(context, key); + // compute the head and tail positions (inclusive) + final long head = Math.max(0, currentPos - reverseTimeScaleUnits + 1); + final long tail = Math.min(sourceRowSet.size() - 1, currentPos + forwardTimeScaleUnits); + + // pop out all values from the current window that are not in the new window + while (!windowPos.isEmpty() && windowPos.front() < head) { + pop(context, windowKeys.remove(), (int)windowIndices.remove()); + windowPos.remove(); } - - // look at the window data and push until satisfied or at the end of the rowset - while (candidatePositionsChunk.size() > 0 && candidatePositionsChunk.get(candidateWindowIndex) <= head) { - final long pos = candidatePositionsChunk.get(candidateWindowIndex); - final long key = candidateRowKeysChunk.get(candidateWindowIndex); - - push(context, key, candidateWindowIndex); - - windowSelector.add(pos); - windowRowKeys.add(key); - - if (++candidateWindowIndex >= candidatePositionsChunk.size()) { - // load the next chunk in order - loadWindowChunks(key + 1); + // check our current values to see if it now matches the window + while(currentInfluencerPos <= tail) { + push(context, currentInfluencerKey, currentInfluencerIndex); + windowKeys.add(currentInfluencerKey); + windowPos.add(currentInfluencerPos); + windowIndices.add(currentInfluencerIndex); + currentInfluencerIndex++; + + if (influencerPosIterator.hasNext()) { + currentInfluencerKey = influencerKeyIterator.next(); + currentInfluencerPos = influencerPosIterator.next(); + } else { + currentInfluencerPos = Long.MAX_VALUE; + currentInfluencerKey = Long.MAX_VALUE; } } - if (windowSelector.isEmpty()) { + if (windowPos.isEmpty()) { reset(context); } } @Override public void close() { - try (final RowSequence.Iterator ignored1 = windowIterator; - final WritableLongChunk ignored2 = candidateRowKeysChunk; - final WritableLongChunk ignored3 = candidatePositionsChunk; - final WritableLongChunk ignored4 = candidateTimestampsChunk; - final WritableLongChunk ignored5 = valuePositionChunk; - final RowSet ignored6 = affectedRows; - final RowSet ignored7 = newModified) { + try (final RowSet.Iterator ignoredIt1 = influencerPosIterator; + final RowSet.Iterator ignoredIt2 = influencerKeyIterator; + final RowSet ignoredRs1 = affectedRows; + final RowSet ignoredRs2 = influencerKeys; + final RowSet ignoredRs3 = affectedRowPositions; + final RowSet ignoredRs4 = influencerPositions; + final RowSet ignoredRs5 = newModified) { } } } @@ -276,8 +219,8 @@ public UpdateByWindowedOperator(@NotNull final MatchPair pair, this.redirContext = redirContext; } - public abstract void push(UpdateContext context, long key, int index); - public abstract void pop(UpdateContext context, long key); + public abstract void push(UpdateContext context, long key, int pos); + public abstract void pop(UpdateContext context, long key, int pos); public abstract void reset(UpdateContext context); // return the first row that affects this key @@ -349,8 +292,17 @@ public long computeLastAffectedKey(long key, @NotNull final RowSet source) { public void initializeFor(@NotNull final UpdateContext context, @NotNull final RowSet updateRowSet) { final UpdateWindowedContext ctx = (UpdateWindowedContext) context; - long windowStartKey = computeFirstAffectingKey(updateRowSet.firstRowKey(), ctx.workingRowSet); - ctx.loadWindowChunks(windowStartKey); + // load all the influencer values this update will need + ctx.loadInfluencerValueChunk(); + + // setup the iterators we will need for managing the windows + ctx.influencerKeyIterator = ctx.influencerKeys.iterator(); + ctx.currentInfluencerKey = ctx.influencerKeyIterator.next(); + if (ctx.influencerPositions != null) { + ctx.influencerPosIterator = ctx.influencerPositions.iterator(); + ctx.currentInfluencerPos = ctx.influencerPosIterator.next(); + } + ctx.currentInfluencerIndex = 0; } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java index 2fcb6311be5..cf3e22cd396 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java @@ -43,13 +43,14 @@ public void close() { } @Override - public void loadCandidateValueChunk(RowSequence windowRowSequence) { + public void loadInfluencerValueChunk() { + int size = influencerKeys.intSize(); // fill the window values chunk if (candidateValuesChunk == null) { - candidateValuesChunk = WritableByteChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + candidateValuesChunk = WritableByteChunk.makeWritableChunk(size); } - try (ChunkSource.FillContext fc = valueSource.makeFillContext(WINDOW_CHUNK_SIZE)){ - valueSource.fillChunk(fc, candidateValuesChunk, windowRowSequence); + try (ChunkSource.FillContext fc = valueSource.makeFillContext(size)){ + valueSource.fillChunk(fc, candidateValuesChunk, influencerKeys); } } } @@ -109,7 +110,7 @@ public void resetForProcess(@NotNull final UpdateContext context, @NotNull final RowSet sourceRowSet, long firstUnmodifiedKey) { final Context ctx = (Context) context; - ctx.workingRowSet = sourceRowSet; + ctx.sourceRowSet = sourceRowSet; } @Override @@ -119,7 +120,6 @@ public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; - ctx.loadDataChunks(inputKeys); doProcessChunk(ctx, inputKeys, keyChunk, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java index ba254497d2a..5e1a2319096 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java @@ -32,13 +32,14 @@ public void close() { } @Override - public void loadCandidateValueChunk(RowSequence windowRowSequence) { + public void loadInfluencerValueChunk() { + int size = influencerKeys.intSize(); // fill the window values chunk if (candidateValuesChunk == null) { - candidateValuesChunk = WritableCharChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + candidateValuesChunk = WritableCharChunk.makeWritableChunk(size); } - try (ChunkSource.FillContext fc = valueSource.makeFillContext(WINDOW_CHUNK_SIZE)){ - valueSource.fillChunk(fc, candidateValuesChunk, windowRowSequence); + try (ChunkSource.FillContext fc = valueSource.makeFillContext(size)){ + valueSource.fillChunk(fc, candidateValuesChunk, influencerKeys); } } } @@ -85,7 +86,7 @@ public void resetForProcess(@NotNull final UpdateContext context, @NotNull final RowSet sourceRowSet, long firstUnmodifiedKey) { final Context ctx = (Context) context; - ctx.workingRowSet = sourceRowSet; + ctx.sourceRowSet = sourceRowSet; } @Override @@ -95,7 +96,6 @@ public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; - ctx.loadDataChunks(inputKeys); doProcessChunk(ctx, inputKeys, keyChunk, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java index e79efbf2719..2b08d08dac0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java @@ -37,13 +37,14 @@ public void close() { } @Override - public void loadCandidateValueChunk(RowSequence windowRowSequence) { + public void loadInfluencerValueChunk() { + int size = influencerKeys.intSize(); // fill the window values chunk if (candidateValuesChunk == null) { - candidateValuesChunk = WritableDoubleChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + candidateValuesChunk = WritableDoubleChunk.makeWritableChunk(size); } - try (ChunkSource.FillContext fc = valueSource.makeFillContext(WINDOW_CHUNK_SIZE)){ - valueSource.fillChunk(fc, candidateValuesChunk, windowRowSequence); + try (ChunkSource.FillContext fc = valueSource.makeFillContext(size)){ + valueSource.fillChunk(fc, candidateValuesChunk, influencerKeys); } } } @@ -97,7 +98,7 @@ public void resetForProcess(@NotNull final UpdateContext context, @NotNull final RowSet sourceRowSet, long firstUnmodifiedKey) { final Context ctx = (Context) context; - ctx.workingRowSet = sourceRowSet; + ctx.sourceRowSet = sourceRowSet; } @Override @@ -107,7 +108,6 @@ public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; - ctx.loadDataChunks(inputKeys); doProcessChunk(ctx, inputKeys, keyChunk, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java index c148984e0ce..224ebf4423b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java @@ -32,13 +32,14 @@ public void close() { } @Override - public void loadCandidateValueChunk(RowSequence windowRowSequence) { + public void loadInfluencerValueChunk() { + int size = influencerKeys.intSize(); // fill the window values chunk if (candidateValuesChunk == null) { - candidateValuesChunk = WritableFloatChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + candidateValuesChunk = WritableFloatChunk.makeWritableChunk(size); } - try (ChunkSource.FillContext fc = valueSource.makeFillContext(WINDOW_CHUNK_SIZE)){ - valueSource.fillChunk(fc, candidateValuesChunk, windowRowSequence); + try (ChunkSource.FillContext fc = valueSource.makeFillContext(size)){ + valueSource.fillChunk(fc, candidateValuesChunk, influencerKeys); } } } @@ -92,7 +93,7 @@ public void resetForProcess(@NotNull final UpdateContext context, @NotNull final RowSet sourceRowSet, long firstUnmodifiedKey) { final Context ctx = (Context) context; - ctx.workingRowSet = sourceRowSet; + ctx.sourceRowSet = sourceRowSet; } @Override @@ -102,7 +103,6 @@ public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; - ctx.loadDataChunks(inputKeys); doProcessChunk(ctx, inputKeys, keyChunk, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java index 110a35a53ca..53c0e3a169a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java @@ -37,13 +37,14 @@ public void close() { } @Override - public void loadCandidateValueChunk(RowSequence windowRowSequence) { + public void loadInfluencerValueChunk() { + int size = influencerKeys.intSize(); // fill the window values chunk if (candidateValuesChunk == null) { - candidateValuesChunk = WritableIntChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + candidateValuesChunk = WritableIntChunk.makeWritableChunk(size); } - try (ChunkSource.FillContext fc = valueSource.makeFillContext(WINDOW_CHUNK_SIZE)){ - valueSource.fillChunk(fc, candidateValuesChunk, windowRowSequence); + try (ChunkSource.FillContext fc = valueSource.makeFillContext(size)){ + valueSource.fillChunk(fc, candidateValuesChunk, influencerKeys); } } } @@ -90,7 +91,7 @@ public void resetForProcess(@NotNull final UpdateContext context, @NotNull final RowSet sourceRowSet, long firstUnmodifiedKey) { final Context ctx = (Context) context; - ctx.workingRowSet = sourceRowSet; + ctx.sourceRowSet = sourceRowSet; } @Override @@ -100,7 +101,6 @@ public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; - ctx.loadDataChunks(inputKeys); doProcessChunk(ctx, inputKeys, keyChunk, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java index 6317af36f92..f36dfb7bb5a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java @@ -37,13 +37,14 @@ public void close() { } @Override - public void loadCandidateValueChunk(RowSequence windowRowSequence) { + public void loadInfluencerValueChunk() { + int size = influencerKeys.intSize(); // fill the window values chunk if (candidateValuesChunk == null) { - candidateValuesChunk = WritableLongChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + candidateValuesChunk = WritableLongChunk.makeWritableChunk(size); } - try (ChunkSource.FillContext fc = valueSource.makeFillContext(WINDOW_CHUNK_SIZE)){ - valueSource.fillChunk(fc, candidateValuesChunk, windowRowSequence); + try (ChunkSource.FillContext fc = valueSource.makeFillContext(size)){ + valueSource.fillChunk(fc, candidateValuesChunk, influencerKeys); } } } @@ -90,7 +91,7 @@ public void resetForProcess(@NotNull final UpdateContext context, @NotNull final RowSet sourceRowSet, long firstUnmodifiedKey) { final Context ctx = (Context) context; - ctx.workingRowSet = sourceRowSet; + ctx.sourceRowSet = sourceRowSet; } @Override @@ -100,7 +101,6 @@ public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; - ctx.loadDataChunks(inputKeys); doProcessChunk(ctx, inputKeys, keyChunk, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java index fc5d2699cbc..b8020af2812 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java @@ -38,13 +38,14 @@ public void close() { } @Override - public void loadCandidateValueChunk(RowSequence windowRowSequence) { + public void loadInfluencerValueChunk() { + int size = influencerKeys.intSize(); // fill the window values chunk if (candidateValuesChunk == null) { - candidateValuesChunk = WritableObjectChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + candidateValuesChunk = WritableObjectChunk.makeWritableChunk(size); } - try (ChunkSource.FillContext fc = valueSource.makeFillContext(WINDOW_CHUNK_SIZE)){ - valueSource.fillChunk(fc, candidateValuesChunk, windowRowSequence); + try (ChunkSource.FillContext fc = valueSource.makeFillContext(size)){ + valueSource.fillChunk(fc, candidateValuesChunk, influencerKeys); } } } @@ -93,7 +94,7 @@ public void resetForProcess(@NotNull final UpdateContext context, @NotNull final RowSet sourceRowSet, long firstUnmodifiedKey) { final Context ctx = (Context) context; - ctx.workingRowSet = sourceRowSet; + ctx.sourceRowSet = sourceRowSet; } @Override @@ -103,7 +104,6 @@ public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; - ctx.loadDataChunks(inputKeys); doProcessChunk(ctx, inputKeys, keyChunk, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java index 3cbee310841..2f8c30b14f0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java @@ -37,13 +37,14 @@ public void close() { } @Override - public void loadCandidateValueChunk(RowSequence windowRowSequence) { + public void loadInfluencerValueChunk() { + int size = influencerKeys.intSize(); // fill the window values chunk if (candidateValuesChunk == null) { - candidateValuesChunk = WritableShortChunk.makeWritableChunk(WINDOW_CHUNK_SIZE); + candidateValuesChunk = WritableShortChunk.makeWritableChunk(size); } - try (ChunkSource.FillContext fc = valueSource.makeFillContext(WINDOW_CHUNK_SIZE)){ - valueSource.fillChunk(fc, candidateValuesChunk, windowRowSequence); + try (ChunkSource.FillContext fc = valueSource.makeFillContext(size)){ + valueSource.fillChunk(fc, candidateValuesChunk, influencerKeys); } } } @@ -90,7 +91,7 @@ public void resetForProcess(@NotNull final UpdateContext context, @NotNull final RowSet sourceRowSet, long firstUnmodifiedKey) { final Context ctx = (Context) context; - ctx.workingRowSet = sourceRowSet; + ctx.sourceRowSet = sourceRowSet; } @Override @@ -100,7 +101,6 @@ public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; - ctx.loadDataChunks(inputKeys); doProcessChunk(ctx, inputKeys, keyChunk, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java index 3fb6cf89d7a..c5e4354f264 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java @@ -48,14 +48,16 @@ protected class Context extends BaseWindowedByteUpdateByOperator.Context { public final SizedSafeCloseable fillContext; public final SizedLongChunk outputValues; - public LinkedList windowValues = new LinkedList<>(); - public long currentVal = NULL_LONG; + // position data for the chunk being currently processed + public SizedLongChunk valuePositionChunk; + protected Context(final int chunkSize) { this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); this.fillContext.ensureCapacity(chunkSize); this.outputValues = new SizedLongChunk<>(chunkSize); + this.valuePositionChunk = new SizedLongChunk<>(chunkSize); } @Override @@ -63,6 +65,7 @@ public void close() { super.close(); outputValues.close(); fillContext.close(); + this.valuePositionChunk.close(); } } @@ -74,8 +77,10 @@ public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize) { @Override public void setChunkSize(@NotNull UpdateContext context, int chunkSize) { - ((Context)context).outputValues.ensureCapacity(chunkSize); - ((Context)context).fillContext.ensureCapacity(chunkSize); + final Context ctx = (Context) context; + ctx.outputValues.ensureCapacity(chunkSize); + ctx.fillContext.ensureCapacity(chunkSize); + ctx.valuePositionChunk.ensureCapacity(chunkSize); } public ByteRollingSumOperator(@NotNull final MatchPair pair, @@ -110,12 +115,9 @@ public ByteRollingSumOperator(@NotNull final MatchPair pair, } @Override - public void push(UpdateContext context, long key, int index) { + public void push(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; - Byte val = ctx.candidateValuesChunk.get(index); - - // add the value to the window buffer - ctx.windowValues.addLast(val); + Byte val = ctx.candidateValuesChunk.get(pos); // increase the running sum if (val != NULL_BYTE) { @@ -128,9 +130,9 @@ public void push(UpdateContext context, long key, int index) { } @Override - public void pop(UpdateContext context, long key) { + public void pop(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; - Byte val = ctx.windowValues.pop(); + Byte val = ctx.candidateValuesChunk.get(pos); // reduce the running sum if (val != NULL_BYTE) { @@ -151,7 +153,17 @@ public void doProcessChunk(@NotNull final BaseWindowedByteUpdateByOperator.Conte @NotNull final Chunk workingChunk) { final Context ctx = (Context) context; + if (timestampColumnName == null) { + // produce position data for the window (will be timestamps for time-based) + // TODO: gotta be a better way than creating two rowsets + try (final RowSet rs = inputKeys.asRowSet(); + final RowSet positions = ctx.sourceRowSet.invert(rs)) { + positions.fillRowKeyChunk(ctx.valuePositionChunk.get()); + } + } + computeTicks(ctx, 0, inputKeys.intSize()); + //noinspection unchecked outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } @@ -163,7 +175,7 @@ private void computeTicks(@NotNull final Context ctx, final WritableLongChunk localOutputValues = ctx.outputValues.get(); for (int ii = runStart; ii < runStart + runLength; ii++) { if (recorder == null) { - ctx.fillWindowTicks(ctx, ctx.valuePositionChunk.get(ii)); + ctx.fillWindowTicks(ctx, ctx.valuePositionChunk.get().get(ii)); } // the sum was computed by push/pop operations localOutputValues.set(ii, ctx.currentVal); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java index 0890f8aac50..79a3749ca06 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java @@ -16,6 +16,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.ChunkSink; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; @@ -54,10 +55,14 @@ protected class Context extends BaseWindowedDoubleUpdateByOperator.Context { public LinkedList windowValues = new LinkedList<>(); + // position data for the chunk being currently processed + public SizedLongChunk valuePositionChunk; + protected Context(final int chunkSize) { this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); this.fillContext.ensureCapacity(chunkSize); this.outputValues = new SizedDoubleChunk<>(chunkSize); + this.valuePositionChunk = new SizedLongChunk<>(chunkSize); } @Override @@ -65,6 +70,7 @@ public void close() { super.close(); outputValues.close(); fillContext.close(); + this.valuePositionChunk.close(); } } @@ -76,8 +82,10 @@ public UpdateContext makeUpdateContext(final int chunkSize) { @Override public void setChunkSize(@NotNull UpdateContext context, int chunkSize) { - ((Context)context).outputValues.ensureCapacity(chunkSize); - ((Context)context).fillContext.ensureCapacity(chunkSize); + final Context ctx = (Context) context; + ctx.outputValues.ensureCapacity(chunkSize); + ctx.fillContext.ensureCapacity(chunkSize); + ctx.valuePositionChunk.ensureCapacity(chunkSize); } public DoubleRollingSumOperator(@NotNull final MatchPair pair, @@ -110,15 +118,16 @@ public DoubleRollingSumOperator(@NotNull final MatchPair pair, } @Override - public void push(UpdateContext context, long key, int index) { + public void push(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; - double val = ctx.candidateValuesChunk.get(index); + double val = ctx.candidateValuesChunk.get(pos); ctx.windowValues.addLast(val); } @Override - public void pop(UpdateContext context, long key) { + public void pop(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; + double val = ctx.candidateValuesChunk.get(pos); ctx.windowValues.pop(); } @@ -134,6 +143,15 @@ public void doProcessChunk(@NotNull final BaseWindowedDoubleUpdateByOperator.Con @NotNull final Chunk workingChunk) { final Context ctx = (Context) context; + if (timestampColumnName == null) { + // produce position data for the window (will be timestamps for time-based) + // TODO: gotta be a better way than creating two rowsets + try (final RowSet rs = inputKeys.asRowSet(); + final RowSet positions = ctx.sourceRowSet.invert(rs)) { + positions.fillRowKeyChunk(ctx.valuePositionChunk.get()); + } + } + computeTicks(ctx, 0, inputKeys.intSize()); //noinspection unchecked outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); @@ -146,7 +164,7 @@ private void computeTicks(@NotNull final Context ctx, final WritableDoubleChunk localOutputValues = ctx.outputValues.get(); for (int ii = runStart; ii < runStart + runLength; ii++) { if (recorder == null) { - ctx.fillWindowTicks(ctx, ctx.valuePositionChunk.get(ii)); + ctx.fillWindowTicks(ctx, ctx.valuePositionChunk.get().get(ii)); } MutableDouble sum = new MutableDouble(NULL_DOUBLE); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java index faf477cfd82..de5b9e08d00 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java @@ -11,6 +11,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.ChunkSink; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; @@ -49,10 +50,14 @@ protected class Context extends BaseWindowedFloatUpdateByOperator.Context { public LinkedList windowValues = new LinkedList<>(); + // position data for the chunk being currently processed + public SizedLongChunk valuePositionChunk; + protected Context(final int chunkSize) { this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); this.fillContext.ensureCapacity(chunkSize); this.outputValues = new SizedFloatChunk<>(chunkSize); + this.valuePositionChunk = new SizedLongChunk<>(chunkSize); } @Override @@ -60,6 +65,7 @@ public void close() { super.close(); outputValues.close(); fillContext.close(); + this.valuePositionChunk.close(); } } @@ -71,8 +77,10 @@ public UpdateContext makeUpdateContext(final int chunkSize) { @Override public void setChunkSize(@NotNull UpdateContext context, int chunkSize) { - ((Context)context).outputValues.ensureCapacity(chunkSize); - ((Context)context).fillContext.ensureCapacity(chunkSize); + final Context ctx = (Context) context; + ctx.outputValues.ensureCapacity(chunkSize); + ctx.fillContext.ensureCapacity(chunkSize); + ctx.valuePositionChunk.ensureCapacity(chunkSize); } public FloatRollingSumOperator(@NotNull final MatchPair pair, @@ -105,15 +113,16 @@ public FloatRollingSumOperator(@NotNull final MatchPair pair, } @Override - public void push(UpdateContext context, long key, int index) { + public void push(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; - float val = ctx.candidateValuesChunk.get(index); + float val = ctx.candidateValuesChunk.get(pos); ctx.windowValues.addLast(val); } @Override - public void pop(UpdateContext context, long key) { + public void pop(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; + float val = ctx.candidateValuesChunk.get(pos); ctx.windowValues.pop(); } @@ -129,6 +138,15 @@ public void doProcessChunk(@NotNull final BaseWindowedFloatUpdateByOperator.Cont @NotNull final Chunk workingChunk) { final Context ctx = (Context) context; + if (timestampColumnName == null) { + // produce position data for the window (will be timestamps for time-based) + // TODO: gotta be a better way than creating two rowsets + try (final RowSet rs = inputKeys.asRowSet(); + final RowSet positions = ctx.sourceRowSet.invert(rs)) { + positions.fillRowKeyChunk(ctx.valuePositionChunk.get()); + } + } + computeTicks(ctx, 0, inputKeys.intSize()); //noinspection unchecked outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); @@ -141,7 +159,7 @@ private void computeTicks(@NotNull final Context ctx, final WritableFloatChunk localOutputValues = ctx.outputValues.get(); for (int ii = runStart; ii < runStart + runLength; ii++) { if (recorder == null) { - ctx.fillWindowTicks(ctx, ctx.valuePositionChunk.get(ii)); + ctx.fillWindowTicks(ctx, ctx.valuePositionChunk.get().get(ii)); } MutableFloat sum = new MutableFloat(NULL_FLOAT); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java index 4ba2014db94..690c58f3346 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java @@ -47,14 +47,16 @@ protected class Context extends BaseWindowedIntUpdateByOperator.Context { public final SizedSafeCloseable fillContext; public final SizedLongChunk outputValues; - public LinkedList windowValues = new LinkedList<>(); - public long currentVal = NULL_LONG; + // position data for the chunk being currently processed + public SizedLongChunk valuePositionChunk; + protected Context(final int chunkSize) { this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); this.fillContext.ensureCapacity(chunkSize); this.outputValues = new SizedLongChunk<>(chunkSize); + this.valuePositionChunk = new SizedLongChunk<>(chunkSize); } @Override @@ -62,6 +64,7 @@ public void close() { super.close(); outputValues.close(); fillContext.close(); + this.valuePositionChunk.close(); } } @@ -73,8 +76,10 @@ public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize) { @Override public void setChunkSize(@NotNull UpdateContext context, int chunkSize) { - ((Context)context).outputValues.ensureCapacity(chunkSize); - ((Context)context).fillContext.ensureCapacity(chunkSize); + final Context ctx = (Context) context; + ctx.outputValues.ensureCapacity(chunkSize); + ctx.fillContext.ensureCapacity(chunkSize); + ctx.valuePositionChunk.ensureCapacity(chunkSize); } public IntRollingSumOperator(@NotNull final MatchPair pair, @@ -107,12 +112,9 @@ public IntRollingSumOperator(@NotNull final MatchPair pair, } @Override - public void push(UpdateContext context, long key, int index) { + public void push(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; - Integer val = ctx.candidateValuesChunk.get(index); - - // add the value to the window buffer - ctx.windowValues.addLast(val); + Integer val = ctx.candidateValuesChunk.get(pos); // increase the running sum if (val != NULL_INT) { @@ -125,9 +127,9 @@ public void push(UpdateContext context, long key, int index) { } @Override - public void pop(UpdateContext context, long key) { + public void pop(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; - Integer val = ctx.windowValues.pop(); + Integer val = ctx.candidateValuesChunk.get(pos); // reduce the running sum if (val != NULL_INT) { @@ -148,7 +150,17 @@ public void doProcessChunk(@NotNull final BaseWindowedIntUpdateByOperator.Contex @NotNull final Chunk workingChunk) { final Context ctx = (Context) context; + if (timestampColumnName == null) { + // produce position data for the window (will be timestamps for time-based) + // TODO: gotta be a better way than creating two rowsets + try (final RowSet rs = inputKeys.asRowSet(); + final RowSet positions = ctx.sourceRowSet.invert(rs)) { + positions.fillRowKeyChunk(ctx.valuePositionChunk.get()); + } + } + computeTicks(ctx, 0, inputKeys.intSize()); + //noinspection unchecked outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } @@ -160,7 +172,7 @@ private void computeTicks(@NotNull final Context ctx, final WritableLongChunk localOutputValues = ctx.outputValues.get(); for (int ii = runStart; ii < runStart + runLength; ii++) { if (recorder == null) { - ctx.fillWindowTicks(ctx, ctx.valuePositionChunk.get(ii)); + ctx.fillWindowTicks(ctx, ctx.valuePositionChunk.get().get(ii)); } // the sum was computed by push/pop operations localOutputValues.set(ii, ctx.currentVal); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java index ecbe2fb28f9..75df7c0365a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java @@ -47,14 +47,16 @@ protected class Context extends BaseWindowedLongUpdateByOperator.Context { public final SizedSafeCloseable fillContext; public final SizedLongChunk outputValues; - public LinkedList windowValues = new LinkedList<>(); - public long currentVal = NULL_LONG; + // position data for the chunk being currently processed + public SizedLongChunk valuePositionChunk; + protected Context(final int chunkSize) { this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); this.fillContext.ensureCapacity(chunkSize); this.outputValues = new SizedLongChunk<>(chunkSize); + this.valuePositionChunk = new SizedLongChunk<>(chunkSize); } @Override @@ -62,6 +64,7 @@ public void close() { super.close(); outputValues.close(); fillContext.close(); + this.valuePositionChunk.close(); } } @@ -73,8 +76,10 @@ public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize) { @Override public void setChunkSize(@NotNull UpdateContext context, int chunkSize) { - ((Context)context).outputValues.ensureCapacity(chunkSize); - ((Context)context).fillContext.ensureCapacity(chunkSize); + final Context ctx = (Context) context; + ctx.outputValues.ensureCapacity(chunkSize); + ctx.fillContext.ensureCapacity(chunkSize); + ctx.valuePositionChunk.ensureCapacity(chunkSize); } public LongRollingSumOperator(@NotNull final MatchPair pair, @@ -107,12 +112,9 @@ public LongRollingSumOperator(@NotNull final MatchPair pair, } @Override - public void push(UpdateContext context, long key, int index) { + public void push(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; - Long val = ctx.candidateValuesChunk.get(index); - - // add the value to the window buffer - ctx.windowValues.addLast(val); + Long val = ctx.candidateValuesChunk.get(pos); // increase the running sum if (val != NULL_LONG) { @@ -125,9 +127,9 @@ public void push(UpdateContext context, long key, int index) { } @Override - public void pop(UpdateContext context, long key) { + public void pop(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; - Long val = ctx.windowValues.pop(); + Long val = ctx.candidateValuesChunk.get(pos); // reduce the running sum if (val != NULL_LONG) { @@ -148,7 +150,17 @@ public void doProcessChunk(@NotNull final BaseWindowedLongUpdateByOperator.Conte @NotNull final Chunk workingChunk) { final Context ctx = (Context) context; + if (timestampColumnName == null) { + // produce position data for the window (will be timestamps for time-based) + // TODO: gotta be a better way than creating two rowsets + try (final RowSet rs = inputKeys.asRowSet(); + final RowSet positions = ctx.sourceRowSet.invert(rs)) { + positions.fillRowKeyChunk(ctx.valuePositionChunk.get()); + } + } + computeTicks(ctx, 0, inputKeys.intSize()); + //noinspection unchecked outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } @@ -160,7 +172,7 @@ private void computeTicks(@NotNull final Context ctx, final WritableLongChunk localOutputValues = ctx.outputValues.get(); for (int ii = runStart; ii < runStart + runLength; ii++) { if (recorder == null) { - ctx.fillWindowTicks(ctx, ctx.valuePositionChunk.get(ii)); + ctx.fillWindowTicks(ctx, ctx.valuePositionChunk.get().get(ii)); } // the sum was computed by push/pop operations localOutputValues.set(ii, ctx.currentVal); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java index 0deb7e2057e..b9fb8dc0f8e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java @@ -42,14 +42,16 @@ protected class Context extends BaseWindowedShortUpdateByOperator.Context { public final SizedSafeCloseable fillContext; public final SizedLongChunk outputValues; - public LinkedList windowValues = new LinkedList<>(); - public long currentVal = NULL_LONG; + // position data for the chunk being currently processed + public SizedLongChunk valuePositionChunk; + protected Context(final int chunkSize) { this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); this.fillContext.ensureCapacity(chunkSize); this.outputValues = new SizedLongChunk<>(chunkSize); + this.valuePositionChunk = new SizedLongChunk<>(chunkSize); } @Override @@ -57,6 +59,7 @@ public void close() { super.close(); outputValues.close(); fillContext.close(); + this.valuePositionChunk.close(); } } @@ -68,8 +71,10 @@ public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize) { @Override public void setChunkSize(@NotNull UpdateContext context, int chunkSize) { - ((Context)context).outputValues.ensureCapacity(chunkSize); - ((Context)context).fillContext.ensureCapacity(chunkSize); + final Context ctx = (Context) context; + ctx.outputValues.ensureCapacity(chunkSize); + ctx.fillContext.ensureCapacity(chunkSize); + ctx.valuePositionChunk.ensureCapacity(chunkSize); } public ShortRollingSumOperator(@NotNull final MatchPair pair, @@ -102,12 +107,9 @@ public ShortRollingSumOperator(@NotNull final MatchPair pair, } @Override - public void push(UpdateContext context, long key, int index) { + public void push(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; - Short val = ctx.candidateValuesChunk.get(index); - - // add the value to the window buffer - ctx.windowValues.addLast(val); + Short val = ctx.candidateValuesChunk.get(pos); // increase the running sum if (val != NULL_SHORT) { @@ -120,9 +122,9 @@ public void push(UpdateContext context, long key, int index) { } @Override - public void pop(UpdateContext context, long key) { + public void pop(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; - Short val = ctx.windowValues.pop(); + Short val = ctx.candidateValuesChunk.get(pos); // reduce the running sum if (val != NULL_SHORT) { @@ -143,7 +145,17 @@ public void doProcessChunk(@NotNull final BaseWindowedShortUpdateByOperator.Cont @NotNull final Chunk workingChunk) { final Context ctx = (Context) context; + if (timestampColumnName == null) { + // produce position data for the window (will be timestamps for time-based) + // TODO: gotta be a better way than creating two rowsets + try (final RowSet rs = inputKeys.asRowSet(); + final RowSet positions = ctx.sourceRowSet.invert(rs)) { + positions.fillRowKeyChunk(ctx.valuePositionChunk.get()); + } + } + computeTicks(ctx, 0, inputKeys.intSize()); + //noinspection unchecked outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } @@ -155,7 +167,7 @@ private void computeTicks(@NotNull final Context ctx, final WritableLongChunk localOutputValues = ctx.outputValues.get(); for (int ii = runStart; ii < runStart + runLength; ii++) { if (recorder == null) { - ctx.fillWindowTicks(ctx, ctx.valuePositionChunk.get(ii)); + ctx.fillWindowTicks(ctx, ctx.valuePositionChunk.get().get(ii)); } // the sum was computed by push/pop operations localOutputValues.set(ii, ctx.currentVal); diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java index 99ef2d656c3..93666b7043d 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java @@ -311,6 +311,7 @@ private long[] rollingSum(short[] values, int prevTicks, int postTicks) { final int head = Math.max(0, i - prevTicks + 1); final int tail = Math.min(values.length - 1, i + postTicks); + // compute everything in this window for (int computeIdx = head; computeIdx <= tail; computeIdx++) { if (!isNull(values[computeIdx])) { diff --git a/engine/updategraph/src/main/java/io/deephaven/engine/liveness/Liveness.java b/engine/updategraph/src/main/java/io/deephaven/engine/liveness/Liveness.java index 9b6476ff4e8..39a5628860b 100644 --- a/engine/updategraph/src/main/java/io/deephaven/engine/liveness/Liveness.java +++ b/engine/updategraph/src/main/java/io/deephaven/engine/liveness/Liveness.java @@ -34,7 +34,7 @@ public final class Liveness { Configuration.getInstance().getBooleanWithDefault("Liveness.heapDump", false); static final boolean CLEANUP_LOG_ENABLED = - Configuration.getInstance().getBooleanWithDefault("Liveness.cleanupLogEnabled", true); + Configuration.getInstance().getBooleanWithDefault("Liveness.cleanupLogEnabled", false); private static final long OUTSTANDING_COUNT_LOG_INTERVAL_MILLIS = 1000L; From 924edce79378e5967752fb055a3723d6104f3a04 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 7 Sep 2022 11:02:02 -0700 Subject: [PATCH 018/123] WIP --- .../impl/UpdateByCumulativeOperator.java | 107 ++++- .../engine/table/impl/UpdateByOperator.java | 93 +--- .../table/impl/UpdateByOperatorFactory.java | 29 +- .../table/impl/UpdateByWindowedOperator.java | 417 ++++++++++++++---- .../engine/table/impl/ZeroKeyUpdateBy.java | 258 ++++++++--- .../ssa/ByteReverseSegmentedSortedArray.java | 22 +- .../impl/ssa/ByteSegmentedSortedArray.java | 22 +- .../ssa/CharReverseSegmentedSortedArray.java | 22 +- .../impl/ssa/CharSegmentedSortedArray.java | 22 +- .../DoubleReverseSegmentedSortedArray.java | 22 +- .../impl/ssa/DoubleSegmentedSortedArray.java | 22 +- .../ssa/FloatReverseSegmentedSortedArray.java | 22 +- .../impl/ssa/FloatSegmentedSortedArray.java | 22 +- .../ssa/IntReverseSegmentedSortedArray.java | 22 +- .../impl/ssa/IntSegmentedSortedArray.java | 22 +- .../ssa/LongReverseSegmentedSortedArray.java | 22 +- .../impl/ssa/LongSegmentedSortedArray.java | 22 +- ...lAwareCharReverseSegmentedSortedArray.java | 22 +- .../NullAwareCharSegmentedSortedArray.java | 22 +- .../ObjectReverseSegmentedSortedArray.java | 22 +- .../impl/ssa/ObjectSegmentedSortedArray.java | 22 +- .../ssa/ShortReverseSegmentedSortedArray.java | 22 +- .../impl/ssa/ShortSegmentedSortedArray.java | 22 +- .../ema/BasePrimitiveEMAOperator.java | 9 +- .../updateby/ema/BigNumberEMAOperator.java | 10 +- .../hashing/TrackerPriorityQueue.java | 155 ------- .../updateby/hashing/UpdateBySlotTracker.java | 406 ----------------- .../internal/BaseByteUpdateByOperator.java | 8 +- .../internal/BaseCharUpdateByOperator.java | 8 +- .../internal/BaseDoubleUpdateByOperator.java | 16 +- .../internal/BaseFloatUpdateByOperator.java | 16 +- .../internal/BaseIntUpdateByOperator.java | 8 +- .../internal/BaseLongUpdateByOperator.java | 8 +- .../internal/BaseObjectUpdateByOperator.java | 8 +- .../internal/BaseShortUpdateByOperator.java | 8 +- .../BaseWindowedByteUpdateByOperator.java | 12 +- .../BaseWindowedCharUpdateByOperator.java | 12 +- .../BaseWindowedDoubleUpdateByOperator.java | 12 +- .../BaseWindowedFloatUpdateByOperator.java | 12 +- .../BaseWindowedIntUpdateByOperator.java | 12 +- .../BaseWindowedLongUpdateByOperator.java | 12 +- .../BaseWindowedObjectUpdateByOperator.java | 12 +- .../BaseWindowedShortUpdateByOperator.java | 12 +- .../LongRecordingUpdateByOperator.java | 18 +- .../rollingsum/ByteRollingSumOperator.java | 22 +- .../rollingsum/DoubleRollingSumOperator.java | 20 +- .../rollingsum/FloatRollingSumOperator.java | 20 +- .../rollingsum/IntRollingSumOperator.java | 22 +- .../rollingsum/LongRollingSumOperator.java | 22 +- .../rollingsum/ShortRollingSumOperator.java | 60 +-- .../table/impl/updateby/TestRollingSum.java | 115 ++++- .../api/updateby/UpdateByOperation.java | 13 + 52 files changed, 1155 insertions(+), 1213 deletions(-) delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/hashing/TrackerPriorityQueue.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/hashing/UpdateBySlotTracker.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java index abb3dda2ccb..ce042b6a3df 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java @@ -1,10 +1,9 @@ package io.deephaven.engine.table.impl; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.rowset.RowSetBuilderSequential; -import io.deephaven.engine.rowset.RowSetFactory; -import io.deephaven.engine.rowset.TrackingRowSet; +import io.deephaven.base.verify.Assert; +import io.deephaven.engine.rowset.*; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.updateby.internal.BaseCharUpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedByteUpdateByOperator; import io.deephaven.tablelogger.Row; @@ -13,21 +12,103 @@ public abstract class UpdateByCumulativeOperator implements UpdateByOperator { public abstract class UpdateCumulativeContext implements UpdateContext { + protected LongSegmentedSortedArray timestampSsa; + protected RowSetBuilderSequential modifiedBuilder; protected RowSet newModified; // store the current subset of rows that need computation protected RowSet affectedRows; - public RowSet determineAffectedRows(@NotNull final TableUpdate upstream, @NotNull final TrackingRowSet source, final boolean upstreamAppendOnly) { + public LongSegmentedSortedArray getTimestampSsa() { + return timestampSsa; + } - long smallestModifiedKey = UpdateByOperator.smallestAffectedKey(upstream.added(), upstream.modified(), upstream.removed(), upstream.shifted(), source); + /** + * Find the smallest valued key that participated in the upstream {@link TableUpdate}. + * + * @param added the added rows + * @param modified the modified rows + * @param removed the removed rows + * @param shifted the shifted rows + * + * @return the smallest key that participated in any part of the update. + */ + long smallestAffectedKey(@NotNull final RowSet added, + @NotNull final RowSet modified, + @NotNull final RowSet removed, + @NotNull final RowSetShiftData shifted, + @NotNull final RowSet affectedIndex) { + + long smallestModifiedKey = Long.MAX_VALUE; + if (removed.isNonempty()) { + smallestModifiedKey = removed.firstRowKey(); + } + + if (added.isNonempty()) { + smallestModifiedKey = Math.min(smallestModifiedKey, added.firstRowKey()); + } - try (final RowSet ignored = affectedRows) { - affectedRows = smallestModifiedKey == Long.MAX_VALUE - ? RowSetFactory.empty() - : source.subSetByKeyRange(smallestModifiedKey, source.lastRowKey()); + if (modified.isNonempty()) { + smallestModifiedKey = Math.min(smallestModifiedKey, modified.firstRowKey()); } + + if (shifted.nonempty()) { + final long firstModKey = modified.isEmpty() ? Long.MAX_VALUE : modified.firstRowKey(); + boolean modShiftFound = !modified.isEmpty(); + boolean affectedFound = false; + try (final RowSequence.Iterator it = affectedIndex.getRowSequenceIterator()) { + for (int shiftIdx = 0; shiftIdx < shifted.size() && (!modShiftFound || !affectedFound); shiftIdx++) { + final long shiftStart = shifted.getBeginRange(shiftIdx); + final long shiftEnd = shifted.getEndRange(shiftIdx); + final long shiftDelta = shifted.getShiftDelta(shiftIdx); + + if (!affectedFound) { + if (it.advance(shiftStart + shiftDelta)) { + final long maybeAffectedKey = it.peekNextKey(); + if (maybeAffectedKey <= shiftEnd + shiftDelta) { + affectedFound = true; + final long keyToCompare = + shiftDelta > 0 ? maybeAffectedKey - shiftDelta : maybeAffectedKey; + smallestModifiedKey = Math.min(smallestModifiedKey, keyToCompare); + } + } else { + affectedFound = true; + } + } + + if (!modShiftFound) { + if (firstModKey <= (shiftEnd + shiftDelta)) { + modShiftFound = true; + // If the first modified key is in the range we should include it + if (firstModKey >= (shiftStart + shiftDelta)) { + smallestModifiedKey = Math.min(smallestModifiedKey, firstModKey - shiftDelta); + } else { + // Otherwise it's not included in any shifts, and since shifts can't reorder rows + // it is the smallest possible value and we've already accounted for it above. + break; + } + } + } + } + } + } + + return smallestModifiedKey; + } + + public RowSet determineAffectedRows(@NotNull final TableUpdate upstream, @NotNull final TrackingRowSet source, final boolean initialStep) { + Assert.assertion(affectedRows==null, "affectedRows should be null when determineAffectedRows() is called"); + if (initialStep) { + affectedRows = source.copy(); + return affectedRows; + } + + long smallestModifiedKey = smallestAffectedKey(upstream.added(), upstream.modified(), upstream.removed(), upstream.shifted(), source); + + affectedRows = smallestModifiedKey == Long.MAX_VALUE + ? RowSetFactory.empty() + : source.subSetByKeyRange(smallestModifiedKey, source.lastRowKey()); return affectedRows; } @@ -85,6 +166,12 @@ public boolean requiresKeys() { return false; } + /*** cumulative operators do not need position data */ + @Override + public boolean requiresPositions() { + return false; + } + /*** cumulative operators always need values */ @Override public boolean requiresValues(@NotNull final UpdateContext context) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java index 35e31b984d1..39b41b9fb95 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java @@ -13,6 +13,7 @@ import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.updateby.internal.BaseCharUpdateByOperator; import io.deephaven.util.SafeCloseable; import org.jetbrains.annotations.NotNull; @@ -88,79 +89,6 @@ static boolean isAppendOnly(@NotNull final TableUpdate update, final long lastKn update.added().firstRowKey() > lastKnownKey; } - /** - * Find the smallest valued key that participated in the upstream {@link TableUpdate}. - * - * @param added the added rows - * @param modified the modified rows - * @param removed the removed rows - * @param shifted the shifted rows - * - * @return the smallest key that participated in any part of the update. - */ - static long smallestAffectedKey(@NotNull final RowSet added, - @NotNull final RowSet modified, - @NotNull final RowSet removed, - @NotNull final RowSetShiftData shifted, - @NotNull final RowSet affectedIndex) { - - long smallestModifiedKey = Long.MAX_VALUE; - if (removed.isNonempty()) { - smallestModifiedKey = removed.firstRowKey(); - } - - if (added.isNonempty()) { - smallestModifiedKey = Math.min(smallestModifiedKey, added.firstRowKey()); - } - - if (modified.isNonempty()) { - smallestModifiedKey = Math.min(smallestModifiedKey, modified.firstRowKey()); - } - - if (shifted.nonempty()) { - final long firstModKey = modified.isEmpty() ? Long.MAX_VALUE : modified.firstRowKey(); - boolean modShiftFound = !modified.isEmpty(); - boolean affectedFound = false; - try (final RowSequence.Iterator it = affectedIndex.getRowSequenceIterator()) { - for (int shiftIdx = 0; shiftIdx < shifted.size() && (!modShiftFound || !affectedFound); shiftIdx++) { - final long shiftStart = shifted.getBeginRange(shiftIdx); - final long shiftEnd = shifted.getEndRange(shiftIdx); - final long shiftDelta = shifted.getShiftDelta(shiftIdx); - - if (!affectedFound) { - if (it.advance(shiftStart + shiftDelta)) { - final long maybeAffectedKey = it.peekNextKey(); - if (maybeAffectedKey <= shiftEnd + shiftDelta) { - affectedFound = true; - final long keyToCompare = - shiftDelta > 0 ? maybeAffectedKey - shiftDelta : maybeAffectedKey; - smallestModifiedKey = Math.min(smallestModifiedKey, keyToCompare); - } - } else { - affectedFound = true; - } - } - - if (!modShiftFound) { - if (firstModKey <= (shiftEnd + shiftDelta)) { - modShiftFound = true; - // If the first modified key is in the range we should include it - if (firstModKey >= (shiftStart + shiftDelta)) { - smallestModifiedKey = Math.min(smallestModifiedKey, firstModKey - shiftDelta); - } else { - // Otherwise it's not included in any shifts, and since shifts can't reorder rows - // it is the smallest possible value and we've already accounted for it above. - break; - } - } - } - } - } - } - - return smallestModifiedKey; - } - /** * A context item for use with {@link Table#updateBy(UpdateByControl, Collection, String...)} for non-bucketed * updates. @@ -173,12 +101,14 @@ interface UpdateContext extends SafeCloseable { * @param source the rowset of the parent table (affected rows will be a subset) */ RowSet determineAffectedRows(@NotNull final TableUpdate upstream, @NotNull final TrackingRowSet source, - final boolean upstreamAppendOnly); + final boolean initialStep); /** * Return the rows computed by the {@Code determineAffectedRows()} */ RowSet getAffectedRows(); + + LongSegmentedSortedArray getTimestampSsa(); } /** @@ -230,10 +160,11 @@ RowSet determineAffectedRows(@NotNull final TableUpdate upstream, @NotNull final * Make an {@link UpdateContext} suitable for use with non-bucketed updates. * * @param chunkSize The expected size of chunks that will be provided during the update, + * @param timestampSsa The timestamp SSA to use for time-based operations (null if using ticks) * @return a new context */ @NotNull - UpdateContext makeUpdateContext(final int chunkSize); + UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa); /** *

      @@ -285,6 +216,14 @@ void initializeFor(@NotNull final UpdateContext context, */ boolean requiresKeys(); + /** + * Query if the operator requires position values for the current stage. This method will always be invoked after an + * appropriate invocation of {@link #initializeFor(UpdateContext, RowSet)} + * + * @return true if the operator requires position indices for this operation + */ + boolean requiresPositions(); + /** * Query if the operator requires values for the current stage. * @@ -316,13 +255,15 @@ void applyOutputShift(@NotNull final UpdateContext context, * @param context the context object * @param inputKeys the keys contained in the chunk * @param keyChunk a {@link LongChunk} containing the keys if requested by {@link #requiresKeys()} or null. + * @param posChunk a {@link LongChunk} containing the positions if requested by {@link #requiresPositions()} or null. * @param valuesChunk the current chunk of working values. * @param postUpdateSourceIndex the resulting source index af */ void processChunk(@NotNull final UpdateContext context, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, - @NotNull final Chunk valuesChunk, + @Nullable final LongChunk posChunk, + @Nullable final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceIndex); /** diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java index 852316932b2..110eef9a830 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java @@ -209,22 +209,13 @@ public Void visit(CumProdSpec p) { @Override public Void visit(@NotNull final RollingSumSpec rs) { - final LongRecordingUpdateByOperator timeStampRecorder; final boolean isTimeBased = rs.prevTimeScale().isTimeBased(); final String timestampCol = rs.prevTimeScale().timestampCol(); - if (isTimeBased) { - timeStampRecorder = makeLongRecordingOperator(source, timestampCol); - ops.add(timeStampRecorder); - } else { - timeStampRecorder = null; - } - Arrays.stream(pairs) .filter(p -> !isTimeBased || !p.rightColumn().equals(timestampCol)) .map(fc -> makeRollingSumOperator(fc, source, - timeStampRecorder, rs)) .forEach(ops::add); return null; @@ -392,42 +383,44 @@ private UpdateByOperator makeForwardFillOperator(MatchPair fc, TableDefaults sou private UpdateByOperator makeRollingSumOperator(@NotNull final MatchPair pair, @NotNull final TableDefaults source, - @Nullable final LongRecordingUpdateByOperator recorder, @NotNull final RollingSumSpec rs) { // noinspection rawtypes final ColumnSource columnSource = source.getColumnSource(pair.rightColumn); + final ColumnSource timestampColumnSource; final Class csType = columnSource.getType(); final String[] affectingColumns; - if (recorder == null) { + if (rs.prevTimeScale().timestampCol() == null) { affectingColumns = new String[] {pair.rightColumn}; + timestampColumnSource = null; } else { affectingColumns = new String[] {rs.prevTimeScale().timestampCol(), pair.rightColumn}; + timestampColumnSource = source.getColumnSource(rs.prevTimeScale().timestampCol()); } final long prevTimeScaleUnits = rs.prevTimeScale().timescaleUnits(); final long fwdTimeScaleUnits = rs.fwdTimeScale().timescaleUnits(); if (csType == Boolean.class || csType == boolean.class) { - return new ByteRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, + return new ByteRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), timestampColumnSource, prevTimeScaleUnits, fwdTimeScaleUnits, ReinterpretUtils.booleanToByteSource(columnSource), redirContext, NULL_BOOLEAN_AS_BYTE); } else if (csType == byte.class || csType == Byte.class) { - return new ByteRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, + return new ByteRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), timestampColumnSource, prevTimeScaleUnits, fwdTimeScaleUnits, columnSource, redirContext, NULL_BYTE); } else if (csType == short.class || csType == Short.class) { - return new ShortRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, + return new ShortRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), timestampColumnSource, prevTimeScaleUnits, fwdTimeScaleUnits, columnSource, redirContext); } else if (csType == int.class || csType == Integer.class) { - return new IntRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, + return new IntRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), timestampColumnSource, prevTimeScaleUnits, fwdTimeScaleUnits, columnSource, redirContext); } else if (csType == long.class || csType == Long.class) { - return new LongRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, + return new LongRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), timestampColumnSource, prevTimeScaleUnits, fwdTimeScaleUnits, columnSource, redirContext); } else if (csType == float.class || csType == Float.class) { - return new FloatRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, + return new FloatRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), timestampColumnSource, prevTimeScaleUnits, fwdTimeScaleUnits, columnSource, redirContext); } else if (csType == double.class || csType == Double.class) { - return new DoubleRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, rs.prevTimeScale().timestampCol(), + return new DoubleRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), timestampColumnSource, prevTimeScaleUnits, fwdTimeScaleUnits, columnSource, redirContext); // } else if (csType == BigDecimal.class) { // return new BigDecimalRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java index 3b9d9a2c688..434c222a5df 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java @@ -2,21 +2,29 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.base.LongRingBuffer; -import io.deephaven.chunk.WritableLongChunk; +import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.chunk.sized.SizedLongChunk; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.table.ChunkSource; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.TableUpdate; -import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; +import io.deephaven.engine.table.impl.sources.ReinterpretUtils; +import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; +import org.apache.commons.lang3.mutable.MutableLong; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; +import static io.deephaven.util.QueryConstants.NULL_LONG; public abstract class UpdateByWindowedOperator implements UpdateByOperator { protected final OperationControl control; - protected final LongRecordingUpdateByOperator recorder; protected final String timestampColumnName; + protected final ColumnSource timestampColumnSource; + protected final long reverseTimeScaleUnits; protected final long forwardTimeScaleUnits; @@ -26,11 +34,17 @@ public abstract class UpdateByWindowedOperator implements UpdateByOperator { protected UpdateBy.UpdateByRedirectionContext redirContext; public abstract class UpdateWindowedContext implements UpdateContext { - public RowSetBuilderSequential modifiedBuilder; - public RowSet newModified; + protected LongSegmentedSortedArray timestampSsa; + + protected RowSetBuilderSequential modifiedBuilder; + protected RowSet newModified; + + public LongSegmentedSortedArray getTimestampSsa() { + return timestampSsa; + } public RowSetBuilderSequential getModifiedBuilder() { - if(modifiedBuilder == null) { + if (modifiedBuilder == null) { modifiedBuilder = RowSetFactory.builderSequential(); } return modifiedBuilder; @@ -44,26 +58,121 @@ public RowSetBuilderSequential getModifiedBuilder() { // items. Because the windows are user-configurable, there may be no overlap between these two sets and we // don't need values for the `affected` rows at all protected RowSet affectedRows; - protected RowSet influencerKeys; + protected RowSet influencerRows; protected long currentInfluencerKey; // candidate data for the window public final int WINDOW_CHUNK_SIZE = 4096; - // persist two iterators, for the head and the tail of the current window - protected RowSet.Iterator influencerPosIterator; - protected RowSet.Iterator influencerKeyIterator; + // persist two iterators, for keys and positions + protected SizedLongChunk influencerKeyChunk; + protected SizedLongChunk influencerPosChunk; + protected LongSegmentedSortedArray.Iterator ssaIterator; // for use with a ticking window protected RowSet affectedRowPositions; protected RowSet influencerPositions; - protected long currentInfluencerPos; + protected long currentInfluencerPosOrTimestamp; protected int currentInfluencerIndex; protected LongRingBuffer windowKeys = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); - protected LongRingBuffer windowPos = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); + protected LongRingBuffer windowPosOrTimestamp = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); protected LongRingBuffer windowIndices = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); + private WritableRowSet computeAffectedRowsTime(final RowSet sourceSet, final RowSet subset, long revNanos, long fwdNanos) { + // swap fwd/rev to get the influencer windows + return computeInfluencerRowsTicks(sourceSet, subset, fwdNanos, revNanos); + } + + private WritableRowSet computeInfluencerRowsTime(final RowSet sourceSet, final RowSet subset, long revTicks, long fwdTicks) { + if (sourceSet.size() == subset.size()) { + return sourceSet.copy(); + } + + int chunkSize = (int) Math.min(subset.size(), 4096); + try (ChunkSource.GetContext getContext = timestampColumnSource.makeGetContext(chunkSize); + RowSequence.Iterator chunkIt = subset.getRowSequenceIterator()) { + final RowSetBuilderSequential builder = RowSetFactory.builderSequential(); + final LongSegmentedSortedArray.Iterator ssaHeadIt = timestampSsa.iterator(false, false); + final LongSegmentedSortedArray.Iterator ssaTailIt = timestampSsa.iterator(false, false); + while (chunkIt.hasMore()) { + RowSequence chunkRs = chunkIt.getNextRowSequenceWithLength(chunkSize); + LongChunk tsChunk = timestampColumnSource.getPrevChunk(getContext, chunkRs).asLongChunk(); + for (int i = 0; i < tsChunk.size(); i++) { + long ts = tsChunk.get(i); + // if this timestamp was null, it wasn't included in any windows and there is nothing to recompute + if (ts != NULL_LONG) { + ssaHeadIt.advanceToBeforeFirst(ts); + final long s = ssaHeadIt.getKey(); + ssaTailIt.advanceToLast(ts); + final long e; + if (ssaHeadIt.hasNext()) { + e = ssaTailIt.nextKey(); + } else { + e = NULL_LONG; + } + builder.appendRange(s, e); + } + } + } + try (final RowSet removedChanges = builder.build()) { + // changed.insert(removedChanges); + } + } + + + + long maxPos = sourceSet.size() - 1; + + // find the first row + + try (final RowSet inverted = sourceSet.invert(subset)) { + final RowSetBuilderSequential builder = RowSetFactory.builderSequential(); + final MutableLong minPos = new MutableLong(0L); + + inverted.forAllRowKeyRanges((s, e) -> { + long sPos = Math.max(s - revTicks, minPos.longValue()); + long ePos = Math.min(e + fwdTicks, maxPos); + builder.appendRange(sPos, ePos); + minPos.setValue(ePos + 1); + }); + + try (final RowSet positions = builder.build()) { + return sourceSet.subSetForPositions(positions); + } + } + } + + + private WritableRowSet computeAffectedRowsTicks(final RowSet sourceSet, final RowSet subset, long revTicks, long fwdTicks) { + // swap fwd/rev to get the influencer windows + return computeInfluencerRowsTicks(sourceSet, subset, fwdTicks, revTicks); + } + + private WritableRowSet computeInfluencerRowsTicks(final RowSet sourceSet, final RowSet subset, long revTicks, long fwdTicks) { + if (sourceSet.size() == subset.size()) { + return sourceSet.copy(); + } + + long maxPos = sourceSet.size() - 1; + + try (final RowSet inverted = sourceSet.invert(subset)) { + final RowSetBuilderSequential builder = RowSetFactory.builderSequential(); + final MutableLong minPos = new MutableLong(0L); + + inverted.forAllRowKeyRanges((s, e) -> { + long sPos = Math.max(s - revTicks, minPos.longValue()); + long ePos = Math.min(e + fwdTicks, maxPos); + builder.appendRange(sPos, ePos); + minPos.setValue(ePos + 1); + }); + + try (final RowSet positions = builder.build()) { + return sourceSet.subSetForPositions(positions); + } + } + } + /*** * This function is only correct if the proper {@code source} rowset is provided. If using buckets, then the * provided rowset must be limited to the rows in the current bucket @@ -71,71 +180,107 @@ public RowSetBuilderSequential getModifiedBuilder() { * * @param upstream the update * @param source the rowset of the parent table (affected rows will be a subset) - * @param upstreamAppendOnly + * @param initialStep whether this is the initial step of building the table */ public RowSet determineAffectedRows(@NotNull final TableUpdate upstream, @NotNull final TrackingRowSet source, - final boolean upstreamAppendOnly) { + final boolean initialStep) { + Assert.assertion(affectedRows==null, "affectedRows should be null when determineAffectedRows() is called"); - // NOTE: this is fast rather than bounding to the smallest set possible. Will result in computing more than - // actually necessary + if (initialStep) { + // all rows are affected initially + affectedRows = source.copy(); + influencerRows = source.copy(); - // TODO: return the minimal set of data for this update - - RowSetBuilderRandom builder = RowSetFactory.builderRandom(); - - if (upstream.removed().isNonempty()) { - // need rows affected by removes in pre-shift space to determine rows to recompute - try (final RowSet prevSource = source.copyPrev()) { - - long s = computeFirstAffectedKey(upstream.removed().firstRowKey(), prevSource); - long e = computeLastAffectedKey(upstream.removed().lastRowKey(), prevSource); - - try (final WritableRowSet tmp = RowSetFactory.fromRange(s,e)) { - // apply shifts to get this back to post-shift - upstream.shifted().apply(tmp); - - builder.addRowSet(tmp); - } + // no need to invert, just create a flat rowset + if (timestampColumnName == null) { + affectedRowPositions = RowSetFactory.flat(source.size()); + influencerPositions = RowSetFactory.flat(source.size()); } + return affectedRows; } - if (upstream.added().isNonempty()) { - // all the new rows need computed - builder.addRowSet(upstream.added()); - - // add the rows affected by the adds - builder.addRange(computeFirstAffectedKey(upstream.added().firstRowKey(), source), - computeLastAffectedKey(upstream.added().lastRowKey(), source)); + if (source.isEmpty()) { + affectedRows = RowSetFactory.empty(); + influencerRows = RowSetFactory.empty(); + if (timestampColumnName == null) { + affectedRowPositions = RowSetFactory.empty(); + influencerPositions = RowSetFactory.empty(); + } + return affectedRows; } - if (upstream.modified().isNonempty()) { - // add the rows affected by the mods - builder.addRange(computeFirstAffectedKey(upstream.modified().firstRowKey(), source), - computeLastAffectedKey(upstream.modified().lastRowKey(), source)); - } + // changed rows are mods+adds + WritableRowSet changed = upstream.added().copy(); + changed.insert(upstream.modified()); - try (final RowSet ignored = affectedRows; - final RowSet ignored2 = influencerKeys; - final RowSet ignored3 = affectedRowPositions; - final RowSet ignored4 = influencerPositions; - final RowSet brs = builder.build()) { + WritableRowSet tmpAffected; - affectedRows = source.intersect(brs); + // compute the affected rows from these changes + if (timestampColumnName == null) { + tmpAffected = computeAffectedRowsTicks(source, changed, reverseTimeScaleUnits, forwardTimeScaleUnits); + } else { + tmpAffected = computeAffectedRowsTime(source, changed, reverseTimeScaleUnits, forwardTimeScaleUnits); + } - WritableRowSet tmp = RowSetFactory.fromRange( - computeFirstAffectingKey(affectedRows.firstRowKey(), source), - computeLastAffectingKey(affectedRows.lastRowKey(), source) - ); - tmp.retain(source); - influencerKeys = tmp; + // add affected rows from any removes - // generate position data rowsets for efficiently computed position offsets + if (upstream.removed().isNonempty()) { if (timestampColumnName == null) { - affectedRowPositions = source.invert(affectedRows); - influencerPositions = source.invert(influencerKeys); + // tick based + try (final RowSet prev = source.copyPrev(); + final WritableRowSet affectedByRemoves = computeAffectedRowsTicks(prev, upstream.removed(), reverseTimeScaleUnits, forwardTimeScaleUnits )) { + // apply shifts to get back to pos-shift space + upstream.shifted().apply(affectedByRemoves); + // retain only the rows that still exist in the source + affectedByRemoves.retain(source); + tmpAffected.insert(affectedByRemoves); + } + } else { + // time-based, first grab all the timestamp data for these removed rows + int size = (int) Math.min(upstream.removed().size(), 4096); + try (ChunkSource.GetContext getContext = timestampColumnSource.makeGetContext(size); + RowSequence.Iterator chunkIt = upstream.removed().getRowSequenceIterator()) { + final RowSetBuilderSequential builder = RowSetFactory.builderSequential(); + final LongSegmentedSortedArray.Iterator ssaHeadIt = timestampSsa.iterator(false, false); + final LongSegmentedSortedArray.Iterator ssaTailIt = timestampSsa.iterator(false, false); + while (chunkIt.hasMore()) { + RowSequence chunkRs = chunkIt.getNextRowSequenceWithLength(size); + LongChunk tsChunk = timestampColumnSource.getPrevChunk(getContext, chunkRs).asLongChunk(); + for (int i = 0; i < tsChunk.size(); i++) { + long ts = tsChunk.get(i); + // if this timestamp was null, it wasn't included in any windows and there is nothing to recompute + if (ts != NULL_LONG) { + ssaHeadIt.advanceToBeforeFirst(ts); + final long s = ssaHeadIt.getKey(); + ssaTailIt.advanceToLast(ts); + final long e; + if (ssaHeadIt.hasNext()) { + e = ssaTailIt.nextKey(); + } else { + e = NULL_LONG; + } + builder.appendRange(s, e); + } + } + } + try (final RowSet removedChanges = builder.build()) { + changed.insert(removedChanges); + } + } } } + affectedRows = tmpAffected; + + // now get influencer rows for the affected + if (timestampColumnName == null) { + influencerRows = computeInfluencerRowsTicks(source, affectedRows, reverseTimeScaleUnits, forwardTimeScaleUnits); + // generate position data rowsets for efficiently computed position offsets + affectedRowPositions = source.invert(affectedRows); + influencerPositions = source.invert(influencerRows); + } else { + + } return affectedRows; } @@ -151,39 +296,100 @@ public void fillWindowTicks(UpdateContext context, long currentPos) { final long tail = Math.min(sourceRowSet.size() - 1, currentPos + forwardTimeScaleUnits); // pop out all values from the current window that are not in the new window - while (!windowPos.isEmpty() && windowPos.front() < head) { + while (!windowPosOrTimestamp.isEmpty() && windowPosOrTimestamp.front() < head) { pop(context, windowKeys.remove(), (int)windowIndices.remove()); - windowPos.remove(); + windowPosOrTimestamp.remove(); } - // check our current values to see if it now matches the window - while(currentInfluencerPos <= tail) { + if (windowPosOrTimestamp.isEmpty()) { + reset(context); + } + + // skip values until they match the window + while(currentInfluencerPosOrTimestamp < head) { + currentInfluencerIndex++; + + if (currentInfluencerIndex < influencerPosChunk.get().size()) { + currentInfluencerPosOrTimestamp = influencerPosChunk.get().get(currentInfluencerIndex); + currentInfluencerKey = influencerKeyChunk.get().get(currentInfluencerIndex); + } else { + currentInfluencerPosOrTimestamp = Long.MAX_VALUE; + currentInfluencerKey = Long.MAX_VALUE; + } + } + + // push matching values + while(currentInfluencerPosOrTimestamp <= tail) { push(context, currentInfluencerKey, currentInfluencerIndex); windowKeys.add(currentInfluencerKey); - windowPos.add(currentInfluencerPos); + windowPosOrTimestamp.add(currentInfluencerPosOrTimestamp); windowIndices.add(currentInfluencerIndex); currentInfluencerIndex++; - if (influencerPosIterator.hasNext()) { - currentInfluencerKey = influencerKeyIterator.next(); - currentInfluencerPos = influencerPosIterator.next(); + if (currentInfluencerIndex < influencerPosChunk.get().size()) { + currentInfluencerPosOrTimestamp = influencerPosChunk.get().get(currentInfluencerIndex); + currentInfluencerKey = influencerKeyChunk.get().get(currentInfluencerIndex); } else { - currentInfluencerPos = Long.MAX_VALUE; + currentInfluencerPosOrTimestamp = Long.MAX_VALUE; currentInfluencerKey = Long.MAX_VALUE; } } + } + + public void fillWindowTime(UpdateContext context, long currentTimestamp) { + // compute the head and tail positions (inclusive) + final long head = currentTimestamp - reverseTimeScaleUnits; + final long tail = currentTimestamp + forwardTimeScaleUnits; + + // pop out all values from the current window that are not in the new window + while (!windowPosOrTimestamp.isEmpty() && windowPosOrTimestamp.front() < head) { + pop(context, windowKeys.remove(), (int)windowIndices.remove()); + windowPosOrTimestamp.remove(); + } - if (windowPos.isEmpty()) { + if (windowPosOrTimestamp.isEmpty()) { reset(context); } + + // skip values until they match the window + while(currentInfluencerPosOrTimestamp < head) { + currentInfluencerIndex++; + + if (ssaIterator.hasNext()) { + ssaIterator.next(); + currentInfluencerPosOrTimestamp = ssaIterator.getValue(); + currentInfluencerKey = ssaIterator.getKey(); + } else { + currentInfluencerPosOrTimestamp = Long.MAX_VALUE; + currentInfluencerKey = Long.MAX_VALUE; + } + } + + // push matching values + while(currentInfluencerPosOrTimestamp <= tail) { + push(context, currentInfluencerKey, currentInfluencerIndex); + windowKeys.add(currentInfluencerKey); + windowPosOrTimestamp.add(currentInfluencerPosOrTimestamp); + windowIndices.add(currentInfluencerIndex); + currentInfluencerIndex++; + + if (ssaIterator.hasNext()) { + ssaIterator.next(); + currentInfluencerPosOrTimestamp = ssaIterator.getValue(); + currentInfluencerKey = ssaIterator.getKey(); + } else { + currentInfluencerPosOrTimestamp = Long.MAX_VALUE; + currentInfluencerKey = Long.MAX_VALUE; + } + } } @Override public void close() { - try (final RowSet.Iterator ignoredIt1 = influencerPosIterator; - final RowSet.Iterator ignoredIt2 = influencerKeyIterator; + try (final SizedLongChunk ignoredChk1 = influencerKeyChunk; + final SizedLongChunk ignoredChk2 = influencerPosChunk; final RowSet ignoredRs1 = affectedRows; - final RowSet ignoredRs2 = influencerKeys; + final RowSet ignoredRs2 = influencerRows; final RowSet ignoredRs3 = affectedRowPositions; final RowSet ignoredRs4 = influencerPositions; final RowSet ignoredRs5 = newModified) { @@ -197,23 +403,22 @@ public void close() { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this operation * @param control the control parameters for operation - * @param timeRecorder an optional recorder for a timestamp column. If this is null, it will be assumed time is - * measured in integer ticks. + * @param timestampColumnName the optional time stamp column for windowing (uses ticks if not provided) * @param redirContext the row redirection context to use for the operation */ public UpdateByWindowedOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, - @Nullable final LongRecordingUpdateByOperator timeRecorder, @Nullable final String timestampColumnName, + @Nullable final ColumnSource timestampColumnSource, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { this.pair = pair; this.affectingColumns = affectingColumns; this.control = control; - this.recorder = timeRecorder; this.timestampColumnName = timestampColumnName; + this.timestampColumnSource = timestampColumnSource == null ? null : ReinterpretUtils.maybeConvertToPrimitive(timestampColumnSource); this.reverseTimeScaleUnits = reverseTimeScaleUnits; this.forwardTimeScaleUnits = forwardTimeScaleUnits; this.redirContext = redirContext; @@ -224,9 +429,9 @@ public UpdateByWindowedOperator(@NotNull final MatchPair pair, public abstract void reset(UpdateContext context); // return the first row that affects this key - public long computeFirstAffectingKey(long key, @NotNull final RowSet source) { + public long computeFirstAffectingKey(long key, @NotNull final RowSet source, final LongSegmentedSortedArray timestampSsa) { - if (recorder == null) { + if (timestampColumnName == null) { // ticks final long keyPos = source.find(key); final long idx = (keyPos < 0) ? -keyPos - reverseTimeScaleUnits : keyPos - reverseTimeScaleUnits + 1; @@ -236,13 +441,15 @@ public long computeFirstAffectingKey(long key, @NotNull final RowSet source) { return source.firstRowKey(); } return source.get(idx); + } else { + } return -1; } // return the last row that affects this key - public long computeLastAffectingKey(long key, @NotNull final RowSet source) { - if (recorder == null) { + public long computeLastAffectingKey(long key, @NotNull final RowSet source, final LongSegmentedSortedArray timestampSsa) { + if (timestampColumnName == null) { // ticks final long keyPos = source.find(key); final long idx = keyPos + forwardTimeScaleUnits; @@ -252,13 +459,15 @@ public long computeLastAffectingKey(long key, @NotNull final RowSet source) { return source.firstRowKey(); } return source.get(idx); + } else { + } return -1; } // return the first row affected by this key - public long computeFirstAffectedKey(long key, @NotNull final RowSet source) { - if (recorder == null) { + public long computeFirstAffectedKey(long key, @NotNull final RowSet source, final LongSegmentedSortedArray timestampSsa) { + if (timestampColumnName == null) { // ticks final long keyPos = source.find(key); final long idx = (keyPos < 0) ? -keyPos - forwardTimeScaleUnits - 1 : keyPos - forwardTimeScaleUnits; @@ -268,13 +477,16 @@ public long computeFirstAffectedKey(long key, @NotNull final RowSet source) { return source.firstRowKey(); } return source.get(idx); + } else { + // time-based, use the SSA +// long ts = this.time } return -1; } // return the last row affected by this key - public long computeLastAffectedKey(long key, @NotNull final RowSet source) { - if (recorder == null) { + public long computeLastAffectedKey(long key, @NotNull final RowSet source, final LongSegmentedSortedArray timestampSsa) { + if (timestampColumnName == null) { // ticks final long keyPos = source.find(key); final long idx = (keyPos < 0) ? -keyPos + reverseTimeScaleUnits - 1 : keyPos + reverseTimeScaleUnits; @@ -284,6 +496,8 @@ public long computeLastAffectedKey(long key, @NotNull final RowSet source) { return source.firstRowKey(); } return source.get(idx); + } else { + } return -1; } @@ -292,15 +506,23 @@ public long computeLastAffectedKey(long key, @NotNull final RowSet source) { public void initializeFor(@NotNull final UpdateContext context, @NotNull final RowSet updateRowSet) { final UpdateWindowedContext ctx = (UpdateWindowedContext) context; - // load all the influencer values this update will need + + // pre=load all the influencer values this update will need ctx.loadInfluencerValueChunk(); - // setup the iterators we will need for managing the windows - ctx.influencerKeyIterator = ctx.influencerKeys.iterator(); - ctx.currentInfluencerKey = ctx.influencerKeyIterator.next(); - if (ctx.influencerPositions != null) { - ctx.influencerPosIterator = ctx.influencerPositions.iterator(); - ctx.currentInfluencerPos = ctx.influencerPosIterator.next(); + // load all the influencer keys + ctx.influencerKeyChunk = new SizedLongChunk(ctx.influencerRows.intSize()); + ctx.influencerRows.fillRowKeyChunk(ctx.influencerKeyChunk.get()); + ctx.currentInfluencerKey = ctx.influencerRows.firstRowKey(); + + if (timestampColumnName == null) { + // load all the influencer positions + ctx.influencerPosChunk = new SizedLongChunk(ctx.influencerRows.intSize()); + ctx.influencerPositions.fillRowKeyChunk(ctx.influencerPosChunk.get()); + ctx.currentInfluencerPosOrTimestamp = ctx.influencerPositions.firstRowKey(); + } else { + ctx.ssaIterator = ctx.timestampSsa.iterator(false, false); + ctx.currentInfluencerPosOrTimestamp = ctx.ssaIterator.nextValue(); } ctx.currentInfluencerIndex = 0; } @@ -348,9 +570,16 @@ public String getTimestampColumnName() { @Override public boolean requiresKeys() { - return true; + return false; + } + + /*** windowed operators need position data when computing ticks */ + @Override + public boolean requiresPositions() { + return this.timestampColumnName == null; } + @Override public boolean requiresValues(@NotNull final UpdateContext context) { // windowed operators don't need current values supplied to them, they only care about windowed values which diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java index f62dc51db28..26968931a31 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java @@ -1,24 +1,25 @@ package io.deephaven.engine.table.impl; import io.deephaven.api.updateby.UpdateByControl; -import io.deephaven.chunk.WritableChunk; -import io.deephaven.chunk.WritableLongChunk; +import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.sized.SizedLongChunk; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.*; +import io.deephaven.engine.table.impl.sources.ReinterpretUtils; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; import io.deephaven.engine.table.impl.util.UpdateSizeCalculator; import io.deephaven.util.SafeCloseable; +import org.apache.commons.lang3.mutable.MutableLong; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import java.util.Arrays; import java.util.Map; -import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; +import static io.deephaven.util.QueryConstants.NULL_LONG; /** * An implementation of {@link UpdateBy} dedicated to zero key computation. @@ -28,8 +29,10 @@ class ZeroKeyUpdateBy extends UpdateBy { final boolean applyShifts; /** store timestamp data in an Ssa (if needed) */ - final LongSegmentedSortedArray tsSsa; - final String tsColumnName; + final String timestampColumnName; + final LongSegmentedSortedArray timestampSsa; + final ChunkSource.WithPrev timestampColumn; + final ModifiedColumnSet timestampColumnSet; /** * Perform an updateBy without any key columns. @@ -70,15 +73,19 @@ protected ZeroKeyUpdateBy(@NotNull final UpdateByOperator[] operators, super(operators, source, redirContext, control); // do we need a timestamp SSA? - this.tsColumnName = Arrays.stream(operators) + this.timestampColumnName = Arrays.stream(operators) .filter(op -> op.getTimestampColumnName() != null) .map(UpdateByOperator::getTimestampColumnName) .findFirst().orElse(null); - if (tsColumnName != null) { - this.tsSsa = new LongSegmentedSortedArray(1024); + if (timestampColumnName != null) { + this.timestampSsa = new LongSegmentedSortedArray(4096); + this.timestampColumn = ReinterpretUtils.maybeConvertToPrimitive(source.getColumnSource(this.timestampColumnName)); + this.timestampColumnSet = source.newModifiedColumnSet(timestampColumnName); } else { - this.tsSsa = null; + this.timestampSsa = null; + this.timestampColumn = null; + this.timestampColumnSet = null; } this.applyShifts = applyShifts; } @@ -87,6 +94,111 @@ ZeroKeyUpdateByListener newListener(@NotNull final String description, @NotNull return new ZeroKeyUpdateByListener(description, source, result); } + private void processUpdateForSsa(TableUpdate upstream) { + final boolean stampModified = upstream.modifiedColumnSet().containsAny(timestampColumnSet); + + final RowSet restampRemovals; + final RowSet restampAdditions; + + // modifies are remove + add operations + if (stampModified) { + restampAdditions = upstream.added().union(upstream.modified()); + restampRemovals = upstream.removed().union(upstream.getModifiedPreShift()); + } else { + restampAdditions = upstream.added(); + restampRemovals = upstream.removed(); + } + + // removes + if (restampRemovals.isNonempty()) { + final int size = (int)Math.min(restampRemovals.size(), 4096); + try (final RowSequence.Iterator it = restampRemovals.getRowSequenceIterator(); + final ChunkSource.GetContext context = timestampColumn.makeGetContext(size); + final WritableLongChunk ssaValues = WritableLongChunk.makeWritableChunk(size); + final WritableLongChunk ssaKeys = WritableLongChunk.makeWritableChunk(size)) { + + MutableLong lastTimestamp = new MutableLong(NULL_LONG); + while (it.hasMore()) { + RowSequence chunkRs = it.getNextRowSequenceWithLength(4096); + + // get the chunks for values and keys + LongChunk valuesChunk = timestampColumn.getPrevChunk(context, chunkRs).asLongChunk(); + LongChunk keysChunk = chunkRs.asRowKeyChunk(); + + // push only non-null values/keys into the Ssa + fillChunkWithNonNull(keysChunk, valuesChunk, ssaKeys, ssaValues, lastTimestamp); + timestampSsa.remove(ssaValues, ssaKeys); + } + } + } + + // shifts + if (upstream.shifted().nonempty()) { + final int size = Math.max(upstream.modified().intSize() + Math.max(upstream.added().intSize(), upstream.removed().intSize()), (int) upstream.shifted().getEffectiveSize()); + try (final RowSet prevRowSet = source.getRowSet().copyPrev(); + final RowSet withoutMods = prevRowSet.minus(upstream.getModifiedPreShift()); + final ColumnSource.GetContext getContext = timestampColumn.makeGetContext(size)) { + + final RowSetShiftData.Iterator sit = upstream.shifted().applyIterator(); + while (sit.hasNext()) { + sit.next(); + try (final RowSet subRowSet = withoutMods.subSetByKeyRange(sit.beginRange(), sit.endRange()); + final RowSet rowSetToShift = subRowSet.minus(upstream.removed())) { + if (rowSetToShift.isEmpty()) { + continue; + } + + final LongChunk shiftValues = timestampColumn.getPrevChunk(getContext, rowSetToShift).asLongChunk(); + + timestampSsa.applyShift(shiftValues, rowSetToShift.asRowKeyChunk(), sit.shiftDelta()); + } + } + } + } + + // adds + if (restampAdditions.isNonempty()) { + final int size = (int)Math.min(restampAdditions.size(), 4096); + try (final RowSequence.Iterator it = restampAdditions.getRowSequenceIterator(); + final ChunkSource.GetContext context = timestampColumn.makeGetContext(size); + final WritableLongChunk ssaValues = WritableLongChunk.makeWritableChunk(size); + final WritableLongChunk ssaKeys = WritableLongChunk.makeWritableChunk(size)) { + MutableLong lastTimestamp = new MutableLong(NULL_LONG); + + while (it.hasMore()) { + RowSequence chunkRs = it.getNextRowSequenceWithLength(4096); + + // get the chunks for values and keys + LongChunk valuesChunk = timestampColumn.getChunk(context, chunkRs).asLongChunk(); + LongChunk keysChunk = chunkRs.asRowKeyChunk(); + + // push only non-null values/keys into the Ssa + fillChunkWithNonNull(keysChunk, valuesChunk, ssaKeys, ssaValues, lastTimestamp); + timestampSsa.insert(ssaValues, ssaKeys); + } + } + } + } + + private void fillChunkWithNonNull(LongChunk keysChunk, LongChunk valuesChunk, WritableLongChunk ssaKeys, WritableLongChunk ssaValues, MutableLong lastTimestamp) { + // reset the insertion chunks + ssaValues.setSize(0); + ssaKeys.setSize(0); + + // add only non-null timestamps to this Ssa + for (int i = 0; i < valuesChunk.size(); i++) { + long ts = valuesChunk.get(i); + if (ts < lastTimestamp.longValue()) { + throw(new IllegalStateException("updateBy time-based operators require non-descending timestamp values")); + } + if (ts != NULL_LONG) { + ssaValues.add(ts); + ssaKeys.add(keysChunk.get(i)); + } + } + } + + void doInitialAdditions() { final TableUpdateImpl fakeUpdate = new TableUpdateImpl(source.getRowSet(), RowSetFactory.empty(), @@ -98,17 +210,20 @@ void doInitialAdditions() { } // add all the SSA data + if (timestampColumnName != null) { + processUpdateForSsa(fakeUpdate); + } try (final UpdateContext ctx = new UpdateContext(fakeUpdate, null, true)) { ctx.setAllAffected(); // do a reprocessing phase for operators that can't add directly - ctx.reprocessRows(RowSetShiftData.EMPTY); + ctx.processRows(RowSetShiftData.EMPTY); } } /** - * An object to hold the transient state during a single {@link ShiftAwareListener#onUpdate(TableUpdate)} update + * An object to hold the transient state during a single {@link InstrumentedTableUpdateListenerAdapter#onUpdate(TableUpdate)} update * cycle. */ private class UpdateContext implements SafeCloseable { @@ -124,6 +239,9 @@ private class UpdateContext implements SafeCloseable { /** true if any operator requested keys */ boolean anyRequireKeys; + /** true if any operator requested positions */ + boolean anyRequirePositions; + /** An array of context objects for each underlying operator */ final UpdateByOperator.UpdateContext[] opContext; @@ -139,6 +257,9 @@ private class UpdateContext implements SafeCloseable { /** A Chunk of longs to store the keys being updated */ final SizedLongChunk keyChunk; + /** A Chunk of longs to store the posiitions of the keys being updated */ + final SizedLongChunk posChunk; + /** A sharedContext to be used with previous values */ SharedContext prevSharedContext; @@ -151,8 +272,7 @@ private class UpdateContext implements SafeCloseable { /** A Long Chunk for previous keys */ WritableLongChunk prevKeyChunk; - - final RowSet affectedRows; + final RowSet rowsToProcess; @SuppressWarnings("resource") UpdateContext(@NotNull final TableUpdate upstream, @@ -167,6 +287,7 @@ private class UpdateContext implements SafeCloseable { this.fillContexts = new SizedSafeCloseable[operators.length]; this.opContext = new UpdateByOperator.UpdateContext[operators.length]; this.keyChunk = new SizedLongChunk<>(); + this.posChunk = new SizedLongChunk<>(); this.inputChunkPopulated = new boolean[operators.length]; if (upstream.shifted().nonempty()) { @@ -181,9 +302,6 @@ private class UpdateContext implements SafeCloseable { this.prevFillContexts = new ChunkSource.FillContext[operators.length]; } - final boolean upstreamAppendOnly = - isInitializeStep || UpdateByOperator.isAppendOnly(upstream, source.getRowSet().lastRowKeyPrev()); - // noinspection unchecked this.postWorkingChunks = new SizedSafeCloseable[operators.length]; for (int opIdx = 0; opIdx < operators.length; opIdx++) { @@ -196,7 +314,7 @@ private class UpdateContext implements SafeCloseable { continue; } - opContext[opIdx] = operators[opIdx].makeUpdateContext(chunkSize); + opContext[opIdx] = operators[opIdx].makeUpdateContext(chunkSize, timestampSsa); final int slotPosition = inputSourceSlots[opIdx]; if (fillContexts[slotPosition] == null) { @@ -226,12 +344,12 @@ private class UpdateContext implements SafeCloseable { } // trigger the operator to determine its own set of affected rows (window-specific), do not close() // since this is managed by the operator context - final RowSet rs = opContext[opIdx].determineAffectedRows(upstream, source.getRowSet(), upstreamAppendOnly); + final RowSet rs = opContext[opIdx].determineAffectedRows(upstream, source.getRowSet(), isInitializeStep); // union the operator rowsets together to get a global set tmp.insert(rs); } - affectedRows = tmp; + rowsToProcess = tmp; } public SharedContext getSharedContext() { @@ -255,6 +373,7 @@ void setChunkSize(int newChunkSize) { this.chunkSize = newChunkSize; this.keyChunk.ensureCapacity(newChunkSize); + this.posChunk.ensureCapacity(newChunkSize); for (int opIdx = 0; opIdx < operators.length; opIdx++) { if (!opAffected[opIdx]) { @@ -277,12 +396,16 @@ void initializeFor(@NotNull final RowSet updateRowSet) { if (opAffected[opIdx]) { operators[opIdx].initializeFor(opContext[opIdx], updateRowSet); anyRequireKeys |= operators[opIdx].requiresKeys(); + anyRequirePositions |= operators[opIdx].requiresPositions(); } } if (anyRequireKeys) { keyChunk.ensureCapacity(chunkSize); } + if (anyRequirePositions) { + posChunk.ensureCapacity(chunkSize); + } } void finishFor() { @@ -300,7 +423,8 @@ void finishFor() { public void close() { sharedContext.close(); keyChunk.close(); - affectedRows.close(); + posChunk.close(); + rowsToProcess.close(); if (prevKeyChunk != null) { prevKeyChunk.close(); @@ -356,31 +480,59 @@ boolean anyModified() { void doUpdate(@NotNull final RowSet updateRowSet, @NotNull final RowSet preShiftUpdateRowSet) { - if (updateRowSet.isEmpty()) { + + } + + /** + * Locate the smallest key that requires reprocessing and then replay the table from that point + */ + private void processRows(RowSetShiftData shifted) { + // Get a sub-index of the source from that minimum reprocessing index and make sure we update our + // contextual chunks and FillContexts to an appropriate size for this step. + final RowSet sourceRowSet = source.getRowSet(); + + final int newChunkSize = (int) Math.min(control.chunkCapacityOrDefault(), rowsToProcess.size()); + setChunkSize(newChunkSize); + + for (int opIndex = 0; opIndex < operators.length; opIndex++) { + if (opAffected[opIndex]) { + final long keyStart = opContext[opIndex].getAffectedRows().firstRowKey(); + final long keyBefore; + try (final RowSet.SearchIterator sit = sourceRowSet.searchIterator()) { + keyBefore = sit.binarySearchValue( + (compareTo, ignored) -> Long.compare(keyStart - 1, compareTo), 1); + } + // apply a shift to keyBefore since the output column is still in prev key space- + + operators[opIndex].resetForProcess(opContext[opIndex], sourceRowSet, keyBefore); + } + } + + // Now iterate rowset to reprocess. + if (rowsToProcess.isEmpty()) { return; } - try (final RowSequence.Iterator okIt = updateRowSet.getRowSequenceIterator(); - final RowSequence.Iterator preShiftOkIt = preShiftUpdateRowSet == updateRowSet ? null - : preShiftUpdateRowSet.getRowSequenceIterator()) { - initializeFor(updateRowSet); + initializeFor(rowsToProcess); - while (okIt.hasMore()) { + try (final RowSet positionsToProcess = anyRequirePositions ? source.getRowSet().invert(rowsToProcess) : null; + final RowSequence.Iterator keyIt = rowsToProcess.getRowSequenceIterator(); + final RowSequence.Iterator posIt = positionsToProcess == null ? null + : positionsToProcess.getRowSequenceIterator()) { + + while (keyIt.hasMore()) { sharedContext.reset(); if (prevSharedContext != null) { prevSharedContext.reset(); } Arrays.fill(inputChunkPopulated, false); - final RowSequence chunkOk = okIt.getNextRowSequenceWithLength(chunkSize); - final RowSequence prevChunkOk = preShiftUpdateRowSet == updateRowSet ? chunkOk - : preShiftOkIt.getNextRowSequenceWithLength(chunkSize); - + final RowSequence chunkOk = keyIt.getNextRowSequenceWithLength(chunkSize); if (anyRequireKeys) { chunkOk.fillRowKeyChunk(keyChunk.get()); - if (prevChunkOk != chunkOk) { - prevChunkOk.fillRowKeyChunk(prevKeyChunk); - } + } + if (anyRequirePositions) { + posIt.getNextRowSequenceWithLength(chunkSize).fillRowKeyChunk(posChunk.get()); } for (int opIdx = 0; opIdx < operators.length; opIdx++) { @@ -403,46 +555,17 @@ void doUpdate(@NotNull final RowSet updateRowSet, currentOp.processChunk(opContext[opIdx], intersect, keyChunk.get(), + posChunk.get(), postWorkingChunks[slotPosition].get(), source.getRowSet()); } } } } - finishFor(); } } - /** - * Locate the smallest key that requires reprocessing and then replay the table from that point - */ - private void reprocessRows(RowSetShiftData shifted) { - // Get a sub-index of the source from that minimum reprocessing index and make sure we update our - // contextual chunks and FillContexts to an appropriate size for this step. - final RowSet sourceRowSet = source.getRowSet(); - - final int newChunkSize = (int) Math.min(control.chunkCapacityOrDefault(), affectedRows.size()); - setChunkSize(newChunkSize); - - for (int opIndex = 0; opIndex < operators.length; opIndex++) { - if (opAffected[opIndex]) { - final long keyStart = opContext[opIndex].getAffectedRows().firstRowKey(); - final long keyBefore; - try (final RowSet.SearchIterator sit = sourceRowSet.searchIterator()) { - keyBefore = sit.binarySearchValue( - (compareTo, ignored) -> Long.compare(keyStart - 1, compareTo), 1); - } - // apply a shift to keyBefore since the output column is still in prev key space- - - operators[opIndex].resetForProcess(opContext[opIndex], sourceRowSet, keyBefore); - } - } - - // Now iterate rowset to reprocess. - doUpdate(affectedRows, affectedRows); - } - /** * Prepare the specified chunk for use. * @@ -481,7 +604,7 @@ private void prepareValuesChunkFor(final int opIdx, } /** - * The Listener for apply an upstream {@link ShiftAwareListener#onUpdate(Update) update} + * The Listener for apply an upstream {@link InstrumentedTableUpdateListenerAdapter#onUpdate(TableUpdate) update} */ class ZeroKeyUpdateByListener extends InstrumentedTableUpdateListenerAdapter { private final QueryTable result; @@ -509,7 +632,12 @@ public ZeroKeyUpdateByListener(@Nullable String description, @Override public void onUpdate(TableUpdate upstream) { + try (final UpdateContext ctx = new UpdateContext(upstream, inputModifiedColumnSets, false)) { + // update the Ssa + if (timestampColumnName != null) { + processUpdateForSsa(upstream); + } if (applyShifts) { if (redirContext.isRedirected()) { @@ -532,7 +660,7 @@ public void onUpdate(TableUpdate upstream) { } // Now do the reprocessing phase. - ctx.reprocessRows(upstream.shifted()); + ctx.processRows(upstream.shifted()); final TableUpdateImpl downstream = new TableUpdateImpl(); // copy these rowSets since TableUpdateImpl#reset will close them with the upstream update diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/ByteReverseSegmentedSortedArray.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/ByteReverseSegmentedSortedArray.java index 4e8e360592a..4f59c601780 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/ByteReverseSegmentedSortedArray.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/ByteReverseSegmentedSortedArray.java @@ -1470,11 +1470,11 @@ public long size() { * one before the first value (so that next must be called) * @return an iterator for this SSA */ - Iterator iterator(boolean disallowExactMatch, boolean isRightSide) { + public Iterator iterator(boolean disallowExactMatch, boolean isRightSide) { return new Iterator(disallowExactMatch, isRightSide); } - final class Iterator { + public final class Iterator { int leafIndex = 0; int indexWithinLeaf = 0; private final boolean disallowExactMatch; @@ -1488,7 +1488,7 @@ final class Iterator { } } - void next() { + public void next() { indexWithinLeaf++; if (leafCount > 1) { if (indexWithinLeaf == leafSizes[leafIndex]) { @@ -1498,7 +1498,7 @@ void next() { } } - boolean hasNext() { + public boolean hasNext() { if (leafCount == 0) { return false; } @@ -1508,7 +1508,7 @@ boolean hasNext() { return leafIndex < leafCount - 1 || (indexWithinLeaf < leafSizes[leafIndex] - 1); } - byte getValue() { + public byte getValue() { if (leafCount == 1) { return directoryValues[indexWithinLeaf]; } @@ -1517,7 +1517,7 @@ byte getValue() { } } - byte nextValue() { + public byte nextValue() { Assert.assertion(hasNext(), "hasNext()"); if (leafCount == 1) { return directoryValues[indexWithinLeaf + 1]; @@ -1529,7 +1529,7 @@ else if (indexWithinLeaf == leafSizes[leafIndex] - 1) { } } - long getKey() { + public long getKey() { if (leafCount == 1) { return directoryRowKeys[indexWithinLeaf]; } @@ -1538,7 +1538,7 @@ long getKey() { } } - long nextKey() { + public long nextKey() { Assert.assertion(hasNext(), "hasNext()"); if (leafCount == 1) { return directoryRowKeys[indexWithinLeaf + 1]; @@ -1556,7 +1556,7 @@ else if (indexWithinLeaf == leafSizes[leafIndex] - 1) { * * @param value the value we are searching for */ - void advanceToLast(byte value) { + public void advanceToLast(byte value) { advanceToInternal(value, true); } @@ -1565,7 +1565,7 @@ void advanceToLast(byte value) { * * @param value the value we are searching for */ - void advanceToBeforeFirst(byte value) { + public void advanceToBeforeFirst(byte value) { advanceToInternal(value, false); if (disallowExactMatch) { if (hasNext() && nextValue() == value) { @@ -1663,7 +1663,7 @@ else if (newLeafIndex > 0) { /** * Advance the iterator to the last value which is equal to the current value. */ - void advanceWhileEqual() { + public void advanceWhileEqual() { final byte value = getValue(); findLastInLeaf(value); while (leafIndex < leafCount - 1) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/ByteSegmentedSortedArray.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/ByteSegmentedSortedArray.java index cfe74d3cacb..8a5e520801e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/ByteSegmentedSortedArray.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/ByteSegmentedSortedArray.java @@ -1469,11 +1469,11 @@ public long size() { * one before the first value (so that next must be called) * @return an iterator for this SSA */ - Iterator iterator(boolean disallowExactMatch, boolean isRightSide) { + public Iterator iterator(boolean disallowExactMatch, boolean isRightSide) { return new Iterator(disallowExactMatch, isRightSide); } - final class Iterator { + public final class Iterator { int leafIndex = 0; int indexWithinLeaf = 0; private final boolean disallowExactMatch; @@ -1487,7 +1487,7 @@ final class Iterator { } } - void next() { + public void next() { indexWithinLeaf++; if (leafCount > 1) { if (indexWithinLeaf == leafSizes[leafIndex]) { @@ -1497,7 +1497,7 @@ void next() { } } - boolean hasNext() { + public boolean hasNext() { if (leafCount == 0) { return false; } @@ -1507,7 +1507,7 @@ boolean hasNext() { return leafIndex < leafCount - 1 || (indexWithinLeaf < leafSizes[leafIndex] - 1); } - byte getValue() { + public byte getValue() { if (leafCount == 1) { return directoryValues[indexWithinLeaf]; } @@ -1516,7 +1516,7 @@ byte getValue() { } } - byte nextValue() { + public byte nextValue() { Assert.assertion(hasNext(), "hasNext()"); if (leafCount == 1) { return directoryValues[indexWithinLeaf + 1]; @@ -1528,7 +1528,7 @@ else if (indexWithinLeaf == leafSizes[leafIndex] - 1) { } } - long getKey() { + public long getKey() { if (leafCount == 1) { return directoryRowKeys[indexWithinLeaf]; } @@ -1537,7 +1537,7 @@ long getKey() { } } - long nextKey() { + public long nextKey() { Assert.assertion(hasNext(), "hasNext()"); if (leafCount == 1) { return directoryRowKeys[indexWithinLeaf + 1]; @@ -1555,7 +1555,7 @@ else if (indexWithinLeaf == leafSizes[leafIndex] - 1) { * * @param value the value we are searching for */ - void advanceToLast(byte value) { + public void advanceToLast(byte value) { advanceToInternal(value, true); } @@ -1564,7 +1564,7 @@ void advanceToLast(byte value) { * * @param value the value we are searching for */ - void advanceToBeforeFirst(byte value) { + public void advanceToBeforeFirst(byte value) { advanceToInternal(value, false); if (disallowExactMatch) { if (hasNext() && nextValue() == value) { @@ -1662,7 +1662,7 @@ else if (newLeafIndex > 0) { /** * Advance the iterator to the last value which is equal to the current value. */ - void advanceWhileEqual() { + public void advanceWhileEqual() { final byte value = getValue(); findLastInLeaf(value); while (leafIndex < leafCount - 1) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/CharReverseSegmentedSortedArray.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/CharReverseSegmentedSortedArray.java index 0757105fbb5..063f50a14ea 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/CharReverseSegmentedSortedArray.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/CharReverseSegmentedSortedArray.java @@ -1465,11 +1465,11 @@ public long size() { * one before the first value (so that next must be called) * @return an iterator for this SSA */ - Iterator iterator(boolean disallowExactMatch, boolean isRightSide) { + public Iterator iterator(boolean disallowExactMatch, boolean isRightSide) { return new Iterator(disallowExactMatch, isRightSide); } - final class Iterator { + public final class Iterator { int leafIndex = 0; int indexWithinLeaf = 0; private final boolean disallowExactMatch; @@ -1483,7 +1483,7 @@ final class Iterator { } } - void next() { + public void next() { indexWithinLeaf++; if (leafCount > 1) { if (indexWithinLeaf == leafSizes[leafIndex]) { @@ -1493,7 +1493,7 @@ void next() { } } - boolean hasNext() { + public boolean hasNext() { if (leafCount == 0) { return false; } @@ -1503,7 +1503,7 @@ boolean hasNext() { return leafIndex < leafCount - 1 || (indexWithinLeaf < leafSizes[leafIndex] - 1); } - char getValue() { + public char getValue() { if (leafCount == 1) { return directoryValues[indexWithinLeaf]; } @@ -1512,7 +1512,7 @@ char getValue() { } } - char nextValue() { + public char nextValue() { Assert.assertion(hasNext(), "hasNext()"); if (leafCount == 1) { return directoryValues[indexWithinLeaf + 1]; @@ -1524,7 +1524,7 @@ else if (indexWithinLeaf == leafSizes[leafIndex] - 1) { } } - long getKey() { + public long getKey() { if (leafCount == 1) { return directoryRowKeys[indexWithinLeaf]; } @@ -1533,7 +1533,7 @@ long getKey() { } } - long nextKey() { + public long nextKey() { Assert.assertion(hasNext(), "hasNext()"); if (leafCount == 1) { return directoryRowKeys[indexWithinLeaf + 1]; @@ -1551,7 +1551,7 @@ else if (indexWithinLeaf == leafSizes[leafIndex] - 1) { * * @param value the value we are searching for */ - void advanceToLast(char value) { + public void advanceToLast(char value) { advanceToInternal(value, true); } @@ -1560,7 +1560,7 @@ void advanceToLast(char value) { * * @param value the value we are searching for */ - void advanceToBeforeFirst(char value) { + public void advanceToBeforeFirst(char value) { advanceToInternal(value, false); if (disallowExactMatch) { if (hasNext() && nextValue() == value) { @@ -1658,7 +1658,7 @@ else if (newLeafIndex > 0) { /** * Advance the iterator to the last value which is equal to the current value. */ - void advanceWhileEqual() { + public void advanceWhileEqual() { final char value = getValue(); findLastInLeaf(value); while (leafIndex < leafCount - 1) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/CharSegmentedSortedArray.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/CharSegmentedSortedArray.java index ab99a442016..c653baf6f93 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/CharSegmentedSortedArray.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/CharSegmentedSortedArray.java @@ -1464,11 +1464,11 @@ public long size() { * one before the first value (so that next must be called) * @return an iterator for this SSA */ - Iterator iterator(boolean disallowExactMatch, boolean isRightSide) { + public Iterator iterator(boolean disallowExactMatch, boolean isRightSide) { return new Iterator(disallowExactMatch, isRightSide); } - final class Iterator { + public final class Iterator { int leafIndex = 0; int indexWithinLeaf = 0; private final boolean disallowExactMatch; @@ -1482,7 +1482,7 @@ final class Iterator { } } - void next() { + public void next() { indexWithinLeaf++; if (leafCount > 1) { if (indexWithinLeaf == leafSizes[leafIndex]) { @@ -1492,7 +1492,7 @@ void next() { } } - boolean hasNext() { + public boolean hasNext() { if (leafCount == 0) { return false; } @@ -1502,7 +1502,7 @@ boolean hasNext() { return leafIndex < leafCount - 1 || (indexWithinLeaf < leafSizes[leafIndex] - 1); } - char getValue() { + public char getValue() { if (leafCount == 1) { return directoryValues[indexWithinLeaf]; } @@ -1511,7 +1511,7 @@ char getValue() { } } - char nextValue() { + public char nextValue() { Assert.assertion(hasNext(), "hasNext()"); if (leafCount == 1) { return directoryValues[indexWithinLeaf + 1]; @@ -1523,7 +1523,7 @@ else if (indexWithinLeaf == leafSizes[leafIndex] - 1) { } } - long getKey() { + public long getKey() { if (leafCount == 1) { return directoryRowKeys[indexWithinLeaf]; } @@ -1532,7 +1532,7 @@ long getKey() { } } - long nextKey() { + public long nextKey() { Assert.assertion(hasNext(), "hasNext()"); if (leafCount == 1) { return directoryRowKeys[indexWithinLeaf + 1]; @@ -1550,7 +1550,7 @@ else if (indexWithinLeaf == leafSizes[leafIndex] - 1) { * * @param value the value we are searching for */ - void advanceToLast(char value) { + public void advanceToLast(char value) { advanceToInternal(value, true); } @@ -1559,7 +1559,7 @@ void advanceToLast(char value) { * * @param value the value we are searching for */ - void advanceToBeforeFirst(char value) { + public void advanceToBeforeFirst(char value) { advanceToInternal(value, false); if (disallowExactMatch) { if (hasNext() && nextValue() == value) { @@ -1657,7 +1657,7 @@ else if (newLeafIndex > 0) { /** * Advance the iterator to the last value which is equal to the current value. */ - void advanceWhileEqual() { + public void advanceWhileEqual() { final char value = getValue(); findLastInLeaf(value); while (leafIndex < leafCount - 1) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/DoubleReverseSegmentedSortedArray.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/DoubleReverseSegmentedSortedArray.java index d14bcb602bc..f598eac7ee5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/DoubleReverseSegmentedSortedArray.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/DoubleReverseSegmentedSortedArray.java @@ -1471,11 +1471,11 @@ public long size() { * one before the first value (so that next must be called) * @return an iterator for this SSA */ - Iterator iterator(boolean disallowExactMatch, boolean isRightSide) { + public Iterator iterator(boolean disallowExactMatch, boolean isRightSide) { return new Iterator(disallowExactMatch, isRightSide); } - final class Iterator { + public final class Iterator { int leafIndex = 0; int indexWithinLeaf = 0; private final boolean disallowExactMatch; @@ -1489,7 +1489,7 @@ final class Iterator { } } - void next() { + public void next() { indexWithinLeaf++; if (leafCount > 1) { if (indexWithinLeaf == leafSizes[leafIndex]) { @@ -1499,7 +1499,7 @@ void next() { } } - boolean hasNext() { + public boolean hasNext() { if (leafCount == 0) { return false; } @@ -1509,7 +1509,7 @@ boolean hasNext() { return leafIndex < leafCount - 1 || (indexWithinLeaf < leafSizes[leafIndex] - 1); } - double getValue() { + public double getValue() { if (leafCount == 1) { return directoryValues[indexWithinLeaf]; } @@ -1518,7 +1518,7 @@ boolean hasNext() { } } - double nextValue() { + public double nextValue() { Assert.assertion(hasNext(), "hasNext()"); if (leafCount == 1) { return directoryValues[indexWithinLeaf + 1]; @@ -1530,7 +1530,7 @@ else if (indexWithinLeaf == leafSizes[leafIndex] - 1) { } } - long getKey() { + public long getKey() { if (leafCount == 1) { return directoryRowKeys[indexWithinLeaf]; } @@ -1539,7 +1539,7 @@ long getKey() { } } - long nextKey() { + public long nextKey() { Assert.assertion(hasNext(), "hasNext()"); if (leafCount == 1) { return directoryRowKeys[indexWithinLeaf + 1]; @@ -1557,7 +1557,7 @@ else if (indexWithinLeaf == leafSizes[leafIndex] - 1) { * * @param value the value we are searching for */ - void advanceToLast(double value) { + public void advanceToLast(double value) { advanceToInternal(value, true); } @@ -1566,7 +1566,7 @@ void advanceToLast(double value) { * * @param value the value we are searching for */ - void advanceToBeforeFirst(double value) { + public void advanceToBeforeFirst(double value) { advanceToInternal(value, false); if (disallowExactMatch) { if (hasNext() && nextValue() == value) { @@ -1664,7 +1664,7 @@ else if (newLeafIndex > 0) { /** * Advance the iterator to the last value which is equal to the current value. */ - void advanceWhileEqual() { + public void advanceWhileEqual() { final double value = getValue(); findLastInLeaf(value); while (leafIndex < leafCount - 1) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/DoubleSegmentedSortedArray.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/DoubleSegmentedSortedArray.java index 76e4bf6aee9..e86cd0ef38e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/DoubleSegmentedSortedArray.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/DoubleSegmentedSortedArray.java @@ -1471,11 +1471,11 @@ public long size() { * one before the first value (so that next must be called) * @return an iterator for this SSA */ - Iterator iterator(boolean disallowExactMatch, boolean isRightSide) { + public Iterator iterator(boolean disallowExactMatch, boolean isRightSide) { return new Iterator(disallowExactMatch, isRightSide); } - final class Iterator { + public final class Iterator { int leafIndex = 0; int indexWithinLeaf = 0; private final boolean disallowExactMatch; @@ -1489,7 +1489,7 @@ final class Iterator { } } - void next() { + public void next() { indexWithinLeaf++; if (leafCount > 1) { if (indexWithinLeaf == leafSizes[leafIndex]) { @@ -1499,7 +1499,7 @@ void next() { } } - boolean hasNext() { + public boolean hasNext() { if (leafCount == 0) { return false; } @@ -1509,7 +1509,7 @@ boolean hasNext() { return leafIndex < leafCount - 1 || (indexWithinLeaf < leafSizes[leafIndex] - 1); } - double getValue() { + public double getValue() { if (leafCount == 1) { return directoryValues[indexWithinLeaf]; } @@ -1518,7 +1518,7 @@ boolean hasNext() { } } - double nextValue() { + public double nextValue() { Assert.assertion(hasNext(), "hasNext()"); if (leafCount == 1) { return directoryValues[indexWithinLeaf + 1]; @@ -1530,7 +1530,7 @@ else if (indexWithinLeaf == leafSizes[leafIndex] - 1) { } } - long getKey() { + public long getKey() { if (leafCount == 1) { return directoryRowKeys[indexWithinLeaf]; } @@ -1539,7 +1539,7 @@ long getKey() { } } - long nextKey() { + public long nextKey() { Assert.assertion(hasNext(), "hasNext()"); if (leafCount == 1) { return directoryRowKeys[indexWithinLeaf + 1]; @@ -1557,7 +1557,7 @@ else if (indexWithinLeaf == leafSizes[leafIndex] - 1) { * * @param value the value we are searching for */ - void advanceToLast(double value) { + public void advanceToLast(double value) { advanceToInternal(value, true); } @@ -1566,7 +1566,7 @@ void advanceToLast(double value) { * * @param value the value we are searching for */ - void advanceToBeforeFirst(double value) { + public void advanceToBeforeFirst(double value) { advanceToInternal(value, false); if (disallowExactMatch) { if (hasNext() && nextValue() == value) { @@ -1664,7 +1664,7 @@ else if (newLeafIndex > 0) { /** * Advance the iterator to the last value which is equal to the current value. */ - void advanceWhileEqual() { + public void advanceWhileEqual() { final double value = getValue(); findLastInLeaf(value); while (leafIndex < leafCount - 1) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/FloatReverseSegmentedSortedArray.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/FloatReverseSegmentedSortedArray.java index 29714df112f..cb8013699be 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/FloatReverseSegmentedSortedArray.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/FloatReverseSegmentedSortedArray.java @@ -1471,11 +1471,11 @@ public long size() { * one before the first value (so that next must be called) * @return an iterator for this SSA */ - Iterator iterator(boolean disallowExactMatch, boolean isRightSide) { + public Iterator iterator(boolean disallowExactMatch, boolean isRightSide) { return new Iterator(disallowExactMatch, isRightSide); } - final class Iterator { + public final class Iterator { int leafIndex = 0; int indexWithinLeaf = 0; private final boolean disallowExactMatch; @@ -1489,7 +1489,7 @@ final class Iterator { } } - void next() { + public void next() { indexWithinLeaf++; if (leafCount > 1) { if (indexWithinLeaf == leafSizes[leafIndex]) { @@ -1499,7 +1499,7 @@ void next() { } } - boolean hasNext() { + public boolean hasNext() { if (leafCount == 0) { return false; } @@ -1509,7 +1509,7 @@ boolean hasNext() { return leafIndex < leafCount - 1 || (indexWithinLeaf < leafSizes[leafIndex] - 1); } - float getValue() { + public float getValue() { if (leafCount == 1) { return directoryValues[indexWithinLeaf]; } @@ -1518,7 +1518,7 @@ float getValue() { } } - float nextValue() { + public float nextValue() { Assert.assertion(hasNext(), "hasNext()"); if (leafCount == 1) { return directoryValues[indexWithinLeaf + 1]; @@ -1530,7 +1530,7 @@ else if (indexWithinLeaf == leafSizes[leafIndex] - 1) { } } - long getKey() { + public long getKey() { if (leafCount == 1) { return directoryRowKeys[indexWithinLeaf]; } @@ -1539,7 +1539,7 @@ long getKey() { } } - long nextKey() { + public long nextKey() { Assert.assertion(hasNext(), "hasNext()"); if (leafCount == 1) { return directoryRowKeys[indexWithinLeaf + 1]; @@ -1557,7 +1557,7 @@ else if (indexWithinLeaf == leafSizes[leafIndex] - 1) { * * @param value the value we are searching for */ - void advanceToLast(float value) { + public void advanceToLast(float value) { advanceToInternal(value, true); } @@ -1566,7 +1566,7 @@ void advanceToLast(float value) { * * @param value the value we are searching for */ - void advanceToBeforeFirst(float value) { + public void advanceToBeforeFirst(float value) { advanceToInternal(value, false); if (disallowExactMatch) { if (hasNext() && nextValue() == value) { @@ -1664,7 +1664,7 @@ else if (newLeafIndex > 0) { /** * Advance the iterator to the last value which is equal to the current value. */ - void advanceWhileEqual() { + public void advanceWhileEqual() { final float value = getValue(); findLastInLeaf(value); while (leafIndex < leafCount - 1) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/FloatSegmentedSortedArray.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/FloatSegmentedSortedArray.java index 188b806e93b..40207e00631 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/FloatSegmentedSortedArray.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/FloatSegmentedSortedArray.java @@ -1471,11 +1471,11 @@ public long size() { * one before the first value (so that next must be called) * @return an iterator for this SSA */ - Iterator iterator(boolean disallowExactMatch, boolean isRightSide) { + public Iterator iterator(boolean disallowExactMatch, boolean isRightSide) { return new Iterator(disallowExactMatch, isRightSide); } - final class Iterator { + public final class Iterator { int leafIndex = 0; int indexWithinLeaf = 0; private final boolean disallowExactMatch; @@ -1489,7 +1489,7 @@ final class Iterator { } } - void next() { + public void next() { indexWithinLeaf++; if (leafCount > 1) { if (indexWithinLeaf == leafSizes[leafIndex]) { @@ -1499,7 +1499,7 @@ void next() { } } - boolean hasNext() { + public boolean hasNext() { if (leafCount == 0) { return false; } @@ -1509,7 +1509,7 @@ boolean hasNext() { return leafIndex < leafCount - 1 || (indexWithinLeaf < leafSizes[leafIndex] - 1); } - float getValue() { + public float getValue() { if (leafCount == 1) { return directoryValues[indexWithinLeaf]; } @@ -1518,7 +1518,7 @@ float getValue() { } } - float nextValue() { + public float nextValue() { Assert.assertion(hasNext(), "hasNext()"); if (leafCount == 1) { return directoryValues[indexWithinLeaf + 1]; @@ -1530,7 +1530,7 @@ else if (indexWithinLeaf == leafSizes[leafIndex] - 1) { } } - long getKey() { + public long getKey() { if (leafCount == 1) { return directoryRowKeys[indexWithinLeaf]; } @@ -1539,7 +1539,7 @@ long getKey() { } } - long nextKey() { + public long nextKey() { Assert.assertion(hasNext(), "hasNext()"); if (leafCount == 1) { return directoryRowKeys[indexWithinLeaf + 1]; @@ -1557,7 +1557,7 @@ else if (indexWithinLeaf == leafSizes[leafIndex] - 1) { * * @param value the value we are searching for */ - void advanceToLast(float value) { + public void advanceToLast(float value) { advanceToInternal(value, true); } @@ -1566,7 +1566,7 @@ void advanceToLast(float value) { * * @param value the value we are searching for */ - void advanceToBeforeFirst(float value) { + public void advanceToBeforeFirst(float value) { advanceToInternal(value, false); if (disallowExactMatch) { if (hasNext() && nextValue() == value) { @@ -1664,7 +1664,7 @@ else if (newLeafIndex > 0) { /** * Advance the iterator to the last value which is equal to the current value. */ - void advanceWhileEqual() { + public void advanceWhileEqual() { final float value = getValue(); findLastInLeaf(value); while (leafIndex < leafCount - 1) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/IntReverseSegmentedSortedArray.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/IntReverseSegmentedSortedArray.java index 7d5e4e2a515..3f2af4a8042 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/IntReverseSegmentedSortedArray.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/IntReverseSegmentedSortedArray.java @@ -1470,11 +1470,11 @@ public long size() { * one before the first value (so that next must be called) * @return an iterator for this SSA */ - Iterator iterator(boolean disallowExactMatch, boolean isRightSide) { + public Iterator iterator(boolean disallowExactMatch, boolean isRightSide) { return new Iterator(disallowExactMatch, isRightSide); } - final class Iterator { + public final class Iterator { int leafIndex = 0; int indexWithinLeaf = 0; private final boolean disallowExactMatch; @@ -1488,7 +1488,7 @@ final class Iterator { } } - void next() { + public void next() { indexWithinLeaf++; if (leafCount > 1) { if (indexWithinLeaf == leafSizes[leafIndex]) { @@ -1498,7 +1498,7 @@ void next() { } } - boolean hasNext() { + public boolean hasNext() { if (leafCount == 0) { return false; } @@ -1508,7 +1508,7 @@ boolean hasNext() { return leafIndex < leafCount - 1 || (indexWithinLeaf < leafSizes[leafIndex] - 1); } - int getValue() { + public int getValue() { if (leafCount == 1) { return directoryValues[indexWithinLeaf]; } @@ -1517,7 +1517,7 @@ int getValue() { } } - int nextValue() { + public int nextValue() { Assert.assertion(hasNext(), "hasNext()"); if (leafCount == 1) { return directoryValues[indexWithinLeaf + 1]; @@ -1529,7 +1529,7 @@ else if (indexWithinLeaf == leafSizes[leafIndex] - 1) { } } - long getKey() { + public long getKey() { if (leafCount == 1) { return directoryRowKeys[indexWithinLeaf]; } @@ -1538,7 +1538,7 @@ long getKey() { } } - long nextKey() { + public long nextKey() { Assert.assertion(hasNext(), "hasNext()"); if (leafCount == 1) { return directoryRowKeys[indexWithinLeaf + 1]; @@ -1556,7 +1556,7 @@ else if (indexWithinLeaf == leafSizes[leafIndex] - 1) { * * @param value the value we are searching for */ - void advanceToLast(int value) { + public void advanceToLast(int value) { advanceToInternal(value, true); } @@ -1565,7 +1565,7 @@ void advanceToLast(int value) { * * @param value the value we are searching for */ - void advanceToBeforeFirst(int value) { + public void advanceToBeforeFirst(int value) { advanceToInternal(value, false); if (disallowExactMatch) { if (hasNext() && nextValue() == value) { @@ -1663,7 +1663,7 @@ else if (newLeafIndex > 0) { /** * Advance the iterator to the last value which is equal to the current value. */ - void advanceWhileEqual() { + public void advanceWhileEqual() { final int value = getValue(); findLastInLeaf(value); while (leafIndex < leafCount - 1) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/IntSegmentedSortedArray.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/IntSegmentedSortedArray.java index 82d1ed81ee2..dbae99bd607 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/IntSegmentedSortedArray.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/IntSegmentedSortedArray.java @@ -1469,11 +1469,11 @@ public long size() { * one before the first value (so that next must be called) * @return an iterator for this SSA */ - Iterator iterator(boolean disallowExactMatch, boolean isRightSide) { + public Iterator iterator(boolean disallowExactMatch, boolean isRightSide) { return new Iterator(disallowExactMatch, isRightSide); } - final class Iterator { + public final class Iterator { int leafIndex = 0; int indexWithinLeaf = 0; private final boolean disallowExactMatch; @@ -1487,7 +1487,7 @@ final class Iterator { } } - void next() { + public void next() { indexWithinLeaf++; if (leafCount > 1) { if (indexWithinLeaf == leafSizes[leafIndex]) { @@ -1497,7 +1497,7 @@ void next() { } } - boolean hasNext() { + public boolean hasNext() { if (leafCount == 0) { return false; } @@ -1507,7 +1507,7 @@ boolean hasNext() { return leafIndex < leafCount - 1 || (indexWithinLeaf < leafSizes[leafIndex] - 1); } - int getValue() { + public int getValue() { if (leafCount == 1) { return directoryValues[indexWithinLeaf]; } @@ -1516,7 +1516,7 @@ int getValue() { } } - int nextValue() { + public int nextValue() { Assert.assertion(hasNext(), "hasNext()"); if (leafCount == 1) { return directoryValues[indexWithinLeaf + 1]; @@ -1528,7 +1528,7 @@ else if (indexWithinLeaf == leafSizes[leafIndex] - 1) { } } - long getKey() { + public long getKey() { if (leafCount == 1) { return directoryRowKeys[indexWithinLeaf]; } @@ -1537,7 +1537,7 @@ long getKey() { } } - long nextKey() { + public long nextKey() { Assert.assertion(hasNext(), "hasNext()"); if (leafCount == 1) { return directoryRowKeys[indexWithinLeaf + 1]; @@ -1555,7 +1555,7 @@ else if (indexWithinLeaf == leafSizes[leafIndex] - 1) { * * @param value the value we are searching for */ - void advanceToLast(int value) { + public void advanceToLast(int value) { advanceToInternal(value, true); } @@ -1564,7 +1564,7 @@ void advanceToLast(int value) { * * @param value the value we are searching for */ - void advanceToBeforeFirst(int value) { + public void advanceToBeforeFirst(int value) { advanceToInternal(value, false); if (disallowExactMatch) { if (hasNext() && nextValue() == value) { @@ -1662,7 +1662,7 @@ else if (newLeafIndex > 0) { /** * Advance the iterator to the last value which is equal to the current value. */ - void advanceWhileEqual() { + public void advanceWhileEqual() { final int value = getValue(); findLastInLeaf(value); while (leafIndex < leafCount - 1) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/LongReverseSegmentedSortedArray.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/LongReverseSegmentedSortedArray.java index 12c9ba22b44..889ea460ab4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/LongReverseSegmentedSortedArray.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/LongReverseSegmentedSortedArray.java @@ -1470,11 +1470,11 @@ public long size() { * one before the first value (so that next must be called) * @return an iterator for this SSA */ - Iterator iterator(boolean disallowExactMatch, boolean isRightSide) { + public Iterator iterator(boolean disallowExactMatch, boolean isRightSide) { return new Iterator(disallowExactMatch, isRightSide); } - final class Iterator { + public final class Iterator { int leafIndex = 0; int indexWithinLeaf = 0; private final boolean disallowExactMatch; @@ -1488,7 +1488,7 @@ final class Iterator { } } - void next() { + public void next() { indexWithinLeaf++; if (leafCount > 1) { if (indexWithinLeaf == leafSizes[leafIndex]) { @@ -1498,7 +1498,7 @@ void next() { } } - boolean hasNext() { + public boolean hasNext() { if (leafCount == 0) { return false; } @@ -1508,7 +1508,7 @@ boolean hasNext() { return leafIndex < leafCount - 1 || (indexWithinLeaf < leafSizes[leafIndex] - 1); } - long getValue() { + public long getValue() { if (leafCount == 1) { return directoryValues[indexWithinLeaf]; } @@ -1517,7 +1517,7 @@ long getValue() { } } - long nextValue() { + public long nextValue() { Assert.assertion(hasNext(), "hasNext()"); if (leafCount == 1) { return directoryValues[indexWithinLeaf + 1]; @@ -1529,7 +1529,7 @@ else if (indexWithinLeaf == leafSizes[leafIndex] - 1) { } } - long getKey() { + public long getKey() { if (leafCount == 1) { return directoryRowKeys[indexWithinLeaf]; } @@ -1538,7 +1538,7 @@ long getKey() { } } - long nextKey() { + public long nextKey() { Assert.assertion(hasNext(), "hasNext()"); if (leafCount == 1) { return directoryRowKeys[indexWithinLeaf + 1]; @@ -1556,7 +1556,7 @@ else if (indexWithinLeaf == leafSizes[leafIndex] - 1) { * * @param value the value we are searching for */ - void advanceToLast(long value) { + public void advanceToLast(long value) { advanceToInternal(value, true); } @@ -1565,7 +1565,7 @@ void advanceToLast(long value) { * * @param value the value we are searching for */ - void advanceToBeforeFirst(long value) { + public void advanceToBeforeFirst(long value) { advanceToInternal(value, false); if (disallowExactMatch) { if (hasNext() && nextValue() == value) { @@ -1663,7 +1663,7 @@ else if (newLeafIndex > 0) { /** * Advance the iterator to the last value which is equal to the current value. */ - void advanceWhileEqual() { + public void advanceWhileEqual() { final long value = getValue(); findLastInLeaf(value); while (leafIndex < leafCount - 1) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/LongSegmentedSortedArray.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/LongSegmentedSortedArray.java index cc3714b24fc..8915cc39fd2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/LongSegmentedSortedArray.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/LongSegmentedSortedArray.java @@ -1469,11 +1469,11 @@ public long size() { * one before the first value (so that next must be called) * @return an iterator for this SSA */ - Iterator iterator(boolean disallowExactMatch, boolean isRightSide) { + public Iterator iterator(boolean disallowExactMatch, boolean isRightSide) { return new Iterator(disallowExactMatch, isRightSide); } - final class Iterator { + public final class Iterator { int leafIndex = 0; int indexWithinLeaf = 0; private final boolean disallowExactMatch; @@ -1487,7 +1487,7 @@ final class Iterator { } } - void next() { + public void next() { indexWithinLeaf++; if (leafCount > 1) { if (indexWithinLeaf == leafSizes[leafIndex]) { @@ -1497,7 +1497,7 @@ void next() { } } - boolean hasNext() { + public boolean hasNext() { if (leafCount == 0) { return false; } @@ -1507,7 +1507,7 @@ boolean hasNext() { return leafIndex < leafCount - 1 || (indexWithinLeaf < leafSizes[leafIndex] - 1); } - long getValue() { + public long getValue() { if (leafCount == 1) { return directoryValues[indexWithinLeaf]; } @@ -1516,7 +1516,7 @@ long getValue() { } } - long nextValue() { + public long nextValue() { Assert.assertion(hasNext(), "hasNext()"); if (leafCount == 1) { return directoryValues[indexWithinLeaf + 1]; @@ -1528,7 +1528,7 @@ else if (indexWithinLeaf == leafSizes[leafIndex] - 1) { } } - long getKey() { + public long getKey() { if (leafCount == 1) { return directoryRowKeys[indexWithinLeaf]; } @@ -1537,7 +1537,7 @@ long getKey() { } } - long nextKey() { + public long nextKey() { Assert.assertion(hasNext(), "hasNext()"); if (leafCount == 1) { return directoryRowKeys[indexWithinLeaf + 1]; @@ -1555,7 +1555,7 @@ else if (indexWithinLeaf == leafSizes[leafIndex] - 1) { * * @param value the value we are searching for */ - void advanceToLast(long value) { + public void advanceToLast(long value) { advanceToInternal(value, true); } @@ -1564,7 +1564,7 @@ void advanceToLast(long value) { * * @param value the value we are searching for */ - void advanceToBeforeFirst(long value) { + public void advanceToBeforeFirst(long value) { advanceToInternal(value, false); if (disallowExactMatch) { if (hasNext() && nextValue() == value) { @@ -1662,7 +1662,7 @@ else if (newLeafIndex > 0) { /** * Advance the iterator to the last value which is equal to the current value. */ - void advanceWhileEqual() { + public void advanceWhileEqual() { final long value = getValue(); findLastInLeaf(value); while (leafIndex < leafCount - 1) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/NullAwareCharReverseSegmentedSortedArray.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/NullAwareCharReverseSegmentedSortedArray.java index d3cdf6650d5..e4c2f6cd430 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/NullAwareCharReverseSegmentedSortedArray.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/NullAwareCharReverseSegmentedSortedArray.java @@ -1471,11 +1471,11 @@ public long size() { * one before the first value (so that next must be called) * @return an iterator for this SSA */ - Iterator iterator(boolean disallowExactMatch, boolean isRightSide) { + public Iterator iterator(boolean disallowExactMatch, boolean isRightSide) { return new Iterator(disallowExactMatch, isRightSide); } - final class Iterator { + public final class Iterator { int leafIndex = 0; int indexWithinLeaf = 0; private final boolean disallowExactMatch; @@ -1489,7 +1489,7 @@ final class Iterator { } } - void next() { + public void next() { indexWithinLeaf++; if (leafCount > 1) { if (indexWithinLeaf == leafSizes[leafIndex]) { @@ -1499,7 +1499,7 @@ void next() { } } - boolean hasNext() { + public boolean hasNext() { if (leafCount == 0) { return false; } @@ -1509,7 +1509,7 @@ boolean hasNext() { return leafIndex < leafCount - 1 || (indexWithinLeaf < leafSizes[leafIndex] - 1); } - char getValue() { + public char getValue() { if (leafCount == 1) { return directoryValues[indexWithinLeaf]; } @@ -1518,7 +1518,7 @@ char getValue() { } } - char nextValue() { + public char nextValue() { Assert.assertion(hasNext(), "hasNext()"); if (leafCount == 1) { return directoryValues[indexWithinLeaf + 1]; @@ -1530,7 +1530,7 @@ else if (indexWithinLeaf == leafSizes[leafIndex] - 1) { } } - long getKey() { + public long getKey() { if (leafCount == 1) { return directoryRowKeys[indexWithinLeaf]; } @@ -1539,7 +1539,7 @@ long getKey() { } } - long nextKey() { + public long nextKey() { Assert.assertion(hasNext(), "hasNext()"); if (leafCount == 1) { return directoryRowKeys[indexWithinLeaf + 1]; @@ -1557,7 +1557,7 @@ else if (indexWithinLeaf == leafSizes[leafIndex] - 1) { * * @param value the value we are searching for */ - void advanceToLast(char value) { + public void advanceToLast(char value) { advanceToInternal(value, true); } @@ -1566,7 +1566,7 @@ void advanceToLast(char value) { * * @param value the value we are searching for */ - void advanceToBeforeFirst(char value) { + public void advanceToBeforeFirst(char value) { advanceToInternal(value, false); if (disallowExactMatch) { if (hasNext() && nextValue() == value) { @@ -1664,7 +1664,7 @@ else if (newLeafIndex > 0) { /** * Advance the iterator to the last value which is equal to the current value. */ - void advanceWhileEqual() { + public void advanceWhileEqual() { final char value = getValue(); findLastInLeaf(value); while (leafIndex < leafCount - 1) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/NullAwareCharSegmentedSortedArray.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/NullAwareCharSegmentedSortedArray.java index c3bc74b1b13..08c81d4db4c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/NullAwareCharSegmentedSortedArray.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/NullAwareCharSegmentedSortedArray.java @@ -1470,11 +1470,11 @@ public long size() { * one before the first value (so that next must be called) * @return an iterator for this SSA */ - Iterator iterator(boolean disallowExactMatch, boolean isRightSide) { + public Iterator iterator(boolean disallowExactMatch, boolean isRightSide) { return new Iterator(disallowExactMatch, isRightSide); } - final class Iterator { + public final class Iterator { int leafIndex = 0; int indexWithinLeaf = 0; private final boolean disallowExactMatch; @@ -1488,7 +1488,7 @@ final class Iterator { } } - void next() { + public void next() { indexWithinLeaf++; if (leafCount > 1) { if (indexWithinLeaf == leafSizes[leafIndex]) { @@ -1498,7 +1498,7 @@ void next() { } } - boolean hasNext() { + public boolean hasNext() { if (leafCount == 0) { return false; } @@ -1508,7 +1508,7 @@ boolean hasNext() { return leafIndex < leafCount - 1 || (indexWithinLeaf < leafSizes[leafIndex] - 1); } - char getValue() { + public char getValue() { if (leafCount == 1) { return directoryValues[indexWithinLeaf]; } @@ -1517,7 +1517,7 @@ char getValue() { } } - char nextValue() { + public char nextValue() { Assert.assertion(hasNext(), "hasNext()"); if (leafCount == 1) { return directoryValues[indexWithinLeaf + 1]; @@ -1529,7 +1529,7 @@ else if (indexWithinLeaf == leafSizes[leafIndex] - 1) { } } - long getKey() { + public long getKey() { if (leafCount == 1) { return directoryRowKeys[indexWithinLeaf]; } @@ -1538,7 +1538,7 @@ long getKey() { } } - long nextKey() { + public long nextKey() { Assert.assertion(hasNext(), "hasNext()"); if (leafCount == 1) { return directoryRowKeys[indexWithinLeaf + 1]; @@ -1556,7 +1556,7 @@ else if (indexWithinLeaf == leafSizes[leafIndex] - 1) { * * @param value the value we are searching for */ - void advanceToLast(char value) { + public void advanceToLast(char value) { advanceToInternal(value, true); } @@ -1565,7 +1565,7 @@ void advanceToLast(char value) { * * @param value the value we are searching for */ - void advanceToBeforeFirst(char value) { + public void advanceToBeforeFirst(char value) { advanceToInternal(value, false); if (disallowExactMatch) { if (hasNext() && nextValue() == value) { @@ -1663,7 +1663,7 @@ else if (newLeafIndex > 0) { /** * Advance the iterator to the last value which is equal to the current value. */ - void advanceWhileEqual() { + public void advanceWhileEqual() { final char value = getValue(); findLastInLeaf(value); while (leafIndex < leafCount - 1) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/ObjectReverseSegmentedSortedArray.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/ObjectReverseSegmentedSortedArray.java index f50688723d3..7995c339d05 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/ObjectReverseSegmentedSortedArray.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/ObjectReverseSegmentedSortedArray.java @@ -1484,11 +1484,11 @@ public long size() { * one before the first value (so that next must be called) * @return an iterator for this SSA */ - Iterator iterator(boolean disallowExactMatch, boolean isRightSide) { + public Iterator iterator(boolean disallowExactMatch, boolean isRightSide) { return new Iterator(disallowExactMatch, isRightSide); } - final class Iterator { + public final class Iterator { int leafIndex = 0; int indexWithinLeaf = 0; private final boolean disallowExactMatch; @@ -1502,7 +1502,7 @@ final class Iterator { } } - void next() { + public void next() { indexWithinLeaf++; if (leafCount > 1) { if (indexWithinLeaf == leafSizes[leafIndex]) { @@ -1512,7 +1512,7 @@ void next() { } } - boolean hasNext() { + public boolean hasNext() { if (leafCount == 0) { return false; } @@ -1522,7 +1522,7 @@ boolean hasNext() { return leafIndex < leafCount - 1 || (indexWithinLeaf < leafSizes[leafIndex] - 1); } - Object getValue() { + public Object getValue() { if (leafCount == 1) { return directoryValues[indexWithinLeaf]; } @@ -1531,7 +1531,7 @@ Object getValue() { } } - Object nextValue() { + public Object nextValue() { Assert.assertion(hasNext(), "hasNext()"); if (leafCount == 1) { return directoryValues[indexWithinLeaf + 1]; @@ -1543,7 +1543,7 @@ else if (indexWithinLeaf == leafSizes[leafIndex] - 1) { } } - long getKey() { + public long getKey() { if (leafCount == 1) { return directoryRowKeys[indexWithinLeaf]; } @@ -1552,7 +1552,7 @@ long getKey() { } } - long nextKey() { + public long nextKey() { Assert.assertion(hasNext(), "hasNext()"); if (leafCount == 1) { return directoryRowKeys[indexWithinLeaf + 1]; @@ -1570,7 +1570,7 @@ else if (indexWithinLeaf == leafSizes[leafIndex] - 1) { * * @param value the value we are searching for */ - void advanceToLast(Object value) { + public void advanceToLast(Object value) { advanceToInternal(value, true); } @@ -1579,7 +1579,7 @@ void advanceToLast(Object value) { * * @param value the value we are searching for */ - void advanceToBeforeFirst(Object value) { + public void advanceToBeforeFirst(Object value) { advanceToInternal(value, false); if (disallowExactMatch) { if (hasNext() && nextValue() == value) { @@ -1677,7 +1677,7 @@ else if (newLeafIndex > 0) { /** * Advance the iterator to the last value which is equal to the current value. */ - void advanceWhileEqual() { + public void advanceWhileEqual() { final Object value = getValue(); findLastInLeaf(value); while (leafIndex < leafCount - 1) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/ObjectSegmentedSortedArray.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/ObjectSegmentedSortedArray.java index bdd9dfe56a7..234f7b1e014 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/ObjectSegmentedSortedArray.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/ObjectSegmentedSortedArray.java @@ -1483,11 +1483,11 @@ public long size() { * one before the first value (so that next must be called) * @return an iterator for this SSA */ - Iterator iterator(boolean disallowExactMatch, boolean isRightSide) { + public Iterator iterator(boolean disallowExactMatch, boolean isRightSide) { return new Iterator(disallowExactMatch, isRightSide); } - final class Iterator { + public final class Iterator { int leafIndex = 0; int indexWithinLeaf = 0; private final boolean disallowExactMatch; @@ -1501,7 +1501,7 @@ final class Iterator { } } - void next() { + public void next() { indexWithinLeaf++; if (leafCount > 1) { if (indexWithinLeaf == leafSizes[leafIndex]) { @@ -1511,7 +1511,7 @@ void next() { } } - boolean hasNext() { + public boolean hasNext() { if (leafCount == 0) { return false; } @@ -1521,7 +1521,7 @@ boolean hasNext() { return leafIndex < leafCount - 1 || (indexWithinLeaf < leafSizes[leafIndex] - 1); } - Object getValue() { + public Object getValue() { if (leafCount == 1) { return directoryValues[indexWithinLeaf]; } @@ -1530,7 +1530,7 @@ Object getValue() { } } - Object nextValue() { + public Object nextValue() { Assert.assertion(hasNext(), "hasNext()"); if (leafCount == 1) { return directoryValues[indexWithinLeaf + 1]; @@ -1542,7 +1542,7 @@ else if (indexWithinLeaf == leafSizes[leafIndex] - 1) { } } - long getKey() { + public long getKey() { if (leafCount == 1) { return directoryRowKeys[indexWithinLeaf]; } @@ -1551,7 +1551,7 @@ long getKey() { } } - long nextKey() { + public long nextKey() { Assert.assertion(hasNext(), "hasNext()"); if (leafCount == 1) { return directoryRowKeys[indexWithinLeaf + 1]; @@ -1569,7 +1569,7 @@ else if (indexWithinLeaf == leafSizes[leafIndex] - 1) { * * @param value the value we are searching for */ - void advanceToLast(Object value) { + public void advanceToLast(Object value) { advanceToInternal(value, true); } @@ -1578,7 +1578,7 @@ void advanceToLast(Object value) { * * @param value the value we are searching for */ - void advanceToBeforeFirst(Object value) { + public void advanceToBeforeFirst(Object value) { advanceToInternal(value, false); if (disallowExactMatch) { if (hasNext() && nextValue() == value) { @@ -1676,7 +1676,7 @@ else if (newLeafIndex > 0) { /** * Advance the iterator to the last value which is equal to the current value. */ - void advanceWhileEqual() { + public void advanceWhileEqual() { final Object value = getValue(); findLastInLeaf(value); while (leafIndex < leafCount - 1) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/ShortReverseSegmentedSortedArray.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/ShortReverseSegmentedSortedArray.java index c11df507bbd..ae9d26ade96 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/ShortReverseSegmentedSortedArray.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/ShortReverseSegmentedSortedArray.java @@ -1470,11 +1470,11 @@ public long size() { * one before the first value (so that next must be called) * @return an iterator for this SSA */ - Iterator iterator(boolean disallowExactMatch, boolean isRightSide) { + public Iterator iterator(boolean disallowExactMatch, boolean isRightSide) { return new Iterator(disallowExactMatch, isRightSide); } - final class Iterator { + public final class Iterator { int leafIndex = 0; int indexWithinLeaf = 0; private final boolean disallowExactMatch; @@ -1488,7 +1488,7 @@ final class Iterator { } } - void next() { + public void next() { indexWithinLeaf++; if (leafCount > 1) { if (indexWithinLeaf == leafSizes[leafIndex]) { @@ -1498,7 +1498,7 @@ void next() { } } - boolean hasNext() { + public boolean hasNext() { if (leafCount == 0) { return false; } @@ -1508,7 +1508,7 @@ boolean hasNext() { return leafIndex < leafCount - 1 || (indexWithinLeaf < leafSizes[leafIndex] - 1); } - short getValue() { + public short getValue() { if (leafCount == 1) { return directoryValues[indexWithinLeaf]; } @@ -1517,7 +1517,7 @@ short getValue() { } } - short nextValue() { + public short nextValue() { Assert.assertion(hasNext(), "hasNext()"); if (leafCount == 1) { return directoryValues[indexWithinLeaf + 1]; @@ -1529,7 +1529,7 @@ else if (indexWithinLeaf == leafSizes[leafIndex] - 1) { } } - long getKey() { + public long getKey() { if (leafCount == 1) { return directoryRowKeys[indexWithinLeaf]; } @@ -1538,7 +1538,7 @@ long getKey() { } } - long nextKey() { + public long nextKey() { Assert.assertion(hasNext(), "hasNext()"); if (leafCount == 1) { return directoryRowKeys[indexWithinLeaf + 1]; @@ -1556,7 +1556,7 @@ else if (indexWithinLeaf == leafSizes[leafIndex] - 1) { * * @param value the value we are searching for */ - void advanceToLast(short value) { + public void advanceToLast(short value) { advanceToInternal(value, true); } @@ -1565,7 +1565,7 @@ void advanceToLast(short value) { * * @param value the value we are searching for */ - void advanceToBeforeFirst(short value) { + public void advanceToBeforeFirst(short value) { advanceToInternal(value, false); if (disallowExactMatch) { if (hasNext() && nextValue() == value) { @@ -1663,7 +1663,7 @@ else if (newLeafIndex > 0) { /** * Advance the iterator to the last value which is equal to the current value. */ - void advanceWhileEqual() { + public void advanceWhileEqual() { final short value = getValue(); findLastInLeaf(value); while (leafIndex < leafCount - 1) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/ShortSegmentedSortedArray.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/ShortSegmentedSortedArray.java index 56085e674e9..7ced3a8052f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/ShortSegmentedSortedArray.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ssa/ShortSegmentedSortedArray.java @@ -1469,11 +1469,11 @@ public long size() { * one before the first value (so that next must be called) * @return an iterator for this SSA */ - Iterator iterator(boolean disallowExactMatch, boolean isRightSide) { + public Iterator iterator(boolean disallowExactMatch, boolean isRightSide) { return new Iterator(disallowExactMatch, isRightSide); } - final class Iterator { + public final class Iterator { int leafIndex = 0; int indexWithinLeaf = 0; private final boolean disallowExactMatch; @@ -1487,7 +1487,7 @@ final class Iterator { } } - void next() { + public void next() { indexWithinLeaf++; if (leafCount > 1) { if (indexWithinLeaf == leafSizes[leafIndex]) { @@ -1497,7 +1497,7 @@ void next() { } } - boolean hasNext() { + public boolean hasNext() { if (leafCount == 0) { return false; } @@ -1507,7 +1507,7 @@ boolean hasNext() { return leafIndex < leafCount - 1 || (indexWithinLeaf < leafSizes[leafIndex] - 1); } - short getValue() { + public short getValue() { if (leafCount == 1) { return directoryValues[indexWithinLeaf]; } @@ -1516,7 +1516,7 @@ short getValue() { } } - short nextValue() { + public short nextValue() { Assert.assertion(hasNext(), "hasNext()"); if (leafCount == 1) { return directoryValues[indexWithinLeaf + 1]; @@ -1528,7 +1528,7 @@ else if (indexWithinLeaf == leafSizes[leafIndex] - 1) { } } - long getKey() { + public long getKey() { if (leafCount == 1) { return directoryRowKeys[indexWithinLeaf]; } @@ -1537,7 +1537,7 @@ long getKey() { } } - long nextKey() { + public long nextKey() { Assert.assertion(hasNext(), "hasNext()"); if (leafCount == 1) { return directoryRowKeys[indexWithinLeaf + 1]; @@ -1555,7 +1555,7 @@ else if (indexWithinLeaf == leafSizes[leafIndex] - 1) { * * @param value the value we are searching for */ - void advanceToLast(short value) { + public void advanceToLast(short value) { advanceToInternal(value, true); } @@ -1564,7 +1564,7 @@ void advanceToLast(short value) { * * @param value the value we are searching for */ - void advanceToBeforeFirst(short value) { + public void advanceToBeforeFirst(short value) { advanceToInternal(value, false); if (disallowExactMatch) { if (hasNext() && nextValue() == value) { @@ -1662,7 +1662,7 @@ else if (newLeafIndex > 0) { /** * Advance the iterator to the last value which is equal to the current value. */ - void advanceWhileEqual() { + public void advanceWhileEqual() { final short value = getValue(); findLastInLeaf(value); while (leafIndex < leafCount - 1) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java index b227b0e633e..4b347716c48 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java @@ -10,6 +10,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.locations.TableDataException; +import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.updateby.internal.BaseDoubleUpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; import org.jetbrains.annotations.NotNull; @@ -30,8 +31,8 @@ class EmaContext extends Context { double oneMinusAlpha; long lastStamp = NULL_LONG; - EmaContext(final double timeScaleUnits, final int chunkSize) { - super(chunkSize); + EmaContext(final double timeScaleUnits, final int chunkSize, final LongSegmentedSortedArray timestampSsa) { + super(chunkSize, timestampSsa); this.alpha = Math.exp(-1 / timeScaleUnits); this.oneMinusAlpha = 1 - alpha; } @@ -65,8 +66,8 @@ public BasePrimitiveEMAOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize) { - return new EmaContext(timeScaleUnits, chunkSize); + public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { + return new EmaContext(timeScaleUnits, chunkSize, timestampSsa); } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java index 1b1859377b6..c730f6dd03f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java @@ -9,9 +9,9 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.locations.TableDataException; +import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; import org.jetbrains.annotations.NotNull; @@ -36,8 +36,8 @@ class EmaContext extends Context { timeRecorder == null ? BigDecimal.ONE.subtract(alpha, control.bigValueContextOrDefault()) : null; long lastStamp = NULL_LONG; - EmaContext(final int chunkSize) { - super(chunkSize); + EmaContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { + super(chunkSize, timestampSsa); } } @@ -76,8 +76,8 @@ public BigNumberEMAOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize) { - return new EmaContext(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { + return new EmaContext(chunkSize, timestampSsa); } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/hashing/TrackerPriorityQueue.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/hashing/TrackerPriorityQueue.java deleted file mode 100644 index 21b48f34eb2..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/hashing/TrackerPriorityQueue.java +++ /dev/null @@ -1,155 +0,0 @@ -package io.deephaven.engine.table.impl.updateby.hashing; - -import io.deephaven.configuration.Configuration; - -/** - * A Min-Heap priority queue used to order {@link UpdateBySlotTracker.UpdateTracker} instances for classifying row keys - * to buckets. - */ -public class TrackerPriorityQueue { - private static final int doublingAllocThreshold = Configuration.getInstance() - .getIntegerWithDefault("TrackerPriorityQueue.doublingAllocThreshold", 4 * 1024 * 1024); - - // Things are nicer (integer division will be bit shift) if this is a power of 2, but it is not mandatory. - private static final int linearAllocStep = - Configuration.getInstance().getIntegerWithDefault("TrackerPriorityQueue.linearAllocStep", 1024 * 1024); - - /** The trackers, slot 0 is unused. */ - private UpdateBySlotTracker.UpdateTracker[] trackers; - - /** - * The size of the queue (invariant: size < start.length - 1). Note since we don't use element 0 in start and end - * arrays, this size does not match the normal invariant in array access where the last element used is an array a[] - * is a[size - 1]; in our case the last element used is a[size]. - */ - private int size = 0; - - /** - * Create a TrackerPriorityQueue with the given initial capacity. - * - * @param initialCapacity how many ranges should we allocate room for - */ - public TrackerPriorityQueue(final int initialCapacity) { - trackers = new UpdateBySlotTracker.UpdateTracker[initialCapacity + 1]; - } - - /** - * Adds an element to the queue. - */ - public void add(final UpdateBySlotTracker.UpdateTracker tracker) { - final int newSize = size + 1; - ensureCapacityFor(newSize); - - trackers[newSize] = tracker; - size = newSize; - fixUp(size); - } - - /** - * Pop the top element from the queue. - * - * @return The item popped - */ - public UpdateBySlotTracker.UpdateTracker pop() { - if (size == 0) { - return null; - } - - final UpdateBySlotTracker.UpdateTracker atTop = trackers[1]; - if (--size > 0) { - trackers[1] = trackers[size + 1]; - fixDown(1); - } - - return atTop; - } - - private void ensureCapacityFor(final int lastIndex) { - final int minCapacity = lastIndex + 1; - if (minCapacity < trackers.length) { - return; - } - - int newCapacity = trackers.length; - while (newCapacity < minCapacity && newCapacity < doublingAllocThreshold) { - newCapacity = 2 * newCapacity; - } - - if (newCapacity < minCapacity) { - final int delta = minCapacity - doublingAllocThreshold; - final int steps = (delta + linearAllocStep - 1) / linearAllocStep; - newCapacity = doublingAllocThreshold + steps * linearAllocStep; - } - - final UpdateBySlotTracker.UpdateTracker[] newTrackers = new UpdateBySlotTracker.UpdateTracker[newCapacity]; - System.arraycopy(trackers, 1, newTrackers, 1, size); - trackers = newTrackers; - } - - /** - * move queue[itemIndex] up the heap until its start is >= that of its parent. - */ - private void fixUp(int itemIndex) { - if (itemIndex <= 1) { - return; - } - - final UpdateBySlotTracker.UpdateTracker item = trackers[itemIndex]; - int parentIndex = itemIndex >> 1; - UpdateBySlotTracker.UpdateTracker parent; - while (itemIndex > 1 && valueOf(item) < valueOf(parent = trackers[parentIndex])) { - trackers[itemIndex] = parent; - itemIndex = parentIndex; - parentIndex = itemIndex >> 1; - } - trackers[itemIndex] = item; - } - - /** - * move queue[itemIndex] down the heap until its start is <= those of its children. - */ - private void fixDown(@SuppressWarnings("SameParameterValue") int itemIndex) { - // Start the smallest child at the left and then adjust - int smallestChildIdx = itemIndex << 1; - if (smallestChildIdx > size) { - return; - } - - final UpdateBySlotTracker.UpdateTracker item = trackers[itemIndex]; - UpdateBySlotTracker.UpdateTracker smallestChild = trackers[smallestChildIdx]; - UpdateBySlotTracker.UpdateTracker nextChild; - // Just pick the smallest of the two values. - if (smallestChildIdx < size && valueOf(nextChild = trackers[smallestChildIdx + 1]) < valueOf(smallestChild)) { - smallestChild = nextChild; - smallestChildIdx++; - } - - if (valueOf(smallestChild) < valueOf(item)) { - trackers[itemIndex] = smallestChild; - itemIndex = smallestChildIdx; - smallestChildIdx = itemIndex << 1; - while (smallestChildIdx <= size) { - smallestChild = trackers[smallestChildIdx]; - if (smallestChildIdx < size - && valueOf(nextChild = trackers[smallestChildIdx + 1]) < valueOf(smallestChild)) { - smallestChild = nextChild; - smallestChildIdx++; - } - - if (valueOf(smallestChild) >= valueOf(item)) { - break; - } - trackers[itemIndex] = smallestChild; - - itemIndex = smallestChildIdx; - smallestChildIdx = itemIndex << 1; - } - - trackers[itemIndex] = item; - } - } - - private long valueOf(final UpdateBySlotTracker.UpdateTracker tracker) { - return tracker.getIterator().currentValue(); - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/hashing/UpdateBySlotTracker.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/hashing/UpdateBySlotTracker.java deleted file mode 100644 index e9c13ae5ded..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/hashing/UpdateBySlotTracker.java +++ /dev/null @@ -1,406 +0,0 @@ -package io.deephaven.engine.table.impl.updateby.hashing; - -import io.deephaven.base.verify.Assert; -import io.deephaven.chunk.LongChunk; -import io.deephaven.engine.rowset.*; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; -import io.deephaven.engine.table.impl.UpdateByOperator; -import io.deephaven.engine.table.impl.sources.LongArraySource; -import io.deephaven.engine.table.impl.sources.ObjectArraySource; -import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; - -/** - *

      - * This class tracks indices by individual bucket. During an update cycle it also provides a means to accumulate updates - * to each bucket so that they can be processed on a per-bucket basis. - *

      - *
      - *

      - * Accumulation of bucketed changes is done by assigning a cookie, incrementally, to each visited bucket. The cookie can - * be then used to retrieve the changes to that bucket which lets the update use space proportional to the number of - * visited buckets in the update, instead of the total number of buckets in the table. - *

      - */ -public class UpdateBySlotTracker { - private static final RowSet EMPTY_INDEX = RowSetFactory.empty(); - private final int chunkSize; - - private long pointer; - - /** how many slots we have allocated */ - private long allocated; - - /** Each time we clear, we add an offset to our cookies, this prevents us from reading old values */ - private long cookieGeneration; - - /** Maintain the cookie for particular visited slots */ - private final LongArraySource slotToCookie = new LongArraySource(); - - /** The actual index for the individual slots */ - private final ObjectArraySource slotIndices = new ObjectArraySource<>(WritableRowSet.class); - - /** A source based upon the current cookie for updates to each individual slot. */ - private final ObjectArraySource slotUpdates = new ObjectArraySource<>(UpdateTracker.class); - - private long largestSlotPosition = -1; - - private boolean updatesApplied = false; - - public interface ModifiedSlotConsumer { - - void accept(@NotNull final UpdateTracker tracker, @NotNull final RowSet slotIndex); - } - - public static class UpdateTracker { - - final int slot; - // We'll use two sequential builders for this since it'll be better than a single random builder, given that - - // we know both sets of addeds and removeds are going to be processed sequentially. - RowSetBuilderSequential addedBuilder; - RowSetBuilderSequential changedBucketAddedBuilder; - RowSet added; - - RowSetBuilderSequential modifiedBuilder; - RowSet modified; - - RowSetBuilderSequential removedBuilder; - RowSetBuilderSequential changedBucketRemovedBuilder; - RowSet removed; - - boolean affectedByShift = false; - - boolean wasAppendOnly = true; - - private RowSet.SearchIterator buckIt; - - private long smallestModifiedKey; - - public UpdateTracker(final int slot) { - this.slot = slot; - } - - public int getSlot() { - return slot; - } - - public long getSmallestModifiedKey() { - return smallestModifiedKey; - } - - @NotNull - public RowSet getAdded() { - if (added == null) { - WritableRowSet newAdded = null; - if (addedBuilder != null) { - newAdded = addedBuilder.build(); - addedBuilder = null; - } - - if (changedBucketAddedBuilder != null) { - if (newAdded == null) { - newAdded = changedBucketAddedBuilder.build(); - } else { - newAdded.insert(changedBucketAddedBuilder.build()); - } - changedBucketAddedBuilder = null; - } - - added = (newAdded == null) ? EMPTY_INDEX : newAdded; - } - return added; - } - - @NotNull - public RowSet getModified() { - if (modified == null) { - modified = modifiedBuilder == null ? EMPTY_INDEX : modifiedBuilder.build(); - modifiedBuilder = null; - } - return modified; - } - - @NotNull - public RowSet getRemoved() { - if (removed == null) { - WritableRowSet newRemoved = null; - if (removedBuilder != null) { - newRemoved = removedBuilder.build(); - removedBuilder = null; - } - - if (changedBucketRemovedBuilder != null) { - if (newRemoved == null) { - newRemoved = changedBucketRemovedBuilder.build(); - } else { - newRemoved.insert(changedBucketRemovedBuilder.build()); - } - changedBucketRemovedBuilder = null; - } - - removed = (newRemoved == null) ? EMPTY_INDEX : newRemoved; - } - return removed; - } - - public boolean wasAppendOnly() { - return wasAppendOnly; - } - - public boolean wasShiftOnly() { - return getRemoved().isEmpty() && getModified().isEmpty() && getAdded().isEmpty() && affectedByShift; - } - - private void applyTo(@NotNull final WritableRowSet slotindex, @NotNull final RowSetShiftData shifts) { - final RowSet removed = getRemoved(); - if (removed.isNonempty()) { - wasAppendOnly = false; - slotindex.remove(removed); - } - - if (shifts.nonempty() && affectedByShift && shifts.apply(slotindex)) { - wasAppendOnly = false; - } - - final RowSet added = getAdded(); - if (added.isNonempty()) { - wasAppendOnly &= added.firstRowKey() > slotindex.lastRowKey(); - slotindex.insert(added); - } - - final RowSet modified = getModified(); - if (modified.isNonempty()) { - wasAppendOnly = false; - } - - this.smallestModifiedKey = UpdateByOperator.smallestAffectedKey(added, - modified, - removed, - shifts, - slotindex); - } - - public void setBucketIterator(@NotNull final RowSet.SearchIterator iter) { - this.buckIt = iter; - } - - public RowSet.SearchIterator getIterator() { - return buckIt; - } - } - - public UpdateBySlotTracker(final int chunkSize) { - this.chunkSize = chunkSize; - } - - public long getModifiedBucketCount() { - return pointer; - } - - /** - * Remove all entries from the tracker. - */ - public void reset() { - // Make sure we clear out references to any UpdateTrackers we may have allocated. - for (int ii = 0; ii < pointer; ii++) { - slotUpdates.setNull(ii); - } - - cookieGeneration += pointer; - if (cookieGeneration > Long.MAX_VALUE / 2) { - cookieGeneration = 0; - } - pointer = 0; - updatesApplied = false; - } - - @NotNull - public RowSet applyUpdates(@NotNull final RowSetShiftData shiftsToApply) { - Assert.eqFalse(updatesApplied, "updatesApplied"); - updatesApplied = true; - RowSetBuilderRandom emptiedSlotsBuilder = null; - slotIndices.ensureCapacity(largestSlotPosition + 1); - - for (int ii = 0; ii < pointer; ii++) { - final UpdateTracker tracker = slotUpdates.getUnsafe(ii); - final WritableRowSet slotIndex = slotIndices.getUnsafe(tracker.getSlot()); - emptiedSlotsBuilder = applyUpdateAndTrackEmpty(tracker, slotIndex, shiftsToApply, emptiedSlotsBuilder); - } - - return emptiedSlotsBuilder == null ? EMPTY_INDEX : emptiedSlotsBuilder.build(); - } - - private RowSetBuilderRandom applyUpdateAndTrackEmpty(@NotNull final UpdateTracker tracker, - @Nullable final WritableRowSet slotIndex, - @NotNull final RowSetShiftData shiftsToApply, - @Nullable RowSetBuilderRandom emptiedSlotsBuilder) { - if (slotIndex == null) { - Assert.assertion(tracker.modifiedBuilder == null && tracker.removedBuilder == null, - "For a missing slot index the update must have been add only"); - slotIndices.set(tracker.slot, (WritableRowSet) tracker.getAdded()); - } else { - tracker.applyTo(slotIndex, shiftsToApply); - - if (slotIndex.isEmpty()) { - if (emptiedSlotsBuilder == null) { - emptiedSlotsBuilder = RowSetFactory.builderRandom(); - } - emptiedSlotsBuilder.addKey(tracker.slot); - } - } - - return emptiedSlotsBuilder; - } - - /** - * For each value, call slotConsumer. - * - * @param slotConsumer the consumer of our values - */ - public void forAllModifiedSlots(@NotNull final ModifiedSlotConsumer slotConsumer) { - for (int ii = 0; ii < pointer; ++ii) { - final UpdateTracker trackerForSlot = slotUpdates.getUnsafe(ii); - slotConsumer.accept(trackerForSlot, slotIndices.getUnsafe(trackerForSlot.slot)); - } - } - - /** - * Add a slot in the main table. - * - * @param slot the slot to add. - */ - public void addToBucket(final int slot, - @NotNull final LongChunk addedChunk, - final int startPos, - final int length) { - largestSlotPosition = Math.max(largestSlotPosition, slot); - final UpdateTracker tracker = getTracker(slot); - if (tracker.addedBuilder == null) { - tracker.addedBuilder = RowSetFactory.builderSequential(); - } - - for (int chunkIdx = startPos; chunkIdx < startPos + length; chunkIdx++) { - tracker.addedBuilder.appendKey(addedChunk.get(chunkIdx)); - } - } - - public void addToBucket(final int slot, - final long keyToAdd) { - largestSlotPosition = Math.max(largestSlotPosition, slot); - final UpdateTracker tracker = getTracker(slot); - if (tracker.changedBucketAddedBuilder == null) { - tracker.changedBucketAddedBuilder = RowSetFactory.builderSequential(); - } - - tracker.changedBucketAddedBuilder.appendKey(keyToAdd); - } - - public void modifyBucket(final int slot, - @NotNull final LongChunk modifiedChunk, - final int startPos, - final int length) { - final UpdateTracker tracker = getTracker(slot); - if (tracker.modifiedBuilder == null) { - tracker.modifiedBuilder = RowSetFactory.builderSequential(); - } - - for (int chunkIdx = startPos; chunkIdx < startPos + length; chunkIdx++) { - tracker.modifiedBuilder.appendKey(modifiedChunk.get(chunkIdx)); - } - } - - public void modifyBucket(final int slot, - final long modifiedKey) { - final UpdateTracker tracker = getTracker(slot); - if (tracker.modifiedBuilder == null) { - tracker.modifiedBuilder = RowSetFactory.builderSequential(); - } - tracker.modifiedBuilder.appendKey(modifiedKey); - } - - public void removeFromBucket(final int slot, - @NotNull final LongChunk removedChunk, - final int startPos, - final int length) { - final UpdateTracker tracker = getTracker(slot); - if (tracker.removedBuilder == null) { - tracker.removedBuilder = RowSetFactory.builderSequential(); - } - for (int chunkIdx = startPos; chunkIdx < startPos + length; chunkIdx++) { - tracker.removedBuilder.appendKey(removedChunk.get(chunkIdx)); - } - } - - public void removeFromBucket(final int slot, - final long removedKey) { - final UpdateTracker tracker = getTracker(slot); - if (tracker.changedBucketRemovedBuilder == null) { - tracker.changedBucketRemovedBuilder = RowSetFactory.builderSequential(); - } - tracker.changedBucketRemovedBuilder.appendKey(removedKey); - } - - public void markForShift(int slot) { - final UpdateTracker tracker = getTracker(slot); - tracker.affectedByShift = true; - } - - public UpdateTracker getTracker(final int slot) { - long cookie = slotToCookie.getLong(slot); - if (!isValidCookie(cookie)) { - cookie = createUpdateForSlot(slot); - } - - final long pointer = getPointerFromCookie(cookie); - return slotUpdates.getUnsafe(pointer); - } - - private long createUpdateForSlot(final int slot) { - if (pointer == allocated || slot == allocated) { - allocated += chunkSize; - slotUpdates.ensureCapacity(allocated); - } - - final long cookie = getCookieFromPointer(pointer); - slotToCookie.ensureCapacity(allocated); - slotToCookie.set(slot, cookie); - slotUpdates.set(pointer, new UpdateTracker(slot)); - pointer++; - - return cookie; - } - - /** - * Is this cookie within our valid range (greater than or equal to our generation, but less than the pointer after - * adjustment? - * - * @param cookie the cookie to check for validity - * - * @return true if the cookie is from the current generation, and references a valid slot in our table - */ - @SuppressWarnings("BooleanMethodIsAlwaysInverted") - private boolean isValidCookie(long cookie) { - return cookie >= cookieGeneration && getPointerFromCookie(cookie) < pointer; - } - - /** - * Get a cookie to return to the user, given a pointer value. - * - * @param pointer the pointer to convert to a cookie - * @return the cookie to return to the user - */ - private long getCookieFromPointer(long pointer) { - return cookieGeneration + pointer; - } - - /** - * Given a valid user's cookie, return the corresponding pointer. - * - * @param cookie the valid cookie - * @return the pointer into modifiedSlots - */ - private long getPointerFromCookie(long cookie) { - return cookie - cookieGeneration; - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java index 8f982b928da..c9b9c31644e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java @@ -20,6 +20,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; import io.deephaven.util.QueryConstants; import org.jetbrains.annotations.NotNull; @@ -49,7 +50,7 @@ protected class Context extends UpdateCumulativeContext { public byte curVal = nullValue; - protected Context(final int chunkSize) { + protected Context(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); this.fillContext.ensureCapacity(chunkSize); this.outputValues = new SizedByteChunk<>(chunkSize); @@ -145,8 +146,8 @@ public Map> getOutputColumns() { @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { + return new Context(chunkSize, timestampSsa); } @Override @@ -205,6 +206,7 @@ public void resetForProcess(@NotNull final UpdateContext context, public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, + @Nullable final LongChunk posChunk, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java index 94fd4f673e4..0fe5c19a4b0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java @@ -10,6 +10,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; import io.deephaven.util.QueryConstants; import org.jetbrains.annotations.NotNull; @@ -38,7 +39,7 @@ protected class Context extends UpdateCumulativeContext { public char curVal = QueryConstants.NULL_CHAR; - protected Context(final int chunkSize) { + protected Context(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); this.fillContext.ensureCapacity(chunkSize); this.outputValues = new SizedCharChunk<>(chunkSize); @@ -121,8 +122,8 @@ public Map> getOutputColumns() { @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { + return new Context(chunkSize, timestampSsa); } @Override @@ -179,6 +180,7 @@ public void resetForProcess(@NotNull final UpdateContext context, public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, + @Nullable final LongChunk posChunk, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java index 006272b01b8..6963f208584 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java @@ -6,24 +6,19 @@ package io.deephaven.engine.table.impl.updateby.internal; import io.deephaven.chunk.Chunk; -import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.attributes.ChunkLengths; -import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.sized.SizedDoubleChunk; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.DoubleArraySource; import io.deephaven.engine.table.impl.sources.DoubleSparseArraySource; import io.deephaven.engine.table.impl.sources.WritableRedirectedColumnSource; -import io.deephaven.engine.table.impl.util.RowRedirection; +import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; -import io.deephaven.util.QueryConstants; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -32,7 +27,6 @@ import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; import static io.deephaven.util.QueryConstants.NULL_DOUBLE; -import static io.deephaven.util.QueryConstants.NULL_LONG; public abstract class BaseDoubleUpdateByOperator extends UpdateByCumulativeOperator { protected final WritableColumnSource outputSource; @@ -54,10 +48,11 @@ protected class Context extends UpdateCumulativeContext { public boolean filledWithPermanentValue = false; - protected Context(final int chunkSize) { + protected Context(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); this.fillContext.ensureCapacity(chunkSize); this.outputValues = new SizedDoubleChunk<>(chunkSize); + this.timestampSsa = timestampSsa; } public RowSetBuilderSequential getModifiedBuilder() { @@ -129,8 +124,8 @@ public Map> getOutputColumns() { @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { + return new Context(chunkSize, timestampSsa); } @Override @@ -178,6 +173,7 @@ public void resetForProcess(@NotNull final UpdateContext context, public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, + @Nullable final LongChunk posChunk, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceRowSet) { final Context ctx = (Context) updateContext; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java index 75fec93e14e..78978009810 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java @@ -1,24 +1,19 @@ package io.deephaven.engine.table.impl.updateby.internal; import io.deephaven.chunk.Chunk; -import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.attributes.ChunkLengths; -import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.sized.SizedFloatChunk; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.FloatArraySource; import io.deephaven.engine.table.impl.sources.FloatSparseArraySource; import io.deephaven.engine.table.impl.sources.WritableRedirectedColumnSource; -import io.deephaven.engine.table.impl.util.RowRedirection; +import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; -import io.deephaven.util.QueryConstants; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -27,7 +22,6 @@ import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; import static io.deephaven.util.QueryConstants.NULL_FLOAT; -import static io.deephaven.util.QueryConstants.NULL_LONG; public abstract class BaseFloatUpdateByOperator extends UpdateByCumulativeOperator { protected final WritableColumnSource outputSource; @@ -49,10 +43,11 @@ protected class Context extends UpdateCumulativeContext { public boolean filledWithPermanentValue = false; - protected Context(final int chunkSize) { + protected Context(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); this.fillContext.ensureCapacity(chunkSize); this.outputValues = new SizedFloatChunk<>(chunkSize); + this.timestampSsa = timestampSsa; } public RowSetBuilderSequential getModifiedBuilder() { @@ -124,8 +119,8 @@ public Map> getOutputColumns() { @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { + return new Context(chunkSize, timestampSsa); } @Override @@ -173,6 +168,7 @@ public void resetForProcess(@NotNull final UpdateContext context, public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, + @Nullable final LongChunk posChunk, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceRowSet) { final Context ctx = (Context) updateContext; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java index f869d8cb81e..89bf7134535 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java @@ -15,6 +15,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; import io.deephaven.util.QueryConstants; import org.jetbrains.annotations.NotNull; @@ -43,7 +44,7 @@ protected class Context extends UpdateCumulativeContext { public int curVal = QueryConstants.NULL_INT; - protected Context(final int chunkSize) { + protected Context(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); this.fillContext.ensureCapacity(chunkSize); this.outputValues = new SizedIntChunk<>(chunkSize); @@ -126,8 +127,8 @@ public Map> getOutputColumns() { @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { + return new Context(chunkSize, timestampSsa); } @Override @@ -184,6 +185,7 @@ public void resetForProcess(@NotNull final UpdateContext context, public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, + @Nullable final LongChunk posChunk, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java index c08ca39f3ad..999a09ce517 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java @@ -15,6 +15,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; import io.deephaven.util.QueryConstants; import org.jetbrains.annotations.NotNull; @@ -43,7 +44,7 @@ protected class Context extends UpdateCumulativeContext { public long curVal = QueryConstants.NULL_LONG; - protected Context(final int chunkSize) { + protected Context(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); this.fillContext.ensureCapacity(chunkSize); this.outputValues = new SizedLongChunk<>(chunkSize); @@ -126,8 +127,8 @@ public Map> getOutputColumns() { @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { + return new Context(chunkSize, timestampSsa); } @Override @@ -184,6 +185,7 @@ public void resetForProcess(@NotNull final UpdateContext context, public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, + @Nullable final LongChunk posChunk, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java index cfb20c79123..39319af9d2d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java @@ -15,6 +15,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; import io.deephaven.util.QueryConstants; import org.jetbrains.annotations.NotNull; @@ -44,7 +45,7 @@ protected class Context extends UpdateCumulativeContext { public T curVal = null; - protected Context(final int chunkSize) { + protected Context(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); this.fillContext.ensureCapacity(chunkSize); this.outputValues = new SizedObjectChunk<>(chunkSize); @@ -129,8 +130,8 @@ public Map> getOutputColumns() { @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { + return new Context(chunkSize, timestampSsa); } @Override @@ -187,6 +188,7 @@ public void resetForProcess(@NotNull final UpdateContext context, public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, + @Nullable final LongChunk posChunk, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java index 06d773c29dd..8be05b14229 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java @@ -15,6 +15,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; import io.deephaven.util.QueryConstants; import org.jetbrains.annotations.NotNull; @@ -43,7 +44,7 @@ protected class Context extends UpdateCumulativeContext { public short curVal = QueryConstants.NULL_SHORT; - protected Context(final int chunkSize) { + protected Context(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); this.fillContext.ensureCapacity(chunkSize); this.outputValues = new SizedShortChunk<>(chunkSize); @@ -126,8 +127,8 @@ public Map> getOutputColumns() { @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { + return new Context(chunkSize, timestampSsa); } @Override @@ -184,6 +185,7 @@ public void resetForProcess(@NotNull final UpdateContext context, public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, + @Nullable final LongChunk posChunk, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java index cf3e22cd396..11ce039fcf6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java @@ -44,13 +44,13 @@ public void close() { @Override public void loadInfluencerValueChunk() { - int size = influencerKeys.intSize(); + int size = influencerRows.intSize(); // fill the window values chunk if (candidateValuesChunk == null) { candidateValuesChunk = WritableByteChunk.makeWritableChunk(size); } try (ChunkSource.FillContext fc = valueSource.makeFillContext(size)){ - valueSource.fillChunk(fc, candidateValuesChunk, influencerKeys); + valueSource.fillChunk(fc, candidateValuesChunk, influencerRows); } } } @@ -58,8 +58,8 @@ public void loadInfluencerValueChunk() { public BaseWindowedByteUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, - @Nullable final LongRecordingUpdateByOperator timeRecorder, @Nullable final String timestampColumnName, + @Nullable final ColumnSource timestampColumnSource, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, @@ -67,7 +67,7 @@ public BaseWindowedByteUpdateByOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); + super(pair, affectingColumns, control, timestampColumnName, timestampColumnSource, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor this.nullValue = getNullValue(); @@ -100,6 +100,7 @@ protected WritableColumnSource makeDenseSource() { protected abstract void doProcessChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, + @Nullable final LongChunk posChunk, @NotNull final Chunk workingChunk); // endregion @@ -117,10 +118,11 @@ public void resetForProcess(@NotNull final UpdateContext context, public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, + @Nullable final LongChunk posChunk, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; - doProcessChunk(ctx, inputKeys, keyChunk, valuesChunk); + doProcessChunk(ctx, inputKeys, keyChunk, posChunk, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java index 5e1a2319096..dbb4ac41952 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java @@ -33,13 +33,13 @@ public void close() { @Override public void loadInfluencerValueChunk() { - int size = influencerKeys.intSize(); + int size = influencerRows.intSize(); // fill the window values chunk if (candidateValuesChunk == null) { candidateValuesChunk = WritableCharChunk.makeWritableChunk(size); } try (ChunkSource.FillContext fc = valueSource.makeFillContext(size)){ - valueSource.fillChunk(fc, candidateValuesChunk, influencerKeys); + valueSource.fillChunk(fc, candidateValuesChunk, influencerRows); } } } @@ -47,8 +47,8 @@ public void loadInfluencerValueChunk() { public BaseWindowedCharUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, - @Nullable final LongRecordingUpdateByOperator timeRecorder, @Nullable final String timestampColumnName, + @Nullable final ColumnSource timestampColumnSource, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, @@ -56,7 +56,7 @@ public BaseWindowedCharUpdateByOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); + super(pair, affectingColumns, control, timestampColumnName, timestampColumnSource, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor // endregion constructor @@ -76,6 +76,7 @@ public BaseWindowedCharUpdateByOperator(@NotNull final MatchPair pair, protected abstract void doProcessChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, + @Nullable final LongChunk posChunk, @NotNull final Chunk workingChunk); // endregion @@ -93,10 +94,11 @@ public void resetForProcess(@NotNull final UpdateContext context, public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, + @Nullable final LongChunk posChunk, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; - doProcessChunk(ctx, inputKeys, keyChunk, valuesChunk); + doProcessChunk(ctx, inputKeys, keyChunk, posChunk, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java index 2b08d08dac0..63bc0023daa 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java @@ -38,13 +38,13 @@ public void close() { @Override public void loadInfluencerValueChunk() { - int size = influencerKeys.intSize(); + int size = influencerRows.intSize(); // fill the window values chunk if (candidateValuesChunk == null) { candidateValuesChunk = WritableDoubleChunk.makeWritableChunk(size); } try (ChunkSource.FillContext fc = valueSource.makeFillContext(size)){ - valueSource.fillChunk(fc, candidateValuesChunk, influencerKeys); + valueSource.fillChunk(fc, candidateValuesChunk, influencerRows); } } } @@ -52,8 +52,8 @@ public void loadInfluencerValueChunk() { public BaseWindowedDoubleUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, - @Nullable final LongRecordingUpdateByOperator timeRecorder, @Nullable final String timestampColumnName, + @Nullable final ColumnSource timestampColumnSource, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, @@ -61,7 +61,7 @@ public BaseWindowedDoubleUpdateByOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); + super(pair, affectingColumns, control, timestampColumnName, timestampColumnSource, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor // endregion constructor @@ -88,6 +88,7 @@ public boolean requiresValues(@NotNull final UpdateContext context) { protected abstract void doProcessChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, + @Nullable final LongChunk posChunk, @NotNull final Chunk workingChunk); // endregion @@ -105,10 +106,11 @@ public void resetForProcess(@NotNull final UpdateContext context, public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, + @Nullable final LongChunk posChunk, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; - doProcessChunk(ctx, inputKeys, keyChunk, valuesChunk); + doProcessChunk(ctx, inputKeys, keyChunk, posChunk, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java index 224ebf4423b..4c7b7343d59 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java @@ -33,13 +33,13 @@ public void close() { @Override public void loadInfluencerValueChunk() { - int size = influencerKeys.intSize(); + int size = influencerRows.intSize(); // fill the window values chunk if (candidateValuesChunk == null) { candidateValuesChunk = WritableFloatChunk.makeWritableChunk(size); } try (ChunkSource.FillContext fc = valueSource.makeFillContext(size)){ - valueSource.fillChunk(fc, candidateValuesChunk, influencerKeys); + valueSource.fillChunk(fc, candidateValuesChunk, influencerRows); } } } @@ -47,8 +47,8 @@ public void loadInfluencerValueChunk() { public BaseWindowedFloatUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, - @Nullable final LongRecordingUpdateByOperator timeRecorder, @Nullable final String timestampColumnName, + @Nullable final ColumnSource timestampColumnSource, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, @@ -56,7 +56,7 @@ public BaseWindowedFloatUpdateByOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); + super(pair, affectingColumns, control, timestampColumnName, timestampColumnSource, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor // endregion constructor @@ -83,6 +83,7 @@ public boolean requiresValues(@NotNull final UpdateContext context) { protected abstract void doProcessChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, + @Nullable final LongChunk posChunk, @NotNull final Chunk workingChunk); // endregion @@ -100,10 +101,11 @@ public void resetForProcess(@NotNull final UpdateContext context, public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, + @Nullable final LongChunk posChunk, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; - doProcessChunk(ctx, inputKeys, keyChunk, valuesChunk); + doProcessChunk(ctx, inputKeys, keyChunk, posChunk, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java index 53c0e3a169a..5b911543f64 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java @@ -38,13 +38,13 @@ public void close() { @Override public void loadInfluencerValueChunk() { - int size = influencerKeys.intSize(); + int size = influencerRows.intSize(); // fill the window values chunk if (candidateValuesChunk == null) { candidateValuesChunk = WritableIntChunk.makeWritableChunk(size); } try (ChunkSource.FillContext fc = valueSource.makeFillContext(size)){ - valueSource.fillChunk(fc, candidateValuesChunk, influencerKeys); + valueSource.fillChunk(fc, candidateValuesChunk, influencerRows); } } } @@ -52,8 +52,8 @@ public void loadInfluencerValueChunk() { public BaseWindowedIntUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, - @Nullable final LongRecordingUpdateByOperator timeRecorder, @Nullable final String timestampColumnName, + @Nullable final ColumnSource timestampColumnSource, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, @@ -61,7 +61,7 @@ public BaseWindowedIntUpdateByOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); + super(pair, affectingColumns, control, timestampColumnName, timestampColumnSource, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor // endregion constructor @@ -81,6 +81,7 @@ public BaseWindowedIntUpdateByOperator(@NotNull final MatchPair pair, protected abstract void doProcessChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, + @Nullable final LongChunk posChunk, @NotNull final Chunk workingChunk); // endregion @@ -98,10 +99,11 @@ public void resetForProcess(@NotNull final UpdateContext context, public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, + @Nullable final LongChunk posChunk, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; - doProcessChunk(ctx, inputKeys, keyChunk, valuesChunk); + doProcessChunk(ctx, inputKeys, keyChunk, posChunk, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java index f36dfb7bb5a..f588a4483d8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java @@ -38,13 +38,13 @@ public void close() { @Override public void loadInfluencerValueChunk() { - int size = influencerKeys.intSize(); + int size = influencerRows.intSize(); // fill the window values chunk if (candidateValuesChunk == null) { candidateValuesChunk = WritableLongChunk.makeWritableChunk(size); } try (ChunkSource.FillContext fc = valueSource.makeFillContext(size)){ - valueSource.fillChunk(fc, candidateValuesChunk, influencerKeys); + valueSource.fillChunk(fc, candidateValuesChunk, influencerRows); } } } @@ -52,8 +52,8 @@ public void loadInfluencerValueChunk() { public BaseWindowedLongUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, - @Nullable final LongRecordingUpdateByOperator timeRecorder, @Nullable final String timestampColumnName, + @Nullable final ColumnSource timestampColumnSource, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, @@ -61,7 +61,7 @@ public BaseWindowedLongUpdateByOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); + super(pair, affectingColumns, control, timestampColumnName, timestampColumnSource, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor // endregion constructor @@ -81,6 +81,7 @@ public BaseWindowedLongUpdateByOperator(@NotNull final MatchPair pair, protected abstract void doProcessChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, + @Nullable final LongChunk posChunk, @NotNull final Chunk workingChunk); // endregion @@ -98,10 +99,11 @@ public void resetForProcess(@NotNull final UpdateContext context, public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, + @Nullable final LongChunk posChunk, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; - doProcessChunk(ctx, inputKeys, keyChunk, valuesChunk); + doProcessChunk(ctx, inputKeys, keyChunk, posChunk, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java index b8020af2812..c604d5012d2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java @@ -39,13 +39,13 @@ public void close() { @Override public void loadInfluencerValueChunk() { - int size = influencerKeys.intSize(); + int size = influencerRows.intSize(); // fill the window values chunk if (candidateValuesChunk == null) { candidateValuesChunk = WritableObjectChunk.makeWritableChunk(size); } try (ChunkSource.FillContext fc = valueSource.makeFillContext(size)){ - valueSource.fillChunk(fc, candidateValuesChunk, influencerKeys); + valueSource.fillChunk(fc, candidateValuesChunk, influencerRows); } } } @@ -53,8 +53,8 @@ public void loadInfluencerValueChunk() { public BaseWindowedObjectUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, - @Nullable final LongRecordingUpdateByOperator timeRecorder, @Nullable final String timestampColumnName, + @Nullable final ColumnSource timestampColumnSource, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, @@ -63,7 +63,7 @@ public BaseWindowedObjectUpdateByOperator(@NotNull final MatchPair pair, , final Class colType // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); + super(pair, affectingColumns, control, timestampColumnName, timestampColumnSource, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor this.colType = colType; @@ -84,6 +84,7 @@ public BaseWindowedObjectUpdateByOperator(@NotNull final MatchPair pair, protected abstract void doProcessChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, + @Nullable final LongChunk posChunk, @NotNull final Chunk workingChunk); // endregion @@ -101,10 +102,11 @@ public void resetForProcess(@NotNull final UpdateContext context, public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, + @Nullable final LongChunk posChunk, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; - doProcessChunk(ctx, inputKeys, keyChunk, valuesChunk); + doProcessChunk(ctx, inputKeys, keyChunk, posChunk, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java index 2f8c30b14f0..92843d51b95 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java @@ -38,13 +38,13 @@ public void close() { @Override public void loadInfluencerValueChunk() { - int size = influencerKeys.intSize(); + int size = influencerRows.intSize(); // fill the window values chunk if (candidateValuesChunk == null) { candidateValuesChunk = WritableShortChunk.makeWritableChunk(size); } try (ChunkSource.FillContext fc = valueSource.makeFillContext(size)){ - valueSource.fillChunk(fc, candidateValuesChunk, influencerKeys); + valueSource.fillChunk(fc, candidateValuesChunk, influencerRows); } } } @@ -52,8 +52,8 @@ public void loadInfluencerValueChunk() { public BaseWindowedShortUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, - @Nullable final LongRecordingUpdateByOperator timeRecorder, @Nullable final String timestampColumnName, + @Nullable final ColumnSource timestampColumnSource, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, @@ -61,7 +61,7 @@ public BaseWindowedShortUpdateByOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); + super(pair, affectingColumns, control, timestampColumnName, timestampColumnSource, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor // endregion constructor @@ -81,6 +81,7 @@ public BaseWindowedShortUpdateByOperator(@NotNull final MatchPair pair, protected abstract void doProcessChunk(@NotNull final Context ctx, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, + @Nullable final LongChunk posChunk, @NotNull final Chunk workingChunk); // endregion @@ -98,10 +99,11 @@ public void resetForProcess(@NotNull final UpdateContext context, public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, + @Nullable final LongChunk posChunk, @NotNull final Chunk valuesChunk, @NotNull final RowSet postUpdateSourceIndex) { final Context ctx = (Context) updateContext; - doProcessChunk(ctx, inputKeys, keyChunk, valuesChunk); + doProcessChunk(ctx, inputKeys, keyChunk, posChunk, valuesChunk); ctx.getModifiedBuilder().appendRowSequence(inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/LongRecordingUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/LongRecordingUpdateByOperator.java index fab250c7cea..a71f9cd4ebe 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/LongRecordingUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/LongRecordingUpdateByOperator.java @@ -2,25 +2,16 @@ import io.deephaven.base.verify.Require; import io.deephaven.chunk.Chunk; -import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.attributes.ChunkLengths; -import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.chunk.attributes.Values; import io.deephaven.datastructures.util.CollectionUtil; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.rowset.RowSetFactory; -import io.deephaven.engine.rowset.RowSetShiftData; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.TableUpdate; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; -import io.deephaven.engine.table.impl.UpdateByOperator; -import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.sources.ReinterpretUtils; +import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -94,13 +85,16 @@ public ColumnSource getColumnSource() { } @Override - public void processChunk(@NotNull UpdateContext updateContext, @NotNull RowSequence inputKeys, @Nullable LongChunk keyChunk, @NotNull Chunk valuesChunk, @NotNull RowSet postUpdateSourceIndex) { + public void processChunk(@NotNull UpdateContext updateContext, @NotNull RowSequence inputKeys, + @Nullable LongChunk keyChunk, + @Nullable LongChunk posChunk, + @NotNull Chunk valuesChunk, @NotNull RowSet postUpdateSourceIndex) { currentContext.addedChunk = valuesChunk.asLongChunk(); } @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, LongSegmentedSortedArray timestampSsa) { return this.currentContext = new RecordingContext(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java index c5e4354f264..e91b3337857 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java @@ -7,11 +7,8 @@ import io.deephaven.api.updateby.OperationControl; 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.chunk.sized.SizedLongChunk; -import io.deephaven.chunk.util.pools.ChunkPoolConstants; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.rowset.chunkattributes.RowKeys; @@ -19,16 +16,13 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.updateby.internal.*; -import io.deephaven.engine.table.impl.util.RowRedirection; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; -import io.deephaven.util.QueryConstants; -import org.apache.commons.lang3.mutable.MutableLong; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import java.util.Collections; -import java.util.LinkedList; import java.util.Map; import static io.deephaven.util.QueryConstants.NULL_LONG; @@ -53,11 +47,12 @@ protected class Context extends BaseWindowedByteUpdateByOperator.Context { // position data for the chunk being currently processed public SizedLongChunk valuePositionChunk; - protected Context(final int chunkSize) { + protected Context(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); this.fillContext.ensureCapacity(chunkSize); this.outputValues = new SizedLongChunk<>(chunkSize); this.valuePositionChunk = new SizedLongChunk<>(chunkSize); + this.timestampSsa = timestampSsa; } @Override @@ -71,8 +66,8 @@ public void close() { @NotNull @Override - public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize) { - return new Context(chunkSize); + public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { + return new Context(chunkSize, timestampSsa); } @Override @@ -86,8 +81,8 @@ public void setChunkSize(@NotNull UpdateContext context, int chunkSize) { public ByteRollingSumOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, - @Nullable final LongRecordingUpdateByOperator recorder, @Nullable final String timestampColumnName, + @Nullable final ColumnSource timestampColumnSource, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final ColumnSource valueSource, @@ -96,7 +91,7 @@ public ByteRollingSumOperator(@NotNull final MatchPair pair, ,final byte nullValue // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, recorder, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource); + super(pair, affectingColumns, control, timestampColumnName, timestampColumnSource, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource); if(redirContext.isRedirected()) { // region create-dense this.maybeInnerSource = new LongArraySource(); @@ -150,6 +145,7 @@ public void reset(UpdateContext context) { public void doProcessChunk(@NotNull final BaseWindowedByteUpdateByOperator.Context context, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, + @Nullable final LongChunk posChunk, @NotNull final Chunk workingChunk) { final Context ctx = (Context) context; @@ -174,7 +170,7 @@ private void computeTicks(@NotNull final Context ctx, final WritableLongChunk localOutputValues = ctx.outputValues.get(); for (int ii = runStart; ii < runStart + runLength; ii++) { - if (recorder == null) { + if (timestampColumnName == null) { ctx.fillWindowTicks(ctx, ctx.valuePositionChunk.get().get(ii)); } // the sum was computed by push/pop operations diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java index 79a3749ca06..7621c193806 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java @@ -9,7 +9,6 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableDoubleChunk; -import io.deephaven.chunk.WritableLongChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.sized.SizedDoubleChunk; import io.deephaven.chunk.sized.SizedLongChunk; @@ -23,13 +22,11 @@ import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedDoubleUpdateByOperator; -import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; import io.deephaven.util.QueryConstants; import org.apache.commons.lang3.mutable.MutableDouble; -import org.apache.commons.lang3.mutable.MutableLong; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -38,7 +35,6 @@ import java.util.Map; import static io.deephaven.util.QueryConstants.NULL_DOUBLE; -import static io.deephaven.util.QueryConstants.NULL_LONG; public class DoubleRollingSumOperator extends BaseWindowedDoubleUpdateByOperator { @@ -58,11 +54,12 @@ protected class Context extends BaseWindowedDoubleUpdateByOperator.Context { // position data for the chunk being currently processed public SizedLongChunk valuePositionChunk; - protected Context(final int chunkSize) { + protected Context(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); this.fillContext.ensureCapacity(chunkSize); this.outputValues = new SizedDoubleChunk<>(chunkSize); this.valuePositionChunk = new SizedLongChunk<>(chunkSize); + this.timestampSsa = timestampSsa; } @Override @@ -76,8 +73,8 @@ public void close() { @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { + return new Context(chunkSize, timestampSsa); } @Override @@ -91,8 +88,8 @@ public void setChunkSize(@NotNull UpdateContext context, int chunkSize) { public DoubleRollingSumOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, - @Nullable final LongRecordingUpdateByOperator recorder, @Nullable final String timestampColumnName, + @Nullable final ColumnSource timestampColumnSource, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final ColumnSource valueSource, @@ -100,7 +97,7 @@ public DoubleRollingSumOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, recorder, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource); + super(pair, affectingColumns, control, timestampColumnName, timestampColumnSource, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource); if(redirContext.isRedirected()) { // region create-dense this.maybeInnerSource = new DoubleArraySource(); @@ -140,6 +137,7 @@ public void reset(UpdateContext context) { public void doProcessChunk(@NotNull final BaseWindowedDoubleUpdateByOperator.Context context, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, + @Nullable final LongChunk posChunk, @NotNull final Chunk workingChunk) { final Context ctx = (Context) context; @@ -163,7 +161,7 @@ private void computeTicks(@NotNull final Context ctx, final WritableDoubleChunk localOutputValues = ctx.outputValues.get(); for (int ii = runStart; ii < runStart + runLength; ii++) { - if (recorder == null) { + if (timestampColumnName == null) { ctx.fillWindowTicks(ctx, ctx.valuePositionChunk.get().get(ii)); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java index de5b9e08d00..dfb760245ca 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java @@ -4,7 +4,6 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableFloatChunk; -import io.deephaven.chunk.WritableLongChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.sized.SizedFloatChunk; import io.deephaven.chunk.sized.SizedLongChunk; @@ -18,13 +17,11 @@ import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedFloatUpdateByOperator; -import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; import io.deephaven.util.QueryConstants; import org.apache.commons.lang3.mutable.MutableFloat; -import org.apache.commons.lang3.mutable.MutableLong; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -33,7 +30,6 @@ import java.util.Map; import static io.deephaven.util.QueryConstants.NULL_FLOAT; -import static io.deephaven.util.QueryConstants.NULL_LONG; public class FloatRollingSumOperator extends BaseWindowedFloatUpdateByOperator { @@ -53,11 +49,12 @@ protected class Context extends BaseWindowedFloatUpdateByOperator.Context { // position data for the chunk being currently processed public SizedLongChunk valuePositionChunk; - protected Context(final int chunkSize) { + protected Context(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); this.fillContext.ensureCapacity(chunkSize); this.outputValues = new SizedFloatChunk<>(chunkSize); this.valuePositionChunk = new SizedLongChunk<>(chunkSize); + this.timestampSsa = timestampSsa; } @Override @@ -71,8 +68,8 @@ public void close() { @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { + return new Context(chunkSize, timestampSsa); } @Override @@ -86,8 +83,8 @@ public void setChunkSize(@NotNull UpdateContext context, int chunkSize) { public FloatRollingSumOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, - @Nullable final LongRecordingUpdateByOperator recorder, @Nullable final String timestampColumnName, + @Nullable final ColumnSource timestampColumnSource, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final ColumnSource valueSource, @@ -95,7 +92,7 @@ public FloatRollingSumOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, recorder, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource); + super(pair, affectingColumns, control, timestampColumnName, timestampColumnSource, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource); if(redirContext.isRedirected()) { // region create-dense this.maybeInnerSource = new FloatArraySource(); @@ -135,6 +132,7 @@ public void reset(UpdateContext context) { public void doProcessChunk(@NotNull final BaseWindowedFloatUpdateByOperator.Context context, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, + @Nullable final LongChunk posChunk, @NotNull final Chunk workingChunk) { final Context ctx = (Context) context; @@ -158,7 +156,7 @@ private void computeTicks(@NotNull final Context ctx, final WritableFloatChunk localOutputValues = ctx.outputValues.get(); for (int ii = runStart; ii < runStart + runLength; ii++) { - if (recorder == null) { + if (timestampColumnName == null) { ctx.fillWindowTicks(ctx, ctx.valuePositionChunk.get().get(ii)); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java index 690c58f3346..ab84050c122 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java @@ -7,11 +7,8 @@ import io.deephaven.api.updateby.OperationControl; 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.chunk.sized.SizedLongChunk; -import io.deephaven.chunk.util.pools.ChunkPoolConstants; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.rowset.chunkattributes.RowKeys; @@ -19,16 +16,13 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.updateby.internal.*; -import io.deephaven.engine.table.impl.util.RowRedirection; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; -import io.deephaven.util.QueryConstants; -import org.apache.commons.lang3.mutable.MutableLong; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import java.util.Collections; -import java.util.LinkedList; import java.util.Map; import static io.deephaven.util.QueryConstants.NULL_LONG; @@ -52,11 +46,12 @@ protected class Context extends BaseWindowedIntUpdateByOperator.Context { // position data for the chunk being currently processed public SizedLongChunk valuePositionChunk; - protected Context(final int chunkSize) { + protected Context(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); this.fillContext.ensureCapacity(chunkSize); this.outputValues = new SizedLongChunk<>(chunkSize); this.valuePositionChunk = new SizedLongChunk<>(chunkSize); + this.timestampSsa = timestampSsa; } @Override @@ -70,8 +65,8 @@ public void close() { @NotNull @Override - public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize) { - return new Context(chunkSize); + public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { + return new Context(chunkSize, timestampSsa); } @Override @@ -85,8 +80,8 @@ public void setChunkSize(@NotNull UpdateContext context, int chunkSize) { public IntRollingSumOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, - @Nullable final LongRecordingUpdateByOperator recorder, @Nullable final String timestampColumnName, + @Nullable final ColumnSource timestampColumnSource, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final ColumnSource valueSource, @@ -94,7 +89,7 @@ public IntRollingSumOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, recorder, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource); + super(pair, affectingColumns, control, timestampColumnName, timestampColumnSource, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource); if(redirContext.isRedirected()) { // region create-dense this.maybeInnerSource = new LongArraySource(); @@ -147,6 +142,7 @@ public void reset(UpdateContext context) { public void doProcessChunk(@NotNull final BaseWindowedIntUpdateByOperator.Context context, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, + @Nullable final LongChunk posChunk, @NotNull final Chunk workingChunk) { final Context ctx = (Context) context; @@ -171,7 +167,7 @@ private void computeTicks(@NotNull final Context ctx, final WritableLongChunk localOutputValues = ctx.outputValues.get(); for (int ii = runStart; ii < runStart + runLength; ii++) { - if (recorder == null) { + if (timestampColumnName == null) { ctx.fillWindowTicks(ctx, ctx.valuePositionChunk.get().get(ii)); } // the sum was computed by push/pop operations diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java index 75df7c0365a..e566096cf5b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java @@ -7,11 +7,8 @@ import io.deephaven.api.updateby.OperationControl; 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.chunk.sized.SizedLongChunk; -import io.deephaven.chunk.util.pools.ChunkPoolConstants; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.rowset.chunkattributes.RowKeys; @@ -19,16 +16,13 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.updateby.internal.*; -import io.deephaven.engine.table.impl.util.RowRedirection; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; -import io.deephaven.util.QueryConstants; -import org.apache.commons.lang3.mutable.MutableLong; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import java.util.Collections; -import java.util.LinkedList; import java.util.Map; import static io.deephaven.util.QueryConstants.NULL_LONG; @@ -52,11 +46,12 @@ protected class Context extends BaseWindowedLongUpdateByOperator.Context { // position data for the chunk being currently processed public SizedLongChunk valuePositionChunk; - protected Context(final int chunkSize) { + protected Context(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); this.fillContext.ensureCapacity(chunkSize); this.outputValues = new SizedLongChunk<>(chunkSize); this.valuePositionChunk = new SizedLongChunk<>(chunkSize); + this.timestampSsa = timestampSsa; } @Override @@ -70,8 +65,8 @@ public void close() { @NotNull @Override - public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize) { - return new Context(chunkSize); + public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { + return new Context(chunkSize, timestampSsa); } @Override @@ -85,8 +80,8 @@ public void setChunkSize(@NotNull UpdateContext context, int chunkSize) { public LongRollingSumOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, - @Nullable final LongRecordingUpdateByOperator recorder, @Nullable final String timestampColumnName, + @Nullable final ColumnSource timestampColumnSource, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final ColumnSource valueSource, @@ -94,7 +89,7 @@ public LongRollingSumOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, recorder, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource); + super(pair, affectingColumns, control, timestampColumnName, timestampColumnSource, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource); if(redirContext.isRedirected()) { // region create-dense this.maybeInnerSource = new LongArraySource(); @@ -147,6 +142,7 @@ public void reset(UpdateContext context) { public void doProcessChunk(@NotNull final BaseWindowedLongUpdateByOperator.Context context, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, + @Nullable final LongChunk posChunk, @NotNull final Chunk workingChunk) { final Context ctx = (Context) context; @@ -171,7 +167,7 @@ private void computeTicks(@NotNull final Context ctx, final WritableLongChunk localOutputValues = ctx.outputValues.get(); for (int ii = runStart; ii < runStart + runLength; ii++) { - if (recorder == null) { + if (timestampColumnName == null) { ctx.fillWindowTicks(ctx, ctx.valuePositionChunk.get().get(ii)); } // the sum was computed by push/pop operations diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java index b9fb8dc0f8e..2c128f9f95b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java @@ -2,11 +2,8 @@ import io.deephaven.api.updateby.OperationControl; 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.chunk.sized.SizedLongChunk; -import io.deephaven.chunk.util.pools.ChunkPoolConstants; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.rowset.chunkattributes.RowKeys; @@ -14,16 +11,13 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.updateby.internal.*; -import io.deephaven.engine.table.impl.util.RowRedirection; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; -import io.deephaven.util.QueryConstants; -import org.apache.commons.lang3.mutable.MutableLong; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import java.util.Collections; -import java.util.LinkedList; import java.util.Map; import static io.deephaven.util.QueryConstants.NULL_LONG; @@ -44,14 +38,11 @@ protected class Context extends BaseWindowedShortUpdateByOperator.Context { public long currentVal = NULL_LONG; - // position data for the chunk being currently processed - public SizedLongChunk valuePositionChunk; - - protected Context(final int chunkSize) { + protected Context(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); this.fillContext.ensureCapacity(chunkSize); this.outputValues = new SizedLongChunk<>(chunkSize); - this.valuePositionChunk = new SizedLongChunk<>(chunkSize); + this.timestampSsa = timestampSsa; } @Override @@ -59,14 +50,13 @@ public void close() { super.close(); outputValues.close(); fillContext.close(); - this.valuePositionChunk.close(); } } @NotNull @Override - public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize) { - return new Context(chunkSize); + public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { + return new Context(chunkSize, timestampSsa); } @Override @@ -74,14 +64,13 @@ public void setChunkSize(@NotNull UpdateContext context, int chunkSize) { final Context ctx = (Context) context; ctx.outputValues.ensureCapacity(chunkSize); ctx.fillContext.ensureCapacity(chunkSize); - ctx.valuePositionChunk.ensureCapacity(chunkSize); } public ShortRollingSumOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, - @Nullable final LongRecordingUpdateByOperator recorder, @Nullable final String timestampColumnName, + @Nullable final ColumnSource timestampColumnSource, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final ColumnSource valueSource, @@ -89,7 +78,7 @@ public ShortRollingSumOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, recorder, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource); + super(pair, affectingColumns, control, timestampColumnName, timestampColumnSource, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource); if(redirContext.isRedirected()) { // region create-dense this.maybeInnerSource = new LongArraySource(); @@ -142,38 +131,49 @@ public void reset(UpdateContext context) { public void doProcessChunk(@NotNull final BaseWindowedShortUpdateByOperator.Context context, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, + @Nullable final LongChunk posChunk, @NotNull final Chunk workingChunk) { final Context ctx = (Context) context; if (timestampColumnName == null) { - // produce position data for the window (will be timestamps for time-based) - // TODO: gotta be a better way than creating two rowsets - try (final RowSet rs = inputKeys.asRowSet(); - final RowSet positions = ctx.sourceRowSet.invert(rs)) { - positions.fillRowKeyChunk(ctx.valuePositionChunk.get()); - } + computeTicks(ctx, posChunk, inputKeys.intSize()); + } else { + computeTime(ctx, inputKeys); } - computeTicks(ctx, 0, inputKeys.intSize()); //noinspection unchecked outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } private void computeTicks(@NotNull final Context ctx, - final int runStart, + @Nullable final LongChunk posChunk, final int runLength) { final WritableLongChunk localOutputValues = ctx.outputValues.get(); - for (int ii = runStart; ii < runStart + runLength; ii++) { - if (recorder == null) { - ctx.fillWindowTicks(ctx, ctx.valuePositionChunk.get().get(ii)); - } + for (int ii = 0; ii < runLength; ii++) { + ctx.fillWindowTicks(ctx, posChunk.get(ii)); // the sum was computed by push/pop operations localOutputValues.set(ii, ctx.currentVal); } } + private void computeTime(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys) { + + final WritableLongChunk localOutputValues = ctx.outputValues.get(); + // get the timestamp values for this chunk + try (final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(inputKeys.intSize())) { + LongChunk timestampChunk = timestampColumnSource.getChunk(context, inputKeys).asLongChunk(); + + for (int ii = 0; ii < inputKeys.intSize(); ii++) { + ctx.fillWindowTime(ctx, timestampChunk.get(ii)); + // the sum was computed by push/pop operations + localOutputValues.set(ii, ctx.currentVal); + } + } + } + @NotNull @Override public Map> getOutputColumns() { diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java index 93666b7043d..05f0084c756 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java @@ -1,13 +1,17 @@ package io.deephaven.engine.table.impl.updateby; +import io.deephaven.api.updateby.BadDataBehavior; +import io.deephaven.api.updateby.OperationControl; import io.deephaven.api.updateby.UpdateByControl; import io.deephaven.api.updateby.UpdateByOperation; import io.deephaven.engine.table.ColumnDefinition; import io.deephaven.engine.table.PartitionedTable; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.impl.*; +import io.deephaven.engine.table.impl.sources.ReinterpretUtils; import io.deephaven.engine.updategraph.UpdateGraphProcessor; import io.deephaven.test.types.OutOfBandTest; +import io.deephaven.time.DateTime; import org.jetbrains.annotations.NotNull; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -15,6 +19,7 @@ import java.io.IOException; import java.math.BigDecimal; import java.math.BigInteger; +import java.time.Duration; import java.util.Arrays; import java.util.List; import java.util.Random; @@ -25,6 +30,8 @@ import static io.deephaven.engine.table.impl.TstUtils.testTable; import static io.deephaven.engine.util.TableTools.*; import static io.deephaven.function.Basic.isNull; +import static io.deephaven.time.DateTimeUtils.MINUTE; +import static io.deephaven.time.DateTimeUtils.convertDateTime; import static io.deephaven.util.QueryConstants.*; import static org.junit.Assert.assertArrayEquals; @@ -142,6 +149,43 @@ private void doTestStaticBucketed(boolean grouped) { }); } + @Test + public void testStaticZeroKeyTimed() { + final QueryTable t = createTestTable(1000, false, false, false, 0xFFFABBBC, + new String[] {"ts"}, new TstUtils.Generator[] {new TstUtils.SortedDateTimeGenerator( + convertDateTime("2022-03-09T09:00:00.000 NY"), + convertDateTime("2022-03-09T16:30:00.000 NY"))}).t; + + final OperationControl skipControl = OperationControl.builder() + .onNullValue(BadDataBehavior.SKIP) + .onNanValue(BadDataBehavior.SKIP).build(); + + final OperationControl resetControl = OperationControl.builder() + .onNullValue(BadDataBehavior.RESET) + .onNanValue(BadDataBehavior.RESET).build(); + + Duration prevTime = Duration.ofMinutes(10); + Duration postTime = Duration.ZERO; + + final Table summed = + t.updateBy(UpdateByOperation.RollingSum("ts", prevTime, postTime, "byteCol", "shortCol", "intCol", "longCol", "floatCol", + "doubleCol", "boolCol" +// TODO: put these back ,"bigIntCol", "bigDecimalCol" + )); + + + DateTime[] ts = (DateTime[])t.getColumn("ts").getDirect(); + long[] timestamps = new long[t.intSize()]; + for (int i = 0; i < t.intSize(); i++) { + timestamps[i] = ts[i].getNanos(); + } + + for (String col : t.getDefinition().getColumnNamesArray()) { + assertWithRollingSumTime(t.getColumn(col).getDirect(), summed.getColumn(col).getDirect(), timestamps, + summed.getColumn(col).getType(), prevTime.toNanos(), postTime.toNanos()); + } + } + // endregion // region Live Tests @@ -327,6 +371,51 @@ private long[] rollingSum(short[] values, int prevTicks, int postTicks) { return result; } + private long[] rollingSumTime(short[] values, long[] timestamps, long prevNanos, long postNanos) { + if (values == null) { + return null; + } + + if (values.length == 0) { + return new long[0]; + } + + long[] result = new long[values.length]; + + int head = 0; + int tail = 0; + + for (int i = 0; i < values.length; i++) { + result[i] = NULL_LONG; + + // set the head and the tail + final long headTime = timestamps[i] - prevNanos; + final long tailTime = timestamps[i] + postNanos; + + // advance head and tail until they are in the correct spots + while (head < values.length && timestamps[head] < headTime) { + head++; + } + + while (tail < values.length && timestamps[tail] < tailTime) { + tail++; + } + + // compute everything in this window + for (int computeIdx = head; computeIdx <= tail; computeIdx++) { + if (!isNull(values[computeIdx])) { + if (result[i] == NULL_LONG) { + result[i] = values[computeIdx]; + } else { + result[i] += values[computeIdx]; + } + } + } + } + + return result; + } + private long[] rollingSum(int[] values, int prevTicks, int postTicks) { if (values == null) { return null; @@ -540,8 +629,32 @@ final void assertWithRollingSumTicks(final @NotNull Object expected, final @NotN final float deltaF = .001f; final double deltaD = .001d; - if (expected instanceof short[]) { + if (expected instanceof byte[]) { + assertArrayEquals(rollingSum((byte[]) expected, prevTicks, postTicks), (long[]) actual); + } else if (expected instanceof short[]) { assertArrayEquals(rollingSum((short[]) expected, prevTicks, postTicks), (long[]) actual); + } else if (expected instanceof int[]) { + assertArrayEquals(rollingSum((int[]) expected, prevTicks, postTicks), (long[]) actual); + } else if (expected instanceof long[]) { + assertArrayEquals(rollingSum((long[]) expected, prevTicks, postTicks), (long[]) actual); + } else if (expected instanceof float[]) { + assertArrayEquals(rollingSum((float[]) expected, prevTicks, postTicks), (float[]) actual, deltaF); + } else if (expected instanceof double[]) { + assertArrayEquals(rollingSum((double[]) expected, prevTicks, postTicks), (double[]) actual, deltaD); + } else if (expected instanceof Boolean[]) { + assertArrayEquals(rollingSum((Boolean[]) expected, prevTicks, postTicks), (long[]) actual); + } else { +// assertArrayEquals(rollingSum((Object[]) expected, type == BigDecimal.class, prevTicks, postTicks), (Object[]) actual); + } + } + + final void assertWithRollingSumTime(final @NotNull Object expected, final @NotNull Object actual, + final @NotNull long[] timestamps, Class type, long prevTime, long postTime) { + final float deltaF = .001f; + final double deltaD = .001d; + + if (expected instanceof short[]) { + assertArrayEquals(rollingSumTime((short[]) expected, timestamps, prevTime, postTime), (long[]) actual); } // if (expected instanceof byte[]) { // assertArrayEquals(rollingSum((byte[]) expected, prevTicks, postTicks), (long[]) actual); diff --git a/table-api/src/main/java/io/deephaven/api/updateby/UpdateByOperation.java b/table-api/src/main/java/io/deephaven/api/updateby/UpdateByOperation.java index 5441048abe7..40ca5b17e48 100644 --- a/table-api/src/main/java/io/deephaven/api/updateby/UpdateByOperation.java +++ b/table-api/src/main/java/io/deephaven/api/updateby/UpdateByOperation.java @@ -240,6 +240,19 @@ static UpdateByOperation RollingSum(long prevTimeTicks, long fwdTimeTicks, Strin return RollingSumSpec.ofTicks(prevTimeTicks, fwdTimeTicks).clause(pairs); } + /** + * Create an {@link RollingSumSpec rolling sum} for the supplied column name pairs, using time as + * the windowing unit. Uses the default OperationControl settings. + * + * @param prevWindowDuration the look-behind window size (in Duration) + * @param fwdWindowDuration the look-ahead window size (in Duration) + * @param pairs The input/output column name pairs + * @return The aggregation + */ + static UpdateByOperation RollingSum(String timestampCol, Duration prevWindowDuration, Duration fwdWindowDuration, String... pairs) { + return RollingSumSpec.ofTime(timestampCol, prevWindowDuration, fwdWindowDuration).clause(pairs); + } + T walk(Visitor visitor); interface Visitor { From 86ecfceb7dcb2e8f756dd72dfdccec65bf1feb58 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Mon, 12 Sep 2022 15:01:45 -0700 Subject: [PATCH 019/123] Tests passing for all operations, including RollingSum (time and tick windows) --- .../impl/UpdateByCumulativeOperator.java | 4 + .../engine/table/impl/UpdateByOperator.java | 7 +- .../table/impl/UpdateByOperatorFactory.java | 26 +- .../table/impl/UpdateByWindowedOperator.java | 263 ++---- .../engine/table/impl/ZeroKeyUpdateBy.java | 9 +- .../BigDecimalRollingSumOperator.java | 201 +++++ .../BigIntegerRollingSumOperator.java | 198 +++++ .../rollingsum/ByteRollingSumOperator.java | 55 +- .../rollingsum/DoubleRollingSumOperator.java | 94 ++- .../rollingsum/FloatRollingSumOperator.java | 94 ++- .../rollingsum/IntRollingSumOperator.java | 55 +- .../rollingsum/LongRollingSumOperator.java | 55 +- .../rollingsum/ShortRollingSumOperator.java | 17 +- .../table/impl/updateby/TestRollingSum.java | 758 +++++++++++++++--- .../api/updateby/spec/RollingSumSpec.java | 4 +- 15 files changed, 1380 insertions(+), 460 deletions(-) create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java index ce042b6a3df..a7becf8f7da 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java @@ -116,6 +116,10 @@ public RowSet getAffectedRows() { return affectedRows; } + public RowSet getInfluencerRows() { + return affectedRows; + } + public RowSetBuilderSequential getModifiedBuilder() { if(modifiedBuilder == null) { modifiedBuilder = RowSetFactory.builderSequential(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java index 39b41b9fb95..320dabd1912 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java @@ -104,10 +104,15 @@ RowSet determineAffectedRows(@NotNull final TableUpdate upstream, @NotNull final final boolean initialStep); /** - * Return the rows computed by the {@Code determineAffectedRows()} + * Return the affected rows computed by the {@Code determineAffectedRows()} */ RowSet getAffectedRows(); + /** + * Return the influencer rows computed by the {@Code determineAffectedRows()} + */ + RowSet getInfluencerRows(); + LongSegmentedSortedArray getTimestampSsa(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java index 110eef9a830..21518522a97 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java @@ -403,31 +403,31 @@ private UpdateByOperator makeRollingSumOperator(@NotNull final MatchPair pair, if (csType == Boolean.class || csType == boolean.class) { return new ByteRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), timestampColumnSource, prevTimeScaleUnits, fwdTimeScaleUnits, - ReinterpretUtils.booleanToByteSource(columnSource), redirContext, NULL_BOOLEAN_AS_BYTE); + redirContext, ReinterpretUtils.booleanToByteSource(columnSource), NULL_BOOLEAN_AS_BYTE); } else if (csType == byte.class || csType == Byte.class) { return new ByteRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), timestampColumnSource, prevTimeScaleUnits, fwdTimeScaleUnits, - columnSource, redirContext, NULL_BYTE); + redirContext, columnSource, NULL_BYTE); } else if (csType == short.class || csType == Short.class) { return new ShortRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), timestampColumnSource, prevTimeScaleUnits, fwdTimeScaleUnits, - columnSource, redirContext); + redirContext, columnSource); } else if (csType == int.class || csType == Integer.class) { return new IntRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), timestampColumnSource, prevTimeScaleUnits, fwdTimeScaleUnits, - columnSource, redirContext); + redirContext, columnSource); } else if (csType == long.class || csType == Long.class) { return new LongRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), timestampColumnSource, prevTimeScaleUnits, fwdTimeScaleUnits, - columnSource, redirContext); + redirContext, columnSource); } else if (csType == float.class || csType == Float.class) { return new FloatRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), timestampColumnSource, prevTimeScaleUnits, fwdTimeScaleUnits, - columnSource, redirContext); + redirContext, columnSource); } else if (csType == double.class || csType == Double.class) { return new DoubleRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), timestampColumnSource, - prevTimeScaleUnits, fwdTimeScaleUnits, columnSource, redirContext); -// } else if (csType == BigDecimal.class) { -// return new BigDecimalRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, -// prevTimeScaleUnits, fwdTimeScaleUnits, columnSource, redirContext, control.mathContextOrDefault()); -// } else if (csType == BigInteger.class) { -// return new BigIntegerRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), recorder, -// prevTimeScaleUnits, fwdTimeScaleUnits, columnSource, redirContext); + prevTimeScaleUnits, fwdTimeScaleUnits, redirContext, columnSource); + } else if (csType == BigDecimal.class) { + return new BigDecimalRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), timestampColumnSource, + prevTimeScaleUnits, fwdTimeScaleUnits, redirContext, columnSource, control.mathContextOrDefault()); + } else if (csType == BigInteger.class) { + return new BigIntegerRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), timestampColumnSource, + prevTimeScaleUnits, fwdTimeScaleUnits, redirContext, columnSource); } throw new IllegalArgumentException("Can not perform RollingSum on type " + csType); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java index 434c222a5df..4500964e7c4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java @@ -4,6 +4,8 @@ import io.deephaven.base.LongRingBuffer; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.WritableChunk; +import io.deephaven.chunk.WritableLongChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.sized.SizedLongChunk; import io.deephaven.engine.rowset.*; @@ -39,6 +41,8 @@ public abstract class UpdateWindowedContext implements UpdateContext { protected RowSetBuilderSequential modifiedBuilder; protected RowSet newModified; + public int nullCount = 0; + public LongSegmentedSortedArray getTimestampSsa() { return timestampSsa; } @@ -64,10 +68,10 @@ public RowSetBuilderSequential getModifiedBuilder() { // candidate data for the window public final int WINDOW_CHUNK_SIZE = 4096; - // persist two iterators, for keys and positions + // allocate some chunks for holding the key, position and timestamp data protected SizedLongChunk influencerKeyChunk; protected SizedLongChunk influencerPosChunk; - protected LongSegmentedSortedArray.Iterator ssaIterator; + protected SizedLongChunk influencerTimestampChunk; // for use with a ticking window protected RowSet affectedRowPositions; @@ -77,73 +81,63 @@ public RowSetBuilderSequential getModifiedBuilder() { protected LongRingBuffer windowKeys = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); protected LongRingBuffer windowPosOrTimestamp = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); - protected LongRingBuffer windowIndices = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); + public LongRingBuffer windowIndices = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); private WritableRowSet computeAffectedRowsTime(final RowSet sourceSet, final RowSet subset, long revNanos, long fwdNanos) { - // swap fwd/rev to get the influencer windows - return computeInfluencerRowsTicks(sourceSet, subset, fwdNanos, revNanos); + // swap fwd/rev to get the affected windows + return computeInfluencerRowsTime(sourceSet, subset, fwdNanos, revNanos); } - private WritableRowSet computeInfluencerRowsTime(final RowSet sourceSet, final RowSet subset, long revTicks, long fwdTicks) { + private WritableRowSet computeInfluencerRowsTime(final RowSet sourceSet, final RowSet subset, long revNanos, long fwdNanos) { if (sourceSet.size() == subset.size()) { return sourceSet.copy(); } int chunkSize = (int) Math.min(subset.size(), 4096); - try (ChunkSource.GetContext getContext = timestampColumnSource.makeGetContext(chunkSize); - RowSequence.Iterator chunkIt = subset.getRowSequenceIterator()) { + try (final RowSequence.Iterator it = subset.getRowSequenceIterator(); + final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(chunkSize)) { final RowSetBuilderSequential builder = RowSetFactory.builderSequential(); - final LongSegmentedSortedArray.Iterator ssaHeadIt = timestampSsa.iterator(false, false); - final LongSegmentedSortedArray.Iterator ssaTailIt = timestampSsa.iterator(false, false); - while (chunkIt.hasMore()) { - RowSequence chunkRs = chunkIt.getNextRowSequenceWithLength(chunkSize); - LongChunk tsChunk = timestampColumnSource.getPrevChunk(getContext, chunkRs).asLongChunk(); - for (int i = 0; i < tsChunk.size(); i++) { - long ts = tsChunk.get(i); - // if this timestamp was null, it wasn't included in any windows and there is nothing to recompute + LongSegmentedSortedArray.Iterator ssaIt = timestampSsa.iterator(false, false); + while (it.hasMore() && ssaIt.hasNext()) { + final RowSequence rs = it.getNextRowSequenceWithLength(chunkSize); + LongChunk timestamps = timestampColumnSource.getChunk(context, rs).asLongChunk(); + + for (int ii = 0; ii < rs.intSize(); ii++) { + // if the timestamp of the row is null, it won't belong to any set and we can ignore it completely + final long ts = timestamps.get(ii); if (ts != NULL_LONG) { - ssaHeadIt.advanceToBeforeFirst(ts); - final long s = ssaHeadIt.getKey(); - ssaTailIt.advanceToLast(ts); - final long e; - if (ssaHeadIt.hasNext()) { - e = ssaTailIt.nextKey(); - } else { - e = NULL_LONG; + // look at every row timestamp, compute the head and tail in nanos + final long head = ts - revNanos; + final long tail = ts + fwdNanos; + + // advance the iterator to the beginning of the window + if (ssaIt.nextValue() < head) { + ssaIt.advanceToBeforeFirst(head); + if (!ssaIt.hasNext()) { + // SSA is exhausted + break; + } } - builder.appendRange(s, e); - } - } - } - try (final RowSet removedChanges = builder.build()) { - // changed.insert(removedChanges); - } - } - + Assert.assertion(ssaIt.hasNext() && ssaIt.nextValue() >= head, "SSA Iterator outside of window"); - long maxPos = sourceSet.size() - 1; - - // find the first row - - try (final RowSet inverted = sourceSet.invert(subset)) { - final RowSetBuilderSequential builder = RowSetFactory.builderSequential(); - final MutableLong minPos = new MutableLong(0L); - - inverted.forAllRowKeyRanges((s, e) -> { - long sPos = Math.max(s - revTicks, minPos.longValue()); - long ePos = Math.min(e + fwdTicks, maxPos); - builder.appendRange(sPos, ePos); - minPos.setValue(ePos + 1); - }); + // step through the SSA and collect keys until outside of the window + while (ssaIt.hasNext() && ssaIt.nextValue() <= tail) { + builder.appendKey(ssaIt.nextKey()); + ssaIt.next(); + } - try (final RowSet positions = builder.build()) { - return sourceSet.subSetForPositions(positions); + if (!ssaIt.hasNext()) { + // SSA is exhausted + break; + } + } + } } + return builder.build(); } } - private WritableRowSet computeAffectedRowsTicks(final RowSet sourceSet, final RowSet subset, long revTicks, long fwdTicks) { // swap fwd/rev to get the influencer windows return computeInfluencerRowsTicks(sourceSet, subset, fwdTicks, revTicks); @@ -225,61 +219,29 @@ public RowSet determineAffectedRows(@NotNull final TableUpdate upstream, @NotNul // add affected rows from any removes if (upstream.removed().isNonempty()) { - if (timestampColumnName == null) { - // tick based - try (final RowSet prev = source.copyPrev(); - final WritableRowSet affectedByRemoves = computeAffectedRowsTicks(prev, upstream.removed(), reverseTimeScaleUnits, forwardTimeScaleUnits )) { - // apply shifts to get back to pos-shift space - upstream.shifted().apply(affectedByRemoves); - // retain only the rows that still exist in the source - affectedByRemoves.retain(source); - tmpAffected.insert(affectedByRemoves); - } - } else { - // time-based, first grab all the timestamp data for these removed rows - int size = (int) Math.min(upstream.removed().size(), 4096); - try (ChunkSource.GetContext getContext = timestampColumnSource.makeGetContext(size); - RowSequence.Iterator chunkIt = upstream.removed().getRowSequenceIterator()) { - final RowSetBuilderSequential builder = RowSetFactory.builderSequential(); - final LongSegmentedSortedArray.Iterator ssaHeadIt = timestampSsa.iterator(false, false); - final LongSegmentedSortedArray.Iterator ssaTailIt = timestampSsa.iterator(false, false); - while (chunkIt.hasMore()) { - RowSequence chunkRs = chunkIt.getNextRowSequenceWithLength(size); - LongChunk tsChunk = timestampColumnSource.getPrevChunk(getContext, chunkRs).asLongChunk(); - for (int i = 0; i < tsChunk.size(); i++) { - long ts = tsChunk.get(i); - // if this timestamp was null, it wasn't included in any windows and there is nothing to recompute - if (ts != NULL_LONG) { - ssaHeadIt.advanceToBeforeFirst(ts); - final long s = ssaHeadIt.getKey(); - ssaTailIt.advanceToLast(ts); - final long e; - if (ssaHeadIt.hasNext()) { - e = ssaTailIt.nextKey(); - } else { - e = NULL_LONG; - } - builder.appendRange(s, e); - } - } - } - try (final RowSet removedChanges = builder.build()) { - changed.insert(removedChanges); - } - } + try (final RowSet prev = source.copyPrev(); + final WritableRowSet affectedByRemoves = timestampColumnName == null + ? computeAffectedRowsTicks(prev, upstream.removed(), reverseTimeScaleUnits, forwardTimeScaleUnits) + : computeAffectedRowsTime(prev, upstream.removed(), reverseTimeScaleUnits, forwardTimeScaleUnits)) { + // apply shifts to get back to pos-shift space + upstream.shifted().apply(affectedByRemoves); + // retain only the rows that still exist in the source + affectedByRemoves.retain(source); + tmpAffected.insert(affectedByRemoves); } } affectedRows = tmpAffected; // now get influencer rows for the affected + if (timestampColumnName == null) { influencerRows = computeInfluencerRowsTicks(source, affectedRows, reverseTimeScaleUnits, forwardTimeScaleUnits); // generate position data rowsets for efficiently computed position offsets affectedRowPositions = source.invert(affectedRows); influencerPositions = source.invert(influencerRows); } else { - + influencerRows = computeInfluencerRowsTime(source, affectedRows, reverseTimeScaleUnits, forwardTimeScaleUnits); } return affectedRows; } @@ -288,9 +250,13 @@ public RowSet getAffectedRows() { return affectedRows; } + public RowSet getInfluencerRows() { + return influencerRows; + } + public abstract void loadInfluencerValueChunk(); - public void fillWindowTicks(UpdateContext context, long currentPos) { + public void fillWindowTicks(UpdateWindowedContext context, long currentPos) { // compute the head and tail positions (inclusive) final long head = Math.max(0, currentPos - reverseTimeScaleUnits + 1); final long tail = Math.min(sourceRowSet.size() - 1, currentPos + forwardTimeScaleUnits); @@ -301,7 +267,8 @@ public void fillWindowTicks(UpdateContext context, long currentPos) { windowPosOrTimestamp.remove(); } - if (windowPosOrTimestamp.isEmpty()) { + // if the window is empty or completly filled with null, call reset() + if (windowPosOrTimestamp.isEmpty() || context.nullCount == windowPosOrTimestamp.size()) { reset(context); } @@ -336,7 +303,7 @@ public void fillWindowTicks(UpdateContext context, long currentPos) { } } - public void fillWindowTime(UpdateContext context, long currentTimestamp) { + public void fillWindowTime(UpdateWindowedContext context, long currentTimestamp) { // compute the head and tail positions (inclusive) final long head = currentTimestamp - reverseTimeScaleUnits; final long tail = currentTimestamp + forwardTimeScaleUnits; @@ -347,7 +314,8 @@ public void fillWindowTime(UpdateContext context, long currentTimestamp) { windowPosOrTimestamp.remove(); } - if (windowPosOrTimestamp.isEmpty()) { + // if the window is empty or completly filled with null, call reset() + if (windowPosOrTimestamp.isEmpty() || context.nullCount == windowPosOrTimestamp.size()) { reset(context); } @@ -355,10 +323,9 @@ public void fillWindowTime(UpdateContext context, long currentTimestamp) { while(currentInfluencerPosOrTimestamp < head) { currentInfluencerIndex++; - if (ssaIterator.hasNext()) { - ssaIterator.next(); - currentInfluencerPosOrTimestamp = ssaIterator.getValue(); - currentInfluencerKey = ssaIterator.getKey(); + if (currentInfluencerIndex < influencerTimestampChunk.get().size()) { + currentInfluencerPosOrTimestamp = influencerTimestampChunk.get().get(currentInfluencerIndex); + currentInfluencerKey = influencerKeyChunk.get().get(currentInfluencerIndex); } else { currentInfluencerPosOrTimestamp = Long.MAX_VALUE; currentInfluencerKey = Long.MAX_VALUE; @@ -373,10 +340,9 @@ public void fillWindowTime(UpdateContext context, long currentTimestamp) { windowIndices.add(currentInfluencerIndex); currentInfluencerIndex++; - if (ssaIterator.hasNext()) { - ssaIterator.next(); - currentInfluencerPosOrTimestamp = ssaIterator.getValue(); - currentInfluencerKey = ssaIterator.getKey(); + if (currentInfluencerIndex < influencerTimestampChunk.get().size()) { + currentInfluencerPosOrTimestamp = influencerTimestampChunk.get().get(currentInfluencerIndex); + currentInfluencerKey = influencerKeyChunk.get().get(currentInfluencerIndex); } else { currentInfluencerPosOrTimestamp = Long.MAX_VALUE; currentInfluencerKey = Long.MAX_VALUE; @@ -388,6 +354,7 @@ public void fillWindowTime(UpdateContext context, long currentTimestamp) { public void close() { try (final SizedLongChunk ignoredChk1 = influencerKeyChunk; final SizedLongChunk ignoredChk2 = influencerPosChunk; + final SizedLongChunk ignoredChk3 = influencerTimestampChunk; final RowSet ignoredRs1 = affectedRows; final RowSet ignoredRs2 = influencerRows; final RowSet ignoredRs3 = affectedRowPositions; @@ -428,80 +395,6 @@ public UpdateByWindowedOperator(@NotNull final MatchPair pair, public abstract void pop(UpdateContext context, long key, int pos); public abstract void reset(UpdateContext context); - // return the first row that affects this key - public long computeFirstAffectingKey(long key, @NotNull final RowSet source, final LongSegmentedSortedArray timestampSsa) { - - if (timestampColumnName == null) { - // ticks - final long keyPos = source.find(key); - final long idx = (keyPos < 0) ? -keyPos - reverseTimeScaleUnits : keyPos - reverseTimeScaleUnits + 1; - if (idx >= source.size()) { - return source.lastRowKey(); - } else if (idx < 0) { - return source.firstRowKey(); - } - return source.get(idx); - } else { - - } - return -1; - } - - // return the last row that affects this key - public long computeLastAffectingKey(long key, @NotNull final RowSet source, final LongSegmentedSortedArray timestampSsa) { - if (timestampColumnName == null) { - // ticks - final long keyPos = source.find(key); - final long idx = keyPos + forwardTimeScaleUnits; - if (idx >= source.size()) { - return source.lastRowKey(); - } else if (idx < 0) { - return source.firstRowKey(); - } - return source.get(idx); - } else { - - } - return -1; - } - - // return the first row affected by this key - public long computeFirstAffectedKey(long key, @NotNull final RowSet source, final LongSegmentedSortedArray timestampSsa) { - if (timestampColumnName == null) { - // ticks - final long keyPos = source.find(key); - final long idx = (keyPos < 0) ? -keyPos - forwardTimeScaleUnits - 1 : keyPos - forwardTimeScaleUnits; - if (idx >= source.size()) { - return source.lastRowKey(); - } else if (idx < 0) { - return source.firstRowKey(); - } - return source.get(idx); - } else { - // time-based, use the SSA -// long ts = this.time - } - return -1; - } - - // return the last row affected by this key - public long computeLastAffectedKey(long key, @NotNull final RowSet source, final LongSegmentedSortedArray timestampSsa) { - if (timestampColumnName == null) { - // ticks - final long keyPos = source.find(key); - final long idx = (keyPos < 0) ? -keyPos + reverseTimeScaleUnits - 1 : keyPos + reverseTimeScaleUnits; - if (idx >= source.size()) { - return source.lastRowKey(); - } else if (idx < 0) { - return source.firstRowKey(); - } - return source.get(idx); - } else { - - } - return -1; - } - @Override public void initializeFor(@NotNull final UpdateContext context, @NotNull final RowSet updateRowSet) { @@ -511,18 +404,21 @@ public void initializeFor(@NotNull final UpdateContext context, ctx.loadInfluencerValueChunk(); // load all the influencer keys - ctx.influencerKeyChunk = new SizedLongChunk(ctx.influencerRows.intSize()); + ctx.influencerKeyChunk = new SizedLongChunk<>(ctx.influencerRows.intSize()); ctx.influencerRows.fillRowKeyChunk(ctx.influencerKeyChunk.get()); ctx.currentInfluencerKey = ctx.influencerRows.firstRowKey(); if (timestampColumnName == null) { // load all the influencer positions - ctx.influencerPosChunk = new SizedLongChunk(ctx.influencerRows.intSize()); + ctx.influencerPosChunk = new SizedLongChunk<>(ctx.influencerRows.intSize()); ctx.influencerPositions.fillRowKeyChunk(ctx.influencerPosChunk.get()); ctx.currentInfluencerPosOrTimestamp = ctx.influencerPositions.firstRowKey(); } else { - ctx.ssaIterator = ctx.timestampSsa.iterator(false, false); - ctx.currentInfluencerPosOrTimestamp = ctx.ssaIterator.nextValue(); + ctx.influencerTimestampChunk = new SizedLongChunk<>(ctx.influencerRows.intSize()); + try (final ChunkSource.FillContext fillContext = timestampColumnSource.makeFillContext(ctx.influencerRows.intSize())) { + timestampColumnSource.fillChunk(fillContext, (WritableChunk) ctx.influencerTimestampChunk.get(), ctx.influencerRows); + } + ctx.currentInfluencerPosOrTimestamp = ctx.influencerTimestampChunk.get().get(0); } ctx.currentInfluencerIndex = 0; } @@ -579,7 +475,6 @@ public boolean requiresPositions() { return this.timestampColumnName == null; } - @Override public boolean requiresValues(@NotNull final UpdateContext context) { // windowed operators don't need current values supplied to them, they only care about windowed values which diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java index 26968931a31..5f4f85b2ffc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java @@ -632,12 +632,13 @@ public ZeroKeyUpdateByListener(@Nullable String description, @Override public void onUpdate(TableUpdate upstream) { + // update the Ssa + if (timestampColumnName != null) { + processUpdateForSsa(upstream); + } try (final UpdateContext ctx = new UpdateContext(upstream, inputModifiedColumnSets, false)) { - // update the Ssa - if (timestampColumnName != null) { - processUpdateForSsa(upstream); - } + if (applyShifts) { if (redirContext.isRedirected()) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java new file mode 100644 index 00000000000..4e008a1372a --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java @@ -0,0 +1,201 @@ +package io.deephaven.engine.table.impl.updateby.rollingsum; + +import io.deephaven.api.updateby.OperationControl; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.WritableObjectChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.chunk.sized.SizedObjectChunk; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.table.*; +import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.UpdateByOperator; +import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; +import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedObjectUpdateByOperator; +import io.deephaven.engine.table.impl.util.SizedSafeCloseable; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.math.BigDecimal; +import java.math.MathContext; +import java.util.Collections; +import java.util.Map; + +public final class BigDecimalRollingSumOperator extends BaseWindowedObjectUpdateByOperator { + @NotNull + private final MathContext mathContext; + + private final WritableColumnSource outputSource; + private final WritableColumnSource maybeInnerSource; + + protected class Context extends BaseWindowedObjectUpdateByOperator.Context { + public final SizedSafeCloseable fillContext; + public final SizedObjectChunk outputValues; + + public BigDecimal currentVal = null; + + protected Context(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { + this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); + this.fillContext.ensureCapacity(chunkSize); + this.outputValues = new SizedObjectChunk<>(chunkSize); + this.timestampSsa = timestampSsa; + } + + @Override + public void close() { + super.close(); + outputValues.close(); + fillContext.close(); + } + } + + @NotNull + @Override + public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { + return new Context(chunkSize, timestampSsa); + } + + @Override + public void setChunkSize(@NotNull UpdateContext context, int chunkSize) { + final Context ctx = (Context) context; + ctx.outputValues.ensureCapacity(chunkSize); + ctx.fillContext.ensureCapacity(chunkSize); + } + + + public BigDecimalRollingSumOperator(@NotNull final MatchPair pair, + @NotNull final String[] affectingColumns, + @NotNull final OperationControl control, + @Nullable final String timestampColumnName, + @Nullable final ColumnSource timestampColumnSource, + final long reverseTimeScaleUnits, + final long forwardTimeScaleUnits, + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, + @NotNull final ColumnSource valueSource + // region extra-constructor-args + // endregion extra-constructor-args + , @NotNull final MathContext mathContext) { + super(pair, affectingColumns, control, timestampColumnName, timestampColumnSource, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource, BigDecimal.class); + this.mathContext = mathContext; + if(redirContext.isRedirected()) { + // region create-dense + this.maybeInnerSource = new ObjectArraySource(BigDecimal.class); + // endregion create-dense + this.outputSource = new WritableRedirectedColumnSource(redirContext.getRowRedirection(), maybeInnerSource, 0); + } else { + this.maybeInnerSource = null; + // region create-sparse + this.outputSource = new ObjectSparseArraySource(BigDecimal.class); + // endregion create-sparse + } + + // region constructor + // endregion constructor + } + + @Override + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + BigDecimal val = (BigDecimal)ctx.candidateValuesChunk.get(pos); + + // increase the running sum + if (val != null) { + if (ctx.currentVal == null) { + ctx.currentVal = val; + } else { + ctx.currentVal = ctx.currentVal.add(val, mathContext); + } + } else { + ctx.nullCount++; + } + } + + @Override + public void pop(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + BigDecimal val = (BigDecimal)ctx.candidateValuesChunk.get(pos); + + // reduce the running sum + if (val != null) { + ctx.currentVal = ctx.currentVal.subtract(val, mathContext); + } else { + ctx.nullCount--; + } + } + + @Override + public void reset(UpdateContext context) { + final Context ctx = (Context) context; + ctx.currentVal = null; + } + + @Override + public void doProcessChunk(@NotNull final BaseWindowedObjectUpdateByOperator.Context context, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @Nullable final LongChunk posChunk, + @NotNull final Chunk workingChunk) { + final Context ctx = (Context) context; + + if (timestampColumnName == null) { + computeTicks(ctx, posChunk, inputKeys.intSize()); + } else { + computeTime(ctx, inputKeys); + } + + //noinspection unchecked + outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); + } + + private void computeTicks(@NotNull final Context ctx, + @Nullable final LongChunk posChunk, + final int runLength) { + + final WritableObjectChunk localOutputValues = ctx.outputValues.get(); + for (int ii = 0; ii < runLength; ii++) { + // the output value is computed by push/pop operations triggered by fillWindow + ctx.fillWindowTicks(ctx, posChunk.get(ii)); + localOutputValues.set(ii, ctx.currentVal); + } + } + + private void computeTime(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys) { + + final WritableObjectChunk localOutputValues = ctx.outputValues.get(); + // get the timestamp values for this chunk + try (final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(inputKeys.intSize())) { + LongChunk timestampChunk = timestampColumnSource.getChunk(context, inputKeys).asLongChunk(); + + for (int ii = 0; ii < inputKeys.intSize(); ii++) { + // the output value is computed by push/pop operations triggered by fillWindow + ctx.fillWindowTime(ctx, timestampChunk.get(ii)); + localOutputValues.set(ii, ctx.currentVal); + } + } + } + + @NotNull + @Override + public Map> getOutputColumns() { + return Collections.singletonMap(pair.leftColumn, outputSource); + } + + @Override + public void startTrackingPrev() { + outputSource.startTrackingPrevValues(); + if(redirContext.isRedirected()) { + maybeInnerSource.startTrackingPrevValues(); + } + } + + @Override + public void applyOutputShift(@NotNull final UpdateContext context, + @NotNull final RowSet subIndexToShift, + final long delta) { + ((ObjectSparseArraySource)outputSource).shift(subIndexToShift, delta); + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java new file mode 100644 index 00000000000..cd631d52863 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java @@ -0,0 +1,198 @@ +package io.deephaven.engine.table.impl.updateby.rollingsum; + +import io.deephaven.api.updateby.OperationControl; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.WritableObjectChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.chunk.sized.SizedObjectChunk; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.table.*; +import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.UpdateByOperator; +import io.deephaven.engine.table.impl.sources.ObjectArraySource; +import io.deephaven.engine.table.impl.sources.ObjectSparseArraySource; +import io.deephaven.engine.table.impl.sources.WritableRedirectedColumnSource; +import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; +import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedObjectUpdateByOperator; +import io.deephaven.engine.table.impl.util.SizedSafeCloseable; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.math.BigInteger; +import java.util.Collections; +import java.util.Map; + +public final class BigIntegerRollingSumOperator extends BaseWindowedObjectUpdateByOperator { + private final WritableColumnSource outputSource; + private final WritableColumnSource maybeInnerSource; + + protected class Context extends BaseWindowedObjectUpdateByOperator.Context { + public final SizedSafeCloseable fillContext; + public final SizedObjectChunk outputValues; + + public BigInteger currentVal = null; + + protected Context(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { + this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); + this.fillContext.ensureCapacity(chunkSize); + this.outputValues = new SizedObjectChunk<>(chunkSize); + this.timestampSsa = timestampSsa; + } + + @Override + public void close() { + super.close(); + outputValues.close(); + fillContext.close(); + } + } + + @NotNull + @Override + public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { + return new Context(chunkSize, timestampSsa); + } + + @Override + public void setChunkSize(@NotNull UpdateContext context, int chunkSize) { + final Context ctx = (Context) context; + ctx.outputValues.ensureCapacity(chunkSize); + ctx.fillContext.ensureCapacity(chunkSize); + } + + + public BigIntegerRollingSumOperator(@NotNull final MatchPair pair, + @NotNull final String[] affectingColumns, + @NotNull final OperationControl control, + @Nullable final String timestampColumnName, + @Nullable final ColumnSource timestampColumnSource, + final long reverseTimeScaleUnits, + final long forwardTimeScaleUnits, + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, + @NotNull final ColumnSource valueSource + // region extra-constructor-args + // endregion extra-constructor-args + ) { + super(pair, affectingColumns, control, timestampColumnName, timestampColumnSource, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource, BigInteger.class); + if(redirContext.isRedirected()) { + // region create-dense + this.maybeInnerSource = new ObjectArraySource(BigInteger.class); + // endregion create-dense + this.outputSource = new WritableRedirectedColumnSource(redirContext.getRowRedirection(), maybeInnerSource, 0); + } else { + this.maybeInnerSource = null; + // region create-sparse + this.outputSource = new ObjectSparseArraySource(BigInteger.class); + // endregion create-sparse + } + + // region constructor + // endregion constructor + } + + @Override + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + BigInteger val = ctx.candidateValuesChunk.get(pos); + + // increase the running sum + if (val != null) { + if (ctx.currentVal == null) { + ctx.currentVal = val; + } else { + ctx.currentVal = ctx.currentVal.add(val); + } + } else { + ctx.nullCount++; + } + } + + @Override + public void pop(UpdateContext context, long key, int pos) { + final BigIntegerRollingSumOperator.Context ctx = (BigIntegerRollingSumOperator.Context) context; + BigInteger val = ctx.candidateValuesChunk.get(pos); + + // reduce the running sum + if (val != null) { + ctx.currentVal = ctx.currentVal.subtract(val); + } else { + ctx.nullCount--; + } + } + + @Override + public void reset(UpdateContext context) { + final BigIntegerRollingSumOperator.Context ctx = (BigIntegerRollingSumOperator.Context) context; + ctx.currentVal = null; + } + + @Override + public void doProcessChunk(@NotNull final BaseWindowedObjectUpdateByOperator.Context context, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @Nullable final LongChunk posChunk, + @NotNull final Chunk workingChunk) { + final BigIntegerRollingSumOperator.Context ctx = (BigIntegerRollingSumOperator.Context) context; + + if (timestampColumnName == null) { + computeTicks(ctx, posChunk, inputKeys.intSize()); + } else { + computeTime(ctx, inputKeys); + } + + //noinspection unchecked + outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); + } + + private void computeTicks(@NotNull final BigIntegerRollingSumOperator.Context ctx, + @Nullable final LongChunk posChunk, + final int runLength) { + + final WritableObjectChunk localOutputValues = ctx.outputValues.get(); + for (int ii = 0; ii < runLength; ii++) { + // the output value is computed by push/pop operations triggered by fillWindow + ctx.fillWindowTicks(ctx, posChunk.get(ii)); + localOutputValues.set(ii, ctx.currentVal); + } + } + + private void computeTime(@NotNull final BigIntegerRollingSumOperator.Context ctx, + @NotNull final RowSequence inputKeys) { + + final WritableObjectChunk localOutputValues = ctx.outputValues.get(); + // get the timestamp values for this chunk + try (final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(inputKeys.intSize())) { + LongChunk timestampChunk = timestampColumnSource.getChunk(context, inputKeys).asLongChunk(); + + for (int ii = 0; ii < inputKeys.intSize(); ii++) { + // the output value is computed by push/pop operations triggered by fillWindow + ctx.fillWindowTime(ctx, timestampChunk.get(ii)); + localOutputValues.set(ii, ctx.currentVal); + } + } + } + + @NotNull + @Override + public Map> getOutputColumns() { + return Collections.singletonMap(pair.leftColumn, outputSource); + } + + @Override + public void startTrackingPrev() { + outputSource.startTrackingPrevValues(); + if(redirContext.isRedirected()) { + maybeInnerSource.startTrackingPrevValues(); + } + } + + @Override + public void applyOutputShift(@NotNull final UpdateContext context, + @NotNull final RowSet subIndexToShift, + final long delta) { + ((ObjectSparseArraySource)outputSource).shift(subIndexToShift, delta); + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java index e91b3337857..219a70d4e0e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java @@ -44,14 +44,10 @@ protected class Context extends BaseWindowedByteUpdateByOperator.Context { public long currentVal = NULL_LONG; - // position data for the chunk being currently processed - public SizedLongChunk valuePositionChunk; - protected Context(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); this.fillContext.ensureCapacity(chunkSize); this.outputValues = new SizedLongChunk<>(chunkSize); - this.valuePositionChunk = new SizedLongChunk<>(chunkSize); this.timestampSsa = timestampSsa; } @@ -60,7 +56,6 @@ public void close() { super.close(); outputValues.close(); fillContext.close(); - this.valuePositionChunk.close(); } } @@ -75,7 +70,6 @@ public void setChunkSize(@NotNull UpdateContext context, int chunkSize) { final Context ctx = (Context) context; ctx.outputValues.ensureCapacity(chunkSize); ctx.fillContext.ensureCapacity(chunkSize); - ctx.valuePositionChunk.ensureCapacity(chunkSize); } public ByteRollingSumOperator(@NotNull final MatchPair pair, @@ -85,8 +79,8 @@ public ByteRollingSumOperator(@NotNull final MatchPair pair, @Nullable final ColumnSource timestampColumnSource, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final ColumnSource valueSource, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, + @NotNull final ColumnSource valueSource // region extra-constructor-args ,final byte nullValue // endregion extra-constructor-args @@ -112,7 +106,7 @@ public ByteRollingSumOperator(@NotNull final MatchPair pair, @Override public void push(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; - Byte val = ctx.candidateValuesChunk.get(pos); + byte val = ctx.candidateValuesChunk.get(pos); // increase the running sum if (val != NULL_BYTE) { @@ -121,17 +115,21 @@ public void push(UpdateContext context, long key, int pos) { } else { ctx.currentVal += val; } + } else { + ctx.nullCount++; } } @Override public void pop(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; - Byte val = ctx.candidateValuesChunk.get(pos); + byte val = ctx.candidateValuesChunk.get(pos); // reduce the running sum if (val != NULL_BYTE) { ctx.currentVal -= val; + } else { + ctx.nullCount--; } } @@ -150,34 +148,43 @@ public void doProcessChunk(@NotNull final BaseWindowedByteUpdateByOperator.Conte final Context ctx = (Context) context; if (timestampColumnName == null) { - // produce position data for the window (will be timestamps for time-based) - // TODO: gotta be a better way than creating two rowsets - try (final RowSet rs = inputKeys.asRowSet(); - final RowSet positions = ctx.sourceRowSet.invert(rs)) { - positions.fillRowKeyChunk(ctx.valuePositionChunk.get()); - } + computeTicks(ctx, posChunk, inputKeys.intSize()); + } else { + computeTime(ctx, inputKeys); } - computeTicks(ctx, 0, inputKeys.intSize()); - //noinspection unchecked outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } private void computeTicks(@NotNull final Context ctx, - final int runStart, + @Nullable final LongChunk posChunk, final int runLength) { final WritableLongChunk localOutputValues = ctx.outputValues.get(); - for (int ii = runStart; ii < runStart + runLength; ii++) { - if (timestampColumnName == null) { - ctx.fillWindowTicks(ctx, ctx.valuePositionChunk.get().get(ii)); - } - // the sum was computed by push/pop operations + for (int ii = 0; ii < runLength; ii++) { + // the output value is computed by push/pop operations triggered by fillWindow + ctx.fillWindowTicks(ctx, posChunk.get(ii)); localOutputValues.set(ii, ctx.currentVal); } } + private void computeTime(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys) { + + final WritableLongChunk localOutputValues = ctx.outputValues.get(); + // get the timestamp values for this chunk + try (final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(inputKeys.intSize())) { + LongChunk timestampChunk = timestampColumnSource.getChunk(context, inputKeys).asLongChunk(); + + for (int ii = 0; ii < inputKeys.intSize(); ii++) { + // the output value is computed by push/pop operations triggered by fillWindow + ctx.fillWindowTime(ctx, timestampChunk.get(ii)); + localOutputValues.set(ii, ctx.currentVal); + } + } + } + @NotNull @Override public Map> getOutputColumns() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java index 7621c193806..b5970ad374c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java @@ -6,9 +6,11 @@ package io.deephaven.engine.table.impl.updateby.rollingsum; import io.deephaven.api.updateby.OperationControl; +import io.deephaven.base.LongRingBuffer; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableDoubleChunk; +import io.deephaven.chunk.WritableLongChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.sized.SizedDoubleChunk; import io.deephaven.chunk.sized.SizedLongChunk; @@ -16,14 +18,12 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.rowset.chunkattributes.RowKeys; -import io.deephaven.engine.table.ChunkSink; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.WritableColumnSource; +import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedDoubleUpdateByOperator; +import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedShortUpdateByOperator; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; import io.deephaven.util.QueryConstants; import org.apache.commons.lang3.mutable.MutableDouble; @@ -35,6 +35,7 @@ import java.util.Map; import static io.deephaven.util.QueryConstants.NULL_DOUBLE; +import static io.deephaven.util.QueryConstants.NULL_LONG; public class DoubleRollingSumOperator extends BaseWindowedDoubleUpdateByOperator { @@ -49,8 +50,6 @@ protected class Context extends BaseWindowedDoubleUpdateByOperator.Context { public final SizedSafeCloseable fillContext; public final SizedDoubleChunk outputValues; - public LinkedList windowValues = new LinkedList<>(); - // position data for the chunk being currently processed public SizedLongChunk valuePositionChunk; @@ -92,8 +91,8 @@ public DoubleRollingSumOperator(@NotNull final MatchPair pair, @Nullable final ColumnSource timestampColumnSource, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final ColumnSource valueSource, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, + @NotNull final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args ) { @@ -118,14 +117,18 @@ public DoubleRollingSumOperator(@NotNull final MatchPair pair, public void push(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; double val = ctx.candidateValuesChunk.get(pos); - ctx.windowValues.addLast(val); + if (val == NULL_DOUBLE) { + ctx.nullCount++; + } } @Override public void pop(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; double val = ctx.candidateValuesChunk.get(pos); - ctx.windowValues.pop(); + if (val == NULL_DOUBLE) { + ctx.nullCount--; + } } @Override @@ -142,41 +145,74 @@ public void doProcessChunk(@NotNull final BaseWindowedDoubleUpdateByOperator.Con final Context ctx = (Context) context; if (timestampColumnName == null) { - // produce position data for the window (will be timestamps for time-based) - // TODO: gotta be a better way than creating two rowsets - try (final RowSet rs = inputKeys.asRowSet(); - final RowSet positions = ctx.sourceRowSet.invert(rs)) { - positions.fillRowKeyChunk(ctx.valuePositionChunk.get()); - } + computeTicks(ctx, posChunk, inputKeys.intSize()); + } else { + computeTime(ctx, inputKeys); } - computeTicks(ctx, 0, inputKeys.intSize()); //noinspection unchecked outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } private void computeTicks(@NotNull final Context ctx, - final int runStart, + @Nullable final LongChunk posChunk, final int runLength) { final WritableDoubleChunk localOutputValues = ctx.outputValues.get(); - for (int ii = runStart; ii < runStart + runLength; ii++) { - if (timestampColumnName == null) { - ctx.fillWindowTicks(ctx, ctx.valuePositionChunk.get().get(ii)); - } + for (int ii = 0; ii < runLength; ii++) { + ctx.fillWindowTicks(ctx, posChunk.get(ii)); - MutableDouble sum = new MutableDouble(NULL_DOUBLE); - ctx.windowValues.forEach(v-> { + double sum = NULL_DOUBLE; + + LongRingBuffer.Iterator it = ctx.windowIndices.iterator(); + while (it.hasNext()) { + double v = ctx.candidateValuesChunk.get((int)it.next()); if (v != QueryConstants.NULL_DOUBLE) { - if (sum.getValue() == NULL_DOUBLE) { - sum.setValue(v); + if (sum == NULL_DOUBLE) { + sum = v; } else { - sum.add(v); + sum += v; } } - }); + } + localOutputValues.set(ii, sum); + } + } - localOutputValues.set(ii, sum.getValue()); + private void computeTime(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys) { + + final WritableDoubleChunk localOutputValues = ctx.outputValues.get(); + // get the timestamp values for this chunk + try (final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(inputKeys.intSize())) { + LongChunk timestampChunk = timestampColumnSource.getChunk(context, inputKeys).asLongChunk(); + + for (int ii = 0; ii < inputKeys.intSize(); ii++) { + long ts = timestampChunk.get(ii); + + // does this value have a valid timestamp + if (ts == NULL_LONG) { + localOutputValues.set(ii, NULL_DOUBLE); + } else { + // the output value is computed by push/pop operations triggered by fillWindow + ctx.fillWindowTime(ctx, timestampChunk.get(ii)); + + double sum = NULL_DOUBLE; + + LongRingBuffer.Iterator it = ctx.windowIndices.iterator(); + while (it.hasNext()) { + double v = ctx.candidateValuesChunk.get((int)it.next()); + if (v != QueryConstants.NULL_DOUBLE) { + if (sum == NULL_DOUBLE) { + sum = v; + } else { + sum += v; + } + } + } + localOutputValues.set(ii, sum); + } + } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java index dfb760245ca..e36938a166c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java @@ -1,9 +1,11 @@ package io.deephaven.engine.table.impl.updateby.rollingsum; import io.deephaven.api.updateby.OperationControl; +import io.deephaven.base.LongRingBuffer; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableFloatChunk; +import io.deephaven.chunk.WritableLongChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.sized.SizedFloatChunk; import io.deephaven.chunk.sized.SizedLongChunk; @@ -11,14 +13,12 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.rowset.chunkattributes.RowKeys; -import io.deephaven.engine.table.ChunkSink; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.WritableColumnSource; +import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedFloatUpdateByOperator; +import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedShortUpdateByOperator; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; import io.deephaven.util.QueryConstants; import org.apache.commons.lang3.mutable.MutableFloat; @@ -30,6 +30,7 @@ import java.util.Map; import static io.deephaven.util.QueryConstants.NULL_FLOAT; +import static io.deephaven.util.QueryConstants.NULL_LONG; public class FloatRollingSumOperator extends BaseWindowedFloatUpdateByOperator { @@ -44,8 +45,6 @@ protected class Context extends BaseWindowedFloatUpdateByOperator.Context { public final SizedSafeCloseable fillContext; public final SizedFloatChunk outputValues; - public LinkedList windowValues = new LinkedList<>(); - // position data for the chunk being currently processed public SizedLongChunk valuePositionChunk; @@ -87,8 +86,8 @@ public FloatRollingSumOperator(@NotNull final MatchPair pair, @Nullable final ColumnSource timestampColumnSource, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final ColumnSource valueSource, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, + @NotNull final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args ) { @@ -113,14 +112,18 @@ public FloatRollingSumOperator(@NotNull final MatchPair pair, public void push(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; float val = ctx.candidateValuesChunk.get(pos); - ctx.windowValues.addLast(val); + if (val == NULL_FLOAT) { + ctx.nullCount++; + } } @Override public void pop(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; float val = ctx.candidateValuesChunk.get(pos); - ctx.windowValues.pop(); + if (val == NULL_FLOAT) { + ctx.nullCount--; + } } @Override @@ -137,41 +140,74 @@ public void doProcessChunk(@NotNull final BaseWindowedFloatUpdateByOperator.Cont final Context ctx = (Context) context; if (timestampColumnName == null) { - // produce position data for the window (will be timestamps for time-based) - // TODO: gotta be a better way than creating two rowsets - try (final RowSet rs = inputKeys.asRowSet(); - final RowSet positions = ctx.sourceRowSet.invert(rs)) { - positions.fillRowKeyChunk(ctx.valuePositionChunk.get()); - } + computeTicks(ctx, posChunk, inputKeys.intSize()); + } else { + computeTime(ctx, inputKeys); } - computeTicks(ctx, 0, inputKeys.intSize()); //noinspection unchecked outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } private void computeTicks(@NotNull final Context ctx, - final int runStart, + @Nullable final LongChunk posChunk, final int runLength) { final WritableFloatChunk localOutputValues = ctx.outputValues.get(); - for (int ii = runStart; ii < runStart + runLength; ii++) { - if (timestampColumnName == null) { - ctx.fillWindowTicks(ctx, ctx.valuePositionChunk.get().get(ii)); - } + for (int ii = 0; ii < runLength; ii++) { + ctx.fillWindowTicks(ctx, posChunk.get(ii)); - MutableFloat sum = new MutableFloat(NULL_FLOAT); - ctx.windowValues.forEach(v-> { + float sum = NULL_FLOAT; + + LongRingBuffer.Iterator it = ctx.windowIndices.iterator(); + while (it.hasNext()) { + float v = ctx.candidateValuesChunk.get((int)it.next()); if (v != QueryConstants.NULL_FLOAT) { - if (sum.getValue() == NULL_FLOAT) { - sum.setValue(v); + if (sum == NULL_FLOAT) { + sum = v; } else { - sum.add(v); + sum += v; } } - }); + } + localOutputValues.set(ii, sum); + } + } - localOutputValues.set(ii, sum.getValue()); + private void computeTime(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys) { + + final WritableFloatChunk localOutputValues = ctx.outputValues.get(); + // get the timestamp values for this chunk + try (final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(inputKeys.intSize())) { + LongChunk timestampChunk = timestampColumnSource.getChunk(context, inputKeys).asLongChunk(); + + for (int ii = 0; ii < inputKeys.intSize(); ii++) { + long ts = timestampChunk.get(ii); + + // does this value have a valid timestamp + if (ts == NULL_LONG) { + localOutputValues.set(ii, NULL_FLOAT); + } else { + // the output value is computed by push/pop operations triggered by fillWindow + ctx.fillWindowTime(ctx, timestampChunk.get(ii)); + + float sum = NULL_FLOAT; + + LongRingBuffer.Iterator it = ctx.windowIndices.iterator(); + while (it.hasNext()) { + float v = ctx.candidateValuesChunk.get((int)it.next()); + if (v != QueryConstants.NULL_FLOAT) { + if (sum == NULL_FLOAT) { + sum = v; + } else { + sum += v; + } + } + } + localOutputValues.set(ii, sum); + } + } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java index ab84050c122..02b5963b515 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java @@ -43,14 +43,10 @@ protected class Context extends BaseWindowedIntUpdateByOperator.Context { public long currentVal = NULL_LONG; - // position data for the chunk being currently processed - public SizedLongChunk valuePositionChunk; - protected Context(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); this.fillContext.ensureCapacity(chunkSize); this.outputValues = new SizedLongChunk<>(chunkSize); - this.valuePositionChunk = new SizedLongChunk<>(chunkSize); this.timestampSsa = timestampSsa; } @@ -59,7 +55,6 @@ public void close() { super.close(); outputValues.close(); fillContext.close(); - this.valuePositionChunk.close(); } } @@ -74,7 +69,6 @@ public void setChunkSize(@NotNull UpdateContext context, int chunkSize) { final Context ctx = (Context) context; ctx.outputValues.ensureCapacity(chunkSize); ctx.fillContext.ensureCapacity(chunkSize); - ctx.valuePositionChunk.ensureCapacity(chunkSize); } public IntRollingSumOperator(@NotNull final MatchPair pair, @@ -84,8 +78,8 @@ public IntRollingSumOperator(@NotNull final MatchPair pair, @Nullable final ColumnSource timestampColumnSource, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final ColumnSource valueSource, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, + @NotNull final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args ) { @@ -109,7 +103,7 @@ public IntRollingSumOperator(@NotNull final MatchPair pair, @Override public void push(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; - Integer val = ctx.candidateValuesChunk.get(pos); + int val = ctx.candidateValuesChunk.get(pos); // increase the running sum if (val != NULL_INT) { @@ -118,17 +112,21 @@ public void push(UpdateContext context, long key, int pos) { } else { ctx.currentVal += val; } + } else { + ctx.nullCount++; } } @Override public void pop(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; - Integer val = ctx.candidateValuesChunk.get(pos); + int val = ctx.candidateValuesChunk.get(pos); // reduce the running sum if (val != NULL_INT) { ctx.currentVal -= val; + } else { + ctx.nullCount--; } } @@ -147,34 +145,43 @@ public void doProcessChunk(@NotNull final BaseWindowedIntUpdateByOperator.Contex final Context ctx = (Context) context; if (timestampColumnName == null) { - // produce position data for the window (will be timestamps for time-based) - // TODO: gotta be a better way than creating two rowsets - try (final RowSet rs = inputKeys.asRowSet(); - final RowSet positions = ctx.sourceRowSet.invert(rs)) { - positions.fillRowKeyChunk(ctx.valuePositionChunk.get()); - } + computeTicks(ctx, posChunk, inputKeys.intSize()); + } else { + computeTime(ctx, inputKeys); } - computeTicks(ctx, 0, inputKeys.intSize()); - //noinspection unchecked outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } private void computeTicks(@NotNull final Context ctx, - final int runStart, + @Nullable final LongChunk posChunk, final int runLength) { final WritableLongChunk localOutputValues = ctx.outputValues.get(); - for (int ii = runStart; ii < runStart + runLength; ii++) { - if (timestampColumnName == null) { - ctx.fillWindowTicks(ctx, ctx.valuePositionChunk.get().get(ii)); - } - // the sum was computed by push/pop operations + for (int ii = 0; ii < runLength; ii++) { + // the output value is computed by push/pop operations triggered by fillWindow + ctx.fillWindowTicks(ctx, posChunk.get(ii)); localOutputValues.set(ii, ctx.currentVal); } } + private void computeTime(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys) { + + final WritableLongChunk localOutputValues = ctx.outputValues.get(); + // get the timestamp values for this chunk + try (final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(inputKeys.intSize())) { + LongChunk timestampChunk = timestampColumnSource.getChunk(context, inputKeys).asLongChunk(); + + for (int ii = 0; ii < inputKeys.intSize(); ii++) { + // the output value is computed by push/pop operations triggered by fillWindow + ctx.fillWindowTime(ctx, timestampChunk.get(ii)); + localOutputValues.set(ii, ctx.currentVal); + } + } + } + @NotNull @Override public Map> getOutputColumns() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java index e566096cf5b..79fbf0df7d1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java @@ -43,14 +43,10 @@ protected class Context extends BaseWindowedLongUpdateByOperator.Context { public long currentVal = NULL_LONG; - // position data for the chunk being currently processed - public SizedLongChunk valuePositionChunk; - protected Context(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); this.fillContext.ensureCapacity(chunkSize); this.outputValues = new SizedLongChunk<>(chunkSize); - this.valuePositionChunk = new SizedLongChunk<>(chunkSize); this.timestampSsa = timestampSsa; } @@ -59,7 +55,6 @@ public void close() { super.close(); outputValues.close(); fillContext.close(); - this.valuePositionChunk.close(); } } @@ -74,7 +69,6 @@ public void setChunkSize(@NotNull UpdateContext context, int chunkSize) { final Context ctx = (Context) context; ctx.outputValues.ensureCapacity(chunkSize); ctx.fillContext.ensureCapacity(chunkSize); - ctx.valuePositionChunk.ensureCapacity(chunkSize); } public LongRollingSumOperator(@NotNull final MatchPair pair, @@ -84,8 +78,8 @@ public LongRollingSumOperator(@NotNull final MatchPair pair, @Nullable final ColumnSource timestampColumnSource, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final ColumnSource valueSource, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, + @NotNull final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args ) { @@ -109,7 +103,7 @@ public LongRollingSumOperator(@NotNull final MatchPair pair, @Override public void push(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; - Long val = ctx.candidateValuesChunk.get(pos); + long val = ctx.candidateValuesChunk.get(pos); // increase the running sum if (val != NULL_LONG) { @@ -118,17 +112,21 @@ public void push(UpdateContext context, long key, int pos) { } else { ctx.currentVal += val; } + } else { + ctx.nullCount++; } } @Override public void pop(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; - Long val = ctx.candidateValuesChunk.get(pos); + long val = ctx.candidateValuesChunk.get(pos); // reduce the running sum if (val != NULL_LONG) { ctx.currentVal -= val; + } else { + ctx.nullCount--; } } @@ -147,34 +145,43 @@ public void doProcessChunk(@NotNull final BaseWindowedLongUpdateByOperator.Conte final Context ctx = (Context) context; if (timestampColumnName == null) { - // produce position data for the window (will be timestamps for time-based) - // TODO: gotta be a better way than creating two rowsets - try (final RowSet rs = inputKeys.asRowSet(); - final RowSet positions = ctx.sourceRowSet.invert(rs)) { - positions.fillRowKeyChunk(ctx.valuePositionChunk.get()); - } + computeTicks(ctx, posChunk, inputKeys.intSize()); + } else { + computeTime(ctx, inputKeys); } - computeTicks(ctx, 0, inputKeys.intSize()); - //noinspection unchecked outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } private void computeTicks(@NotNull final Context ctx, - final int runStart, + @Nullable final LongChunk posChunk, final int runLength) { final WritableLongChunk localOutputValues = ctx.outputValues.get(); - for (int ii = runStart; ii < runStart + runLength; ii++) { - if (timestampColumnName == null) { - ctx.fillWindowTicks(ctx, ctx.valuePositionChunk.get().get(ii)); - } - // the sum was computed by push/pop operations + for (int ii = 0; ii < runLength; ii++) { + // the output value is computed by push/pop operations triggered by fillWindow + ctx.fillWindowTicks(ctx, posChunk.get(ii)); localOutputValues.set(ii, ctx.currentVal); } } + private void computeTime(@NotNull final Context ctx, + @NotNull final RowSequence inputKeys) { + + final WritableLongChunk localOutputValues = ctx.outputValues.get(); + // get the timestamp values for this chunk + try (final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(inputKeys.intSize())) { + LongChunk timestampChunk = timestampColumnSource.getChunk(context, inputKeys).asLongChunk(); + + for (int ii = 0; ii < inputKeys.intSize(); ii++) { + // the output value is computed by push/pop operations triggered by fillWindow + ctx.fillWindowTime(ctx, timestampChunk.get(ii)); + localOutputValues.set(ii, ctx.currentVal); + } + } + } + @NotNull @Override public Map> getOutputColumns() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java index 2c128f9f95b..add2b323dcb 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java @@ -73,8 +73,8 @@ public ShortRollingSumOperator(@NotNull final MatchPair pair, @Nullable final ColumnSource timestampColumnSource, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final ColumnSource valueSource, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, + @NotNull final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args ) { @@ -98,7 +98,7 @@ public ShortRollingSumOperator(@NotNull final MatchPair pair, @Override public void push(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; - Short val = ctx.candidateValuesChunk.get(pos); + short val = ctx.candidateValuesChunk.get(pos); // increase the running sum if (val != NULL_SHORT) { @@ -107,17 +107,21 @@ public void push(UpdateContext context, long key, int pos) { } else { ctx.currentVal += val; } + } else { + ctx.nullCount++; } } @Override public void pop(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; - Short val = ctx.candidateValuesChunk.get(pos); + short val = ctx.candidateValuesChunk.get(pos); // reduce the running sum if (val != NULL_SHORT) { ctx.currentVal -= val; + } else { + ctx.nullCount--; } } @@ -141,7 +145,6 @@ public void doProcessChunk(@NotNull final BaseWindowedShortUpdateByOperator.Cont computeTime(ctx, inputKeys); } - //noinspection unchecked outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } @@ -152,8 +155,8 @@ private void computeTicks(@NotNull final Context ctx, final WritableLongChunk localOutputValues = ctx.outputValues.get(); for (int ii = 0; ii < runLength; ii++) { + // the output value is computed by push/pop operations triggered by fillWindow ctx.fillWindowTicks(ctx, posChunk.get(ii)); - // the sum was computed by push/pop operations localOutputValues.set(ii, ctx.currentVal); } } @@ -167,8 +170,8 @@ private void computeTime(@NotNull final Context ctx, LongChunk timestampChunk = timestampColumnSource.getChunk(context, inputKeys).asLongChunk(); for (int ii = 0; ii < inputKeys.intSize(); ii++) { + // the output value is computed by push/pop operations triggered by fillWindow ctx.fillWindowTime(ctx, timestampChunk.get(ii)); - // the sum was computed by push/pop operations localOutputValues.set(ii, ctx.currentVal); } } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java index 05f0084c756..72338357b43 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java @@ -69,7 +69,6 @@ public void testStaticZeroKeyFwdWindow() { } } - @Test public void testStaticZeroKeyFwdRevWindow() { final QueryTable t = createTestTable(10000, false, false, false, 0x31313131).t; @@ -85,6 +84,114 @@ public void testStaticZeroKeyFwdRevWindow() { } } + @Test + public void testStaticZeroKeyTimed() { + final QueryTable t = createTestTable(10000, false, false, false, 0xFFFABBBC, + new String[] {"ts"}, new TstUtils.Generator[] {new TstUtils.SortedDateTimeGenerator( + convertDateTime("2022-03-09T09:00:00.000 NY"), + convertDateTime("2022-03-09T16:30:00.000 NY"))}).t; + + final OperationControl skipControl = OperationControl.builder() + .onNullValue(BadDataBehavior.SKIP) + .onNanValue(BadDataBehavior.SKIP).build(); + + final OperationControl resetControl = OperationControl.builder() + .onNullValue(BadDataBehavior.RESET) + .onNanValue(BadDataBehavior.RESET).build(); + + Duration prevTime = Duration.ofMinutes(10); + Duration postTime = Duration.ZERO; + + final Table summed = + t.updateBy(UpdateByOperation.RollingSum("ts", prevTime, postTime, "byteCol", "shortCol", "intCol", "longCol", "floatCol", + "doubleCol", "boolCol", "bigIntCol", "bigDecimalCol" + )); + + + DateTime[] ts = (DateTime[])t.getColumn("ts").getDirect(); + long[] timestamps = new long[t.intSize()]; + for (int i = 0; i < t.intSize(); i++) { + timestamps[i] = ts[i].getNanos(); + } + + for (String col : t.getDefinition().getColumnNamesArray()) { + assertWithRollingSumTime(t.getColumn(col).getDirect(), summed.getColumn(col).getDirect(), timestamps, + summed.getColumn(col).getType(), prevTime.toNanos(), postTime.toNanos()); + } + } + + @Test + public void testStaticZeroKeyFwdWindowTimed() { + final QueryTable t = createTestTable(10000, false, false, false, 0xFFFABBBC, + new String[] {"ts"}, new TstUtils.Generator[] {new TstUtils.SortedDateTimeGenerator( + convertDateTime("2022-03-09T09:00:00.000 NY"), + convertDateTime("2022-03-09T16:30:00.000 NY"))}).t; + + final OperationControl skipControl = OperationControl.builder() + .onNullValue(BadDataBehavior.SKIP) + .onNanValue(BadDataBehavior.SKIP).build(); + + final OperationControl resetControl = OperationControl.builder() + .onNullValue(BadDataBehavior.RESET) + .onNanValue(BadDataBehavior.RESET).build(); + + Duration prevTime = Duration.ZERO; + Duration postTime = Duration.ofMinutes(10); + + final Table summed = + t.updateBy(UpdateByOperation.RollingSum("ts", prevTime, postTime, "byteCol", "shortCol", "intCol", "longCol", "floatCol", + "doubleCol", "boolCol", "bigIntCol", "bigDecimalCol" + )); + + + DateTime[] ts = (DateTime[])t.getColumn("ts").getDirect(); + long[] timestamps = new long[t.intSize()]; + for (int i = 0; i < t.intSize(); i++) { + timestamps[i] = ts[i].getNanos(); + } + + for (String col : t.getDefinition().getColumnNamesArray()) { + assertWithRollingSumTime(t.getColumn(col).getDirect(), summed.getColumn(col).getDirect(), timestamps, + summed.getColumn(col).getType(), prevTime.toNanos(), postTime.toNanos()); + } + } + + @Test + public void testStaticZeroKeyFwdRevWindowTimed() { + final QueryTable t = createTestTable(10000, false, false, false, 0xFFFABBBC, + new String[] {"ts"}, new TstUtils.Generator[] {new TstUtils.SortedDateTimeGenerator( + convertDateTime("2022-03-09T09:00:00.000 NY"), + convertDateTime("2022-03-09T16:30:00.000 NY"))}).t; + + final OperationControl skipControl = OperationControl.builder() + .onNullValue(BadDataBehavior.SKIP) + .onNanValue(BadDataBehavior.SKIP).build(); + + final OperationControl resetControl = OperationControl.builder() + .onNullValue(BadDataBehavior.RESET) + .onNanValue(BadDataBehavior.RESET).build(); + + Duration prevTime = Duration.ofMinutes(10); + Duration postTime = Duration.ofMinutes(10); + + final Table summed = + t.updateBy(UpdateByOperation.RollingSum("ts", prevTime, postTime, "byteCol", "shortCol", "intCol", "longCol", "floatCol", + "doubleCol", "boolCol", "bigIntCol", "bigDecimalCol" + )); + + + DateTime[] ts = (DateTime[])t.getColumn("ts").getDirect(); + long[] timestamps = new long[t.intSize()]; + for (int i = 0; i < t.intSize(); i++) { + timestamps[i] = ts[i].getNanos(); + } + + for (String col : t.getDefinition().getColumnNamesArray()) { + assertWithRollingSumTime(t.getColumn(col).getDirect(), summed.getColumn(col).getDirect(), timestamps, + summed.getColumn(col).getType(), prevTime.toNanos(), postTime.toNanos()); + } + } + // endregion // region Bucketed Tests @@ -130,8 +237,7 @@ private void doTestStaticBucketed(boolean grouped) { final Table summed = t.updateBy(UpdateByOperation.RollingSum(prevTicks, postTicks, "byteCol", "shortCol", "intCol", "longCol", "floatCol", - "doubleCol", "boolCol" -// TODO: put these back ,"bigIntCol", "bigDecimalCol" + "doubleCol", "boolCol", "bigIntCol", "bigDecimalCol" ), "Sym"); @@ -150,40 +256,49 @@ private void doTestStaticBucketed(boolean grouped) { } @Test - public void testStaticZeroKeyTimed() { - final QueryTable t = createTestTable(1000, false, false, false, 0xFFFABBBC, - new String[] {"ts"}, new TstUtils.Generator[] {new TstUtils.SortedDateTimeGenerator( - convertDateTime("2022-03-09T09:00:00.000 NY"), - convertDateTime("2022-03-09T16:30:00.000 NY"))}).t; + public void testStaticBucketedTimed() { + doTestStaticBucketedTimed(false, Duration.ofMinutes(10), Duration.ZERO); + } - final OperationControl skipControl = OperationControl.builder() - .onNullValue(BadDataBehavior.SKIP) - .onNanValue(BadDataBehavior.SKIP).build(); + @Test + public void testStaticBucketedFwdWindowTimed() { + doTestStaticBucketedTimed(false, Duration.ZERO, Duration.ofMinutes(10)); + } - final OperationControl resetControl = OperationControl.builder() - .onNullValue(BadDataBehavior.RESET) - .onNanValue(BadDataBehavior.RESET).build(); + @Test + public void testStaticBucketedFwdRevWindowTimed() { + doTestStaticBucketedTimed(false, Duration.ofMinutes(10), Duration.ofMinutes(10)); + } - Duration prevTime = Duration.ofMinutes(10); - Duration postTime = Duration.ZERO; + private void doTestStaticBucketedTimed(boolean grouped, Duration prevTime, Duration postTime) { + final QueryTable t = createTestTable(10000, true, grouped, false, 0xFFFABBBC, + new String[] {"ts"}, new TstUtils.Generator[] {new TstUtils.SortedDateTimeGenerator( + convertDateTime("2022-03-09T09:00:00.000 NY"), + convertDateTime("2022-03-09T16:30:00.000 NY"))}).t; final Table summed = t.updateBy(UpdateByOperation.RollingSum("ts", prevTime, postTime, "byteCol", "shortCol", "intCol", "longCol", "floatCol", - "doubleCol", "boolCol" -// TODO: put these back ,"bigIntCol", "bigDecimalCol" - )); + "doubleCol", "boolCol", "bigIntCol", "bigDecimalCol" + ), "Sym"); - DateTime[] ts = (DateTime[])t.getColumn("ts").getDirect(); - long[] timestamps = new long[t.intSize()]; - for (int i = 0; i < t.intSize(); i++) { - timestamps[i] = ts[i].getNanos(); - } + final PartitionedTable preOp = t.partitionBy("Sym"); + final PartitionedTable postOp = summed.partitionBy("Sym"); - for (String col : t.getDefinition().getColumnNamesArray()) { - assertWithRollingSumTime(t.getColumn(col).getDirect(), summed.getColumn(col).getDirect(), timestamps, - summed.getColumn(col).getType(), prevTime.toNanos(), postTime.toNanos()); - } + String[] columns = t.getDefinition().getColumnStream().map(ColumnDefinition::getName).toArray(String[]::new); + + preOp.partitionedTransform(postOp, (source, actual) -> { + DateTime[] ts = (DateTime[])source.getColumn("ts").getDirect(); + long[] timestamps = new long[source.intSize()]; + for (int i = 0; i < source.intSize(); i++) { + timestamps[i] = ts[i].getNanos(); + } + Arrays.stream(columns).forEach(col -> { + assertWithRollingSumTime(source.getColumn(col).getDirect(), actual.getColumn(col).getDirect(), timestamps, + actual.getColumn(col).getType(), prevTime.toNanos(), postTime.toNanos()); + }); + return source; + }); } // endregion @@ -222,6 +337,47 @@ protected Table e() { } } + @Test + public void testZeroKeyAppendOnlyTimed() { + doTestAppendOnlyTimed(false); + } + + @Test + public void testBucketedAppendOnlyTimed() { + doTestAppendOnlyTimed(true); + } + + + private void doTestAppendOnlyTimed(boolean bucketed) { + final CreateResult result = createTestTable(10000, bucketed, false, true, 0x31313131, + new String[] {"ts"}, new TstUtils.Generator[] {new TstUtils.SortedDateTimeGenerator( + convertDateTime("2022-03-09T09:00:00.000 NY"), + convertDateTime("2022-03-09T16:30:00.000 NY"))}); + final QueryTable t = result.t; + t.setAttribute(Table.ADD_ONLY_TABLE_ATTRIBUTE, Boolean.TRUE); + + t.setAttribute(Table.ADD_ONLY_TABLE_ATTRIBUTE, Boolean.TRUE); + + Duration prevTime = Duration.ofMinutes(10); + Duration postTime = Duration.ZERO; + + final EvalNugget[] nuggets = new EvalNugget[] { + new EvalNugget() { + @Override + protected Table e() { + return bucketed ? t.updateBy(UpdateByOperation.RollingSum("ts", prevTime, postTime), "Sym") + : t.updateBy(UpdateByOperation.RollingSum("ts", prevTime, postTime)); + } + } + }; + + final Random billy = new Random(0xB177B177); + for (int ii = 0; ii < 100; ii++) { + UpdateGraphProcessor.DEFAULT.runWithinUnitTestCycle(() -> generateAppends(100, billy, t, result.infos)); + TstUtils.validate("Table", nuggets); + } + } + @Test public void testZeroKeyGeneralTicking() { final CreateResult result = createTestTable(10000, false, false, true, 0x31313131); @@ -270,9 +426,7 @@ protected Table e() { @Test public void testBucketedGeneralTicking() { - final CreateResult result = createTestTable(100, true, false, true, 0x31313131); - -// final CreateResult result = createTestTable(10000, true, false, true, 0x31313131); + final CreateResult result = createTestTable(10000, true, false, true, 0x31313131); final QueryTable t = result.t; final EvalNugget[] nuggets = new EvalNugget[] { @@ -371,51 +525,6 @@ private long[] rollingSum(short[] values, int prevTicks, int postTicks) { return result; } - private long[] rollingSumTime(short[] values, long[] timestamps, long prevNanos, long postNanos) { - if (values == null) { - return null; - } - - if (values.length == 0) { - return new long[0]; - } - - long[] result = new long[values.length]; - - int head = 0; - int tail = 0; - - for (int i = 0; i < values.length; i++) { - result[i] = NULL_LONG; - - // set the head and the tail - final long headTime = timestamps[i] - prevNanos; - final long tailTime = timestamps[i] + postNanos; - - // advance head and tail until they are in the correct spots - while (head < values.length && timestamps[head] < headTime) { - head++; - } - - while (tail < values.length && timestamps[tail] < tailTime) { - tail++; - } - - // compute everything in this window - for (int computeIdx = head; computeIdx <= tail; computeIdx++) { - if (!isNull(values[computeIdx])) { - if (result[i] == NULL_LONG) { - result[i] = values[computeIdx]; - } else { - result[i] += values[computeIdx]; - } - } - } - } - - return result; - } - private long[] rollingSum(int[] values, int prevTicks, int postTicks) { if (values == null) { return null; @@ -625,53 +734,464 @@ public static Object[] rollingSum(Object[] values, final boolean isBD, int prevT return result; } - final void assertWithRollingSumTicks(final @NotNull Object expected, final @NotNull Object actual, Class type, int prevTicks, int postTicks) { - final float deltaF = .001f; - final double deltaD = .001d; + private long[] rollingSumTime(byte[] values, long[] timestamps, long prevNanos, long postNanos) { + if (values == null) { + return null; + } - if (expected instanceof byte[]) { - assertArrayEquals(rollingSum((byte[]) expected, prevTicks, postTicks), (long[]) actual); - } else if (expected instanceof short[]) { - assertArrayEquals(rollingSum((short[]) expected, prevTicks, postTicks), (long[]) actual); - } else if (expected instanceof int[]) { - assertArrayEquals(rollingSum((int[]) expected, prevTicks, postTicks), (long[]) actual); - } else if (expected instanceof long[]) { - assertArrayEquals(rollingSum((long[]) expected, prevTicks, postTicks), (long[]) actual); - } else if (expected instanceof float[]) { - assertArrayEquals(rollingSum((float[]) expected, prevTicks, postTicks), (float[]) actual, deltaF); - } else if (expected instanceof double[]) { - assertArrayEquals(rollingSum((double[]) expected, prevTicks, postTicks), (double[]) actual, deltaD); - } else if (expected instanceof Boolean[]) { - assertArrayEquals(rollingSum((Boolean[]) expected, prevTicks, postTicks), (long[]) actual); - } else { -// assertArrayEquals(rollingSum((Object[]) expected, type == BigDecimal.class, prevTicks, postTicks), (Object[]) actual); + if (values.length == 0) { + return new long[0]; } - } - final void assertWithRollingSumTime(final @NotNull Object expected, final @NotNull Object actual, - final @NotNull long[] timestamps, Class type, long prevTime, long postTime) { - final float deltaF = .001f; - final double deltaD = .001d; + long[] result = new long[values.length]; - if (expected instanceof short[]) { - assertArrayEquals(rollingSumTime((short[]) expected, timestamps, prevTime, postTime), (long[]) actual); + int head = 0; + int tail = 0; + + for (int i = 0; i < values.length; i++) { + result[i] = NULL_LONG; + + // check the current timestamp. skip if NULL + if (timestamps[i] == NULL_LONG) { + continue; + } + + // set the head and the tail + final long headTime = timestamps[i] - prevNanos; + final long tailTime = timestamps[i] + postNanos; + + // advance head and tail until they are in the correct spots + while (head < values.length && timestamps[head] < headTime) { + head++; + } + + while (tail < values.length && timestamps[tail] <= tailTime) { + tail++; + } + + // compute everything in this window + for (int computeIdx = head; computeIdx < tail; computeIdx++) { + if (!isNull(values[computeIdx])) { + if (result[i] == NULL_LONG) { + result[i] = values[computeIdx]; + } else { + result[i] += values[computeIdx]; + } + } + } + } + + return result; + } + + private long[] rollingSumTime(short[] values, long[] timestamps, long prevNanos, long postNanos) { + if (values == null) { + return null; + } + + if (values.length == 0) { + return new long[0]; + } + + long[] result = new long[values.length]; + + int head = 0; + int tail = 0; + + for (int i = 0; i < values.length; i++) { + result[i] = NULL_LONG; + + // check the current timestamp. skip if NULL + if (timestamps[i] == NULL_LONG) { + continue; + } + + // set the head and the tail + final long headTime = timestamps[i] - prevNanos; + final long tailTime = timestamps[i] + postNanos; + + // advance head and tail until they are in the correct spots + while (head < values.length && timestamps[head] < headTime) { + head++; + } + + while (tail < values.length && timestamps[tail] <= tailTime) { + tail++; + } + + // compute everything in this window + for (int computeIdx = head; computeIdx < tail; computeIdx++) { + if (!isNull(values[computeIdx])) { + if (result[i] == NULL_LONG) { + result[i] = values[computeIdx]; + } else { + result[i] += values[computeIdx]; + } + } + } + } + + return result; + } + + private long[] rollingSumTime(int[] values, long[] timestamps, long prevNanos, long postNanos) { + if (values == null) { + return null; + } + + if (values.length == 0) { + return new long[0]; + } + + long[] result = new long[values.length]; + + int head = 0; + int tail = 0; + + for (int i = 0; i < values.length; i++) { + result[i] = NULL_LONG; + + // check the current timestamp. skip if NULL + if (timestamps[i] == NULL_LONG) { + continue; + } + + // set the head and the tail + final long headTime = timestamps[i] - prevNanos; + final long tailTime = timestamps[i] + postNanos; + + // advance head and tail until they are in the correct spots + while (head < values.length && timestamps[head] < headTime) { + head++; + } + + while (tail < values.length && timestamps[tail] <= tailTime) { + tail++; + } + + // compute everything in this window + for (int computeIdx = head; computeIdx < tail; computeIdx++) { + if (!isNull(values[computeIdx])) { + if (result[i] == NULL_LONG) { + result[i] = values[computeIdx]; + } else { + result[i] += values[computeIdx]; + } + } + } + } + + return result; + } + + private long[] rollingSumTime(long[] values, long[] timestamps, long prevNanos, long postNanos) { + if (values == null) { + return null; + } + + if (values.length == 0) { + return new long[0]; + } + + long[] result = new long[values.length]; + + int head = 0; + int tail = 0; + + for (int i = 0; i < values.length; i++) { + result[i] = NULL_LONG; + + // check the current timestamp. skip if NULL + if (timestamps[i] == NULL_LONG) { + continue; + } + + // set the head and the tail + final long headTime = timestamps[i] - prevNanos; + final long tailTime = timestamps[i] + postNanos; + + // advance head and tail until they are in the correct spots + while (head < values.length && timestamps[head] < headTime) { + head++; + } + + while (tail < values.length && timestamps[tail] <= tailTime) { + tail++; + } + + // compute everything in this window + for (int computeIdx = head; computeIdx < tail; computeIdx++) { + if (!isNull(values[computeIdx])) { + if (result[i] == NULL_LONG) { + result[i] = values[computeIdx]; + } else { + result[i] += values[computeIdx]; + } + } + } + } + + return result; + } + + private float[] rollingSumTime(float[] values, long[] timestamps, long prevNanos, long postNanos) { + if (values == null) { + return null; + } + + if (values.length == 0) { + return new float[0]; + } + + float[] result = new float[values.length]; + + int head = 0; + int tail = 0; + + for (int i = 0; i < values.length; i++) { + result[i] = NULL_FLOAT; + + // check the current timestamp. skip if NULL + if (timestamps[i] == NULL_LONG) { + continue; + } + + // set the head and the tail + final long headTime = timestamps[i] - prevNanos; + final long tailTime = timestamps[i] + postNanos; + + // advance head and tail until they are in the correct spots + while (head < values.length && timestamps[head] < headTime) { + head++; + } + + while (tail < values.length && timestamps[tail] <= tailTime) { + tail++; + } + + // compute everything in this window + for (int computeIdx = head; computeIdx < tail; computeIdx++) { + if (!isNull(values[computeIdx])) { + if (result[i] == NULL_FLOAT) { + result[i] = values[computeIdx]; + } else { + result[i] += values[computeIdx]; + } + } + } + } + + return result; + } + + private double[] rollingSumTime(double[] values, long[] timestamps, long prevNanos, long postNanos) { + if (values == null) { + return null; + } + + if (values.length == 0) { + return new double[0]; + } + + double[] result = new double[values.length]; + + int head = 0; + int tail = 0; + + for (int i = 0; i < values.length; i++) { + result[i] = NULL_DOUBLE; + + // check the current timestamp. skip if NULL + if (timestamps[i] == NULL_LONG) { + continue; + } + + // set the head and the tail + final long headTime = timestamps[i] - prevNanos; + final long tailTime = timestamps[i] + postNanos; + + // advance head and tail until they are in the correct spots + while (head < values.length && timestamps[head] < headTime) { + head++; + } + + while (tail < values.length && timestamps[tail] <= tailTime) { + tail++; + } + + // compute everything in this window + for (int computeIdx = head; computeIdx < tail; computeIdx++) { + if (!isNull(values[computeIdx])) { + if (result[i] == NULL_DOUBLE) { + result[i] = values[computeIdx]; + } else { + result[i] += values[computeIdx]; + } + } + } + } + + return result; + } + + private long[] rollingSumTime(Boolean[] values, long[] timestamps, long prevNanos, long postNanos) { + if (values == null) { + return null; + } + + if (values.length == 0) { + return new long[0]; + } + + long[] result = new long[values.length]; + + int head = 0; + int tail = 0; + + for (int i = 0; i < values.length; i++) { + result[i] = NULL_LONG; + + // check the current timestamp. skip if NULL + if (timestamps[i] == NULL_LONG) { + continue; + } + + // set the head and the tail + final long headTime = timestamps[i] - prevNanos; + final long tailTime = timestamps[i] + postNanos; + + // advance head and tail until they are in the correct spots + while (head < values.length && timestamps[head] < headTime) { + head++; + } + + while (tail < values.length && timestamps[tail] <= tailTime) { + tail++; + } + + // compute everything in this window + for (int computeIdx = head; computeIdx < tail; computeIdx++) { + if (!isNull(values[computeIdx])) { + if (result[i] == NULL_LONG) { + result[i] = values[computeIdx] ? 1 : 0; + } else { + result[i] += (values[computeIdx] ? 1 : 0); + } + } + } + } + + return result; + } + + private Object[] rollingSumTime(Object[] values, long[] timestamps, final boolean isBD, long prevNanos, long postNanos) { + if (values == null) { + return null; + } + + if (values.length == 0) { + return new Object[0]; + } + + Object[] result = new Object[values.length]; + + int head = 0; + int tail = 0; + + for (int i = 0; i < values.length; i++) { + result[i] = null; + + // check the current timestamp. skip if NULL + if (timestamps[i] == NULL_LONG) { + continue; + } + + // set the head and the tail + final long headTime = timestamps[i] - prevNanos; + final long tailTime = timestamps[i] + postNanos; + + // advance head and tail until they are in the correct spots + while (head < values.length && timestamps[head] < headTime) { + head++; + } + + while (tail < values.length && timestamps[tail] <= tailTime) { + tail++; + } + + // compute everything in this window + for (int computeIdx = head; computeIdx < tail; computeIdx++) { + if (values[computeIdx] != null) { + if (result[i] == null) { + result[i] = values[computeIdx]; + } else { + if (isBD) { + result[i] = ((BigDecimal) result[i]).add((BigDecimal) values[computeIdx], + UpdateByControl.mathContextDefault()); + } else { + result[i] = ((BigInteger) result[i]).add((BigInteger) values[computeIdx]); + } + } + } + } + } + + return result; + } + + + final void assertWithRollingSumTicks(final @NotNull Object expected, final @NotNull Object actual, Class type, int prevTicks, int postTicks) { + final float deltaF = .001f; + final double deltaD = .001d; + + if (expected instanceof byte[]) { + assertArrayEquals(rollingSum((byte[]) expected, prevTicks, postTicks), (long[]) actual); + } else if (expected instanceof short[]) { + assertArrayEquals(rollingSum((short[]) expected, prevTicks, postTicks), (long[]) actual); + } else if (expected instanceof int[]) { + assertArrayEquals(rollingSum((int[]) expected, prevTicks, postTicks), (long[]) actual); + } else if (expected instanceof long[]) { + assertArrayEquals(rollingSum((long[]) expected, prevTicks, postTicks), (long[]) actual); + } else if (expected instanceof float[]) { + assertArrayEquals(rollingSum((float[]) expected, prevTicks, postTicks), (float[]) actual, deltaF); + } else if (expected instanceof double[]) { + assertArrayEquals(rollingSum((double[]) expected, prevTicks, postTicks), (double[]) actual, deltaD); + } else if (expected instanceof Boolean[]) { + assertArrayEquals(rollingSum((Boolean[]) expected, prevTicks, postTicks), (long[]) actual); + } else { + if (type == BigDecimal.class) { + assertArrayEquals(rollingSum((Object[]) expected, true, prevTicks, postTicks), (Object[]) actual); + } else if (type == BigInteger.class) { + assertArrayEquals(rollingSum((Object[]) expected, false, prevTicks, postTicks), (Object[]) actual); + } + } + } + + final void assertWithRollingSumTime(final @NotNull Object expected, final @NotNull Object actual, + final @NotNull long[] timestamps, Class type, long prevTime, long postTime) { + final float deltaF = .001f; + final double deltaD = .001d; + + if (expected instanceof byte[]) { + assertArrayEquals(rollingSumTime((byte[]) expected, timestamps, prevTime, postTime), (long[]) actual); + } else if (expected instanceof short[]) { + assertArrayEquals(rollingSumTime((short[]) expected, timestamps, prevTime, postTime), (long[]) actual); + } else if (expected instanceof int[]) { + assertArrayEquals(rollingSumTime((int[]) expected, timestamps, prevTime, postTime), (long[]) actual); + } else if (expected instanceof long[]) { + assertArrayEquals(rollingSumTime((long[]) expected, timestamps, prevTime, postTime), (long[]) actual); + } else if (expected instanceof float[]) { + assertArrayEquals(rollingSumTime((float[]) expected, timestamps, prevTime, postTime), (float[]) actual, deltaF); + } else if (expected instanceof double[]) { + assertArrayEquals(rollingSumTime((double[]) expected, timestamps, prevTime, postTime), (double[]) actual, deltaD); + } else if (expected instanceof Boolean[]) { + assertArrayEquals(rollingSumTime((Boolean[]) expected, timestamps, prevTime, postTime), (long[]) actual); + } else { + if (type == BigDecimal.class) { + assertArrayEquals(rollingSumTime((Object[]) expected, timestamps, true, prevTime, postTime), (Object[]) actual); + } else if (type == BigInteger.class) { + assertArrayEquals(rollingSumTime((Object[]) expected, timestamps, false, prevTime, postTime), (Object[]) actual); + } } -// if (expected instanceof byte[]) { -// assertArrayEquals(rollingSum((byte[]) expected, prevTicks, postTicks), (long[]) actual); -// } else if (expected instanceof short[]) { -// assertArrayEquals(rollingSum((short[]) expected, prevTicks, postTicks), (long[]) actual); -// } else if (expected instanceof int[]) { -// assertArrayEquals(rollingSum((int[]) expected, prevTicks, postTicks), (long[]) actual); -// } else if (expected instanceof long[]) { -// assertArrayEquals(rollingSum((long[]) expected, prevTicks, postTicks), (long[]) actual); -// } else if (expected instanceof float[]) { -// assertArrayEquals(rollingSum((float[]) expected, prevTicks, postTicks), (float[]) actual, deltaF); -// } else if (expected instanceof double[]) { -// assertArrayEquals(rollingSum((double[]) expected, prevTicks, postTicks), (double[]) actual, deltaD); -// } else if (expected instanceof Boolean[]) { -// assertArrayEquals(rollingSum((Boolean[]) expected, prevTicks, postTicks), (long[]) actual); -// } else { -// assertArrayEquals(rollingSum((Object[]) expected, type == BigDecimal.class, prevTicks, postTicks), (Object[]) actual); -// } } } diff --git a/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java b/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java index 906b519b949..4f8e767ad19 100644 --- a/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java +++ b/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java @@ -109,8 +109,8 @@ public final boolean applicableTo(Class inputType) { || inputType.equals(int.class) || inputType.equals(long.class) || inputType.equals(short.class) || inputType.equals(byte.class) -// // is boxed numeric? -// || Number.class.isAssignableFrom(inputType) + // is boxed numeric? + || Number.class.isAssignableFrom(inputType) // is boolean? || inputType == boolean.class || inputType == Boolean.class; From 63bc13029acf8b849b515cd4c4b53d6f4d5cd3af Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 13 Sep 2022 15:27:10 -0700 Subject: [PATCH 020/123] Added Pairwise floating point storage and tests (100% coverage) --- .../engine/table/impl/UpdateByOperator.java | 29 -- .../internal/PairwiseFloatRingBuffer.java | 239 ++++++++++++ .../rollingsum/DoubleRollingSumOperator.java | 69 ++-- .../rollingsum/FloatRollingSumOperator.java | 69 ++-- .../updateby/PairwiseFloatRingBufferTest.java | 340 ++++++++++++++++++ .../table/impl/updateby/TestRollingSum.java | 10 +- .../replicators/ReplicateUpdateBy.java | 7 + 7 files changed, 648 insertions(+), 115 deletions(-) create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java create mode 100644 engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/PairwiseFloatRingBufferTest.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java index 320dabd1912..ab136b67c0f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java @@ -27,35 +27,6 @@ * interface, the pattern of calls will be as follows. * *
        - *
      1. Removes - *
          - *
        • {@link #initializeFor(UpdateContext, RowSet, UpdateBy.UpdateType)}
        • - *
        • {@link #removeChunk(UpdateContext, LongChunk, Chunk, long)}
        • - *
        • {@link #finishFor(UpdateContext, UpdateBy.UpdateType)}
        • - *
        - *
      2. - *
      3. Shifts - *
          - *
        • {@link #initializeFor(UpdateContext, RowSet, UpdateBy.UpdateType)}
        • - *
        • {@link #applyShift(UpdateContext, RowSet, RowSetShiftData)}
        • - *
        • {@link #finishFor(UpdateContext, UpdateBy.UpdateType)}
        • - *
        - *
      4. - *
      5. Modifies - *
          - *
        • {@link #initializeFor(UpdateContext, RowSet, UpdateBy.UpdateType)}
        • - *
        • {@link #modifyChunk(UpdateContext, LongChunk, LongChunk, Chunk, Chunk, long)}
        • - *
        • {@link #finishFor(UpdateContext, UpdateBy.UpdateType)}
        • - *
        - *
      6. - *
      7. Adds - *
          - *
        • {@link #initializeFor(UpdateContext, RowSet, UpdateBy.UpdateType)}
        • - *
        • {@link #addChunk(UpdateContext, RowSequence, LongChunk, Chunk, long)}
        • - *
        • {@link #finishFor(UpdateContext, UpdateBy.UpdateType)}
        • - *
        - *
      8. - * *
      9. Reprocess *
          *
        • {@link #resetForProcess(UpdateContext, RowSet, long)}
        • diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java new file mode 100644 index 00000000000..78db0a8c4ee --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java @@ -0,0 +1,239 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.engine.table.impl.updateby.internal; + +import io.deephaven.chunk.WritableFloatChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.util.SafeCloseable; + +import java.util.BitSet; +import java.util.NoSuchElementException; + +/*** + * Store this data in the form of a binary tree where the latter half of the chunk is treated as a ring buffer and pairwise results of the `FloatFunction` are stored in the parent nodes. We do lazy evaluation by maintaining a 'dirty' `BitSet` and compute the ultimate pairwise result only when requested by `evaluate()' + * + * To keep the parent-node finding math easy and consistent between the ring buffer and the computation tree, the binary tree is shifted by one index so the root (and final result of computation) ends up in index 1 (instead of 0 which is un-used) + */ + +public class PairwiseFloatRingBuffer implements SafeCloseable { + // use a sized float chunk for underlying storage + private WritableFloatChunk storageChunk; + private final BitSet dirtyBits; + private final FloatFunction pairwiseFunction; + private final float emptyVal; + + // this measures internal storage capacity (chunk is twice this size) + private int capacity; + private int chunkSize; + + private int head; + private int tail; + + @FunctionalInterface + public interface FloatFunction { + /** + * Applies this function to the given arguments. + * + * @param a the first function argument + * @param b the second function argument + * @return the function result + */ + float apply(float a, float b); + } + + /** + * Create a ring buffer for Float values that will perform pairwise evaluation of the internal data values using an efficient binary-tree implementation to compute only changed values. The buffer will grow exponentially as items are pushed into it but will not shrink as values are removed + * + * @param initialSize the minimum size for the structure to hold + * @param emptyVal an innocuous value that will not affect the user-provided function results. for example, 0.0f for performing addition/subtraction, 1.0f for performing multiplication/division + * @param pairwiseFunction the user provided function for evaluation, takes two float parameters and returns a float. This function will be applied repeatedly to pairs of data values until the final result is available + */ + public PairwiseFloatRingBuffer(int initialSize, float emptyVal, FloatFunction pairwiseFunction) { + // increase to next power of two + this.capacity = Integer.highestOneBit(initialSize) * 2; + this.chunkSize = capacity * 2; + this.storageChunk = WritableFloatChunk.makeWritableChunk(chunkSize); + this.dirtyBits = new BitSet(chunkSize); + this.pairwiseFunction = pairwiseFunction; + this.emptyVal = emptyVal; + + this.storageChunk.fillWithValue(0, chunkSize, emptyVal); + this.head = this.tail = this.capacity; + } + + public float evaluate() { + // work through all the dirty bits from high to low until none remain + int bit = chunkSize; + while (!dirtyBits.isEmpty()) { + int nextSetBit = dirtyBits.previousSetBit(bit); + final int left = nextSetBit & 0xFFFFFFFE; // clear the final bit to force evenness + final int right = left + 1; + // this isn't the typical parent = (n-1)/2 because the tree is right-shifted by one + final int parent = left / 2; + + // load the data values + final float leftVal = storageChunk.get(left); + final float rightVal = storageChunk.get(right); + final float parentVal = storageChunk.get(parent); + + dirtyBits.clear(left, right + 1); // clear() excludes `toIndex` so add one to clear `right` as well + + final float computeVal = pairwiseFunction.apply(leftVal, rightVal); + if (parentVal != computeVal) { + storageChunk.set(parent, computeVal); + // mark the parent dirty (if not the last) + if (parent > 1) { + dirtyBits.set(parent); + } + } else { + final int x = 5; + } + bit = left; + } + // final value is in index 1 + return storageChunk.get(1); + } + + private void grow() { + int oldCapacity = capacity; + int oldChunkSize = chunkSize; + + // double the current capacity + capacity *= 2; + chunkSize = capacity * 2; + + // transfer to the new chunk + WritableFloatChunk oldChunk = storageChunk; + storageChunk = WritableFloatChunk.makeWritableChunk(chunkSize); + + // fill the pairwise tree (0 to capacity) with empty value + storageChunk.fillWithValue(0, capacity, emptyVal); + + // move the data to the new chunk, note that we store the ring data in the second half of the array + if (tail > head) { + storageChunk.copyFromTypedChunk(oldChunk, head, capacity, tail - head); + tail = capacity + tail - head; + } else { + storageChunk.copyFromTypedChunk(oldChunk, head, capacity, oldChunkSize - head); + storageChunk.copyFromTypedChunk(oldChunk, oldCapacity, oldChunkSize - head + capacity, tail - oldCapacity); + tail = capacity + oldCapacity - 1; + } + // fill the unused storage with the empty value + storageChunk.fillWithValue(tail, chunkSize - tail, emptyVal); + + // free the old data chunk + oldChunk.close(); + head = capacity; + + // TODO: investigate moving precomputed results also. Since we are re-ordering the data values, would be + // tricky to maintain order but a recursive function could probably do it efficiently. For now, make life easy + // by setting all input dirty so the tree is recomputed on next `evaluate()` + dirtyBits.clear(); + dirtyBits.set(head, tail, true); + } + + public void push(float val) { + if (isFull()) { + grow(); + } + // add the new data + storageChunk.set(tail, val); + dirtyBits.set(tail); + // move the tail + tail = ((tail + 1) % capacity) + capacity; + } + + public void pushEmptyValue() { + push(emptyVal); + } + + public float pop() { + if (isEmpty()) { + throw new NoSuchElementException(); + } + float val = storageChunk.get(head); + storageChunk.set(head, emptyVal); + dirtyBits.set(head); + // move the head + head = ((head + 1) % capacity) + capacity; + return val; + } + + public boolean isFull() { + return ((tail + 1) % capacity) + capacity == head; + } + + public int size() { + return tail >= head + ? (tail - head) : + (tail + (capacity - head)); + } + + public boolean isEmpty() { + return tail == head; + } + + public float peek(float onEmpty) { + if (isEmpty()) { + return onEmpty; + } + return storageChunk.get(head); + } + + public float poll(float onEmpty) { + if (isEmpty()) { + return onEmpty; + } + float e = storageChunk.get(head); + head = (head + 1) % capacity + capacity; + return e; + } + + public float front() { + return front(0); + } + + public float front(int offset) { + if (offset < 0 || offset >= size()) { + throw new NoSuchElementException(); + } + return storageChunk.get((head + offset) % capacity + capacity); + } + + public float back() { + if (isEmpty()) { + throw new NoSuchElementException(); + } + return tail == capacity ? storageChunk.get(chunkSize - 1) : storageChunk.get(tail - 1); + } + + public float peekBack(float onEmpty) { + if (isEmpty()) { + return onEmpty; + } + return tail == capacity ? storageChunk.get(chunkSize - 1) : storageChunk.get(tail - 1); + } + + public float element() { + if (isEmpty()) { + throw new NoSuchElementException(); + } + return storageChunk.get(head); + } + + public int capacity() { + return capacity - 1; + } + + public int remaining() { + return capacity() - size(); + } + + @Override + public void close() { + try (final WritableFloatChunk ignoredChunk = storageChunk) { + // close the closable items + } + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java index b5970ad374c..ed8ad72fcec 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java @@ -6,11 +6,9 @@ package io.deephaven.engine.table.impl.updateby.rollingsum; import io.deephaven.api.updateby.OperationControl; -import io.deephaven.base.LongRingBuffer; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableDoubleChunk; -import io.deephaven.chunk.WritableLongChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.sized.SizedDoubleChunk; import io.deephaven.chunk.sized.SizedLongChunk; @@ -23,19 +21,15 @@ import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedDoubleUpdateByOperator; -import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedShortUpdateByOperator; +import io.deephaven.engine.table.impl.updateby.internal.PairwiseDoubleRingBuffer; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; -import io.deephaven.util.QueryConstants; -import org.apache.commons.lang3.mutable.MutableDouble; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import java.util.Collections; -import java.util.LinkedList; import java.util.Map; import static io.deephaven.util.QueryConstants.NULL_DOUBLE; -import static io.deephaven.util.QueryConstants.NULL_LONG; public class DoubleRollingSumOperator extends BaseWindowedDoubleUpdateByOperator { @@ -53,12 +47,22 @@ protected class Context extends BaseWindowedDoubleUpdateByOperator.Context { // position data for the chunk being currently processed public SizedLongChunk valuePositionChunk; + public PairwiseDoubleRingBuffer pairwiseSum; + protected Context(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); this.fillContext.ensureCapacity(chunkSize); this.outputValues = new SizedDoubleChunk<>(chunkSize); this.valuePositionChunk = new SizedLongChunk<>(chunkSize); this.timestampSsa = timestampSsa; + final int initialSize; + if (timestampColumnName == null) { + // we know exactly the size and will never grow when using ticks + initialSize = (int)(reverseTimeScaleUnits + forwardTimeScaleUnits + 1); + } else { + initialSize = 64; // too big and the log(m) operation costs but growth also costs + } + this.pairwiseSum = new PairwiseDoubleRingBuffer(initialSize, 0.0f, (a, b) -> a + b); } @Override @@ -67,6 +71,7 @@ public void close() { outputValues.close(); fillContext.close(); this.valuePositionChunk.close(); + this.pairwiseSum.close(); } } @@ -117,7 +122,11 @@ public DoubleRollingSumOperator(@NotNull final MatchPair pair, public void push(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; double val = ctx.candidateValuesChunk.get(pos); - if (val == NULL_DOUBLE) { + + if (val != NULL_DOUBLE) { + ctx.pairwiseSum.push(val); + } else { + ctx.pairwiseSum.pushEmptyValue(); ctx.nullCount++; } } @@ -129,6 +138,7 @@ public void pop(UpdateContext context, long key, int pos) { if (val == NULL_DOUBLE) { ctx.nullCount--; } + ctx.pairwiseSum.pop(); } @Override @@ -162,20 +172,12 @@ private void computeTicks(@NotNull final Context ctx, for (int ii = 0; ii < runLength; ii++) { ctx.fillWindowTicks(ctx, posChunk.get(ii)); - double sum = NULL_DOUBLE; - - LongRingBuffer.Iterator it = ctx.windowIndices.iterator(); - while (it.hasNext()) { - double v = ctx.candidateValuesChunk.get((int)it.next()); - if (v != QueryConstants.NULL_DOUBLE) { - if (sum == NULL_DOUBLE) { - sum = v; - } else { - sum += v; - } - } + if (ctx.pairwiseSum.isEmpty() || ctx.pairwiseSum.size() == ctx.nullCount) { + localOutputValues.set(ii, NULL_DOUBLE); + } else { + double val = ctx.pairwiseSum.evaluate(); + localOutputValues.set(ii, val); } - localOutputValues.set(ii, sum); } } @@ -188,29 +190,14 @@ private void computeTime(@NotNull final Context ctx, LongChunk timestampChunk = timestampColumnSource.getChunk(context, inputKeys).asLongChunk(); for (int ii = 0; ii < inputKeys.intSize(); ii++) { - long ts = timestampChunk.get(ii); + // the output value is computed by push/pop operations triggered by fillWindow + ctx.fillWindowTime(ctx, timestampChunk.get(ii)); - // does this value have a valid timestamp - if (ts == NULL_LONG) { + if (ctx.pairwiseSum.isEmpty() || ctx.pairwiseSum.size() == ctx.nullCount) { localOutputValues.set(ii, NULL_DOUBLE); } else { - // the output value is computed by push/pop operations triggered by fillWindow - ctx.fillWindowTime(ctx, timestampChunk.get(ii)); - - double sum = NULL_DOUBLE; - - LongRingBuffer.Iterator it = ctx.windowIndices.iterator(); - while (it.hasNext()) { - double v = ctx.candidateValuesChunk.get((int)it.next()); - if (v != QueryConstants.NULL_DOUBLE) { - if (sum == NULL_DOUBLE) { - sum = v; - } else { - sum += v; - } - } - } - localOutputValues.set(ii, sum); + double val = ctx.pairwiseSum.evaluate(); + localOutputValues.set(ii, val); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java index e36938a166c..eadf635c01f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java @@ -1,11 +1,9 @@ package io.deephaven.engine.table.impl.updateby.rollingsum; import io.deephaven.api.updateby.OperationControl; -import io.deephaven.base.LongRingBuffer; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableFloatChunk; -import io.deephaven.chunk.WritableLongChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.sized.SizedFloatChunk; import io.deephaven.chunk.sized.SizedLongChunk; @@ -18,19 +16,15 @@ import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedFloatUpdateByOperator; -import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedShortUpdateByOperator; +import io.deephaven.engine.table.impl.updateby.internal.PairwiseFloatRingBuffer; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; -import io.deephaven.util.QueryConstants; -import org.apache.commons.lang3.mutable.MutableFloat; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import java.util.Collections; -import java.util.LinkedList; import java.util.Map; import static io.deephaven.util.QueryConstants.NULL_FLOAT; -import static io.deephaven.util.QueryConstants.NULL_LONG; public class FloatRollingSumOperator extends BaseWindowedFloatUpdateByOperator { @@ -48,12 +42,22 @@ protected class Context extends BaseWindowedFloatUpdateByOperator.Context { // position data for the chunk being currently processed public SizedLongChunk valuePositionChunk; + public PairwiseFloatRingBuffer pairwiseSum; + protected Context(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); this.fillContext.ensureCapacity(chunkSize); this.outputValues = new SizedFloatChunk<>(chunkSize); this.valuePositionChunk = new SizedLongChunk<>(chunkSize); this.timestampSsa = timestampSsa; + final int initialSize; + if (timestampColumnName == null) { + // we know exactly the size and will never grow when using ticks + initialSize = (int)(reverseTimeScaleUnits + forwardTimeScaleUnits + 1); + } else { + initialSize = 64; // too big and the log(m) operation costs but growth also costs + } + this.pairwiseSum = new PairwiseFloatRingBuffer(initialSize, 0.0f, (a, b) -> a + b); } @Override @@ -62,6 +66,7 @@ public void close() { outputValues.close(); fillContext.close(); this.valuePositionChunk.close(); + this.pairwiseSum.close(); } } @@ -112,7 +117,11 @@ public FloatRollingSumOperator(@NotNull final MatchPair pair, public void push(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; float val = ctx.candidateValuesChunk.get(pos); - if (val == NULL_FLOAT) { + + if (val != NULL_FLOAT) { + ctx.pairwiseSum.push(val); + } else { + ctx.pairwiseSum.pushEmptyValue(); ctx.nullCount++; } } @@ -124,6 +133,7 @@ public void pop(UpdateContext context, long key, int pos) { if (val == NULL_FLOAT) { ctx.nullCount--; } + ctx.pairwiseSum.pop(); } @Override @@ -157,20 +167,12 @@ private void computeTicks(@NotNull final Context ctx, for (int ii = 0; ii < runLength; ii++) { ctx.fillWindowTicks(ctx, posChunk.get(ii)); - float sum = NULL_FLOAT; - - LongRingBuffer.Iterator it = ctx.windowIndices.iterator(); - while (it.hasNext()) { - float v = ctx.candidateValuesChunk.get((int)it.next()); - if (v != QueryConstants.NULL_FLOAT) { - if (sum == NULL_FLOAT) { - sum = v; - } else { - sum += v; - } - } + if (ctx.pairwiseSum.isEmpty() || ctx.pairwiseSum.size() == ctx.nullCount) { + localOutputValues.set(ii, NULL_FLOAT); + } else { + float val = ctx.pairwiseSum.evaluate(); + localOutputValues.set(ii, val); } - localOutputValues.set(ii, sum); } } @@ -183,29 +185,14 @@ private void computeTime(@NotNull final Context ctx, LongChunk timestampChunk = timestampColumnSource.getChunk(context, inputKeys).asLongChunk(); for (int ii = 0; ii < inputKeys.intSize(); ii++) { - long ts = timestampChunk.get(ii); + // the output value is computed by push/pop operations triggered by fillWindow + ctx.fillWindowTime(ctx, timestampChunk.get(ii)); - // does this value have a valid timestamp - if (ts == NULL_LONG) { + if (ctx.pairwiseSum.isEmpty() || ctx.pairwiseSum.size() == ctx.nullCount) { localOutputValues.set(ii, NULL_FLOAT); } else { - // the output value is computed by push/pop operations triggered by fillWindow - ctx.fillWindowTime(ctx, timestampChunk.get(ii)); - - float sum = NULL_FLOAT; - - LongRingBuffer.Iterator it = ctx.windowIndices.iterator(); - while (it.hasNext()) { - float v = ctx.candidateValuesChunk.get((int)it.next()); - if (v != QueryConstants.NULL_FLOAT) { - if (sum == NULL_FLOAT) { - sum = v; - } else { - sum += v; - } - } - } - localOutputValues.set(ii, sum); + float val = ctx.pairwiseSum.evaluate(); + localOutputValues.set(ii, val); } } } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/PairwiseFloatRingBufferTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/PairwiseFloatRingBufferTest.java new file mode 100644 index 00000000000..c277a5860b0 --- /dev/null +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/PairwiseFloatRingBufferTest.java @@ -0,0 +1,340 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.engine.table.impl.updateby; + +import io.deephaven.engine.table.impl.updateby.internal.PairwiseFloatRingBuffer; +import io.deephaven.test.types.OutOfBandTest; +import junit.framework.TestCase; +import org.junit.experimental.categories.Category; + +import java.util.NoSuchElementException; + +import static io.deephaven.util.QueryConstants.NULL_FLOAT; + +@Category(OutOfBandTest.class) +public class PairwiseFloatRingBufferTest extends TestCase { + + private void assertEmpty(PairwiseFloatRingBuffer rb) { + assertTrue(rb.isEmpty()); + assertEquals(0, rb.size()); + + try { + rb.pop(); + fail("queue should be empty"); + } catch (NoSuchElementException x) { + // expected + } + + try { + rb.element(); + fail("queue should be empty"); + } catch (NoSuchElementException x) { + // expected + } + } + + private void assertFull(PairwiseFloatRingBuffer rb) { + assertFalse(rb.isEmpty()); + assertEquals(rb.capacity(), rb.size()); + } + + private void assertNotEmpty(PairwiseFloatRingBuffer rb, int expectedSize, float expectedHead) { + assertFalse(rb.isEmpty()); + assertEquals(expectedSize, rb.size()); + + assertTrue(expectedHead == rb.peek(Long.MIN_VALUE)); + try { + assertTrue(expectedHead == rb.element()); + } catch (NoSuchElementException x) { + fail("queue should not be empty"); + } + } + + private void assertAdd(PairwiseFloatRingBuffer rb, float newHead, int expectedSize, float expectedHead) { + rb.push(newHead); + assertNotEmpty(rb, expectedSize, expectedHead); + } + + private void assertRemove(PairwiseFloatRingBuffer rb, int expectedSize, float expectedHead) { + assertNotEmpty(rb, expectedSize, expectedHead); + try { + assertTrue(expectedHead == rb.pop()); + } catch (NoSuchElementException x) { + fail("queue should not be empty"); + } + } + + float A = 'A'; + float B = 'B'; + float C = 'C'; + float D = 'D'; + float E = 'E'; + float F = 'F'; + float G = 'G'; + + public void testAddRemove() { + + PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(3, NULL_FLOAT, Float::min); + + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + // take this opportunity to test some front()/back() functions + assertEquals(rb.front(), A); + assertEquals(rb.front(0), A); + assertEquals(rb.front(1), B); + assertEquals(rb.front(2), C); + assertEquals(rb.back(),C); + assertEquals(rb.peekBack(NULL_FLOAT),C); + + assertFull(rb); + + assertRemove(rb, 3, A); + assertRemove(rb, 2, B); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertRemove(rb, 2, A); + assertAdd(rb, C, 2, B); + assertRemove(rb, 2, B); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertRemove(rb, 1, A); + assertEmpty(rb); + assertAdd(rb, B, 1, B); + assertRemove(rb, 1, B); + assertEmpty(rb); + assertAdd(rb, C, 1, C); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, D, 1, D); + assertRemove(rb, 1, D); + assertEmpty(rb); + assertAdd(rb, E, 1, E); + assertRemove(rb, 1, E); + assertEmpty(rb); + assertAdd(rb, F, 1, F); + assertRemove(rb, 1, F); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertFull(rb); + + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + assertAdd(rb, F, 6, A); + + assertRemove(rb, 6, A); + assertRemove(rb, 5, B); + assertRemove(rb, 4, C); + assertRemove(rb, 3, D); + assertRemove(rb, 2, E); + assertRemove(rb, 1, F); + assertEmpty(rb); + } + + public void testGrowSimple() { + PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(5, NULL_FLOAT, Float::min); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + + // this will grow; the elements are in a single contiguous block + assertAdd(rb, F, 6, A); + + assertRemove(rb, 6, A); + assertRemove(rb, 5, B); + assertRemove(rb, 4, C); + assertRemove(rb, 3, D); + assertRemove(rb, 2, E); + assertRemove(rb, 1, F); + assertEmpty(rb); + + rb.pushEmptyValue(); + assertEquals(rb.front(), NULL_FLOAT); + try { + rb.front(-1); + fail("expected a NoSuchElement exception"); + } catch (NoSuchElementException x) { + // expected + } + try { + rb.front(5); + fail("expected a NoSuchElement exception"); + } catch (NoSuchElementException x) { + // expected + } + assertEquals(rb.poll(0.0f), NULL_FLOAT); + } + + public void testGrowComplex() { + PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(5, NULL_FLOAT, Float::min); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertRemove(rb, 3, A); + assertRemove(rb, 2, B); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + + // this will grow; the elements are in two blocks + assertAdd(rb, F, 6, A); + + assertRemove(rb, 6, A); + assertRemove(rb, 5, B); + assertRemove(rb, 4, C); + assertRemove(rb, 3, D); + assertRemove(rb, 2, E); + assertRemove(rb, 1, F); + assertEmpty(rb); + } + + public void testWhenEmpty() { + PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(5, NULL_FLOAT, Float::min); + try { + rb.back(); + fail("expected a NoSuchElement exception"); + } catch (NoSuchElementException x) { + // expected + } + try { + rb.pop(); + fail("expected a NoSuchElement exception"); + } catch (NoSuchElementException x) { + // expected + } + try { + rb.front(); + fail("expected a NoSuchElement exception"); + } catch (NoSuchElementException x) { + // expected + } + assertEquals(rb.poll((float)-1), (float)-1); + assertEquals(rb.peek((float)-1), (float)-1); + assertEquals(rb.peekBack((float)-1), (float)-1); + } + + public void testBack() { + PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(5, NULL_FLOAT, Float::min); + + assertAdd(rb, A, 1, A); + assertEquals(rb.back(), A); + + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertEquals(rb.back(), C); + } + + public void testBackTailIsZero() { + PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(5, NULL_FLOAT, Float::min); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + + assertRemove(rb, 5, A); + assertAdd(rb, F, 5, B); + assertEquals(rb.back(), F); + } + + public void testLargeAmounts() { + try (final PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(3, 0.0f, Float::sum)) { + + // move the head and tail off zero + for (float i = 0; i < 1000; i++) { + rb.push(i); + } + for (float i = 0; i < 1000; i++) { + rb.pop(); + } + + + for (float i = 0; i < 10_000; i++) + rb.push(i); + + for (float i = 10_000; i < 1_000_000; i++) { + rb.push(i); + assertEquals(i - 10_000 + 1, rb.front(1)); + assertEquals(i - 10_000, rb.pop()); + assertEquals(rb.remaining(), rb.capacity() - rb.size()); + } + } + } + + public void testEvaluateMinLargeAmounts() { + try (final PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(3, Float.MAX_VALUE, Float::min)) { + + for (float i = 0; i < 10_000; i++) + rb.push(i); + + for (float i = 10_000; i < 1_000_000; i++) { + rb.push(i); + assertEquals(i - 10_000 + 1, rb.front(1)); + assertEquals(i - 10_000, rb.evaluate()); // front of queue is min + assertEquals(i - 10_000, rb.pop()); + } + } + } + + public void testEvaluateMaxLargeAmounts() { + try (final PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(3, Float.MIN_VALUE, Float::max)) { + for (float i = 0; i < 10_000; i++) + rb.push(i); + + for (float i = 10_000; i < 1_000_000; i++) { + rb.push(i); + assertEquals(i - 10_000 + 1, rb.front(1)); + assertEquals(i, rb.evaluate()); // last value added is max + assertEquals(i - 10_000, rb.pop()); + } + } + } + + public void testEvaluateSumLargeAmounts() { + try (final PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(3, 0.0f, Float::sum)) { + + float runningSum = 0.0f; + + for (float i = 0; i < 1_000; i++) { + rb.push(i); + runningSum += i; + } + + // stay small enough to avoid floating errors but prove the concept + for (float i = 1_000; i < 10_000; i++) { + rb.push(i); + runningSum += i; // add the current value + assertEquals(i - 1_000 + 1, rb.front(1)); + + assertEquals(runningSum, rb.evaluate()); + + assertEquals(i - 1_000, rb.pop()); + runningSum -= i - 1_000; // remove the value 1_0000 ago + + assertEquals(runningSum, rb.evaluate()); + } + } + } +} diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java index 72338357b43..bdf86be1b61 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java @@ -1141,8 +1141,9 @@ private Object[] rollingSumTime(Object[] values, long[] timestamps, final boolea final void assertWithRollingSumTicks(final @NotNull Object expected, final @NotNull Object actual, Class type, int prevTicks, int postTicks) { - final float deltaF = .001f; - final double deltaD = .001d; + // looking for gross errors like missing entries (NOTE: pairwise results are more accurate than true rolling) + final float deltaF = .02f; + final double deltaD = .02d; if (expected instanceof byte[]) { assertArrayEquals(rollingSum((byte[]) expected, prevTicks, postTicks), (long[]) actual); @@ -1169,8 +1170,9 @@ final void assertWithRollingSumTicks(final @NotNull Object expected, final @NotN final void assertWithRollingSumTime(final @NotNull Object expected, final @NotNull Object actual, final @NotNull long[] timestamps, Class type, long prevTime, long postTime) { - final float deltaF = .001f; - final double deltaD = .001d; + // looking for gross errors like missing entries (NOTE: pairwise results are more accurate than true rolling) + final float deltaF = .02f; + final double deltaD = .02d; if (expected instanceof byte[]) { assertArrayEquals(rollingSumTime((byte[]) expected, timestamps, prevTime, postTime), (long[]) actual); diff --git a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java index bed6f65e788..204b440a188 100644 --- a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java +++ b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java @@ -104,6 +104,13 @@ public static void main(String[] args) throws IOException { replicateNumericOperator( "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java", "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java"); + + ReplicatePrimitiveCode.floatToAllFloatingPoints( + "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java", + exemptions); + ReplicatePrimitiveCode.floatToAllFloatingPoints( + "engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/PairwiseFloatRingBufferTest.java", + exemptions); } private static void replicateNumericOperator(@NotNull final String shortClass, @NotNull final String floatClass) From 82a865c3cda46b853bd88ea841a929d8767df164 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 13 Sep 2022 15:28:48 -0700 Subject: [PATCH 021/123] added Double versions of the pairwise structures --- .../internal/PairwiseDoubleRingBuffer.java | 244 +++++++++++++ .../PairwiseDoubleRingBufferTest.java | 345 ++++++++++++++++++ 2 files changed, 589 insertions(+) create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java create mode 100644 engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/PairwiseDoubleRingBufferTest.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java new file mode 100644 index 00000000000..87c46dd558b --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java @@ -0,0 +1,244 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit PairwiseFloatRingBuffer and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +package io.deephaven.engine.table.impl.updateby.internal; + +import io.deephaven.chunk.WritableDoubleChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.util.SafeCloseable; + +import java.util.BitSet; +import java.util.NoSuchElementException; + +/*** + * Store this data in the form of a binary tree where the latter half of the chunk is treated as a ring buffer and pairwise results of the `DoubleFunction` are stored in the parent nodes. We do lazy evaluation by maintaining a 'dirty' `BitSet` and compute the ultimate pairwise result only when requested by `evaluate()' + * + * To keep the parent-node finding math easy and consistent between the ring buffer and the computation tree, the binary tree is shifted by one index so the root (and final result of computation) ends up in index 1 (instead of 0 which is un-used) + */ + +public class PairwiseDoubleRingBuffer implements SafeCloseable { + // use a sized double chunk for underlying storage + private WritableDoubleChunk storageChunk; + private final BitSet dirtyBits; + private final DoubleFunction pairwiseFunction; + private final double emptyVal; + + // this measures internal storage capacity (chunk is twice this size) + private int capacity; + private int chunkSize; + + private int head; + private int tail; + + @FunctionalInterface + public interface DoubleFunction { + /** + * Applies this function to the given arguments. + * + * @param a the first function argument + * @param b the second function argument + * @return the function result + */ + double apply(double a, double b); + } + + /** + * Create a ring buffer for Double values that will perform pairwise evaluation of the internal data values using an efficient binary-tree implementation to compute only changed values. The buffer will grow exponentially as items are pushed into it but will not shrink as values are removed + * + * @param initialSize the minimum size for the structure to hold + * @param emptyVal an innocuous value that will not affect the user-provided function results. for example, 0.0f for performing addition/subtraction, 1.0f for performing multiplication/division + * @param pairwiseFunction the user provided function for evaluation, takes two double parameters and returns a double. This function will be applied repeatedly to pairs of data values until the final result is available + */ + public PairwiseDoubleRingBuffer(int initialSize, double emptyVal, DoubleFunction pairwiseFunction) { + // increase to next power of two + this.capacity = Integer.highestOneBit(initialSize) * 2; + this.chunkSize = capacity * 2; + this.storageChunk = WritableDoubleChunk.makeWritableChunk(chunkSize); + this.dirtyBits = new BitSet(chunkSize); + this.pairwiseFunction = pairwiseFunction; + this.emptyVal = emptyVal; + + this.storageChunk.fillWithValue(0, chunkSize, emptyVal); + this.head = this.tail = this.capacity; + } + + public double evaluate() { + // work through all the dirty bits from high to low until none remain + int bit = chunkSize; + while (!dirtyBits.isEmpty()) { + int nextSetBit = dirtyBits.previousSetBit(bit); + final int left = nextSetBit & 0xFFFFFFFE; // clear the final bit to force evenness + final int right = left + 1; + // this isn't the typical parent = (n-1)/2 because the tree is right-shifted by one + final int parent = left / 2; + + // load the data values + final double leftVal = storageChunk.get(left); + final double rightVal = storageChunk.get(right); + final double parentVal = storageChunk.get(parent); + + dirtyBits.clear(left, right + 1); // clear() excludes `toIndex` so add one to clear `right` as well + + final double computeVal = pairwiseFunction.apply(leftVal, rightVal); + if (parentVal != computeVal) { + storageChunk.set(parent, computeVal); + // mark the parent dirty (if not the last) + if (parent > 1) { + dirtyBits.set(parent); + } + } else { + final int x = 5; + } + bit = left; + } + // final value is in index 1 + return storageChunk.get(1); + } + + private void grow() { + int oldCapacity = capacity; + int oldChunkSize = chunkSize; + + // double the current capacity + capacity *= 2; + chunkSize = capacity * 2; + + // transfer to the new chunk + WritableDoubleChunk oldChunk = storageChunk; + storageChunk = WritableDoubleChunk.makeWritableChunk(chunkSize); + + // fill the pairwise tree (0 to capacity) with empty value + storageChunk.fillWithValue(0, capacity, emptyVal); + + // move the data to the new chunk, note that we store the ring data in the second half of the array + if (tail > head) { + storageChunk.copyFromTypedChunk(oldChunk, head, capacity, tail - head); + tail = capacity + tail - head; + } else { + storageChunk.copyFromTypedChunk(oldChunk, head, capacity, oldChunkSize - head); + storageChunk.copyFromTypedChunk(oldChunk, oldCapacity, oldChunkSize - head + capacity, tail - oldCapacity); + tail = capacity + oldCapacity - 1; + } + // fill the unused storage with the empty value + storageChunk.fillWithValue(tail, chunkSize - tail, emptyVal); + + // free the old data chunk + oldChunk.close(); + head = capacity; + + // TODO: investigate moving precomputed results also. Since we are re-ordering the data values, would be + // tricky to maintain order but a recursive function could probably do it efficiently. For now, make life easy + // by setting all input dirty so the tree is recomputed on next `evaluate()` + dirtyBits.clear(); + dirtyBits.set(head, tail, true); + } + + public void push(double val) { + if (isFull()) { + grow(); + } + // add the new data + storageChunk.set(tail, val); + dirtyBits.set(tail); + // move the tail + tail = ((tail + 1) % capacity) + capacity; + } + + public void pushEmptyValue() { + push(emptyVal); + } + + public double pop() { + if (isEmpty()) { + throw new NoSuchElementException(); + } + double val = storageChunk.get(head); + storageChunk.set(head, emptyVal); + dirtyBits.set(head); + // move the head + head = ((head + 1) % capacity) + capacity; + return val; + } + + public boolean isFull() { + return ((tail + 1) % capacity) + capacity == head; + } + + public int size() { + return tail >= head + ? (tail - head) : + (tail + (capacity - head)); + } + + public boolean isEmpty() { + return tail == head; + } + + public double peek(double onEmpty) { + if (isEmpty()) { + return onEmpty; + } + return storageChunk.get(head); + } + + public double poll(double onEmpty) { + if (isEmpty()) { + return onEmpty; + } + double e = storageChunk.get(head); + head = (head + 1) % capacity + capacity; + return e; + } + + public double front() { + return front(0); + } + + public double front(int offset) { + if (offset < 0 || offset >= size()) { + throw new NoSuchElementException(); + } + return storageChunk.get((head + offset) % capacity + capacity); + } + + public double back() { + if (isEmpty()) { + throw new NoSuchElementException(); + } + return tail == capacity ? storageChunk.get(chunkSize - 1) : storageChunk.get(tail - 1); + } + + public double peekBack(double onEmpty) { + if (isEmpty()) { + return onEmpty; + } + return tail == capacity ? storageChunk.get(chunkSize - 1) : storageChunk.get(tail - 1); + } + + public double element() { + if (isEmpty()) { + throw new NoSuchElementException(); + } + return storageChunk.get(head); + } + + public int capacity() { + return capacity - 1; + } + + public int remaining() { + return capacity() - size(); + } + + @Override + public void close() { + try (final WritableDoubleChunk ignoredChunk = storageChunk) { + // close the closable items + } + } +} diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/PairwiseDoubleRingBufferTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/PairwiseDoubleRingBufferTest.java new file mode 100644 index 00000000000..bd331a66f6f --- /dev/null +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/PairwiseDoubleRingBufferTest.java @@ -0,0 +1,345 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit PairwiseFloatRingBufferTest and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +package io.deephaven.engine.table.impl.updateby; + +import io.deephaven.engine.table.impl.updateby.internal.PairwiseDoubleRingBuffer; +import io.deephaven.test.types.OutOfBandTest; +import junit.framework.TestCase; +import org.junit.experimental.categories.Category; + +import java.util.NoSuchElementException; + +import static io.deephaven.util.QueryConstants.NULL_DOUBLE; + +@Category(OutOfBandTest.class) +public class PairwiseDoubleRingBufferTest extends TestCase { + + private void assertEmpty(PairwiseDoubleRingBuffer rb) { + assertTrue(rb.isEmpty()); + assertEquals(0, rb.size()); + + try { + rb.pop(); + fail("queue should be empty"); + } catch (NoSuchElementException x) { + // expected + } + + try { + rb.element(); + fail("queue should be empty"); + } catch (NoSuchElementException x) { + // expected + } + } + + private void assertFull(PairwiseDoubleRingBuffer rb) { + assertFalse(rb.isEmpty()); + assertEquals(rb.capacity(), rb.size()); + } + + private void assertNotEmpty(PairwiseDoubleRingBuffer rb, int expectedSize, double expectedHead) { + assertFalse(rb.isEmpty()); + assertEquals(expectedSize, rb.size()); + + assertTrue(expectedHead == rb.peek(Long.MIN_VALUE)); + try { + assertTrue(expectedHead == rb.element()); + } catch (NoSuchElementException x) { + fail("queue should not be empty"); + } + } + + private void assertAdd(PairwiseDoubleRingBuffer rb, double newHead, int expectedSize, double expectedHead) { + rb.push(newHead); + assertNotEmpty(rb, expectedSize, expectedHead); + } + + private void assertRemove(PairwiseDoubleRingBuffer rb, int expectedSize, double expectedHead) { + assertNotEmpty(rb, expectedSize, expectedHead); + try { + assertTrue(expectedHead == rb.pop()); + } catch (NoSuchElementException x) { + fail("queue should not be empty"); + } + } + + double A = 'A'; + double B = 'B'; + double C = 'C'; + double D = 'D'; + double E = 'E'; + double F = 'F'; + double G = 'G'; + + public void testAddRemove() { + + PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(3, NULL_DOUBLE, Double::min); + + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + // take this opportunity to test some front()/back() functions + assertEquals(rb.front(), A); + assertEquals(rb.front(0), A); + assertEquals(rb.front(1), B); + assertEquals(rb.front(2), C); + assertEquals(rb.back(),C); + assertEquals(rb.peekBack(NULL_DOUBLE),C); + + assertFull(rb); + + assertRemove(rb, 3, A); + assertRemove(rb, 2, B); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertRemove(rb, 2, A); + assertAdd(rb, C, 2, B); + assertRemove(rb, 2, B); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertRemove(rb, 1, A); + assertEmpty(rb); + assertAdd(rb, B, 1, B); + assertRemove(rb, 1, B); + assertEmpty(rb); + assertAdd(rb, C, 1, C); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, D, 1, D); + assertRemove(rb, 1, D); + assertEmpty(rb); + assertAdd(rb, E, 1, E); + assertRemove(rb, 1, E); + assertEmpty(rb); + assertAdd(rb, F, 1, F); + assertRemove(rb, 1, F); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertFull(rb); + + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + assertAdd(rb, F, 6, A); + + assertRemove(rb, 6, A); + assertRemove(rb, 5, B); + assertRemove(rb, 4, C); + assertRemove(rb, 3, D); + assertRemove(rb, 2, E); + assertRemove(rb, 1, F); + assertEmpty(rb); + } + + public void testGrowSimple() { + PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(5, NULL_DOUBLE, Double::min); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + + // this will grow; the elements are in a single contiguous block + assertAdd(rb, F, 6, A); + + assertRemove(rb, 6, A); + assertRemove(rb, 5, B); + assertRemove(rb, 4, C); + assertRemove(rb, 3, D); + assertRemove(rb, 2, E); + assertRemove(rb, 1, F); + assertEmpty(rb); + + rb.pushEmptyValue(); + assertEquals(rb.front(), NULL_DOUBLE); + try { + rb.front(-1); + fail("expected a NoSuchElement exception"); + } catch (NoSuchElementException x) { + // expected + } + try { + rb.front(5); + fail("expected a NoSuchElement exception"); + } catch (NoSuchElementException x) { + // expected + } + assertEquals(rb.poll(0.0f), NULL_DOUBLE); + } + + public void testGrowComplex() { + PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(5, NULL_DOUBLE, Double::min); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertRemove(rb, 3, A); + assertRemove(rb, 2, B); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + + // this will grow; the elements are in two blocks + assertAdd(rb, F, 6, A); + + assertRemove(rb, 6, A); + assertRemove(rb, 5, B); + assertRemove(rb, 4, C); + assertRemove(rb, 3, D); + assertRemove(rb, 2, E); + assertRemove(rb, 1, F); + assertEmpty(rb); + } + + public void testWhenEmpty() { + PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(5, NULL_DOUBLE, Double::min); + try { + rb.back(); + fail("expected a NoSuchElement exception"); + } catch (NoSuchElementException x) { + // expected + } + try { + rb.pop(); + fail("expected a NoSuchElement exception"); + } catch (NoSuchElementException x) { + // expected + } + try { + rb.front(); + fail("expected a NoSuchElement exception"); + } catch (NoSuchElementException x) { + // expected + } + assertEquals(rb.poll((double)-1), (double)-1); + assertEquals(rb.peek((double)-1), (double)-1); + assertEquals(rb.peekBack((double)-1), (double)-1); + } + + public void testBack() { + PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(5, NULL_DOUBLE, Double::min); + + assertAdd(rb, A, 1, A); + assertEquals(rb.back(), A); + + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertEquals(rb.back(), C); + } + + public void testBackTailIsZero() { + PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(5, NULL_DOUBLE, Double::min); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + + assertRemove(rb, 5, A); + assertAdd(rb, F, 5, B); + assertEquals(rb.back(), F); + } + + public void testLargeAmounts() { + try (final PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(3, 0.0f, Double::sum)) { + + // move the head and tail off zero + for (double i = 0; i < 1000; i++) { + rb.push(i); + } + for (double i = 0; i < 1000; i++) { + rb.pop(); + } + + + for (double i = 0; i < 10_000; i++) + rb.push(i); + + for (double i = 10_000; i < 1_000_000; i++) { + rb.push(i); + assertEquals(i - 10_000 + 1, rb.front(1)); + assertEquals(i - 10_000, rb.pop()); + assertEquals(rb.remaining(), rb.capacity() - rb.size()); + } + } + } + + public void testEvaluateMinLargeAmounts() { + try (final PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(3, Double.MAX_VALUE, Double::min)) { + + for (double i = 0; i < 10_000; i++) + rb.push(i); + + for (double i = 10_000; i < 1_000_000; i++) { + rb.push(i); + assertEquals(i - 10_000 + 1, rb.front(1)); + assertEquals(i - 10_000, rb.evaluate()); // front of queue is min + assertEquals(i - 10_000, rb.pop()); + } + } + } + + public void testEvaluateMaxLargeAmounts() { + try (final PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(3, Double.MIN_VALUE, Double::max)) { + for (double i = 0; i < 10_000; i++) + rb.push(i); + + for (double i = 10_000; i < 1_000_000; i++) { + rb.push(i); + assertEquals(i - 10_000 + 1, rb.front(1)); + assertEquals(i, rb.evaluate()); // last value added is max + assertEquals(i - 10_000, rb.pop()); + } + } + } + + public void testEvaluateSumLargeAmounts() { + try (final PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(3, 0.0f, Double::sum)) { + + double runningSum = 0.0f; + + for (double i = 0; i < 1_000; i++) { + rb.push(i); + runningSum += i; + } + + // stay small enough to avoid doubleing errors but prove the concept + for (double i = 1_000; i < 10_000; i++) { + rb.push(i); + runningSum += i; // add the current value + assertEquals(i - 1_000 + 1, rb.front(1)); + + assertEquals(runningSum, rb.evaluate()); + + assertEquals(i - 1_000, rb.pop()); + runningSum -= i - 1_000; // remove the value 1_0000 ago + + assertEquals(runningSum, rb.evaluate()); + } + } + } +} From 3c9fbb48db25cf38285f097035674d4682259006 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Thu, 15 Sep 2022 09:43:15 -0700 Subject: [PATCH 022/123] refactor the pairwise to use a list of dirty indices instead of bitset --- .../java/io/deephaven/engine/table/Table.java | 2 +- .../engine/rowset/impl/OrderedLongSet.java | 18 +- .../impl/OrderedLongSetBuilderSequential.java | 4 +- .../impl/RspBitmapBuilderSequential.java | 66 +++++++- .../impl/BucketedPartitionedUpdateBy.java | 35 ++-- .../engine/table/impl/QueryTable.java | 6 +- .../engine/table/impl/TableDefaults.java | 2 +- .../engine/table/impl/UncoalescedTable.java | 2 +- .../deephaven/engine/table/impl/UpdateBy.java | 2 +- .../impl/UpdateByCumulativeOperator.java | 33 ++-- .../engine/table/impl/UpdateByOperator.java | 23 +-- .../table/impl/UpdateByOperatorFactory.java | 60 ++++--- .../table/impl/UpdateByWindowedOperator.java | 113 +++++++------ .../engine/table/impl/ZeroKeyUpdateBy.java | 63 ++++--- .../impl/by/PartitionByChunkedOperator.java | 159 ++++++++++++++---- .../ema/BasePrimitiveEMAOperator.java | 13 +- .../updateby/ema/BigDecimalEMAOperator.java | 3 +- .../updateby/ema/BigNumberEMAOperator.java | 6 +- .../internal/PairwiseDoubleRingBuffer.java | 80 +++++++-- .../internal/PairwiseFloatRingBuffer.java | 80 +++++++-- .../BigDecimalRollingSumOperator.java | 64 +++---- .../PairwiseDoubleRingBufferTest.java | 37 ++-- .../updateby/PairwiseFloatRingBufferTest.java | 37 ++-- .../table/impl/updateby/TestRollingSum.java | 61 ++++--- .../src/main/resources/dh-tests.prop | 4 +- .../api/updateby/UpdateByOperation.java | 15 +- .../api/updateby/spec/RollingSumSpec.java | 109 ++++++------ 27 files changed, 733 insertions(+), 364 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 d43d897ca5b..5ce94a55c02 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 @@ -934,7 +934,7 @@ public static AsOfMatchRule of(ReverseAsOfJoinRule rule) { */ @ConcurrentMethod PartitionedTable partitionedAggBy(Collection aggregations, boolean preserveEmpty, - Table initialGroups, @NotNull Collection keyColumnNames); + Table initialGroups, @NotNull Collection keyColumnNames); /** * Convenience method that performs an {@link #aggBy} and wraps the result in a {@link PartitionedTable}. If diff --git a/engine/rowset/src/main/java/io/deephaven/engine/rowset/impl/OrderedLongSet.java b/engine/rowset/src/main/java/io/deephaven/engine/rowset/impl/OrderedLongSet.java index 2ebaa672f09..e4abef7b76e 100644 --- a/engine/rowset/src/main/java/io/deephaven/engine/rowset/impl/OrderedLongSet.java +++ b/engine/rowset/src/main/java/io/deephaven/engine/rowset/impl/OrderedLongSet.java @@ -3,6 +3,7 @@ */ package io.deephaven.engine.rowset.impl; +import io.deephaven.chunk.util.LongChunkIterator; import io.deephaven.configuration.Configuration; import io.deephaven.engine.rowset.RowSequenceFactory; import io.deephaven.engine.rowset.RowSet; @@ -39,7 +40,7 @@ public interface OrderedLongSet { @FinalDefault default OrderedLongSet ixInsert(final LongChunk keys, final int offset, - final int length) { + final int length) { if (length <= 1) { if (length == 0) { return this; @@ -69,7 +70,7 @@ default OrderedLongSet ixInsert(final LongChunk keys, final int @FinalDefault default OrderedLongSet ixRemove(final LongChunk keys, final int offset, - final int length) { + final int length) { if (ixIsEmpty()) { return this; } @@ -190,7 +191,7 @@ default void ixValidate() { * @return A new {@link OrderedLongSet} containing the specified slice of {@code keys} */ static OrderedLongSet fromChunk(final LongChunk keys, final int offset, final int length, - final boolean disposable) { + final boolean disposable) { if (length == 0) { return EMPTY; } @@ -237,13 +238,13 @@ public OrderedLongSet ixInsertRange(final long startKey, final long endKey) { @Override public OrderedLongSet ixInsertSecondHalf(final LongChunk keys, final int offset, - final int length) { + final int length) { return fromChunk(keys, offset, length, false); } @Override public OrderedLongSet ixRemoveSecondHalf(final LongChunk keys, final int offset, - final int length) { + final int length) { throw new IllegalStateException(); } @@ -489,6 +490,13 @@ default void appendOrderedLongSet(final long shiftAmount, final OrderedLongSet i }); } + default void appendOrderedRowKeysChunk(LongChunk chunk, int offset, int length) { + LongChunkIterator it = new LongChunkIterator(chunk, offset, length); + while (it.hasNext()) { + appendKey(it.nextLong()); + } + } + @Override default void accept(final long firstKey, final long lastKey) { appendRange(firstKey, lastKey); diff --git a/engine/rowset/src/main/java/io/deephaven/engine/rowset/impl/OrderedLongSetBuilderSequential.java b/engine/rowset/src/main/java/io/deephaven/engine/rowset/impl/OrderedLongSetBuilderSequential.java index 9869c40feb6..d7c9db247bd 100644 --- a/engine/rowset/src/main/java/io/deephaven/engine/rowset/impl/OrderedLongSetBuilderSequential.java +++ b/engine/rowset/src/main/java/io/deephaven/engine/rowset/impl/OrderedLongSetBuilderSequential.java @@ -3,9 +3,9 @@ */ package io.deephaven.engine.rowset.impl; -import io.deephaven.engine.rowset.impl.singlerange.*; -import io.deephaven.engine.rowset.impl.sortedranges.SortedRanges; import io.deephaven.engine.rowset.impl.rsp.RspBitmap; +import io.deephaven.engine.rowset.impl.singlerange.SingleRange; +import io.deephaven.engine.rowset.impl.sortedranges.SortedRanges; import io.deephaven.util.annotations.TestUseOnly; public class OrderedLongSetBuilderSequential extends RspBitmapBuilderSequential { diff --git a/engine/rowset/src/main/java/io/deephaven/engine/rowset/impl/RspBitmapBuilderSequential.java b/engine/rowset/src/main/java/io/deephaven/engine/rowset/impl/RspBitmapBuilderSequential.java index 336f5be99ef..f3bff5ab81e 100644 --- a/engine/rowset/src/main/java/io/deephaven/engine/rowset/impl/RspBitmapBuilderSequential.java +++ b/engine/rowset/src/main/java/io/deephaven/engine/rowset/impl/RspBitmapBuilderSequential.java @@ -3,8 +3,10 @@ */ package io.deephaven.engine.rowset.impl; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.util.LongChunkIterator; import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.impl.OrderedLongSet; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.rowset.impl.rsp.DisposableRspBitmap; import io.deephaven.engine.rowset.impl.rsp.RspArray; import io.deephaven.engine.rowset.impl.rsp.RspBitmap; @@ -119,6 +121,68 @@ public void appendOrderedLongSet(final long shiftAmount, final OrderedLongSet ix rb.appendShiftedUnsafeNoWriteCheck(shiftAmount, (RspBitmap) ix, acquire); } + @Override + public void appendOrderedRowKeysChunk(LongChunk chunk, int offset, int length) { + if (length == 0) { + return; + } + + if (rb != null) { + appendKeyChunkRb(chunk, offset, length); + } else { + appendKeyChunk(chunk, offset, length); + } + } + + private void appendKeyChunkRb(LongChunk chunk, int offset, int length) { + // flush to the rb before appending + if (pendingStart != -1) { + flushPendingRange(); + } + if (pendingContainerKey != -1) { + flushPendingContainer(); + } + + // single key? + if (length == 1) { + rb.appendUnsafeNoWriteCheck(chunk.get(offset)); + return; + } + + // single range? + final int lastOffsetInclusive = offset + length - 1; + final long first = chunk.get(offset); + final long last = chunk.get(lastOffsetInclusive); + if (last - first + 1 == length) { + rb.appendRangeUnsafeNoWriteCheck(first, last); + return; + } + + rb.addValuesUnsafeNoWriteCheck(chunk, offset, length); + } + + private void appendKeyChunk(LongChunk chunk, int offset, int length) { + // single key? + if (length == 1) { + appendKey(chunk.get(offset)); + return; + } + + // single range? + final int lastOffsetInclusive = offset + length - 1; + final long first = chunk.get(offset); + final long last = chunk.get(lastOffsetInclusive); + if (last - first + 1 == length) { + appendRange(first, last); + return; + } + + final LongChunkIterator it = new LongChunkIterator(chunk, offset, length); + while (it.hasNext()) { + appendKey(it.nextLong()); + } + } + protected void flushPendingRange() { final long pendingStartOnEntry = pendingStart; pendingStart = -1; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java index 19b76c16d5a..10d85035ae3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java @@ -54,12 +54,12 @@ public static Table compute(@NotNull final String description, } protected BucketedPartitionedUpdateBy(@NotNull final String description, - @NotNull final UpdateByOperator[] operators, - @NotNull final QueryTable source, - @NotNull final Map> resultSources, - @NotNull final Collection byColumns, - @NotNull final UpdateByRedirectionContext redirContext, - @NotNull final UpdateByControl control) { + @NotNull final UpdateByOperator[] operators, + @NotNull final QueryTable source, + @NotNull final Map> resultSources, + @NotNull final Collection byColumns, + @NotNull final UpdateByRedirectionContext redirContext, + @NotNull final UpdateByControl control) { super(operators, source, redirContext, control); // create a source-listener that will listen to the source updates and apply the shifts to the output columns @@ -86,13 +86,15 @@ public void onUpdate(@NotNull final TableUpdate upstream) { } } super.onUpdate(upstream); - }}); + } + }); recorders = new LinkedList<>(); listener = newListener(description); // create a listener and recorder for the source table as first entry - BucketedPartitionedUpdateByListenerRecorder recorder = new BucketedPartitionedUpdateByListenerRecorder(description, source, resultTable); + BucketedPartitionedUpdateByListenerRecorder recorder = + new BucketedPartitionedUpdateByListenerRecorder(description, source, resultTable); recorder.setMergedListener(listener); source.listenForUpdates(recorder); @@ -115,9 +117,9 @@ public void onUpdate(@NotNull final TableUpdate upstream) { final PartitionedTable pt = sourceListenerTable.partitionedAggBy(List.of(), true, null, byColumns); final PartitionedTable transformed = pt.transform(t -> { // create the table - Table newTable = ZeroKeyUpdateBy.compute( + Table newTable = ZeroKeyUpdateBy.compute( description, - (QueryTable)t, + (QueryTable) t, operators, resultSources, redirContext, @@ -125,7 +127,8 @@ public void onUpdate(@NotNull final TableUpdate upstream) { false); if (listener != null) { - BucketedPartitionedUpdateByListenerRecorder recorder = new BucketedPartitionedUpdateByListenerRecorder(description, newTable, resultTable); + BucketedPartitionedUpdateByListenerRecorder recorder = + new BucketedPartitionedUpdateByListenerRecorder(description, newTable, resultTable); recorder.setMergedListener(listener); newTable.listenForUpdates(recorder); @@ -149,10 +152,12 @@ private final class BucketedPartitionedUpdateByListenerRecorder extends Listener private final ModifiedColumnSet.Transformer modifiedColumnsTransformer; - BucketedPartitionedUpdateByListenerRecorder(@NotNull String description, @NotNull final Table constituent, @NotNull final Table dependent) { + BucketedPartitionedUpdateByListenerRecorder(@NotNull String description, @NotNull final Table constituent, + @NotNull final Table dependent) { super(description, constituent, dependent); - modifiedColumnsTransformer = ((QueryTable) constituent).newModifiedColumnSetTransformer((QueryTable)dependent, constituent.getDefinition().getColumnNamesArray()); - } + modifiedColumnsTransformer = ((QueryTable) constituent).newModifiedColumnSetTransformer( + (QueryTable) dependent, constituent.getDefinition().getColumnNamesArray()); + } } /** @@ -184,7 +189,7 @@ protected void process() { ListenerRecorder sourceRecorder = recorders.peekFirst(); downstream.added = sourceRecorder.getAdded().copy(); downstream.removed = sourceRecorder.getRemoved().copy(); - downstream.shifted = sourceRecorder.getShifted(); + downstream.shifted = sourceRecorder.getShifted(); // union the modifies from all the tables (including source) downstream.modifiedColumnSet = resultTable.getModifiedColumnSetForUpdates(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java index c5802827718..9c9fa141a92 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java @@ -457,7 +457,8 @@ public Object[] getRecord(long rowNo, String... columnNames) { } @Override - public PartitionedTable partitionBy(final boolean dropKeys, @NotNull final Collection columns) { + public PartitionedTable partitionBy(final boolean dropKeys, + @NotNull final Collection columns) { if (isStream()) { throw streamUnsupported("partitionBy"); } @@ -479,7 +480,8 @@ public PartitionedTable partitionBy(final boolean dropKeys, @NotNull final Colle @Override public PartitionedTable partitionedAggBy(final Collection aggregations, - final boolean preserveEmpty, @Nullable final Table initialGroups, @NotNull final Collection columns) { + final boolean preserveEmpty, @Nullable final Table initialGroups, + @NotNull final Collection columns) { if (isStream()) { throw streamUnsupported("partitionedAggBy"); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/TableDefaults.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/TableDefaults.java index 7df408e5fb4..32b2568b175 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/TableDefaults.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/TableDefaults.java @@ -593,7 +593,7 @@ default PartitionedTable partitionBy(String... keyColumnNames) { @Override @ConcurrentMethod default PartitionedTable partitionedAggBy(final Collection aggregations, - final boolean preserveEmpty, @Nullable final Table initialGroups, String... keyColumnNames) { + final boolean preserveEmpty, @Nullable final Table initialGroups, String... keyColumnNames) { return partitionedAggBy(aggregations, preserveEmpty, initialGroups, ColumnName.from(keyColumnNames)); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UncoalescedTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UncoalescedTable.java index bd5e9272f01..6693d72f1b6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UncoalescedTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UncoalescedTable.java @@ -386,7 +386,7 @@ public PartitionedTable partitionBy(boolean dropKeys, @NotNull Collection aggregations, boolean preserveEmpty, - Table initialGroups, String... keyColumnNames) { + Table initialGroups, String... keyColumnNames) { return coalesce().partitionedAggBy(aggregations, preserveEmpty, initialGroups, keyColumnNames); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java index c044e50e395..d4244472218 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java @@ -108,7 +108,7 @@ public void processUpdateForRedirection(@NotNull final TableUpdate upstream, fin } private boolean shiftRedirectedKey(@NotNull final RowSet.SearchIterator iterator, final long delta, - final long key) { + final long key) { final long inner = rowRedirection.remove(key); if (inner != NULL_ROW_KEY) { rowRedirection.put(key + delta, inner); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java index a7becf8f7da..77e56be6cbd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java @@ -35,10 +35,10 @@ public LongSegmentedSortedArray getTimestampSsa() { * @return the smallest key that participated in any part of the update. */ long smallestAffectedKey(@NotNull final RowSet added, - @NotNull final RowSet modified, - @NotNull final RowSet removed, - @NotNull final RowSetShiftData shifted, - @NotNull final RowSet affectedIndex) { + @NotNull final RowSet modified, + @NotNull final RowSet removed, + @NotNull final RowSetShiftData shifted, + @NotNull final RowSet affectedIndex) { long smallestModifiedKey = Long.MAX_VALUE; if (removed.isNonempty()) { @@ -58,7 +58,8 @@ long smallestAffectedKey(@NotNull final RowSet added, boolean modShiftFound = !modified.isEmpty(); boolean affectedFound = false; try (final RowSequence.Iterator it = affectedIndex.getRowSequenceIterator()) { - for (int shiftIdx = 0; shiftIdx < shifted.size() && (!modShiftFound || !affectedFound); shiftIdx++) { + for (int shiftIdx = 0; shiftIdx < shifted.size() + && (!modShiftFound || !affectedFound); shiftIdx++) { final long shiftStart = shifted.getBeginRange(shiftIdx); final long shiftEnd = shifted.getEndRange(shiftIdx); final long shiftDelta = shifted.getShiftDelta(shiftIdx); @@ -97,14 +98,17 @@ long smallestAffectedKey(@NotNull final RowSet added, return smallestModifiedKey; } - public RowSet determineAffectedRows(@NotNull final TableUpdate upstream, @NotNull final TrackingRowSet source, final boolean initialStep) { - Assert.assertion(affectedRows==null, "affectedRows should be null when determineAffectedRows() is called"); + public RowSet determineAffectedRows(@NotNull final TableUpdate upstream, @NotNull final TrackingRowSet source, + final boolean initialStep) { + Assert.assertion(affectedRows == null, + "affectedRows should be null when determineAffectedRows() is called"); if (initialStep) { affectedRows = source.copy(); return affectedRows; } - long smallestModifiedKey = smallestAffectedKey(upstream.added(), upstream.modified(), upstream.removed(), upstream.shifted(), source); + long smallestModifiedKey = smallestAffectedKey(upstream.added(), upstream.modified(), upstream.removed(), + upstream.shifted(), source); affectedRows = smallestModifiedKey == Long.MAX_VALUE ? RowSetFactory.empty() @@ -121,7 +125,7 @@ public RowSet getInfluencerRows() { } public RowSetBuilderSequential getModifiedBuilder() { - if(modifiedBuilder == null) { + if (modifiedBuilder == null) { modifiedBuilder = RowSetFactory.builderSequential(); } return modifiedBuilder; @@ -130,31 +134,30 @@ public RowSetBuilderSequential getModifiedBuilder() { @Override public void close() { try (final RowSet ignored = affectedRows; - final RowSet ignored2 = newModified) { + final RowSet ignored2 = newModified) { } } } @Override public void initializeFor(@NotNull final UpdateContext context, - @NotNull final RowSet updateRowSet) { - } + @NotNull final RowSet updateRowSet) {} @Override public void finishFor(@NotNull final UpdateContext context) { - UpdateCumulativeContext ctx = (UpdateCumulativeContext)context; + UpdateCumulativeContext ctx = (UpdateCumulativeContext) context; ctx.newModified = ctx.getModifiedBuilder().build(); } @NotNull final public RowSet getAdditionalModifications(@NotNull final UpdateContext context) { - UpdateCumulativeContext ctx = (UpdateCumulativeContext)context; + UpdateCumulativeContext ctx = (UpdateCumulativeContext) context; return ctx.newModified; } @Override final public boolean anyModified(@NotNull final UpdateContext context) { - UpdateCumulativeContext ctx = (UpdateCumulativeContext)context; + UpdateCumulativeContext ctx = (UpdateCumulativeContext) context; return ctx.newModified != null && ctx.newModified.isNonempty(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java index ab136b67c0f..88d0bd148c4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java @@ -72,7 +72,7 @@ interface UpdateContext extends SafeCloseable { * @param source the rowset of the parent table (affected rows will be a subset) */ RowSet determineAffectedRows(@NotNull final TableUpdate upstream, @NotNull final TrackingRowSet source, - final boolean initialStep); + final boolean initialStep); /** * Return the affected rows computed by the {@Code determineAffectedRows()} @@ -209,8 +209,8 @@ void initializeFor(@NotNull final UpdateContext context, boolean requiresValues(@NotNull final UpdateContext context); /** - * Set the chunk size to be used for operations. This is used during the processing phase - * when the chunks allocated during the normal processing phase may not be large enough. + * Set the chunk size to be used for operations. This is used during the processing phase when the chunks allocated + * during the normal processing phase may not be large enough. * * @param context the context object * @param chunkSize the new chunk size @@ -231,16 +231,17 @@ void applyOutputShift(@NotNull final UpdateContext context, * @param context the context object * @param inputKeys the keys contained in the chunk * @param keyChunk a {@link LongChunk} containing the keys if requested by {@link #requiresKeys()} or null. - * @param posChunk a {@link LongChunk} containing the positions if requested by {@link #requiresPositions()} or null. + * @param posChunk a {@link LongChunk} containing the positions if requested by {@link #requiresPositions()} or + * null. * @param valuesChunk the current chunk of working values. * @param postUpdateSourceIndex the resulting source index af */ void processChunk(@NotNull final UpdateContext context, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @Nullable final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex); + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @Nullable final LongChunk posChunk, + @Nullable final Chunk valuesChunk, + @NotNull final RowSet postUpdateSourceIndex); /** * Reset the operator to the state at the `firstModifiedKey` for non-bucketed operation. This is invoked immediately @@ -253,6 +254,6 @@ void processChunk(@NotNull final UpdateContext context, * @param firstUnmodifiedKey the first unmodified key after which we will reprocess rows. */ void resetForProcess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceIndex, - final long firstUnmodifiedKey); + @NotNull final RowSet sourceIndex, + final long firstUnmodifiedKey); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java index 21518522a97..1181d0ed131 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java @@ -241,28 +241,36 @@ private UpdateByOperator makeEmaOperator(@NotNull final MatchPair pair, final long timeScaleUnits = ema.timeScale().timescaleUnits(); if (csType == byte.class || csType == Byte.class) { - return new ByteEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, ema.timeScale().timestampCol(), timeScaleUnits, + return new ByteEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, + ema.timeScale().timestampCol(), timeScaleUnits, columnSource, redirContext); } else if (csType == short.class || csType == Short.class) { - return new ShortEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, ema.timeScale().timestampCol(), timeScaleUnits, + return new ShortEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, + ema.timeScale().timestampCol(), timeScaleUnits, columnSource, redirContext); } else if (csType == int.class || csType == Integer.class) { - return new IntEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, ema.timeScale().timestampCol(), timeScaleUnits, + return new IntEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, + ema.timeScale().timestampCol(), timeScaleUnits, columnSource, redirContext); } else if (csType == long.class || csType == Long.class) { - return new LongEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, ema.timeScale().timestampCol(), timeScaleUnits, + return new LongEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, + ema.timeScale().timestampCol(), timeScaleUnits, columnSource, redirContext); } else if (csType == float.class || csType == Float.class) { - return new FloatEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, ema.timeScale().timestampCol(), timeScaleUnits, + return new FloatEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, + ema.timeScale().timestampCol(), timeScaleUnits, columnSource, redirContext); } else if (csType == double.class || csType == Double.class) { - return new DoubleEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, ema.timeScale().timestampCol(), + return new DoubleEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, + ema.timeScale().timestampCol(), timeScaleUnits, columnSource, redirContext); } else if (csType == BigDecimal.class) { - return new BigDecimalEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, ema.timeScale().timestampCol(), + return new BigDecimalEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, + ema.timeScale().timestampCol(), timeScaleUnits, columnSource, redirContext); } else if (csType == BigInteger.class) { - return new BigIntegerEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, ema.timeScale().timestampCol(), + return new BigIntegerEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, + ema.timeScale().timestampCol(), timeScaleUnits, columnSource, redirContext); } @@ -382,8 +390,8 @@ private UpdateByOperator makeForwardFillOperator(MatchPair fc, TableDefaults sou } private UpdateByOperator makeRollingSumOperator(@NotNull final MatchPair pair, - @NotNull final TableDefaults source, - @NotNull final RollingSumSpec rs) { + @NotNull final TableDefaults source, + @NotNull final RollingSumSpec rs) { // noinspection rawtypes final ColumnSource columnSource = source.getColumnSource(pair.rightColumn); final ColumnSource timestampColumnSource; @@ -402,31 +410,41 @@ private UpdateByOperator makeRollingSumOperator(@NotNull final MatchPair pair, final long fwdTimeScaleUnits = rs.fwdTimeScale().timescaleUnits(); if (csType == Boolean.class || csType == boolean.class) { - return new ByteRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), timestampColumnSource, prevTimeScaleUnits, fwdTimeScaleUnits, + return new ByteRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), + rs.prevTimeScale().timestampCol(), timestampColumnSource, prevTimeScaleUnits, fwdTimeScaleUnits, redirContext, ReinterpretUtils.booleanToByteSource(columnSource), NULL_BOOLEAN_AS_BYTE); } else if (csType == byte.class || csType == Byte.class) { - return new ByteRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), timestampColumnSource, prevTimeScaleUnits, fwdTimeScaleUnits, - redirContext, columnSource, NULL_BYTE); + return new ByteRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), + rs.prevTimeScale().timestampCol(), timestampColumnSource, prevTimeScaleUnits, fwdTimeScaleUnits, + redirContext, columnSource, NULL_BYTE); } else if (csType == short.class || csType == Short.class) { - return new ShortRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), timestampColumnSource, prevTimeScaleUnits, fwdTimeScaleUnits, + return new ShortRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), + rs.prevTimeScale().timestampCol(), timestampColumnSource, prevTimeScaleUnits, fwdTimeScaleUnits, redirContext, columnSource); } else if (csType == int.class || csType == Integer.class) { - return new IntRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), timestampColumnSource, prevTimeScaleUnits, fwdTimeScaleUnits, + return new IntRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), + rs.prevTimeScale().timestampCol(), timestampColumnSource, prevTimeScaleUnits, fwdTimeScaleUnits, redirContext, columnSource); } else if (csType == long.class || csType == Long.class) { - return new LongRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), timestampColumnSource, prevTimeScaleUnits, fwdTimeScaleUnits, + return new LongRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), + rs.prevTimeScale().timestampCol(), timestampColumnSource, prevTimeScaleUnits, fwdTimeScaleUnits, redirContext, columnSource); } else if (csType == float.class || csType == Float.class) { - return new FloatRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), timestampColumnSource, prevTimeScaleUnits, fwdTimeScaleUnits, + return new FloatRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), + rs.prevTimeScale().timestampCol(), timestampColumnSource, prevTimeScaleUnits, fwdTimeScaleUnits, redirContext, columnSource); } else if (csType == double.class || csType == Double.class) { - return new DoubleRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), timestampColumnSource, + return new DoubleRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), + rs.prevTimeScale().timestampCol(), timestampColumnSource, prevTimeScaleUnits, fwdTimeScaleUnits, redirContext, columnSource); } else if (csType == BigDecimal.class) { - return new BigDecimalRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), timestampColumnSource, - prevTimeScaleUnits, fwdTimeScaleUnits, redirContext, columnSource, control.mathContextOrDefault()); + return new BigDecimalRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), + rs.prevTimeScale().timestampCol(), timestampColumnSource, + prevTimeScaleUnits, fwdTimeScaleUnits, redirContext, columnSource, + control.mathContextOrDefault()); } else if (csType == BigInteger.class) { - return new BigIntegerRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), timestampColumnSource, + return new BigIntegerRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), + rs.prevTimeScale().timestampCol(), timestampColumnSource, prevTimeScaleUnits, fwdTimeScaleUnits, redirContext, columnSource); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java index 4500964e7c4..dcc24e13f4c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java @@ -57,9 +57,9 @@ public RowSetBuilderSequential getModifiedBuilder() { // store a local copy of the source rowset (may not be needed) public RowSet sourceRowSet = null; - // there are two sets of rows we will be tracking. `affected` rows need to be recomputed because of this + // there are two sets of rows we will be tracking. `affected` rows need to be recomputed because of this // update and `influencer` rows contain the data that will be used to compute the new values for the `affected` - // items. Because the windows are user-configurable, there may be no overlap between these two sets and we + // items. Because the windows are user-configurable, there may be no overlap between these two sets and we // don't need values for the `affected` rows at all protected RowSet affectedRows; protected RowSet influencerRows; @@ -83,19 +83,21 @@ public RowSetBuilderSequential getModifiedBuilder() { protected LongRingBuffer windowPosOrTimestamp = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); public LongRingBuffer windowIndices = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); - private WritableRowSet computeAffectedRowsTime(final RowSet sourceSet, final RowSet subset, long revNanos, long fwdNanos) { + private WritableRowSet computeAffectedRowsTime(final RowSet sourceSet, final RowSet subset, long revNanos, + long fwdNanos) { // swap fwd/rev to get the affected windows return computeInfluencerRowsTime(sourceSet, subset, fwdNanos, revNanos); } - private WritableRowSet computeInfluencerRowsTime(final RowSet sourceSet, final RowSet subset, long revNanos, long fwdNanos) { + private WritableRowSet computeInfluencerRowsTime(final RowSet sourceSet, final RowSet subset, long revNanos, + long fwdNanos) { if (sourceSet.size() == subset.size()) { return sourceSet.copy(); } int chunkSize = (int) Math.min(subset.size(), 4096); try (final RowSequence.Iterator it = subset.getRowSequenceIterator(); - final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(chunkSize)) { + final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(chunkSize)) { final RowSetBuilderSequential builder = RowSetFactory.builderSequential(); LongSegmentedSortedArray.Iterator ssaIt = timestampSsa.iterator(false, false); while (it.hasMore() && ssaIt.hasNext()) { @@ -103,7 +105,8 @@ private WritableRowSet computeInfluencerRowsTime(final RowSet sourceSet, final R LongChunk timestamps = timestampColumnSource.getChunk(context, rs).asLongChunk(); for (int ii = 0; ii < rs.intSize(); ii++) { - // if the timestamp of the row is null, it won't belong to any set and we can ignore it completely + // if the timestamp of the row is null, it won't belong to any set and we can ignore it + // completely final long ts = timestamps.get(ii); if (ts != NULL_LONG) { // look at every row timestamp, compute the head and tail in nanos @@ -119,7 +122,8 @@ private WritableRowSet computeInfluencerRowsTime(final RowSet sourceSet, final R } } - Assert.assertion(ssaIt.hasNext() && ssaIt.nextValue() >= head, "SSA Iterator outside of window"); + Assert.assertion(ssaIt.hasNext() && ssaIt.nextValue() >= head, + "SSA Iterator outside of window"); // step through the SSA and collect keys until outside of the window while (ssaIt.hasNext() && ssaIt.nextValue() <= tail) { @@ -138,12 +142,14 @@ private WritableRowSet computeInfluencerRowsTime(final RowSet sourceSet, final R } } - private WritableRowSet computeAffectedRowsTicks(final RowSet sourceSet, final RowSet subset, long revTicks, long fwdTicks) { + private WritableRowSet computeAffectedRowsTicks(final RowSet sourceSet, final RowSet subset, long revTicks, + long fwdTicks) { // swap fwd/rev to get the influencer windows return computeInfluencerRowsTicks(sourceSet, subset, fwdTicks, revTicks); } - private WritableRowSet computeInfluencerRowsTicks(final RowSet sourceSet, final RowSet subset, long revTicks, long fwdTicks) { + private WritableRowSet computeInfluencerRowsTicks(final RowSet sourceSet, final RowSet subset, long revTicks, + long fwdTicks) { if (sourceSet.size() == subset.size()) { return sourceSet.copy(); } @@ -162,23 +168,23 @@ private WritableRowSet computeInfluencerRowsTicks(final RowSet sourceSet, final }); try (final RowSet positions = builder.build()) { - return sourceSet.subSetForPositions(positions); + return sourceSet.subSetForPositions(positions); } } } /*** - * This function is only correct if the proper {@code source} rowset is provided. If using buckets, then the - * provided rowset must be limited to the rows in the current bucket - * only + * This function is only correct if the proper {@code source} rowset is provided. If using buckets, then the + * provided rowset must be limited to the rows in the current bucket only * * @param upstream the update * @param source the rowset of the parent table (affected rows will be a subset) * @param initialStep whether this is the initial step of building the table */ public RowSet determineAffectedRows(@NotNull final TableUpdate upstream, @NotNull final TrackingRowSet source, - final boolean initialStep) { - Assert.assertion(affectedRows==null, "affectedRows should be null when determineAffectedRows() is called"); + final boolean initialStep) { + Assert.assertion(affectedRows == null, + "affectedRows should be null when determineAffectedRows() is called"); if (initialStep) { // all rows are affected initially @@ -220,9 +226,11 @@ public RowSet determineAffectedRows(@NotNull final TableUpdate upstream, @NotNul if (upstream.removed().isNonempty()) { try (final RowSet prev = source.copyPrev(); - final WritableRowSet affectedByRemoves = timestampColumnName == null - ? computeAffectedRowsTicks(prev, upstream.removed(), reverseTimeScaleUnits, forwardTimeScaleUnits) - : computeAffectedRowsTime(prev, upstream.removed(), reverseTimeScaleUnits, forwardTimeScaleUnits)) { + final WritableRowSet affectedByRemoves = timestampColumnName == null + ? computeAffectedRowsTicks(prev, upstream.removed(), reverseTimeScaleUnits, + forwardTimeScaleUnits) + : computeAffectedRowsTime(prev, upstream.removed(), reverseTimeScaleUnits, + forwardTimeScaleUnits)) { // apply shifts to get back to pos-shift space upstream.shifted().apply(affectedByRemoves); // retain only the rows that still exist in the source @@ -236,12 +244,14 @@ public RowSet determineAffectedRows(@NotNull final TableUpdate upstream, @NotNul // now get influencer rows for the affected if (timestampColumnName == null) { - influencerRows = computeInfluencerRowsTicks(source, affectedRows, reverseTimeScaleUnits, forwardTimeScaleUnits); + influencerRows = + computeInfluencerRowsTicks(source, affectedRows, reverseTimeScaleUnits, forwardTimeScaleUnits); // generate position data rowsets for efficiently computed position offsets affectedRowPositions = source.invert(affectedRows); influencerPositions = source.invert(influencerRows); } else { - influencerRows = computeInfluencerRowsTime(source, affectedRows, reverseTimeScaleUnits, forwardTimeScaleUnits); + influencerRows = + computeInfluencerRowsTime(source, affectedRows, reverseTimeScaleUnits, forwardTimeScaleUnits); } return affectedRows; } @@ -263,7 +273,7 @@ public void fillWindowTicks(UpdateWindowedContext context, long currentPos) { // pop out all values from the current window that are not in the new window while (!windowPosOrTimestamp.isEmpty() && windowPosOrTimestamp.front() < head) { - pop(context, windowKeys.remove(), (int)windowIndices.remove()); + pop(context, windowKeys.remove(), (int) windowIndices.remove()); windowPosOrTimestamp.remove(); } @@ -273,7 +283,7 @@ public void fillWindowTicks(UpdateWindowedContext context, long currentPos) { } // skip values until they match the window - while(currentInfluencerPosOrTimestamp < head) { + while (currentInfluencerPosOrTimestamp < head) { currentInfluencerIndex++; if (currentInfluencerIndex < influencerPosChunk.get().size()) { @@ -286,7 +296,7 @@ public void fillWindowTicks(UpdateWindowedContext context, long currentPos) { } // push matching values - while(currentInfluencerPosOrTimestamp <= tail) { + while (currentInfluencerPosOrTimestamp <= tail) { push(context, currentInfluencerKey, currentInfluencerIndex); windowKeys.add(currentInfluencerKey); windowPosOrTimestamp.add(currentInfluencerPosOrTimestamp); @@ -310,7 +320,7 @@ public void fillWindowTime(UpdateWindowedContext context, long currentTimestamp) // pop out all values from the current window that are not in the new window while (!windowPosOrTimestamp.isEmpty() && windowPosOrTimestamp.front() < head) { - pop(context, windowKeys.remove(), (int)windowIndices.remove()); + pop(context, windowKeys.remove(), (int) windowIndices.remove()); windowPosOrTimestamp.remove(); } @@ -320,7 +330,7 @@ public void fillWindowTime(UpdateWindowedContext context, long currentTimestamp) } // skip values until they match the window - while(currentInfluencerPosOrTimestamp < head) { + while (currentInfluencerPosOrTimestamp < head) { currentInfluencerIndex++; if (currentInfluencerIndex < influencerTimestampChunk.get().size()) { @@ -333,7 +343,7 @@ public void fillWindowTime(UpdateWindowedContext context, long currentTimestamp) } // push matching values - while(currentInfluencerPosOrTimestamp <= tail) { + while (currentInfluencerPosOrTimestamp <= tail) { push(context, currentInfluencerKey, currentInfluencerIndex); windowKeys.add(currentInfluencerKey); windowPosOrTimestamp.add(currentInfluencerPosOrTimestamp); @@ -353,13 +363,13 @@ public void fillWindowTime(UpdateWindowedContext context, long currentTimestamp) @Override public void close() { try (final SizedLongChunk ignoredChk1 = influencerKeyChunk; - final SizedLongChunk ignoredChk2 = influencerPosChunk; - final SizedLongChunk ignoredChk3 = influencerTimestampChunk; - final RowSet ignoredRs1 = affectedRows; - final RowSet ignoredRs2 = influencerRows; - final RowSet ignoredRs3 = affectedRowPositions; - final RowSet ignoredRs4 = influencerPositions; - final RowSet ignoredRs5 = newModified) { + final SizedLongChunk ignoredChk2 = influencerPosChunk; + final SizedLongChunk ignoredChk3 = influencerTimestampChunk; + final RowSet ignoredRs1 = affectedRows; + final RowSet ignoredRs2 = influencerRows; + final RowSet ignoredRs3 = affectedRowPositions; + final RowSet ignoredRs4 = influencerPositions; + final RowSet ignoredRs5 = newModified) { } } } @@ -370,34 +380,37 @@ public void close() { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this operation * @param control the control parameters for operation - * @param timestampColumnName the optional time stamp column for windowing (uses ticks if not provided) + * @param timestampColumnName the optional time stamp column for windowing (uses ticks if not provided) * @param redirContext the row redirection context to use for the operation */ public UpdateByWindowedOperator(@NotNull final MatchPair pair, - @NotNull final String[] affectingColumns, - @NotNull final OperationControl control, - @Nullable final String timestampColumnName, - @Nullable final ColumnSource timestampColumnSource, - final long reverseTimeScaleUnits, - final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { + @NotNull final String[] affectingColumns, + @NotNull final OperationControl control, + @Nullable final String timestampColumnName, + @Nullable final ColumnSource timestampColumnSource, + final long reverseTimeScaleUnits, + final long forwardTimeScaleUnits, + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { this.pair = pair; this.affectingColumns = affectingColumns; this.control = control; this.timestampColumnName = timestampColumnName; - this.timestampColumnSource = timestampColumnSource == null ? null : ReinterpretUtils.maybeConvertToPrimitive(timestampColumnSource); + this.timestampColumnSource = + timestampColumnSource == null ? null : ReinterpretUtils.maybeConvertToPrimitive(timestampColumnSource); this.reverseTimeScaleUnits = reverseTimeScaleUnits; this.forwardTimeScaleUnits = forwardTimeScaleUnits; this.redirContext = redirContext; } public abstract void push(UpdateContext context, long key, int pos); + public abstract void pop(UpdateContext context, long key, int pos); + public abstract void reset(UpdateContext context); @Override public void initializeFor(@NotNull final UpdateContext context, - @NotNull final RowSet updateRowSet) { + @NotNull final RowSet updateRowSet) { final UpdateWindowedContext ctx = (UpdateWindowedContext) context; // pre=load all the influencer values this update will need @@ -415,8 +428,10 @@ public void initializeFor(@NotNull final UpdateContext context, ctx.currentInfluencerPosOrTimestamp = ctx.influencerPositions.firstRowKey(); } else { ctx.influencerTimestampChunk = new SizedLongChunk<>(ctx.influencerRows.intSize()); - try (final ChunkSource.FillContext fillContext = timestampColumnSource.makeFillContext(ctx.influencerRows.intSize())) { - timestampColumnSource.fillChunk(fillContext, (WritableChunk) ctx.influencerTimestampChunk.get(), ctx.influencerRows); + try (final ChunkSource.FillContext fillContext = + timestampColumnSource.makeFillContext(ctx.influencerRows.intSize())) { + timestampColumnSource.fillChunk(fillContext, + (WritableChunk) ctx.influencerTimestampChunk.get(), ctx.influencerRows); } ctx.currentInfluencerPosOrTimestamp = ctx.influencerTimestampChunk.get().get(0); } @@ -425,19 +440,19 @@ public void initializeFor(@NotNull final UpdateContext context, @Override public void finishFor(@NotNull final UpdateContext context) { - UpdateWindowedContext ctx = (UpdateWindowedContext)context; + UpdateWindowedContext ctx = (UpdateWindowedContext) context; ctx.newModified = ctx.getModifiedBuilder().build(); } @NotNull final public RowSet getAdditionalModifications(@NotNull final UpdateContext context) { - UpdateWindowedContext ctx = (UpdateWindowedContext)context; + UpdateWindowedContext ctx = (UpdateWindowedContext) context; return ctx.newModified; } @Override final public boolean anyModified(@NotNull final UpdateContext context) { - UpdateWindowedContext ctx = (UpdateWindowedContext)context; + UpdateWindowedContext ctx = (UpdateWindowedContext) context; return ctx.newModified != null && ctx.newModified.isNonempty(); } @@ -456,7 +471,7 @@ public String[] getAffectingColumnNames() { @NotNull @Override public String[] getOutputColumnNames() { - return new String[] { pair.leftColumn }; + return new String[] {pair.leftColumn}; } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java index 5f4f85b2ffc..e87647ec4a1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java @@ -80,7 +80,8 @@ protected ZeroKeyUpdateBy(@NotNull final UpdateByOperator[] operators, if (timestampColumnName != null) { this.timestampSsa = new LongSegmentedSortedArray(4096); - this.timestampColumn = ReinterpretUtils.maybeConvertToPrimitive(source.getColumnSource(this.timestampColumnName)); + this.timestampColumn = + ReinterpretUtils.maybeConvertToPrimitive(source.getColumnSource(this.timestampColumnName)); this.timestampColumnSet = source.newModifiedColumnSet(timestampColumnName); } else { this.timestampSsa = null; @@ -111,18 +112,19 @@ private void processUpdateForSsa(TableUpdate upstream) { // removes if (restampRemovals.isNonempty()) { - final int size = (int)Math.min(restampRemovals.size(), 4096); + final int size = (int) Math.min(restampRemovals.size(), 4096); try (final RowSequence.Iterator it = restampRemovals.getRowSequenceIterator(); - final ChunkSource.GetContext context = timestampColumn.makeGetContext(size); - final WritableLongChunk ssaValues = WritableLongChunk.makeWritableChunk(size); - final WritableLongChunk ssaKeys = WritableLongChunk.makeWritableChunk(size)) { + final ChunkSource.GetContext context = timestampColumn.makeGetContext(size); + final WritableLongChunk ssaValues = WritableLongChunk.makeWritableChunk(size); + final WritableLongChunk ssaKeys = WritableLongChunk.makeWritableChunk(size)) { MutableLong lastTimestamp = new MutableLong(NULL_LONG); while (it.hasMore()) { RowSequence chunkRs = it.getNextRowSequenceWithLength(4096); // get the chunks for values and keys - LongChunk valuesChunk = timestampColumn.getPrevChunk(context, chunkRs).asLongChunk(); + LongChunk valuesChunk = + timestampColumn.getPrevChunk(context, chunkRs).asLongChunk(); LongChunk keysChunk = chunkRs.asRowKeyChunk(); // push only non-null values/keys into the Ssa @@ -134,21 +136,24 @@ private void processUpdateForSsa(TableUpdate upstream) { // shifts if (upstream.shifted().nonempty()) { - final int size = Math.max(upstream.modified().intSize() + Math.max(upstream.added().intSize(), upstream.removed().intSize()), (int) upstream.shifted().getEffectiveSize()); + final int size = Math.max( + upstream.modified().intSize() + Math.max(upstream.added().intSize(), upstream.removed().intSize()), + (int) upstream.shifted().getEffectiveSize()); try (final RowSet prevRowSet = source.getRowSet().copyPrev(); - final RowSet withoutMods = prevRowSet.minus(upstream.getModifiedPreShift()); - final ColumnSource.GetContext getContext = timestampColumn.makeGetContext(size)) { + final RowSet withoutMods = prevRowSet.minus(upstream.getModifiedPreShift()); + final ColumnSource.GetContext getContext = timestampColumn.makeGetContext(size)) { final RowSetShiftData.Iterator sit = upstream.shifted().applyIterator(); while (sit.hasNext()) { sit.next(); try (final RowSet subRowSet = withoutMods.subSetByKeyRange(sit.beginRange(), sit.endRange()); - final RowSet rowSetToShift = subRowSet.minus(upstream.removed())) { + final RowSet rowSetToShift = subRowSet.minus(upstream.removed())) { if (rowSetToShift.isEmpty()) { continue; } - final LongChunk shiftValues = timestampColumn.getPrevChunk(getContext, rowSetToShift).asLongChunk(); + final LongChunk shiftValues = + timestampColumn.getPrevChunk(getContext, rowSetToShift).asLongChunk(); timestampSsa.applyShift(shiftValues, rowSetToShift.asRowKeyChunk(), sit.shiftDelta()); } @@ -158,11 +163,11 @@ private void processUpdateForSsa(TableUpdate upstream) { // adds if (restampAdditions.isNonempty()) { - final int size = (int)Math.min(restampAdditions.size(), 4096); + final int size = (int) Math.min(restampAdditions.size(), 4096); try (final RowSequence.Iterator it = restampAdditions.getRowSequenceIterator(); - final ChunkSource.GetContext context = timestampColumn.makeGetContext(size); - final WritableLongChunk ssaValues = WritableLongChunk.makeWritableChunk(size); - final WritableLongChunk ssaKeys = WritableLongChunk.makeWritableChunk(size)) { + final ChunkSource.GetContext context = timestampColumn.makeGetContext(size); + final WritableLongChunk ssaValues = WritableLongChunk.makeWritableChunk(size); + final WritableLongChunk ssaKeys = WritableLongChunk.makeWritableChunk(size)) { MutableLong lastTimestamp = new MutableLong(NULL_LONG); while (it.hasMore()) { @@ -180,7 +185,9 @@ private void processUpdateForSsa(TableUpdate upstream) { } } - private void fillChunkWithNonNull(LongChunk keysChunk, LongChunk valuesChunk, WritableLongChunk ssaKeys, WritableLongChunk ssaValues, MutableLong lastTimestamp) { + private void fillChunkWithNonNull(LongChunk keysChunk, LongChunk valuesChunk, + WritableLongChunk ssaKeys, WritableLongChunk ssaValues, + MutableLong lastTimestamp) { // reset the insertion chunks ssaValues.setSize(0); ssaKeys.setSize(0); @@ -189,7 +196,8 @@ private void fillChunkWithNonNull(LongChunk keysChunk, LongChunk for (int i = 0; i < valuesChunk.size(); i++) { long ts = valuesChunk.get(i); if (ts < lastTimestamp.longValue()) { - throw(new IllegalStateException("updateBy time-based operators require non-descending timestamp values")); + throw (new IllegalStateException( + "updateBy time-based operators require non-descending timestamp values")); } if (ts != NULL_LONG) { ssaValues.add(ts); @@ -223,8 +231,8 @@ void doInitialAdditions() { } /** - * An object to hold the transient state during a single {@link InstrumentedTableUpdateListenerAdapter#onUpdate(TableUpdate)} update - * cycle. + * An object to hold the transient state during a single + * {@link InstrumentedTableUpdateListenerAdapter#onUpdate(TableUpdate)} update cycle. */ private class UpdateContext implements SafeCloseable { /** The expected size of chunks to the various update stages */ @@ -344,7 +352,8 @@ private class UpdateContext implements SafeCloseable { } // trigger the operator to determine its own set of affected rows (window-specific), do not close() // since this is managed by the operator context - final RowSet rs = opContext[opIdx].determineAffectedRows(upstream, source.getRowSet(), isInitializeStep); + final RowSet rs = + opContext[opIdx].determineAffectedRows(upstream, source.getRowSet(), isInitializeStep); // union the operator rowsets together to get a global set tmp.insert(rs); @@ -515,10 +524,11 @@ private void processRows(RowSetShiftData shifted) { initializeFor(rowsToProcess); - try (final RowSet positionsToProcess = anyRequirePositions ? source.getRowSet().invert(rowsToProcess) : null; - final RowSequence.Iterator keyIt = rowsToProcess.getRowSequenceIterator(); - final RowSequence.Iterator posIt = positionsToProcess == null ? null - : positionsToProcess.getRowSequenceIterator()) { + try (final RowSet positionsToProcess = + anyRequirePositions ? source.getRowSet().invert(rowsToProcess) : null; + final RowSequence.Iterator keyIt = rowsToProcess.getRowSequenceIterator(); + final RowSequence.Iterator posIt = positionsToProcess == null ? null + : positionsToProcess.getRowSequenceIterator()) { while (keyIt.hasMore()) { sharedContext.reset(); @@ -547,7 +557,7 @@ private void processRows(RowSetShiftData shifted) { if (chunkOk.firstRowKey() <= opContext[opIdx].getAffectedRows().lastRowKey() && chunkOk.lastRowKey() >= opContext[opIdx].getAffectedRows().firstRowKey()) { try (final RowSet rs = chunkOk.asRowSet(); - final RowSet intersect = rs.intersect(opContext[opIdx].getAffectedRows())) { + final RowSet intersect = rs.intersect(opContext[opIdx].getAffectedRows())) { prepareValuesChunkFor(opIdx, slotPosition, false, true, intersect, intersect, null, postWorkingChunks[slotPosition].get(), @@ -644,7 +654,8 @@ public void onUpdate(TableUpdate upstream) { if (redirContext.isRedirected()) { redirContext.processUpdateForRedirection(upstream, source.getRowSet()); } else { - // We will not mess with shifts if we are using a redirection because we'll have applied the shift + // We will not mess with shifts if we are using a redirection because we'll have applied the + // shift // to the redirection index already by now. if (upstream.shifted().nonempty()) { try (final RowSet prevIdx = source.getRowSet().copyPrev()) { 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 9e44f58d09f..fc33144ea15 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 @@ -21,7 +21,6 @@ import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.rowset.impl.AdaptiveOrderedLongSetBuilderRandom; -import io.deephaven.engine.rowset.impl.OrderedLongSet; import io.deephaven.engine.rowset.impl.WritableRowSetImpl; import io.deephaven.engine.table.ColumnDefinition; import io.deephaven.engine.table.ColumnSource; @@ -54,11 +53,13 @@ public final class PartitionByChunkedOperator implements IterativeChunkedAggrega // region nonexistent table sentinels /** - * Sentinel value for the row set builder belonging to a table that was never created because either the result + * Sentinel value for the row set builders belonging to a table that was never created because either the result * table was no longer live or the aggregation update listener was no longer live. Should be used for assignment and * reference equality tests, only. */ - private static final RowSetBuilderRandom NONEXISTENT_TABLE_ROW_SET_BUILDER = RowSetFactory.builderRandom(); + private static final RowSetBuilderRandom NONEXISTENT_TABLE_ROW_SET_BUILDER_RANDOM = RowSetFactory.builderRandom(); + private static final RowSetBuilderSequential NONEXISTENT_TABLE_ROW_SET_BUILDER_SEQUENTIAL = + RowSetFactory.builderSequential(); /** * Helper value for the following sentinel values but not used directly for comparison @@ -92,6 +93,7 @@ public interface AttributeCopier { private final String callSite; private final ObjectArraySource tables; + private final ObjectArraySource addedRowSetBuildersSequential; private final ObjectArraySource addedRowSetBuilders; private final ObjectArraySource removedRowSetBuilders; private final ObjectArraySource modifiedRowSetBuilders; @@ -100,6 +102,8 @@ public interface AttributeCopier { private final ModifiedColumnSet resultModifiedColumnSet; private final ModifiedColumnSet.Transformer upstreamToResultTransformer; + private volatile boolean initialized; + private volatile Table resultTable; private volatile LivenessReferent aggregationUpdateListener; @@ -142,10 +146,12 @@ public interface AttributeCopier { @NotNull final String... keyColumnNames) { this.parentTable = parentTable; this.resultName = resultName; + this.initialized = false; callSite = QueryPerformanceRecorder.getCallerLine(); tables = new ObjectArraySource<>(QueryTable.class); + addedRowSetBuildersSequential = new ObjectArraySource<>(RowSetBuilderSequential.class); addedRowSetBuilders = new ObjectArraySource<>(RowSetBuilderRandom.class); // Note: Sub-tables always share their ColumnSource map with the parent table, so they can all use this result @@ -202,7 +208,13 @@ public void addChunk(final BucketedContext bucketedContext, final Chunk inputRowKeysAsOrdered = (LongChunk) inputRowKeys; - accumulateToRowSet(addedRowSetBuilders, inputRowKeysAsOrdered, 0, chunkSize, destination); + if (!initialized) { + accumulateToBuilderSequential(addedRowSetBuildersSequential, inputRowKeysAsOrdered, 0, chunkSize, + destination); + } else { + accumulateToBuilderRandom(addedRowSetBuilders, inputRowKeysAsOrdered, 0, chunkSize, destination); + } if (stepUpdatedDestinations != null) { stepUpdatedDestinations.insert(destination); } @@ -318,7 +337,11 @@ public boolean addChunk(final SingletonContext singletonContext, final int chunk @Override public boolean addRowSet(SingletonContext context, RowSet rowSet, long destination) { - accumulateToRowSet(addedRowSetBuilders, rowSet, destination); + if (!initialized) { + accumulateToBuilderSequential(addedRowSetBuildersSequential, rowSet, destination); + } else { + accumulateToBuilderRandom(addedRowSetBuilders, rowSet, destination); + } if (stepUpdatedDestinations != null) { stepUpdatedDestinations.insert(destination); } @@ -333,7 +356,7 @@ public boolean removeChunk(final SingletonContext singletonContext, final int ch Assert.neqNull(stepUpdatedDestinations, "stepUpdatedDestinations"); // noinspection unchecked final LongChunk inputRowKeysAsOrdered = (LongChunk) inputRowKeys; - accumulateToRowSet(removedRowSetBuilders, inputRowKeysAsOrdered, 0, chunkSize, destination); + accumulateToBuilderRandom(removedRowSetBuilders, inputRowKeysAsOrdered, 0, chunkSize, destination); stepUpdatedDestinations.insert(destination); return false; } @@ -370,16 +393,56 @@ public boolean modifyRowKeys(final SingletonContext context, @NotNull final Long } // noinspection unchecked final LongChunk rowKeysAsOrdered = (LongChunk) rowKeys; - accumulateToRowSet(modifiedRowSetBuilders, rowKeysAsOrdered, 0, rowKeys.size(), destination); + accumulateToBuilderRandom(modifiedRowSetBuilders, rowKeysAsOrdered, 0, rowKeys.size(), destination); stepUpdatedDestinations.insert(destination); return false; } - private static void accumulateToRowSet(@NotNull final ObjectArraySource rowSetColumn, - @NotNull final LongChunk rowKeysToAdd, - final int start, final int length, final long destination) { + private static void accumulateToBuilderSequential( + @NotNull final ObjectArraySource rowSetColumn, + @NotNull final LongChunk rowKeysToAdd, + final int start, final int length, final long destination) { + final RowSetBuilderSequential builder = rowSetColumn.getUnsafe(destination); + if (builder == NONEXISTENT_TABLE_ROW_SET_BUILDER_SEQUENTIAL) { + return; + } + // slice the chunk to the start and length + LongChunk sliced = rowKeysToAdd.slice(start, length); + if (builder == null) { + // create (and store) a new builder, fill with these keys + final RowSetBuilderSequential newBuilder = RowSetFactory.builderSequential(); + newBuilder.appendOrderedRowKeysChunk(sliced); + rowSetColumn.set(destination, newBuilder); + return; + } + // add the keys to the stored builder + builder.appendOrderedRowKeysChunk(sliced); + } + + private static void accumulateToBuilderSequential( + @NotNull final ObjectArraySource rowSetColumn, + @NotNull final RowSet rowSetToAdd, final long destination) { + final RowSetBuilderSequential builder = rowSetColumn.getUnsafe(destination); + if (builder == NONEXISTENT_TABLE_ROW_SET_BUILDER_SEQUENTIAL) { + return; + } + if (builder == null) { + // create (and store) a new builder, fill with this rowset + final RowSetBuilderSequential newBuilder = RowSetFactory.builderSequential(); + newBuilder.appendRowSequence(rowSetToAdd); + rowSetColumn.set(destination, newBuilder); + return; + } + // add the rowset to the stored builder + builder.appendRowSequence(rowSetToAdd); + } + + + private static void accumulateToBuilderRandom(@NotNull final ObjectArraySource rowSetColumn, + @NotNull final LongChunk rowKeysToAdd, + final int start, final int length, final long destination) { final RowSetBuilderRandom builder = rowSetColumn.getUnsafe(destination); - if (builder == NONEXISTENT_TABLE_ROW_SET_BUILDER) { + if (builder == NONEXISTENT_TABLE_ROW_SET_BUILDER_RANDOM) { return; } // slice the chunk to the start and length @@ -395,10 +458,10 @@ private static void accumulateToRowSet(@NotNull final ObjectArraySource rowSetColumn, - @NotNull final RowSet rowSetToAdd, final long destination) { + private static void accumulateToBuilderRandom(@NotNull final ObjectArraySource rowSetColumn, + @NotNull final RowSet rowSetToAdd, final long destination) { final RowSetBuilderRandom builder = rowSetColumn.getUnsafe(destination); - if (builder == NONEXISTENT_TABLE_ROW_SET_BUILDER) { + if (builder == NONEXISTENT_TABLE_ROW_SET_BUILDER_RANDOM) { return; } if (builder == null) { @@ -464,6 +527,9 @@ private boolean appendShifts(@NotNull final LongChunk preShif public void ensureCapacity(final long tableSize) { tables.ensureCapacity(tableSize); addedRowSetBuilders.ensureCapacity(tableSize); + if (!initialized) { + addedRowSetBuildersSequential.ensureCapacity(tableSize); + } if (parentTable.isRefreshing()) { removedRowSetBuilders.ensureCapacity(tableSize); modifiedRowSetBuilders.ensureCapacity(tableSize); @@ -488,8 +554,8 @@ public void propagateInitialState(@NotNull final QueryTable resultTable) { final boolean setCallSite = QueryPerformanceRecorder.setCallsite(callSite); try (final ResettableWritableObjectChunk tablesResettableChunk = ResettableWritableObjectChunk.makeResettableChunk(); - final ResettableWritableObjectChunk addedRowSetsResettableChunk = - ResettableWritableObjectChunk.makeResettableChunk(); + final ResettableWritableObjectChunk addedRowSetsResettableChunk = + !initialized ? ResettableWritableObjectChunk.makeResettableChunk() : null; final RowSequence.Iterator initialDestinationsIterator = initialDestinations.getRowSequenceIterator()) { @@ -497,14 +563,15 @@ public void propagateInitialState(@NotNull final QueryTable resultTable) { final WritableObjectChunk tablesBackingChunk = tablesResettableChunk.asWritableObjectChunk(); // noinspection unchecked - final WritableObjectChunk addedRowSetsBackingChunk = + final WritableObjectChunk addedRowSetsBackingChunk = addedRowSetsResettableChunk.asWritableObjectChunk(); while (initialDestinationsIterator.hasMore()) { final long firstSliceDestination = initialDestinationsIterator.peekNextKey(); final long firstBackingChunkDestination = tables.resetWritableChunkToBackingStore(tablesResettableChunk, firstSliceDestination); - addedRowSetBuilders.resetWritableChunkToBackingStore(addedRowSetsResettableChunk, + // use the sequential builders during initialization + addedRowSetBuildersSequential.resetWritableChunkToBackingStore(addedRowSetsResettableChunk, firstSliceDestination); final long lastBackingChunkDestination = firstBackingChunkDestination + tablesBackingChunk.size() - 1; @@ -515,7 +582,7 @@ public void propagateInitialState(@NotNull final QueryTable resultTable) { final int backingChunkOffset = Math.toIntExact(destinationToInitialize - firstBackingChunkDestination); final WritableRowSet initialRowSet = - extractAndClearRowSet(addedRowSetsBackingChunk, backingChunkOffset); + extractAndClearBuilderSequential(addedRowSetsBackingChunk, backingChunkOffset); final QueryTable newTable = makeSubTable(initialRowSet); tablesBackingChunk.set(backingChunkOffset, newTable); }); @@ -524,6 +591,7 @@ public void propagateInitialState(@NotNull final QueryTable resultTable) { if (setCallSite) { QueryPerformanceRecorder.clearCallsite(); } + initialized = true; } } } @@ -639,7 +707,8 @@ private void propagateResurrectedDestinations(@NotNull final RowSequence resurre final TableUpdateImpl downstream = new TableUpdateImpl(); - downstream.added = nullToEmpty(extractAndClearRowSet(addedRowSetsBackingChunk, backingChunkOffset)); + downstream.added = + nullToEmpty(extractAndClearBuilderRandom(addedRowSetsBackingChunk, backingChunkOffset)); downstream.removed = RowSetFactory.empty(); downstream.modified = RowSetFactory.empty(); downstream.shifted = RowSetShiftData.EMPTY; @@ -713,16 +782,16 @@ private void propagateNewDestinations(@NotNull final RowSequence newDestinations final int backingChunkOffset = Math.toIntExact(newDestination - firstBackingChunkDestination); if (allowCreation) { final WritableRowSet newRowSet = - extractAndClearRowSet(addedRowSetsBackingChunk, backingChunkOffset); + extractAndClearBuilderRandom(addedRowSetsBackingChunk, backingChunkOffset); final QueryTable newTable = makeSubTable(newRowSet); linkTableReferences(newTable); tablesBackingChunk.set(backingChunkOffset, newTable); } else { // We will never try to create this table again, or accumulate further state for it. tablesBackingChunk.set(backingChunkOffset, NONEXISTENT_TABLE); - addedRowSetsBackingChunk.set(backingChunkOffset, NONEXISTENT_TABLE_ROW_SET_BUILDER); - removedRowSetsBackingChunk.set(backingChunkOffset, NONEXISTENT_TABLE_ROW_SET_BUILDER); - modifiedRowSetsBackingChunk.set(backingChunkOffset, NONEXISTENT_TABLE_ROW_SET_BUILDER); + addedRowSetsBackingChunk.set(backingChunkOffset, NONEXISTENT_TABLE_ROW_SET_BUILDER_RANDOM); + removedRowSetsBackingChunk.set(backingChunkOffset, NONEXISTENT_TABLE_ROW_SET_BUILDER_RANDOM); + modifiedRowSetsBackingChunk.set(backingChunkOffset, NONEXISTENT_TABLE_ROW_SET_BUILDER_RANDOM); shiftDataBuildersBackingChunk.set(backingChunkOffset, NONEXISTENT_TABLE_SHIFT_BUILDER); } }); @@ -790,7 +859,7 @@ private void propagateUpdatesToRemovedDestinations(@NotNull final RowSequence re downstream.added = RowSetFactory.empty(); downstream.removed = - nullToEmpty(extractAndClearRowSet(removedRowSetsBackingChunk, backingChunkOffset)); + nullToEmpty(extractAndClearBuilderRandom(removedRowSetsBackingChunk, backingChunkOffset)); downstream.modified = RowSetFactory.empty(); downstream.shifted = RowSetShiftData.EMPTY; downstream.modifiedColumnSet = ModifiedColumnSet.EMPTY; @@ -811,7 +880,10 @@ private void propagateUpdatesToModifiedDestinations(@NotNull final RowSequence m try (final ResettableWritableObjectChunk tablesResettableChunk = ResettableWritableObjectChunk.makeResettableChunk(); final ResettableWritableObjectChunk addedRowSetsResettableChunk = - ResettableWritableObjectChunk.makeResettableChunk(); + initialized ? ResettableWritableObjectChunk.makeResettableChunk() : null; + final ResettableWritableObjectChunk addedRowSetsSequentialResettableChunk = + !initialized ? ResettableWritableObjectChunk.makeResettableChunk() : null; + final ResettableWritableObjectChunk removedRowSetsResettableChunk = ResettableWritableObjectChunk.makeResettableChunk(); final ResettableWritableObjectChunk modifiedRowSetsResettableChunk = @@ -826,6 +898,7 @@ private void propagateUpdatesToModifiedDestinations(@NotNull final RowSequence m // noinspection unchecked final WritableObjectChunk addedRowSetsBackingChunk = addedRowSetsResettableChunk.asWritableObjectChunk(); + // noinspection unchecked final WritableObjectChunk removedRowSetsBackingChunk = removedRowSetsResettableChunk.asWritableObjectChunk(); @@ -843,6 +916,7 @@ private void propagateUpdatesToModifiedDestinations(@NotNull final RowSequence m // The (valid) assumption is that the other write-through resets will address the same range. addedRowSetBuilders.resetWritableChunkToBackingStore(addedRowSetsResettableChunk, firstSliceDestination); + removedRowSetBuilders.resetWritableChunkToBackingStore(removedRowSetsResettableChunk, firstSliceDestination); modifiedRowSetBuilders.resetWritableChunkToBackingStore(modifiedRowSetsResettableChunk, @@ -866,11 +940,13 @@ private void propagateUpdatesToModifiedDestinations(@NotNull final RowSequence m final TableUpdateImpl downstream = new TableUpdateImpl(); - downstream.added = nullToEmpty(extractAndClearRowSet(addedRowSetsBackingChunk, backingChunkOffset)); + downstream.added = + nullToEmpty(extractAndClearBuilderRandom(addedRowSetsBackingChunk, backingChunkOffset)); + downstream.removed = - nullToEmpty(extractAndClearRowSet(removedRowSetsBackingChunk, backingChunkOffset)); + nullToEmpty(extractAndClearBuilderRandom(removedRowSetsBackingChunk, backingChunkOffset)); downstream.modified = stepValuesModified - ? nullToEmpty(extractAndClearRowSet(modifiedRowSetsBackingChunk, backingChunkOffset)) + ? nullToEmpty(extractAndClearBuilderRandom(modifiedRowSetsBackingChunk, backingChunkOffset)) : RowSetFactory.empty(); downstream.shifted = extractAndClearShiftDataBuilder(shiftDataBuildersBackingChunk, backingChunkOffset); @@ -895,11 +971,26 @@ private void propagateUpdatesToModifiedDestinations(@NotNull final RowSequence m } } - private static WritableRowSet extractAndClearRowSet( + private static WritableRowSet extractAndClearBuilderRandom( @NotNull final WritableObjectChunk rowSetChunk, final int offset) { final RowSetBuilderRandom builder = rowSetChunk.get(offset); - Assert.neq(builder, "builder", NONEXISTENT_TABLE_ROW_SET_BUILDER, "NONEXISTENT_TABLE_ROW_SET_BUILDER"); + Assert.neq(builder, "builder", NONEXISTENT_TABLE_ROW_SET_BUILDER_RANDOM, + "NONEXISTENT_TABLE_ROW_SET_BUILDER_RANDOM"); + if (builder != null) { + final WritableRowSet rowSet = builder.build(); + rowSetChunk.set(offset, null); + return rowSet; + } + return null; + } + + private static WritableRowSet extractAndClearBuilderSequential( + @NotNull final WritableObjectChunk rowSetChunk, + final int offset) { + final RowSetBuilderSequential builder = rowSetChunk.get(offset); + Assert.neq(builder, "builder", NONEXISTENT_TABLE_ROW_SET_BUILDER_SEQUENTIAL, + "NONEXISTENT_TABLE_ROW_SET_BUILDER_SEQUENTIAL"); if (builder != null) { final WritableRowSet rowSet = builder.build(); rowSetChunk.set(offset, null); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java index 4b347716c48..3d034460249 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java @@ -66,7 +66,8 @@ public BasePrimitiveEMAOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { + public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize, + final LongSegmentedSortedArray timestampSsa) { return new EmaContext(timeScaleUnits, chunkSize, timestampSsa); } @@ -78,8 +79,8 @@ public void initializeFor(@NotNull final UpdateByOperator.UpdateContext updateCo @Override protected void doProcessChunk(@NotNull final Context context, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { + @NotNull final RowSequence inputKeys, + @NotNull final Chunk workingChunk) { final EmaContext ctx = (EmaContext) context; if (timeRecorder == null) { computeWithTicks(ctx, workingChunk, 0, inputKeys.intSize()); @@ -91,8 +92,8 @@ protected void doProcessChunk(@NotNull final Context context, @Override public void resetForProcess(@NotNull final UpdateByOperator.UpdateContext context, - @NotNull final RowSet sourceIndex, - final long firstUnmodifiedKey) { + @NotNull final RowSet sourceIndex, + final long firstUnmodifiedKey) { super.resetForProcess(context, sourceIndex, firstUnmodifiedKey); if (timeRecorder == null) { @@ -106,7 +107,7 @@ public void resetForProcess(@NotNull final UpdateByOperator.UpdateContext contex ctx.lastStamp = NULL_LONG; } else { // If it hasn't been reset to null, then it's possible that the value at that position was null, in - // which case we must have ignored it, and so we have to actually keep looking backwards until we find + // which case we must have ignored it, and so we have to actually keep looking backwards until we find // something not null. ctx.lastStamp = locateFirstValidPreviousTimestamp(sourceIndex, firstUnmodifiedKey); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java index 263036794b4..dfd1afe5f4d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java @@ -41,7 +41,8 @@ public BigDecimalEMAOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, timestampColumnName, timeScaleUnits, valueSource, redirContext); + super(pair, affectingColumns, control, timeRecorder, timestampColumnName, timeScaleUnits, valueSource, + redirContext); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java index c730f6dd03f..c538051cb68 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java @@ -98,8 +98,8 @@ protected void doProcessChunk(@NotNull final Context updateContext, @Override public void resetForProcess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceIndex, - final long firstUnmodifiedKey) { + @NotNull final RowSet sourceIndex, + final long firstUnmodifiedKey) { super.resetForProcess(context, sourceIndex, firstUnmodifiedKey); if (timeRecorder == null) { @@ -114,7 +114,7 @@ public void resetForProcess(@NotNull final UpdateContext context, ctx.lastStamp = NULL_LONG; } else { // If it hasn't been reset to null, then it's possible that the value at that position was null, in - // which case we must have ignored it, and so we have to actually keep looking backwards until we find + // which case we must have ignored it, and so we have to actually keep looking backwards until we find // something not null. ctx.lastStamp = locateFirstValidPreviousTimestamp(sourceIndex, firstUnmodifiedKey); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java index 87c46dd558b..4aed616b19e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java @@ -8,11 +8,11 @@ */ package io.deephaven.engine.table.impl.updateby.internal; +import gnu.trove.list.array.TIntArrayList; import io.deephaven.chunk.WritableDoubleChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.util.SafeCloseable; -import java.util.BitSet; import java.util.NoSuchElementException; /*** @@ -24,7 +24,10 @@ public class PairwiseDoubleRingBuffer implements SafeCloseable { // use a sized double chunk for underlying storage private WritableDoubleChunk storageChunk; - private final BitSet dirtyBits; +// private final BitSet dirtyBits; + private final TIntArrayList dirtyIndices; + private boolean allDirty; + private final DoubleFunction pairwiseFunction; private final double emptyVal; @@ -59,21 +62,55 @@ public PairwiseDoubleRingBuffer(int initialSize, double emptyVal, DoubleFunction this.capacity = Integer.highestOneBit(initialSize) * 2; this.chunkSize = capacity * 2; this.storageChunk = WritableDoubleChunk.makeWritableChunk(chunkSize); - this.dirtyBits = new BitSet(chunkSize); +// this.dirtyBits = new BitSet(chunkSize); + this.dirtyIndices = new TIntArrayList(chunkSize); this.pairwiseFunction = pairwiseFunction; this.emptyVal = emptyVal; this.storageChunk.fillWithValue(0, chunkSize, emptyVal); this.head = this.tail = this.capacity; + this.allDirty = false; + } + + private void evaluateRangeFast(int start, int end) { + // everything in this range needs recomputed + for (int left = start & 0xFFFFFFFE; left < end; left += 2) { + final int right = left + 1; + final int parent = left / 2; + + // load the data values + final double leftVal = storageChunk.get(left); + final double rightVal = storageChunk.get(right); + + // compute & stpre + final double computeVal = pairwiseFunction.apply(leftVal, rightVal); + storageChunk.set(parent, computeVal); + + // mark the parent dirty + dirtyIndices.add(parent); + } } public double evaluate() { + // if all dirty, recompute all values + if (allDirty) { + if (head < tail) { + evaluateRangeFast(head, tail); + } else { + evaluateRangeFast(head, chunkSize); + evaluateRangeFast(capacity, tail); + } + } else { + // sort so consecutive values are adjacent + dirtyIndices.sort(); + } + // work through all the dirty bits from high to low until none remain - int bit = chunkSize; - while (!dirtyBits.isEmpty()) { - int nextSetBit = dirtyBits.previousSetBit(bit); - final int left = nextSetBit & 0xFFFFFFFE; // clear the final bit to force evenness + int dirtyIndex = 0; + while (dirtyIndex < dirtyIndices.size()) { + final int left = dirtyIndices.get(dirtyIndex) & 0xFFFFFFFE; // clear the final bit to force evenness final int right = left + 1; + // this isn't the typical parent = (n-1)/2 because the tree is right-shifted by one final int parent = left / 2; @@ -82,20 +119,25 @@ public double evaluate() { final double rightVal = storageChunk.get(right); final double parentVal = storageChunk.get(parent); - dirtyBits.clear(left, right + 1); // clear() excludes `toIndex` so add one to clear `right` as well - final double computeVal = pairwiseFunction.apply(leftVal, rightVal); if (parentVal != computeVal) { storageChunk.set(parent, computeVal); // mark the parent dirty (if not the last) if (parent > 1) { - dirtyBits.set(parent); + dirtyIndices.add(parent); } + } + // how far should we advance + final int nextIndex = dirtyIndex + 1; + if (nextIndex < dirtyIndices.size() && dirtyIndices.get(nextIndex) == right) { + dirtyIndex += 2; } else { - final int x = 5; + dirtyIndex++; } - bit = left; } + allDirty = false; + dirtyIndices.clear(); + // final value is in index 1 return storageChunk.get(1); } @@ -134,8 +176,8 @@ private void grow() { // TODO: investigate moving precomputed results also. Since we are re-ordering the data values, would be // tricky to maintain order but a recursive function could probably do it efficiently. For now, make life easy // by setting all input dirty so the tree is recomputed on next `evaluate()` - dirtyBits.clear(); - dirtyBits.set(head, tail, true); + this.dirtyIndices.clear(); + allDirty = true; } public void push(double val) { @@ -144,7 +186,10 @@ public void push(double val) { } // add the new data storageChunk.set(tail, val); - dirtyBits.set(tail); + if (!allDirty) { + dirtyIndices.add(tail); + } + // move the tail tail = ((tail + 1) % capacity) + capacity; } @@ -159,7 +204,10 @@ public double pop() { } double val = storageChunk.get(head); storageChunk.set(head, emptyVal); - dirtyBits.set(head); + if (!allDirty) { + dirtyIndices.add(head); + } + // move the head head = ((head + 1) % capacity) + capacity; return val; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java index 78db0a8c4ee..647a86dd7ad 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java @@ -3,11 +3,11 @@ */ package io.deephaven.engine.table.impl.updateby.internal; +import gnu.trove.list.array.TIntArrayList; import io.deephaven.chunk.WritableFloatChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.util.SafeCloseable; -import java.util.BitSet; import java.util.NoSuchElementException; /*** @@ -19,7 +19,10 @@ public class PairwiseFloatRingBuffer implements SafeCloseable { // use a sized float chunk for underlying storage private WritableFloatChunk storageChunk; - private final BitSet dirtyBits; +// private final BitSet dirtyBits; + private final TIntArrayList dirtyIndices; + private boolean allDirty; + private final FloatFunction pairwiseFunction; private final float emptyVal; @@ -54,21 +57,55 @@ public PairwiseFloatRingBuffer(int initialSize, float emptyVal, FloatFunction pa this.capacity = Integer.highestOneBit(initialSize) * 2; this.chunkSize = capacity * 2; this.storageChunk = WritableFloatChunk.makeWritableChunk(chunkSize); - this.dirtyBits = new BitSet(chunkSize); +// this.dirtyBits = new BitSet(chunkSize); + this.dirtyIndices = new TIntArrayList(chunkSize); this.pairwiseFunction = pairwiseFunction; this.emptyVal = emptyVal; this.storageChunk.fillWithValue(0, chunkSize, emptyVal); this.head = this.tail = this.capacity; + this.allDirty = false; + } + + private void evaluateRangeFast(int start, int end) { + // everything in this range needs recomputed + for (int left = start & 0xFFFFFFFE; left < end; left += 2) { + final int right = left + 1; + final int parent = left / 2; + + // load the data values + final float leftVal = storageChunk.get(left); + final float rightVal = storageChunk.get(right); + + // compute & stpre + final float computeVal = pairwiseFunction.apply(leftVal, rightVal); + storageChunk.set(parent, computeVal); + + // mark the parent dirty + dirtyIndices.add(parent); + } } public float evaluate() { + // if all dirty, recompute all values + if (allDirty) { + if (head < tail) { + evaluateRangeFast(head, tail); + } else { + evaluateRangeFast(head, chunkSize); + evaluateRangeFast(capacity, tail); + } + } else { + // sort so consecutive values are adjacent + dirtyIndices.sort(); + } + // work through all the dirty bits from high to low until none remain - int bit = chunkSize; - while (!dirtyBits.isEmpty()) { - int nextSetBit = dirtyBits.previousSetBit(bit); - final int left = nextSetBit & 0xFFFFFFFE; // clear the final bit to force evenness + int dirtyIndex = 0; + while (dirtyIndex < dirtyIndices.size()) { + final int left = dirtyIndices.get(dirtyIndex) & 0xFFFFFFFE; // clear the final bit to force evenness final int right = left + 1; + // this isn't the typical parent = (n-1)/2 because the tree is right-shifted by one final int parent = left / 2; @@ -77,20 +114,25 @@ public float evaluate() { final float rightVal = storageChunk.get(right); final float parentVal = storageChunk.get(parent); - dirtyBits.clear(left, right + 1); // clear() excludes `toIndex` so add one to clear `right` as well - final float computeVal = pairwiseFunction.apply(leftVal, rightVal); if (parentVal != computeVal) { storageChunk.set(parent, computeVal); // mark the parent dirty (if not the last) if (parent > 1) { - dirtyBits.set(parent); + dirtyIndices.add(parent); } + } + // how far should we advance + final int nextIndex = dirtyIndex + 1; + if (nextIndex < dirtyIndices.size() && dirtyIndices.get(nextIndex) == right) { + dirtyIndex += 2; } else { - final int x = 5; + dirtyIndex++; } - bit = left; } + allDirty = false; + dirtyIndices.clear(); + // final value is in index 1 return storageChunk.get(1); } @@ -129,8 +171,8 @@ private void grow() { // TODO: investigate moving precomputed results also. Since we are re-ordering the data values, would be // tricky to maintain order but a recursive function could probably do it efficiently. For now, make life easy // by setting all input dirty so the tree is recomputed on next `evaluate()` - dirtyBits.clear(); - dirtyBits.set(head, tail, true); + this.dirtyIndices.clear(); + allDirty = true; } public void push(float val) { @@ -139,7 +181,10 @@ public void push(float val) { } // add the new data storageChunk.set(tail, val); - dirtyBits.set(tail); + if (!allDirty) { + dirtyIndices.add(tail); + } + // move the tail tail = ((tail + 1) % capacity) + capacity; } @@ -154,7 +199,10 @@ public float pop() { } float val = storageChunk.get(head); storageChunk.set(head, emptyVal); - dirtyBits.set(head); + if (!allDirty) { + dirtyIndices.add(head); + } + // move the head head = ((head + 1) % capacity) + capacity; return val; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java index 4e008a1372a..92b527a7073 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java @@ -54,7 +54,8 @@ public void close() { @NotNull @Override - public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { + public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize, + final LongSegmentedSortedArray timestampSsa) { return new Context(chunkSize, timestampSsa); } @@ -67,24 +68,26 @@ public void setChunkSize(@NotNull UpdateContext context, int chunkSize) { public BigDecimalRollingSumOperator(@NotNull final MatchPair pair, - @NotNull final String[] affectingColumns, - @NotNull final OperationControl control, - @Nullable final String timestampColumnName, - @Nullable final ColumnSource timestampColumnSource, - final long reverseTimeScaleUnits, - final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, - @NotNull final ColumnSource valueSource - // region extra-constructor-args - // endregion extra-constructor-args - , @NotNull final MathContext mathContext) { - super(pair, affectingColumns, control, timestampColumnName, timestampColumnSource, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource, BigDecimal.class); + @NotNull final String[] affectingColumns, + @NotNull final OperationControl control, + @Nullable final String timestampColumnName, + @Nullable final ColumnSource timestampColumnSource, + final long reverseTimeScaleUnits, + final long forwardTimeScaleUnits, + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, + @NotNull final ColumnSource valueSource + // region extra-constructor-args + // endregion extra-constructor-args + , @NotNull final MathContext mathContext) { + super(pair, affectingColumns, control, timestampColumnName, timestampColumnSource, reverseTimeScaleUnits, + forwardTimeScaleUnits, redirContext, valueSource, BigDecimal.class); this.mathContext = mathContext; - if(redirContext.isRedirected()) { + if (redirContext.isRedirected()) { // region create-dense this.maybeInnerSource = new ObjectArraySource(BigDecimal.class); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(redirContext.getRowRedirection(), maybeInnerSource, 0); + this.outputSource = + new WritableRedirectedColumnSource(redirContext.getRowRedirection(), maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -99,7 +102,7 @@ public BigDecimalRollingSumOperator(@NotNull final MatchPair pair, @Override public void push(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; - BigDecimal val = (BigDecimal)ctx.candidateValuesChunk.get(pos); + BigDecimal val = (BigDecimal) ctx.candidateValuesChunk.get(pos); // increase the running sum if (val != null) { @@ -116,7 +119,7 @@ public void push(UpdateContext context, long key, int pos) { @Override public void pop(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; - BigDecimal val = (BigDecimal)ctx.candidateValuesChunk.get(pos); + BigDecimal val = (BigDecimal) ctx.candidateValuesChunk.get(pos); // reduce the running sum if (val != null) { @@ -134,10 +137,10 @@ public void reset(UpdateContext context) { @Override public void doProcessChunk(@NotNull final BaseWindowedObjectUpdateByOperator.Context context, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @NotNull final Chunk workingChunk) { + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @Nullable final LongChunk posChunk, + @NotNull final Chunk workingChunk) { final Context ctx = (Context) context; if (timestampColumnName == null) { @@ -146,13 +149,13 @@ public void doProcessChunk(@NotNull final BaseWindowedObjectUpdateByOperator posChunk, - final int runLength) { + @Nullable final LongChunk posChunk, + final int runLength) { final WritableObjectChunk localOutputValues = ctx.outputValues.get(); for (int ii = 0; ii < runLength; ii++) { @@ -163,12 +166,13 @@ private void computeTicks(@NotNull final Context ctx, } private void computeTime(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys) { + @NotNull final RowSequence inputKeys) { final WritableObjectChunk localOutputValues = ctx.outputValues.get(); // get the timestamp values for this chunk try (final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(inputKeys.intSize())) { - LongChunk timestampChunk = timestampColumnSource.getChunk(context, inputKeys).asLongChunk(); + LongChunk timestampChunk = + timestampColumnSource.getChunk(context, inputKeys).asLongChunk(); for (int ii = 0; ii < inputKeys.intSize(); ii++) { // the output value is computed by push/pop operations triggered by fillWindow @@ -187,15 +191,15 @@ public Map> getOutputColumns() { @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if(redirContext.isRedirected()) { + if (redirContext.isRedirected()) { maybeInnerSource.startTrackingPrevValues(); } } @Override public void applyOutputShift(@NotNull final UpdateContext context, - @NotNull final RowSet subIndexToShift, - final long delta) { - ((ObjectSparseArraySource)outputSource).shift(subIndexToShift, delta); + @NotNull final RowSet subIndexToShift, + final long delta) { + ((ObjectSparseArraySource) outputSource).shift(subIndexToShift, delta); } } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/PairwiseDoubleRingBufferTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/PairwiseDoubleRingBufferTest.java index bd331a66f6f..96c48fba846 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/PairwiseDoubleRingBufferTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/PairwiseDoubleRingBufferTest.java @@ -39,11 +39,6 @@ private void assertEmpty(PairwiseDoubleRingBuffer rb) { } } - private void assertFull(PairwiseDoubleRingBuffer rb) { - assertFalse(rb.isEmpty()); - assertEquals(rb.capacity(), rb.size()); - } - private void assertNotEmpty(PairwiseDoubleRingBuffer rb, int expectedSize, double expectedHead) { assertFalse(rb.isEmpty()); assertEquals(expectedSize, rb.size()); @@ -84,6 +79,16 @@ public void testAddRemove() { assertEmpty(rb); + // move the head and tail off zero + for (double i = 0; i < 1000; i++) { + rb.push(i); + } + for (double i = 0; i < 1000; i++) { + rb.pop(); + } + + assertEmpty(rb); + assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); @@ -95,8 +100,6 @@ public void testAddRemove() { assertEquals(rb.back(),C); assertEquals(rb.peekBack(NULL_DOUBLE),C); - assertFull(rb); - assertRemove(rb, 3, A); assertRemove(rb, 2, B); assertRemove(rb, 1, C); @@ -133,7 +136,6 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertFull(rb); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); @@ -275,7 +277,6 @@ public void testLargeAmounts() { rb.pop(); } - for (double i = 0; i < 10_000; i++) rb.push(i); @@ -290,7 +291,6 @@ public void testLargeAmounts() { public void testEvaluateMinLargeAmounts() { try (final PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(3, Double.MAX_VALUE, Double::min)) { - for (double i = 0; i < 10_000; i++) rb.push(i); @@ -342,4 +342,21 @@ public void testEvaluateSumLargeAmounts() { } } } + + public void testEvaluationEdgeCase() { + try (final PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(3, -Double.MAX_VALUE, Double::max)) { + // move the head and tail off zero + for (double i = 0; i < 500; i++) { + rb.push(i); + } + for (double i = 0; i < 500; i++) { + rb.pop(); + } + + for (double i = 0; i < 100; i++) { + rb.push(i); + } + assertEquals((double)99, rb.evaluate()); // last value added is max + } + } } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/PairwiseFloatRingBufferTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/PairwiseFloatRingBufferTest.java index c277a5860b0..5112a54532b 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/PairwiseFloatRingBufferTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/PairwiseFloatRingBufferTest.java @@ -34,11 +34,6 @@ private void assertEmpty(PairwiseFloatRingBuffer rb) { } } - private void assertFull(PairwiseFloatRingBuffer rb) { - assertFalse(rb.isEmpty()); - assertEquals(rb.capacity(), rb.size()); - } - private void assertNotEmpty(PairwiseFloatRingBuffer rb, int expectedSize, float expectedHead) { assertFalse(rb.isEmpty()); assertEquals(expectedSize, rb.size()); @@ -79,6 +74,16 @@ public void testAddRemove() { assertEmpty(rb); + // move the head and tail off zero + for (float i = 0; i < 1000; i++) { + rb.push(i); + } + for (float i = 0; i < 1000; i++) { + rb.pop(); + } + + assertEmpty(rb); + assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); @@ -90,8 +95,6 @@ public void testAddRemove() { assertEquals(rb.back(),C); assertEquals(rb.peekBack(NULL_FLOAT),C); - assertFull(rb); - assertRemove(rb, 3, A); assertRemove(rb, 2, B); assertRemove(rb, 1, C); @@ -128,7 +131,6 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertFull(rb); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); @@ -270,7 +272,6 @@ public void testLargeAmounts() { rb.pop(); } - for (float i = 0; i < 10_000; i++) rb.push(i); @@ -285,7 +286,6 @@ public void testLargeAmounts() { public void testEvaluateMinLargeAmounts() { try (final PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(3, Float.MAX_VALUE, Float::min)) { - for (float i = 0; i < 10_000; i++) rb.push(i); @@ -337,4 +337,21 @@ public void testEvaluateSumLargeAmounts() { } } } + + public void testEvaluationEdgeCase() { + try (final PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(3, -Float.MAX_VALUE, Float::max)) { + // move the head and tail off zero + for (float i = 0; i < 500; i++) { + rb.push(i); + } + for (float i = 0; i < 500; i++) { + rb.pop(); + } + + for (float i = 0; i < 100; i++) { + rb.push(i); + } + assertEquals((float)99, rb.evaluate()); // last value added is max + } + } } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java index bdf86be1b61..b8de83b6630 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java @@ -103,12 +103,12 @@ public void testStaticZeroKeyTimed() { Duration postTime = Duration.ZERO; final Table summed = - t.updateBy(UpdateByOperation.RollingSum("ts", prevTime, postTime, "byteCol", "shortCol", "intCol", "longCol", "floatCol", - "doubleCol", "boolCol", "bigIntCol", "bigDecimalCol" - )); + t.updateBy(UpdateByOperation.RollingSum("ts", prevTime, postTime, "byteCol", "shortCol", "intCol", + "longCol", "floatCol", + "doubleCol", "boolCol", "bigIntCol", "bigDecimalCol")); - DateTime[] ts = (DateTime[])t.getColumn("ts").getDirect(); + DateTime[] ts = (DateTime[]) t.getColumn("ts").getDirect(); long[] timestamps = new long[t.intSize()]; for (int i = 0; i < t.intSize(); i++) { timestamps[i] = ts[i].getNanos(); @@ -139,12 +139,12 @@ public void testStaticZeroKeyFwdWindowTimed() { Duration postTime = Duration.ofMinutes(10); final Table summed = - t.updateBy(UpdateByOperation.RollingSum("ts", prevTime, postTime, "byteCol", "shortCol", "intCol", "longCol", "floatCol", - "doubleCol", "boolCol", "bigIntCol", "bigDecimalCol" - )); + t.updateBy(UpdateByOperation.RollingSum("ts", prevTime, postTime, "byteCol", "shortCol", "intCol", + "longCol", "floatCol", + "doubleCol", "boolCol", "bigIntCol", "bigDecimalCol")); - DateTime[] ts = (DateTime[])t.getColumn("ts").getDirect(); + DateTime[] ts = (DateTime[]) t.getColumn("ts").getDirect(); long[] timestamps = new long[t.intSize()]; for (int i = 0; i < t.intSize(); i++) { timestamps[i] = ts[i].getNanos(); @@ -175,12 +175,12 @@ public void testStaticZeroKeyFwdRevWindowTimed() { Duration postTime = Duration.ofMinutes(10); final Table summed = - t.updateBy(UpdateByOperation.RollingSum("ts", prevTime, postTime, "byteCol", "shortCol", "intCol", "longCol", "floatCol", - "doubleCol", "boolCol", "bigIntCol", "bigDecimalCol" - )); + t.updateBy(UpdateByOperation.RollingSum("ts", prevTime, postTime, "byteCol", "shortCol", "intCol", + "longCol", "floatCol", + "doubleCol", "boolCol", "bigIntCol", "bigDecimalCol")); - DateTime[] ts = (DateTime[])t.getColumn("ts").getDirect(); + DateTime[] ts = (DateTime[]) t.getColumn("ts").getDirect(); long[] timestamps = new long[t.intSize()]; for (int i = 0; i < t.intSize(); i++) { timestamps[i] = ts[i].getNanos(); @@ -236,9 +236,9 @@ private void doTestStaticBucketed(boolean grouped) { int postTicks = 10; final Table summed = - t.updateBy(UpdateByOperation.RollingSum(prevTicks, postTicks, "byteCol", "shortCol", "intCol", "longCol", "floatCol", - "doubleCol", "boolCol", "bigIntCol", "bigDecimalCol" - ), "Sym"); + t.updateBy(UpdateByOperation.RollingSum(prevTicks, postTicks, "byteCol", "shortCol", "intCol", + "longCol", "floatCol", + "doubleCol", "boolCol", "bigIntCol", "bigDecimalCol"), "Sym"); final PartitionedTable preOp = t.partitionBy("Sym"); @@ -277,9 +277,9 @@ private void doTestStaticBucketedTimed(boolean grouped, Duration prevTime, Durat convertDateTime("2022-03-09T16:30:00.000 NY"))}).t; final Table summed = - t.updateBy(UpdateByOperation.RollingSum("ts", prevTime, postTime, "byteCol", "shortCol", "intCol", "longCol", "floatCol", - "doubleCol", "boolCol", "bigIntCol", "bigDecimalCol" - ), "Sym"); + t.updateBy(UpdateByOperation.RollingSum("ts", prevTime, postTime, "byteCol", "shortCol", "intCol", + "longCol", "floatCol", + "doubleCol", "boolCol", "bigIntCol", "bigDecimalCol"), "Sym"); final PartitionedTable preOp = t.partitionBy("Sym"); @@ -288,13 +288,14 @@ private void doTestStaticBucketedTimed(boolean grouped, Duration prevTime, Durat String[] columns = t.getDefinition().getColumnStream().map(ColumnDefinition::getName).toArray(String[]::new); preOp.partitionedTransform(postOp, (source, actual) -> { - DateTime[] ts = (DateTime[])source.getColumn("ts").getDirect(); + DateTime[] ts = (DateTime[]) source.getColumn("ts").getDirect(); long[] timestamps = new long[source.intSize()]; for (int i = 0; i < source.intSize(); i++) { timestamps[i] = ts[i].getNanos(); } Arrays.stream(columns).forEach(col -> { - assertWithRollingSumTime(source.getColumn(col).getDirect(), actual.getColumn(col).getDirect(), timestamps, + assertWithRollingSumTime(source.getColumn(col).getDirect(), actual.getColumn(col).getDirect(), + timestamps, actual.getColumn(col).getType(), prevTime.toNanos(), postTime.toNanos()); }); return source; @@ -1084,7 +1085,8 @@ private long[] rollingSumTime(Boolean[] values, long[] timestamps, long prevNano return result; } - private Object[] rollingSumTime(Object[] values, long[] timestamps, final boolean isBD, long prevNanos, long postNanos) { + private Object[] rollingSumTime(Object[] values, long[] timestamps, final boolean isBD, long prevNanos, + long postNanos) { if (values == null) { return null; } @@ -1140,7 +1142,8 @@ private Object[] rollingSumTime(Object[] values, long[] timestamps, final boolea } - final void assertWithRollingSumTicks(final @NotNull Object expected, final @NotNull Object actual, Class type, int prevTicks, int postTicks) { + final void assertWithRollingSumTicks(final @NotNull Object expected, final @NotNull Object actual, Class type, + int prevTicks, int postTicks) { // looking for gross errors like missing entries (NOTE: pairwise results are more accurate than true rolling) final float deltaF = .02f; final double deltaD = .02d; @@ -1169,7 +1172,7 @@ final void assertWithRollingSumTicks(final @NotNull Object expected, final @NotN } final void assertWithRollingSumTime(final @NotNull Object expected, final @NotNull Object actual, - final @NotNull long[] timestamps, Class type, long prevTime, long postTime) { + final @NotNull long[] timestamps, Class type, long prevTime, long postTime) { // looking for gross errors like missing entries (NOTE: pairwise results are more accurate than true rolling) final float deltaF = .02f; final double deltaD = .02d; @@ -1183,16 +1186,20 @@ final void assertWithRollingSumTime(final @NotNull Object expected, final @NotNu } else if (expected instanceof long[]) { assertArrayEquals(rollingSumTime((long[]) expected, timestamps, prevTime, postTime), (long[]) actual); } else if (expected instanceof float[]) { - assertArrayEquals(rollingSumTime((float[]) expected, timestamps, prevTime, postTime), (float[]) actual, deltaF); + assertArrayEquals(rollingSumTime((float[]) expected, timestamps, prevTime, postTime), (float[]) actual, + deltaF); } else if (expected instanceof double[]) { - assertArrayEquals(rollingSumTime((double[]) expected, timestamps, prevTime, postTime), (double[]) actual, deltaD); + assertArrayEquals(rollingSumTime((double[]) expected, timestamps, prevTime, postTime), (double[]) actual, + deltaD); } else if (expected instanceof Boolean[]) { assertArrayEquals(rollingSumTime((Boolean[]) expected, timestamps, prevTime, postTime), (long[]) actual); } else { if (type == BigDecimal.class) { - assertArrayEquals(rollingSumTime((Object[]) expected, timestamps, true, prevTime, postTime), (Object[]) actual); + assertArrayEquals(rollingSumTime((Object[]) expected, timestamps, true, prevTime, postTime), + (Object[]) actual); } else if (type == BigInteger.class) { - assertArrayEquals(rollingSumTime((Object[]) expected, timestamps, false, prevTime, postTime), (Object[]) actual); + assertArrayEquals(rollingSumTime((Object[]) expected, timestamps, false, prevTime, postTime), + (Object[]) actual); } } } diff --git a/props/test-configs/src/main/resources/dh-tests.prop b/props/test-configs/src/main/resources/dh-tests.prop index e270fa9e6de..c6b87ff2799 100644 --- a/props/test-configs/src/main/resources/dh-tests.prop +++ b/props/test-configs/src/main/resources/dh-tests.prop @@ -104,4 +104,6 @@ BarrageMessageProducer.maxSnapshotCellCount=50 BarrageStreamGenerator.batchSize=4 #BarrageTable.debug=true -#BarrageMessageProducer.debug=true \ No newline at end of file +#BarrageMessageProducer.debug=true + +ReleaseTracker.captureStackTraces=true \ No newline at end of file diff --git a/table-api/src/main/java/io/deephaven/api/updateby/UpdateByOperation.java b/table-api/src/main/java/io/deephaven/api/updateby/UpdateByOperation.java index 40ca5b17e48..500b03188ad 100644 --- a/table-api/src/main/java/io/deephaven/api/updateby/UpdateByOperation.java +++ b/table-api/src/main/java/io/deephaven/api/updateby/UpdateByOperation.java @@ -216,8 +216,8 @@ static UpdateByOperation Ema(OperationControl control, String timestampColumn, D } /** - * Create an {@link RollingSumSpec rolling sum} for the supplied column name pairs, using ticks as - * the windowing unit. Uses the default OperationControl settings. + * Create an {@link RollingSumSpec rolling sum} for the supplied column name pairs, using ticks as the windowing + * unit. Uses the default OperationControl settings. * * @param prevTimeTicks the look-behind window size (in rows/ticks) * @param pairs The input/output column name pairs @@ -228,8 +228,8 @@ static UpdateByOperation RollingSum(long prevTimeTicks, String... pairs) { } /** - * Create an {@link RollingSumSpec rolling sum} for the supplied column name pairs, using ticks as - * the windowing unit. Uses the default OperationControl settings. + * Create an {@link RollingSumSpec rolling sum} for the supplied column name pairs, using ticks as the windowing + * unit. Uses the default OperationControl settings. * * @param prevTimeTicks the look-behind window size (in rows/ticks) * @param fwdTimeTicks the look-ahead window size (in rows/ticks) @@ -241,15 +241,16 @@ static UpdateByOperation RollingSum(long prevTimeTicks, long fwdTimeTicks, Strin } /** - * Create an {@link RollingSumSpec rolling sum} for the supplied column name pairs, using time as - * the windowing unit. Uses the default OperationControl settings. + * Create an {@link RollingSumSpec rolling sum} for the supplied column name pairs, using time as the windowing + * unit. Uses the default OperationControl settings. * * @param prevWindowDuration the look-behind window size (in Duration) * @param fwdWindowDuration the look-ahead window size (in Duration) * @param pairs The input/output column name pairs * @return The aggregation */ - static UpdateByOperation RollingSum(String timestampCol, Duration prevWindowDuration, Duration fwdWindowDuration, String... pairs) { + static UpdateByOperation RollingSum(String timestampCol, Duration prevWindowDuration, Duration fwdWindowDuration, + String... pairs) { return RollingSumSpec.ofTime(timestampCol, prevWindowDuration, fwdWindowDuration).clause(pairs); } diff --git a/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java b/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java index 4f8e767ad19..27cf0ca85e4 100644 --- a/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java +++ b/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java @@ -29,8 +29,10 @@ public static RollingSumSpec ofTime(final String timestampCol, Duration prevWind return of(TimeScale.ofTime(timestampCol, prevWindowDuration)); } - public static RollingSumSpec ofTime(final String timestampCol, Duration prevWindowDuration, Duration fwdWindowDuration) { - return of(TimeScale.ofTime(timestampCol, prevWindowDuration), TimeScale.ofTime(timestampCol, fwdWindowDuration)); + public static RollingSumSpec ofTime(final String timestampCol, Duration prevWindowDuration, + Duration fwdWindowDuration) { + return of(TimeScale.ofTime(timestampCol, prevWindowDuration), + TimeScale.ofTime(timestampCol, fwdWindowDuration)); } // general use contructors @@ -43,48 +45,51 @@ public static RollingSumSpec of(TimeScale prevTimeScale, TimeScale fwdTimeScale) return ImmutableRollingSumSpec.builder().prevTimeScale(prevTimeScale).fwdTimeScale(fwdTimeScale).build(); } -// public static RollingSumSpec of(TimeScale prevTimeScale) { -// return ImmutableWindowedOpSpec.builder().prevTimeScale(prevTimeScale).build(); -// } -// -// public static RollingSumSpec of(OperationControl control, TimeScale prevTimeScale, TimeScale fwdTimeScale) { -// return ImmutableWindowedOpSpec.builder().control(control).prevTimeScale(prevTimeScale).fwdTimeScale(fwdTimeScale).build(); -// } -// -// public static RollingSumSpec ofTime(final OperationControl control, -// final String timestampCol, -// long prevWindowTimeScaleNanos) { -// return of(control, TimeScale.ofTime(timestampCol, prevWindowTimeScaleNanos)); -// } -// -// public static RollingSumSpec ofTime(final OperationControl control, -// final String timestampCol, -// long prevWindowTimeScaleNanos, -// long fwdWindowTimeScaleNanos) { -// return of(control, TimeScale.ofTime(timestampCol, prevWindowTimeScaleNanos), TimeScale.ofTime(timestampCol, fwdWindowTimeScaleNanos)); -// } -// -// public static RollingSumSpec ofTime(final OperationControl control, -// final String timestampCol, -// Duration prevWindowDuration) { -// return of(control, TimeScale.ofTime(timestampCol, prevWindowDuration)); -// } -// -// -// public static RollingSumSpec ofTime(final OperationControl control, -// final String timestampCol, -// Duration prevWindowDuration, -// Duration fwdWindowDuration) { -// return of(control, TimeScale.ofTime(timestampCol, prevWindowDuration), TimeScale.ofTime(timestampCol, fwdWindowDuration)); -// } -// -// public static RollingSumSpec ofTicks(OperationControl control, long prevTickWindow) { -// return of(control, TimeScale.ofTicks(prevTickWindow)); -// } -// -// public static RollingSumSpec ofTicks(OperationControl control, long prevTickWindow, long fwdTickWindow) { -// return of(control, TimeScale.ofTicks(prevTickWindow), TimeScale.ofTicks(fwdTickWindow)); -// } + // public static RollingSumSpec of(TimeScale prevTimeScale) { + // return ImmutableWindowedOpSpec.builder().prevTimeScale(prevTimeScale).build(); + // } + // + // public static RollingSumSpec of(OperationControl control, TimeScale prevTimeScale, TimeScale fwdTimeScale) { + // return + // ImmutableWindowedOpSpec.builder().control(control).prevTimeScale(prevTimeScale).fwdTimeScale(fwdTimeScale).build(); + // } + // + // public static RollingSumSpec ofTime(final OperationControl control, + // final String timestampCol, + // long prevWindowTimeScaleNanos) { + // return of(control, TimeScale.ofTime(timestampCol, prevWindowTimeScaleNanos)); + // } + // + // public static RollingSumSpec ofTime(final OperationControl control, + // final String timestampCol, + // long prevWindowTimeScaleNanos, + // long fwdWindowTimeScaleNanos) { + // return of(control, TimeScale.ofTime(timestampCol, prevWindowTimeScaleNanos), TimeScale.ofTime(timestampCol, + // fwdWindowTimeScaleNanos)); + // } + // + // public static RollingSumSpec ofTime(final OperationControl control, + // final String timestampCol, + // Duration prevWindowDuration) { + // return of(control, TimeScale.ofTime(timestampCol, prevWindowDuration)); + // } + // + // + // public static RollingSumSpec ofTime(final OperationControl control, + // final String timestampCol, + // Duration prevWindowDuration, + // Duration fwdWindowDuration) { + // return of(control, TimeScale.ofTime(timestampCol, prevWindowDuration), TimeScale.ofTime(timestampCol, + // fwdWindowDuration)); + // } + // + // public static RollingSumSpec ofTicks(OperationControl control, long prevTickWindow) { + // return of(control, TimeScale.ofTicks(prevTickWindow)); + // } + // + // public static RollingSumSpec ofTicks(OperationControl control, long prevTickWindow, long fwdTickWindow) { + // return of(control, TimeScale.ofTicks(prevTickWindow), TimeScale.ofTicks(fwdTickWindow)); + // } public abstract Optional control(); @@ -104,16 +109,16 @@ public final OperationControl controlOrDefault() { @Override public final boolean applicableTo(Class inputType) { return - // is primitive numeric? - inputType.equals(double.class) || inputType.equals(float.class) - || inputType.equals(int.class) || inputType.equals(long.class) || inputType.equals(short.class) - || inputType.equals(byte.class) + // is primitive numeric? + inputType.equals(double.class) || inputType.equals(float.class) + || inputType.equals(int.class) || inputType.equals(long.class) || inputType.equals(short.class) + || inputType.equals(byte.class) - // is boxed numeric? - || Number.class.isAssignableFrom(inputType) + // is boxed numeric? + || Number.class.isAssignableFrom(inputType) - // is boolean? - || inputType == boolean.class || inputType == Boolean.class; + // is boolean? + || inputType == boolean.class || inputType == Boolean.class; } @Override From dfa8f29ba768b157ac32461a767a813c37e5dda5 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Thu, 15 Sep 2022 15:26:04 -0700 Subject: [PATCH 023/123] added replicated ring buffers and tests --- .../base/ringbuffer/ByteRingBuffer.java | 192 +++++++++ .../base/ringbuffer/CharRingBuffer.java | 187 +++++++++ .../base/ringbuffer/DoubleRingBuffer.java | 192 +++++++++ .../base/ringbuffer/FloatRingBuffer.java | 192 +++++++++ .../base/ringbuffer/IntRingBuffer.java | 192 +++++++++ .../base/{ => ringbuffer}/LongRingBuffer.java | 13 +- .../base/ringbuffer/ObjectRingBuffer.java | 192 +++++++++ .../base/ringbuffer/ShortRingBuffer.java | 192 +++++++++ .../base/ringbuffer/ByteRingBufferTest.java | 372 ++++++++++++++++++ .../base/ringbuffer/CharRingBufferTest.java | 367 +++++++++++++++++ .../base/ringbuffer/DoubleRingBufferTest.java | 372 ++++++++++++++++++ .../base/ringbuffer/FloatRingBufferTest.java | 372 ++++++++++++++++++ .../base/ringbuffer/IntRingBufferTest.java | 372 ++++++++++++++++++ .../{ => ringbuffer}/LongRingBufferTest.java | 29 +- .../base/ringbuffer/ObjectRingBufferTest.java | 372 ++++++++++++++++++ .../base/ringbuffer/ShortRingBufferTest.java | 372 ++++++++++++++++++ .../engine/table/impl/SortListener.java | 2 +- .../table/impl/UpdateByWindowedOperator.java | 29 +- replication/static/build.gradle | 3 + .../replicators/ReplicateRingBuffers.java | 38 ++ 20 files changed, 4020 insertions(+), 32 deletions(-) create mode 100644 Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java create mode 100644 Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java create mode 100644 Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java create mode 100644 Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java create mode 100644 Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java rename Base/src/main/java/io/deephaven/base/{ => ringbuffer}/LongRingBuffer.java (90%) create mode 100644 Base/src/main/java/io/deephaven/base/ringbuffer/ObjectRingBuffer.java create mode 100644 Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java create mode 100644 Base/src/test/java/io/deephaven/base/ringbuffer/ByteRingBufferTest.java create mode 100644 Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java create mode 100644 Base/src/test/java/io/deephaven/base/ringbuffer/DoubleRingBufferTest.java create mode 100644 Base/src/test/java/io/deephaven/base/ringbuffer/FloatRingBufferTest.java create mode 100644 Base/src/test/java/io/deephaven/base/ringbuffer/IntRingBufferTest.java rename Base/src/test/java/io/deephaven/base/{ => ringbuffer}/LongRingBufferTest.java (90%) create mode 100644 Base/src/test/java/io/deephaven/base/ringbuffer/ObjectRingBufferTest.java create mode 100644 Base/src/test/java/io/deephaven/base/ringbuffer/ShortRingBufferTest.java create mode 100644 replication/static/src/main/java/io/deephaven/replicators/ReplicateRingBuffers.java diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java new file mode 100644 index 00000000000..f83655e9ba8 --- /dev/null +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java @@ -0,0 +1,192 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit CharRingBuffer and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +package io.deephaven.base.ringbuffer; + +import java.io.Serializable; +import java.util.NoSuchElementException; + +/** + * A trivial circular buffer for primitive values, like java.util.concurrent.ArrayBlockingQueue but without all the + * synchronization and collection cruft. Storage is between head (incl.) and tail (excl.) wrapping around the end of the + * array. If the buffer is *not* growable, it will make room for a new element by dropping off the oldest element in the + * buffer instead. + */ +public class ByteRingBuffer implements Serializable { + protected final boolean growable; + protected byte[] storage; + protected int head, tail; + + private void grow() { + if (growable) { + byte[] newStorage = new byte[storage.length * 2]; + if (tail > head) { + System.arraycopy(storage, head, newStorage, 0, tail - head); + tail = tail - head; + } else { + System.arraycopy(storage, head, newStorage, 0, storage.length - head); + System.arraycopy(storage, 0, newStorage, storage.length - head, tail); + tail += storage.length - head; + } + head = 0; + storage = newStorage; + } else { + head = (head + 1) % storage.length; + } + } + + public boolean isFull() { + return (tail + 1) % storage.length == head; + } + + public ByteRingBuffer(int capacity) { + this(capacity, true); + } + + public ByteRingBuffer(int capacity, boolean growable) { + this.growable = growable; + this.storage = new byte[capacity + 1]; + this.tail = this.head = 0; + } + + public boolean isEmpty() { + return tail == head; + } + + public int size() { + return tail >= head ? (tail - head) : (tail + (storage.length - head)); + } + + public int capacity() { + return storage.length - 1; + } + + public int remaining() { + return capacity() - size(); + } + + public void clear() { + tail = head = 0; + } + + public boolean add(byte e) { + if (isFull()) { + grow(); + } + storage[tail] = e; + tail = (tail + 1) % storage.length; + return true; + } + + public byte addOverwrite(byte e, byte notFullResult) { + byte result = notFullResult; + if (isFull()) { + result = remove(); + } + storage[tail] = e; + tail = (tail + 1) % storage.length; + return result; + } + + public boolean offer(byte e) { + if (isFull()) { + return false; + } + storage[tail] = e; + tail = (tail + 1) % storage.length; + return true; + } + + public byte remove() { + if (isEmpty()) { + throw new NoSuchElementException(); + } + byte e = storage[head]; + head = (head + 1) % storage.length; + return e; + } + + public byte poll(byte onEmpty) { + if (isEmpty()) { + return onEmpty; + } + byte e = storage[head]; + head = (head + 1) % storage.length; + return e; + } + + public byte element() { + if (isEmpty()) { + throw new NoSuchElementException(); + } + return storage[head]; + } + + public byte peek(byte onEmpty) { + if (isEmpty()) { + return onEmpty; + } + return storage[head]; + } + + public byte front() { + return front(0); + } + + public byte front(int offset) { + if (offset >= size()) { + throw new NoSuchElementException(); + } + return storage[(head + offset) % storage.length]; + } + + public byte back() { + if (isEmpty()) { + throw new NoSuchElementException(); + } + return tail == 0 ? storage[storage.length - 1] : storage[tail - 1]; + } + + public byte peekBack(byte onEmpty) { + if (isEmpty()) { + return onEmpty; + } + return tail == 0 ? storage[storage.length - 1] : storage[tail - 1]; + } + + public Iterator iterator() { + return new Iterator(); + } + + public class Iterator { + int count = -1; + + public boolean hasNext() { + return count + 1 < size(); + } + + public byte next() { + count++; + return storage[(head + count) % storage.length]; + } + + public void remove() { + throw new UnsupportedOperationException(); + } + } + + public byte[] getAll() { + int n = size(), h = head; + byte[] result = new byte[n]; + for (int i = 0; i < n; ++i) { + result[i] = storage[h]; + h = (h + 1) % storage.length; + } + return result; + } +} diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java new file mode 100644 index 00000000000..56a698a09fe --- /dev/null +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java @@ -0,0 +1,187 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.base.ringbuffer; + +import java.io.Serializable; +import java.util.NoSuchElementException; + +/** + * A trivial circular buffer for primitive values, like java.util.concurrent.ArrayBlockingQueue but without all the + * synchronization and collection cruft. Storage is between head (incl.) and tail (excl.) wrapping around the end of the + * array. If the buffer is *not* growable, it will make room for a new element by dropping off the oldest element in the + * buffer instead. + */ +public class CharRingBuffer implements Serializable { + protected final boolean growable; + protected char[] storage; + protected int head, tail; + + private void grow() { + if (growable) { + char[] newStorage = new char[storage.length * 2]; + if (tail > head) { + System.arraycopy(storage, head, newStorage, 0, tail - head); + tail = tail - head; + } else { + System.arraycopy(storage, head, newStorage, 0, storage.length - head); + System.arraycopy(storage, 0, newStorage, storage.length - head, tail); + tail += storage.length - head; + } + head = 0; + storage = newStorage; + } else { + head = (head + 1) % storage.length; + } + } + + public boolean isFull() { + return (tail + 1) % storage.length == head; + } + + public CharRingBuffer(int capacity) { + this(capacity, true); + } + + public CharRingBuffer(int capacity, boolean growable) { + this.growable = growable; + this.storage = new char[capacity + 1]; + this.tail = this.head = 0; + } + + public boolean isEmpty() { + return tail == head; + } + + public int size() { + return tail >= head ? (tail - head) : (tail + (storage.length - head)); + } + + public int capacity() { + return storage.length - 1; + } + + public int remaining() { + return capacity() - size(); + } + + public void clear() { + tail = head = 0; + } + + public boolean add(char e) { + if (isFull()) { + grow(); + } + storage[tail] = e; + tail = (tail + 1) % storage.length; + return true; + } + + public char addOverwrite(char e, char notFullResult) { + char result = notFullResult; + if (isFull()) { + result = remove(); + } + storage[tail] = e; + tail = (tail + 1) % storage.length; + return result; + } + + public boolean offer(char e) { + if (isFull()) { + return false; + } + storage[tail] = e; + tail = (tail + 1) % storage.length; + return true; + } + + public char remove() { + if (isEmpty()) { + throw new NoSuchElementException(); + } + char e = storage[head]; + head = (head + 1) % storage.length; + return e; + } + + public char poll(char onEmpty) { + if (isEmpty()) { + return onEmpty; + } + char e = storage[head]; + head = (head + 1) % storage.length; + return e; + } + + public char element() { + if (isEmpty()) { + throw new NoSuchElementException(); + } + return storage[head]; + } + + public char peek(char onEmpty) { + if (isEmpty()) { + return onEmpty; + } + return storage[head]; + } + + public char front() { + return front(0); + } + + public char front(int offset) { + if (offset >= size()) { + throw new NoSuchElementException(); + } + return storage[(head + offset) % storage.length]; + } + + public char back() { + if (isEmpty()) { + throw new NoSuchElementException(); + } + return tail == 0 ? storage[storage.length - 1] : storage[tail - 1]; + } + + public char peekBack(char onEmpty) { + if (isEmpty()) { + return onEmpty; + } + return tail == 0 ? storage[storage.length - 1] : storage[tail - 1]; + } + + public Iterator iterator() { + return new Iterator(); + } + + public class Iterator { + int count = -1; + + public boolean hasNext() { + return count + 1 < size(); + } + + public char next() { + count++; + return storage[(head + count) % storage.length]; + } + + public void remove() { + throw new UnsupportedOperationException(); + } + } + + public char[] getAll() { + int n = size(), h = head; + char[] result = new char[n]; + for (int i = 0; i < n; ++i) { + result[i] = storage[h]; + h = (h + 1) % storage.length; + } + return result; + } +} diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java new file mode 100644 index 00000000000..f8f3f14ea0a --- /dev/null +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java @@ -0,0 +1,192 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit CharRingBuffer and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +package io.deephaven.base.ringbuffer; + +import java.io.Serializable; +import java.util.NoSuchElementException; + +/** + * A trivial circular buffer for primitive values, like java.util.concurrent.ArrayBlockingQueue but without all the + * synchronization and collection cruft. Storage is between head (incl.) and tail (excl.) wrapping around the end of the + * array. If the buffer is *not* growable, it will make room for a new element by dropping off the oldest element in the + * buffer instead. + */ +public class DoubleRingBuffer implements Serializable { + protected final boolean growable; + protected double[] storage; + protected int head, tail; + + private void grow() { + if (growable) { + double[] newStorage = new double[storage.length * 2]; + if (tail > head) { + System.arraycopy(storage, head, newStorage, 0, tail - head); + tail = tail - head; + } else { + System.arraycopy(storage, head, newStorage, 0, storage.length - head); + System.arraycopy(storage, 0, newStorage, storage.length - head, tail); + tail += storage.length - head; + } + head = 0; + storage = newStorage; + } else { + head = (head + 1) % storage.length; + } + } + + public boolean isFull() { + return (tail + 1) % storage.length == head; + } + + public DoubleRingBuffer(int capacity) { + this(capacity, true); + } + + public DoubleRingBuffer(int capacity, boolean growable) { + this.growable = growable; + this.storage = new double[capacity + 1]; + this.tail = this.head = 0; + } + + public boolean isEmpty() { + return tail == head; + } + + public int size() { + return tail >= head ? (tail - head) : (tail + (storage.length - head)); + } + + public int capacity() { + return storage.length - 1; + } + + public int remaining() { + return capacity() - size(); + } + + public void clear() { + tail = head = 0; + } + + public boolean add(double e) { + if (isFull()) { + grow(); + } + storage[tail] = e; + tail = (tail + 1) % storage.length; + return true; + } + + public double addOverwrite(double e, double notFullResult) { + double result = notFullResult; + if (isFull()) { + result = remove(); + } + storage[tail] = e; + tail = (tail + 1) % storage.length; + return result; + } + + public boolean offer(double e) { + if (isFull()) { + return false; + } + storage[tail] = e; + tail = (tail + 1) % storage.length; + return true; + } + + public double remove() { + if (isEmpty()) { + throw new NoSuchElementException(); + } + double e = storage[head]; + head = (head + 1) % storage.length; + return e; + } + + public double poll(double onEmpty) { + if (isEmpty()) { + return onEmpty; + } + double e = storage[head]; + head = (head + 1) % storage.length; + return e; + } + + public double element() { + if (isEmpty()) { + throw new NoSuchElementException(); + } + return storage[head]; + } + + public double peek(double onEmpty) { + if (isEmpty()) { + return onEmpty; + } + return storage[head]; + } + + public double front() { + return front(0); + } + + public double front(int offset) { + if (offset >= size()) { + throw new NoSuchElementException(); + } + return storage[(head + offset) % storage.length]; + } + + public double back() { + if (isEmpty()) { + throw new NoSuchElementException(); + } + return tail == 0 ? storage[storage.length - 1] : storage[tail - 1]; + } + + public double peekBack(double onEmpty) { + if (isEmpty()) { + return onEmpty; + } + return tail == 0 ? storage[storage.length - 1] : storage[tail - 1]; + } + + public Iterator iterator() { + return new Iterator(); + } + + public class Iterator { + int count = -1; + + public boolean hasNext() { + return count + 1 < size(); + } + + public double next() { + count++; + return storage[(head + count) % storage.length]; + } + + public void remove() { + throw new UnsupportedOperationException(); + } + } + + public double[] getAll() { + int n = size(), h = head; + double[] result = new double[n]; + for (int i = 0; i < n; ++i) { + result[i] = storage[h]; + h = (h + 1) % storage.length; + } + return result; + } +} diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java new file mode 100644 index 00000000000..cb1ad5f4e1d --- /dev/null +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java @@ -0,0 +1,192 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit CharRingBuffer and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +package io.deephaven.base.ringbuffer; + +import java.io.Serializable; +import java.util.NoSuchElementException; + +/** + * A trivial circular buffer for primitive values, like java.util.concurrent.ArrayBlockingQueue but without all the + * synchronization and collection cruft. Storage is between head (incl.) and tail (excl.) wrapping around the end of the + * array. If the buffer is *not* growable, it will make room for a new element by dropping off the oldest element in the + * buffer instead. + */ +public class FloatRingBuffer implements Serializable { + protected final boolean growable; + protected float[] storage; + protected int head, tail; + + private void grow() { + if (growable) { + float[] newStorage = new float[storage.length * 2]; + if (tail > head) { + System.arraycopy(storage, head, newStorage, 0, tail - head); + tail = tail - head; + } else { + System.arraycopy(storage, head, newStorage, 0, storage.length - head); + System.arraycopy(storage, 0, newStorage, storage.length - head, tail); + tail += storage.length - head; + } + head = 0; + storage = newStorage; + } else { + head = (head + 1) % storage.length; + } + } + + public boolean isFull() { + return (tail + 1) % storage.length == head; + } + + public FloatRingBuffer(int capacity) { + this(capacity, true); + } + + public FloatRingBuffer(int capacity, boolean growable) { + this.growable = growable; + this.storage = new float[capacity + 1]; + this.tail = this.head = 0; + } + + public boolean isEmpty() { + return tail == head; + } + + public int size() { + return tail >= head ? (tail - head) : (tail + (storage.length - head)); + } + + public int capacity() { + return storage.length - 1; + } + + public int remaining() { + return capacity() - size(); + } + + public void clear() { + tail = head = 0; + } + + public boolean add(float e) { + if (isFull()) { + grow(); + } + storage[tail] = e; + tail = (tail + 1) % storage.length; + return true; + } + + public float addOverwrite(float e, float notFullResult) { + float result = notFullResult; + if (isFull()) { + result = remove(); + } + storage[tail] = e; + tail = (tail + 1) % storage.length; + return result; + } + + public boolean offer(float e) { + if (isFull()) { + return false; + } + storage[tail] = e; + tail = (tail + 1) % storage.length; + return true; + } + + public float remove() { + if (isEmpty()) { + throw new NoSuchElementException(); + } + float e = storage[head]; + head = (head + 1) % storage.length; + return e; + } + + public float poll(float onEmpty) { + if (isEmpty()) { + return onEmpty; + } + float e = storage[head]; + head = (head + 1) % storage.length; + return e; + } + + public float element() { + if (isEmpty()) { + throw new NoSuchElementException(); + } + return storage[head]; + } + + public float peek(float onEmpty) { + if (isEmpty()) { + return onEmpty; + } + return storage[head]; + } + + public float front() { + return front(0); + } + + public float front(int offset) { + if (offset >= size()) { + throw new NoSuchElementException(); + } + return storage[(head + offset) % storage.length]; + } + + public float back() { + if (isEmpty()) { + throw new NoSuchElementException(); + } + return tail == 0 ? storage[storage.length - 1] : storage[tail - 1]; + } + + public float peekBack(float onEmpty) { + if (isEmpty()) { + return onEmpty; + } + return tail == 0 ? storage[storage.length - 1] : storage[tail - 1]; + } + + public Iterator iterator() { + return new Iterator(); + } + + public class Iterator { + int count = -1; + + public boolean hasNext() { + return count + 1 < size(); + } + + public float next() { + count++; + return storage[(head + count) % storage.length]; + } + + public void remove() { + throw new UnsupportedOperationException(); + } + } + + public float[] getAll() { + int n = size(), h = head; + float[] result = new float[n]; + for (int i = 0; i < n; ++i) { + result[i] = storage[h]; + h = (h + 1) % storage.length; + } + return result; + } +} diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java new file mode 100644 index 00000000000..98012f97a9e --- /dev/null +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java @@ -0,0 +1,192 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit CharRingBuffer and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +package io.deephaven.base.ringbuffer; + +import java.io.Serializable; +import java.util.NoSuchElementException; + +/** + * A trivial circular buffer for primitive values, like java.util.concurrent.ArrayBlockingQueue but without all the + * synchronization and collection cruft. Storage is between head (incl.) and tail (excl.) wrapping around the end of the + * array. If the buffer is *not* growable, it will make room for a new element by dropping off the oldest element in the + * buffer instead. + */ +public class IntRingBuffer implements Serializable { + protected final boolean growable; + protected int[] storage; + protected int head, tail; + + private void grow() { + if (growable) { + int[] newStorage = new int[storage.length * 2]; + if (tail > head) { + System.arraycopy(storage, head, newStorage, 0, tail - head); + tail = tail - head; + } else { + System.arraycopy(storage, head, newStorage, 0, storage.length - head); + System.arraycopy(storage, 0, newStorage, storage.length - head, tail); + tail += storage.length - head; + } + head = 0; + storage = newStorage; + } else { + head = (head + 1) % storage.length; + } + } + + public boolean isFull() { + return (tail + 1) % storage.length == head; + } + + public IntRingBuffer(int capacity) { + this(capacity, true); + } + + public IntRingBuffer(int capacity, boolean growable) { + this.growable = growable; + this.storage = new int[capacity + 1]; + this.tail = this.head = 0; + } + + public boolean isEmpty() { + return tail == head; + } + + public int size() { + return tail >= head ? (tail - head) : (tail + (storage.length - head)); + } + + public int capacity() { + return storage.length - 1; + } + + public int remaining() { + return capacity() - size(); + } + + public void clear() { + tail = head = 0; + } + + public boolean add(int e) { + if (isFull()) { + grow(); + } + storage[tail] = e; + tail = (tail + 1) % storage.length; + return true; + } + + public int addOverwrite(int e, int notFullResult) { + int result = notFullResult; + if (isFull()) { + result = remove(); + } + storage[tail] = e; + tail = (tail + 1) % storage.length; + return result; + } + + public boolean offer(int e) { + if (isFull()) { + return false; + } + storage[tail] = e; + tail = (tail + 1) % storage.length; + return true; + } + + public int remove() { + if (isEmpty()) { + throw new NoSuchElementException(); + } + int e = storage[head]; + head = (head + 1) % storage.length; + return e; + } + + public int poll(int onEmpty) { + if (isEmpty()) { + return onEmpty; + } + int e = storage[head]; + head = (head + 1) % storage.length; + return e; + } + + public int element() { + if (isEmpty()) { + throw new NoSuchElementException(); + } + return storage[head]; + } + + public int peek(int onEmpty) { + if (isEmpty()) { + return onEmpty; + } + return storage[head]; + } + + public int front() { + return front(0); + } + + public int front(int offset) { + if (offset >= size()) { + throw new NoSuchElementException(); + } + return storage[(head + offset) % storage.length]; + } + + public int back() { + if (isEmpty()) { + throw new NoSuchElementException(); + } + return tail == 0 ? storage[storage.length - 1] : storage[tail - 1]; + } + + public int peekBack(int onEmpty) { + if (isEmpty()) { + return onEmpty; + } + return tail == 0 ? storage[storage.length - 1] : storage[tail - 1]; + } + + public Iterator iterator() { + return new Iterator(); + } + + public class Iterator { + int count = -1; + + public boolean hasNext() { + return count + 1 < size(); + } + + public int next() { + count++; + return storage[(head + count) % storage.length]; + } + + public void remove() { + throw new UnsupportedOperationException(); + } + } + + public int[] getAll() { + int n = size(), h = head; + int[] result = new int[n]; + for (int i = 0; i < n; ++i) { + result[i] = storage[h]; + h = (h + 1) % storage.length; + } + return result; + } +} diff --git a/Base/src/main/java/io/deephaven/base/LongRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java similarity index 90% rename from Base/src/main/java/io/deephaven/base/LongRingBuffer.java rename to Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java index 05a69035972..3f447516632 100644 --- a/Base/src/main/java/io/deephaven/base/LongRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java @@ -1,22 +1,23 @@ /** * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending */ -package io.deephaven.base; +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit CharRingBuffer and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +package io.deephaven.base.ringbuffer; import java.io.Serializable; import java.util.NoSuchElementException; /** - * A trivial circular buffer for primitive longs, like java.util.concurrent.ArrayBlockingQueue but without all the + * A trivial circular buffer for primitive values, like java.util.concurrent.ArrayBlockingQueue but without all the * synchronization and collection cruft. Storage is between head (incl.) and tail (excl.) wrapping around the end of the * array. If the buffer is *not* growable, it will make room for a new element by dropping off the oldest element in the * buffer instead. */ public class LongRingBuffer implements Serializable { - - public static final boolean GROWABLE_NO = false; - public static final boolean GROWABLE_YES = true; - protected final boolean growable; protected long[] storage; protected int head, tail; diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/ObjectRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/ObjectRingBuffer.java new file mode 100644 index 00000000000..d3c6b6910a3 --- /dev/null +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/ObjectRingBuffer.java @@ -0,0 +1,192 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit CharRingBuffer and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +package io.deephaven.base.ringbuffer; + +import java.io.Serializable; +import java.util.NoSuchElementException; + +/** + * A trivial circular buffer for primitive values, like java.util.concurrent.ArrayBlockingQueue but without all the + * synchronization and collection cruft. Storage is between head (incl.) and tail (excl.) wrapping around the end of the + * array. If the buffer is *not* growable, it will make room for a new element by dropping off the oldest element in the + * buffer instead. + */ +public class ObjectRingBuffer implements Serializable { + protected final boolean growable; + protected Object[] storage; + protected int head, tail; + + private void grow() { + if (growable) { + Object[] newStorage = new Object[storage.length * 2]; + if (tail > head) { + System.arraycopy(storage, head, newStorage, 0, tail - head); + tail = tail - head; + } else { + System.arraycopy(storage, head, newStorage, 0, storage.length - head); + System.arraycopy(storage, 0, newStorage, storage.length - head, tail); + tail += storage.length - head; + } + head = 0; + storage = newStorage; + } else { + head = (head + 1) % storage.length; + } + } + + public boolean isFull() { + return (tail + 1) % storage.length == head; + } + + public ObjectRingBuffer(int capacity) { + this(capacity, true); + } + + public ObjectRingBuffer(int capacity, boolean growable) { + this.growable = growable; + this.storage = new Object[capacity + 1]; + this.tail = this.head = 0; + } + + public boolean isEmpty() { + return tail == head; + } + + public int size() { + return tail >= head ? (tail - head) : (tail + (storage.length - head)); + } + + public int capacity() { + return storage.length - 1; + } + + public int remaining() { + return capacity() - size(); + } + + public void clear() { + tail = head = 0; + } + + public boolean add(Object e) { + if (isFull()) { + grow(); + } + storage[tail] = e; + tail = (tail + 1) % storage.length; + return true; + } + + public Object addOverwrite(Object e, Object notFullResult) { + Object result = notFullResult; + if (isFull()) { + result = remove(); + } + storage[tail] = e; + tail = (tail + 1) % storage.length; + return result; + } + + public boolean offer(Object e) { + if (isFull()) { + return false; + } + storage[tail] = e; + tail = (tail + 1) % storage.length; + return true; + } + + public Object remove() { + if (isEmpty()) { + throw new NoSuchElementException(); + } + Object e = storage[head]; + head = (head + 1) % storage.length; + return e; + } + + public Object poll(Object onEmpty) { + if (isEmpty()) { + return onEmpty; + } + Object e = storage[head]; + head = (head + 1) % storage.length; + return e; + } + + public Object element() { + if (isEmpty()) { + throw new NoSuchElementException(); + } + return storage[head]; + } + + public Object peek(Object onEmpty) { + if (isEmpty()) { + return onEmpty; + } + return storage[head]; + } + + public Object front() { + return front(0); + } + + public Object front(int offset) { + if (offset >= size()) { + throw new NoSuchElementException(); + } + return storage[(head + offset) % storage.length]; + } + + public Object back() { + if (isEmpty()) { + throw new NoSuchElementException(); + } + return tail == 0 ? storage[storage.length - 1] : storage[tail - 1]; + } + + public Object peekBack(Object onEmpty) { + if (isEmpty()) { + return onEmpty; + } + return tail == 0 ? storage[storage.length - 1] : storage[tail - 1]; + } + + public Iterator iterator() { + return new Iterator(); + } + + public class Iterator { + int count = -1; + + public boolean hasNext() { + return count + 1 < size(); + } + + public Object next() { + count++; + return storage[(head + count) % storage.length]; + } + + public void remove() { + throw new UnsupportedOperationException(); + } + } + + public Object[] getAll() { + int n = size(), h = head; + Object[] result = new Object[n]; + for (int i = 0; i < n; ++i) { + result[i] = storage[h]; + h = (h + 1) % storage.length; + } + return result; + } +} diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java new file mode 100644 index 00000000000..03f7a6095f3 --- /dev/null +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java @@ -0,0 +1,192 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit CharRingBuffer and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +package io.deephaven.base.ringbuffer; + +import java.io.Serializable; +import java.util.NoSuchElementException; + +/** + * A trivial circular buffer for primitive values, like java.util.concurrent.ArrayBlockingQueue but without all the + * synchronization and collection cruft. Storage is between head (incl.) and tail (excl.) wrapping around the end of the + * array. If the buffer is *not* growable, it will make room for a new element by dropping off the oldest element in the + * buffer instead. + */ +public class ShortRingBuffer implements Serializable { + protected final boolean growable; + protected short[] storage; + protected int head, tail; + + private void grow() { + if (growable) { + short[] newStorage = new short[storage.length * 2]; + if (tail > head) { + System.arraycopy(storage, head, newStorage, 0, tail - head); + tail = tail - head; + } else { + System.arraycopy(storage, head, newStorage, 0, storage.length - head); + System.arraycopy(storage, 0, newStorage, storage.length - head, tail); + tail += storage.length - head; + } + head = 0; + storage = newStorage; + } else { + head = (head + 1) % storage.length; + } + } + + public boolean isFull() { + return (tail + 1) % storage.length == head; + } + + public ShortRingBuffer(int capacity) { + this(capacity, true); + } + + public ShortRingBuffer(int capacity, boolean growable) { + this.growable = growable; + this.storage = new short[capacity + 1]; + this.tail = this.head = 0; + } + + public boolean isEmpty() { + return tail == head; + } + + public int size() { + return tail >= head ? (tail - head) : (tail + (storage.length - head)); + } + + public int capacity() { + return storage.length - 1; + } + + public int remaining() { + return capacity() - size(); + } + + public void clear() { + tail = head = 0; + } + + public boolean add(short e) { + if (isFull()) { + grow(); + } + storage[tail] = e; + tail = (tail + 1) % storage.length; + return true; + } + + public short addOverwrite(short e, short notFullResult) { + short result = notFullResult; + if (isFull()) { + result = remove(); + } + storage[tail] = e; + tail = (tail + 1) % storage.length; + return result; + } + + public boolean offer(short e) { + if (isFull()) { + return false; + } + storage[tail] = e; + tail = (tail + 1) % storage.length; + return true; + } + + public short remove() { + if (isEmpty()) { + throw new NoSuchElementException(); + } + short e = storage[head]; + head = (head + 1) % storage.length; + return e; + } + + public short poll(short onEmpty) { + if (isEmpty()) { + return onEmpty; + } + short e = storage[head]; + head = (head + 1) % storage.length; + return e; + } + + public short element() { + if (isEmpty()) { + throw new NoSuchElementException(); + } + return storage[head]; + } + + public short peek(short onEmpty) { + if (isEmpty()) { + return onEmpty; + } + return storage[head]; + } + + public short front() { + return front(0); + } + + public short front(int offset) { + if (offset >= size()) { + throw new NoSuchElementException(); + } + return storage[(head + offset) % storage.length]; + } + + public short back() { + if (isEmpty()) { + throw new NoSuchElementException(); + } + return tail == 0 ? storage[storage.length - 1] : storage[tail - 1]; + } + + public short peekBack(short onEmpty) { + if (isEmpty()) { + return onEmpty; + } + return tail == 0 ? storage[storage.length - 1] : storage[tail - 1]; + } + + public Iterator iterator() { + return new Iterator(); + } + + public class Iterator { + int count = -1; + + public boolean hasNext() { + return count + 1 < size(); + } + + public short next() { + count++; + return storage[(head + count) % storage.length]; + } + + public void remove() { + throw new UnsupportedOperationException(); + } + } + + public short[] getAll() { + int n = size(), h = head; + short[] result = new short[n]; + for (int i = 0; i < n; ++i) { + result[i] = storage[h]; + h = (h + 1) % storage.length; + } + return result; + } +} diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/ByteRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/ByteRingBufferTest.java new file mode 100644 index 00000000000..6f7ca5452db --- /dev/null +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/ByteRingBufferTest.java @@ -0,0 +1,372 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit CharRingBufferTest and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +package io.deephaven.base.ringbuffer; + +import junit.framework.TestCase; + +import java.util.NoSuchElementException; + +public class ByteRingBufferTest extends TestCase { + + final byte SENTINEL = Byte.MIN_VALUE; + + private void assertEmpty(ByteRingBuffer rb) { + assertTrue(rb.isEmpty()); + assertEquals(0, rb.size()); + + assertEquals(SENTINEL, rb.peek(SENTINEL)); + try { + rb.element(); + fail("queue should be empty"); + } catch (NoSuchElementException x) { + // expected + } + + assertEquals(SENTINEL, rb.poll(SENTINEL)); + try { + rb.remove(); + fail("queue should be empty"); + } catch (NoSuchElementException x) { + // expected + } + } + + private void assertFull(ByteRingBuffer rb) { + assertFalse(rb.isEmpty()); + assertEquals(rb.capacity(), rb.size()); + } + + private void assertNotEmpty(ByteRingBuffer rb, int expectedSize, byte expectedHead) { + assertFalse(rb.isEmpty()); + assertEquals(expectedSize, rb.size()); + + assertTrue(expectedHead == rb.peek(SENTINEL)); + try { + assertTrue(expectedHead == rb.element()); + } catch (NoSuchElementException x) { + fail("queue should not be empty"); + } + } + + private void assertAdd(ByteRingBuffer rb, byte newHead, int expectedSize, byte expectedHead) { + assertTrue(rb.add(newHead)); + assertNotEmpty(rb, expectedSize, expectedHead); + } + + private void assertOffer(ByteRingBuffer rb, byte newHead, int expectedSize, byte expectedHead) { + assertTrue(rb.offer(newHead)); + assertNotEmpty(rb, expectedSize, expectedHead); + } + + private void assertPoll(ByteRingBuffer rb, int expectedSize, byte expectedHead) { + assertNotEmpty(rb, expectedSize, expectedHead); + assertTrue(expectedHead == rb.poll(SENTINEL)); + } + + private void assertRemove(ByteRingBuffer rb, int expectedSize, byte expectedHead) { + assertNotEmpty(rb, expectedSize, expectedHead); + try { + assertTrue(expectedHead == rb.remove()); + } catch (NoSuchElementException x) { + fail("queue should not be empty"); + } + } + + byte A = 'A'; + byte B = 'B'; + byte C = 'C'; + byte D = 'D'; + byte E = 'E'; + byte F = 'F'; + + public void testAddRemove() { + + ByteRingBuffer rb = new ByteRingBuffer(3); + + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertFull(rb); + + assertRemove(rb, 3, A); + assertRemove(rb, 2, B); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertRemove(rb, 2, A); + assertAdd(rb, C, 2, B); + assertRemove(rb, 2, B); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertRemove(rb, 1, A); + assertEmpty(rb); + assertAdd(rb, B, 1, B); + assertRemove(rb, 1, B); + assertEmpty(rb); + assertAdd(rb, C, 1, C); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, D, 1, D); + assertRemove(rb, 1, D); + assertEmpty(rb); + assertAdd(rb, E, 1, E); + assertRemove(rb, 1, E); + assertEmpty(rb); + assertAdd(rb, F, 1, F); + assertRemove(rb, 1, F); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertFull(rb); + + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + assertAdd(rb, F, 6, A); + + assertRemove(rb, 6, A); + assertRemove(rb, 5, B); + assertRemove(rb, 4, C); + assertRemove(rb, 3, D); + assertRemove(rb, 2, E); + assertRemove(rb, 1, F); + assertEmpty(rb); + } + + public void testOfferPoll() { + ByteRingBuffer rb = new ByteRingBuffer(3); + + assertEmpty(rb); + + assertOffer(rb, A, 1, A); + assertOffer(rb, B, 2, A); + assertOffer(rb, C, 3, A); + + assertFull(rb); + + assertPoll(rb, 3, A); + assertPoll(rb, 2, B); + assertPoll(rb, 1, C); + assertEmpty(rb); + + assertOffer(rb, A, 1, A); + assertOffer(rb, B, 2, A); + assertPoll(rb, 2, A); + assertOffer(rb, C, 2, B); + assertPoll(rb, 2, B); + assertPoll(rb, 1, C); + assertEmpty(rb); + + assertOffer(rb, A, 1, A); + assertPoll(rb, 1, A); + assertEmpty(rb); + assertOffer(rb, B, 1, B); + assertPoll(rb, 1, B); + assertEmpty(rb); + assertOffer(rb, C, 1, C); + assertPoll(rb, 1, C); + assertEmpty(rb); + + assertOffer(rb, D, 1, D); + assertPoll(rb, 1, D); + assertEmpty(rb); + assertOffer(rb, E, 1, E); + assertPoll(rb, 1, E); + assertEmpty(rb); + assertOffer(rb, F, 1, F); + assertPoll(rb, 1, F); + assertEmpty(rb); + + assertOffer(rb, A, 1, A); + assertOffer(rb, B, 2, A); + assertOffer(rb, C, 3, A); + assertFull(rb); + + assertAdd(rb, D, 4, A); // need one add to grow it + assertOffer(rb, E, 5, A); // NOTE: assumes capacity grows by at least a factor of two + assertOffer(rb, F, 6, A); + + assertPoll(rb, 6, A); + assertPoll(rb, 5, B); + assertPoll(rb, 4, C); + assertPoll(rb, 3, D); + assertPoll(rb, 2, E); + assertPoll(rb, 1, F); + assertEmpty(rb); + } + + public void testGrowSimple() { + ByteRingBuffer rb = new ByteRingBuffer(5); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + assertFull(rb); + + // this will grow; the elements are in a single contiguous block + assertAdd(rb, F, 6, A); + + assertRemove(rb, 6, A); + assertRemove(rb, 5, B); + assertRemove(rb, 4, C); + assertRemove(rb, 3, D); + assertRemove(rb, 2, E); + assertRemove(rb, 1, F); + assertEmpty(rb); + } + + public void testGrowComplex() { + ByteRingBuffer rb = new ByteRingBuffer(5); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertRemove(rb, 3, A); + assertRemove(rb, 2, B); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + assertFull(rb); + + // this will grow; the elements are in two blocks + assertAdd(rb, F, 6, A); + + assertRemove(rb, 6, A); + assertRemove(rb, 5, B); + assertRemove(rb, 4, C); + assertRemove(rb, 3, D); + assertRemove(rb, 2, E); + assertRemove(rb, 1, F); + assertEmpty(rb); + } + + public void testIterator() { + ByteRingBuffer rb = new ByteRingBuffer(3); + + ByteRingBuffer.Iterator iter = rb.iterator(); + assertFalse(iter.hasNext()); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertFull(rb); + + iter = rb.iterator(); + assertTrue(iter.hasNext()); + assertEquals(A, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(B, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(C, iter.next()); + assertFalse(iter.hasNext()); + + assertRemove(rb, 3, A); + assertAdd(rb, D, 3, B); + + iter = rb.iterator(); + assertTrue(iter.hasNext()); + assertEquals(B, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(C, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(D, iter.next()); + assertFalse(iter.hasNext()); + + assertRemove(rb, 3, B); + + iter = rb.iterator(); + assertEquals(C, iter.next()); + assertEquals(D, iter.next()); + assertFalse(iter.hasNext()); + + assertAdd(rb, E, 3, C); + assertAdd(rb, F, 4, C); + + iter = rb.iterator(); + assertTrue(iter.hasNext()); + assertEquals(C, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(D, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(E, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(F, iter.next()); + assertFalse(iter.hasNext()); + } + + public void testBack() { + ByteRingBuffer rb = new ByteRingBuffer(5); + + assertAdd(rb, A, 1, A); + assertEquals(rb.back(), A); + + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertEquals(rb.back(), C); + } + + public void testBackWhenEmpty() { + ByteRingBuffer rb = new ByteRingBuffer(5); + try { + rb.back(); + fail("expected a NoSuchElement exception"); + } catch (NoSuchElementException x) { + // expected + } + } + + public void testBackTailIsZero() { + ByteRingBuffer rb = new ByteRingBuffer(5); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + assertFull(rb); + + assertRemove(rb, 5, A); + assertAdd(rb, F, 5, B); + assertEquals(rb.back(), F); + } + + public void testLargeAmounts() { + ByteRingBuffer rb = new ByteRingBuffer(3); + + for (int i = 0; i < 100; i++) + rb.add((byte)i); + + for (int i = 100; i < 200; i++) { + rb.add((byte)i); + assertEquals((byte)(i - 100 + 1), rb.front(1)); + assertEquals((byte)(i - 100), rb.poll(SENTINEL)); + } + + for (int i = 200; i < 300; i++) { + if (i < 299) + assertEquals((byte)(i - 100 + 1), rb.front(1)); + assertEquals((byte)(i - 100), rb.poll(SENTINEL)); + } + } +} diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java new file mode 100644 index 00000000000..2b21ff2e92c --- /dev/null +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java @@ -0,0 +1,367 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.base.ringbuffer; + +import junit.framework.TestCase; + +import java.util.NoSuchElementException; + +public class CharRingBufferTest extends TestCase { + + final char SENTINEL = Character.MIN_VALUE; + + private void assertEmpty(CharRingBuffer rb) { + assertTrue(rb.isEmpty()); + assertEquals(0, rb.size()); + + assertEquals(SENTINEL, rb.peek(SENTINEL)); + try { + rb.element(); + fail("queue should be empty"); + } catch (NoSuchElementException x) { + // expected + } + + assertEquals(SENTINEL, rb.poll(SENTINEL)); + try { + rb.remove(); + fail("queue should be empty"); + } catch (NoSuchElementException x) { + // expected + } + } + + private void assertFull(CharRingBuffer rb) { + assertFalse(rb.isEmpty()); + assertEquals(rb.capacity(), rb.size()); + } + + private void assertNotEmpty(CharRingBuffer rb, int expectedSize, char expectedHead) { + assertFalse(rb.isEmpty()); + assertEquals(expectedSize, rb.size()); + + assertTrue(expectedHead == rb.peek(SENTINEL)); + try { + assertTrue(expectedHead == rb.element()); + } catch (NoSuchElementException x) { + fail("queue should not be empty"); + } + } + + private void assertAdd(CharRingBuffer rb, char newHead, int expectedSize, char expectedHead) { + assertTrue(rb.add(newHead)); + assertNotEmpty(rb, expectedSize, expectedHead); + } + + private void assertOffer(CharRingBuffer rb, char newHead, int expectedSize, char expectedHead) { + assertTrue(rb.offer(newHead)); + assertNotEmpty(rb, expectedSize, expectedHead); + } + + private void assertPoll(CharRingBuffer rb, int expectedSize, char expectedHead) { + assertNotEmpty(rb, expectedSize, expectedHead); + assertTrue(expectedHead == rb.poll(SENTINEL)); + } + + private void assertRemove(CharRingBuffer rb, int expectedSize, char expectedHead) { + assertNotEmpty(rb, expectedSize, expectedHead); + try { + assertTrue(expectedHead == rb.remove()); + } catch (NoSuchElementException x) { + fail("queue should not be empty"); + } + } + + char A = 'A'; + char B = 'B'; + char C = 'C'; + char D = 'D'; + char E = 'E'; + char F = 'F'; + + public void testAddRemove() { + + CharRingBuffer rb = new CharRingBuffer(3); + + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertFull(rb); + + assertRemove(rb, 3, A); + assertRemove(rb, 2, B); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertRemove(rb, 2, A); + assertAdd(rb, C, 2, B); + assertRemove(rb, 2, B); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertRemove(rb, 1, A); + assertEmpty(rb); + assertAdd(rb, B, 1, B); + assertRemove(rb, 1, B); + assertEmpty(rb); + assertAdd(rb, C, 1, C); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, D, 1, D); + assertRemove(rb, 1, D); + assertEmpty(rb); + assertAdd(rb, E, 1, E); + assertRemove(rb, 1, E); + assertEmpty(rb); + assertAdd(rb, F, 1, F); + assertRemove(rb, 1, F); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertFull(rb); + + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + assertAdd(rb, F, 6, A); + + assertRemove(rb, 6, A); + assertRemove(rb, 5, B); + assertRemove(rb, 4, C); + assertRemove(rb, 3, D); + assertRemove(rb, 2, E); + assertRemove(rb, 1, F); + assertEmpty(rb); + } + + public void testOfferPoll() { + CharRingBuffer rb = new CharRingBuffer(3); + + assertEmpty(rb); + + assertOffer(rb, A, 1, A); + assertOffer(rb, B, 2, A); + assertOffer(rb, C, 3, A); + + assertFull(rb); + + assertPoll(rb, 3, A); + assertPoll(rb, 2, B); + assertPoll(rb, 1, C); + assertEmpty(rb); + + assertOffer(rb, A, 1, A); + assertOffer(rb, B, 2, A); + assertPoll(rb, 2, A); + assertOffer(rb, C, 2, B); + assertPoll(rb, 2, B); + assertPoll(rb, 1, C); + assertEmpty(rb); + + assertOffer(rb, A, 1, A); + assertPoll(rb, 1, A); + assertEmpty(rb); + assertOffer(rb, B, 1, B); + assertPoll(rb, 1, B); + assertEmpty(rb); + assertOffer(rb, C, 1, C); + assertPoll(rb, 1, C); + assertEmpty(rb); + + assertOffer(rb, D, 1, D); + assertPoll(rb, 1, D); + assertEmpty(rb); + assertOffer(rb, E, 1, E); + assertPoll(rb, 1, E); + assertEmpty(rb); + assertOffer(rb, F, 1, F); + assertPoll(rb, 1, F); + assertEmpty(rb); + + assertOffer(rb, A, 1, A); + assertOffer(rb, B, 2, A); + assertOffer(rb, C, 3, A); + assertFull(rb); + + assertAdd(rb, D, 4, A); // need one add to grow it + assertOffer(rb, E, 5, A); // NOTE: assumes capacity grows by at least a factor of two + assertOffer(rb, F, 6, A); + + assertPoll(rb, 6, A); + assertPoll(rb, 5, B); + assertPoll(rb, 4, C); + assertPoll(rb, 3, D); + assertPoll(rb, 2, E); + assertPoll(rb, 1, F); + assertEmpty(rb); + } + + public void testGrowSimple() { + CharRingBuffer rb = new CharRingBuffer(5); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + assertFull(rb); + + // this will grow; the elements are in a single contiguous block + assertAdd(rb, F, 6, A); + + assertRemove(rb, 6, A); + assertRemove(rb, 5, B); + assertRemove(rb, 4, C); + assertRemove(rb, 3, D); + assertRemove(rb, 2, E); + assertRemove(rb, 1, F); + assertEmpty(rb); + } + + public void testGrowComplex() { + CharRingBuffer rb = new CharRingBuffer(5); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertRemove(rb, 3, A); + assertRemove(rb, 2, B); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + assertFull(rb); + + // this will grow; the elements are in two blocks + assertAdd(rb, F, 6, A); + + assertRemove(rb, 6, A); + assertRemove(rb, 5, B); + assertRemove(rb, 4, C); + assertRemove(rb, 3, D); + assertRemove(rb, 2, E); + assertRemove(rb, 1, F); + assertEmpty(rb); + } + + public void testIterator() { + CharRingBuffer rb = new CharRingBuffer(3); + + CharRingBuffer.Iterator iter = rb.iterator(); + assertFalse(iter.hasNext()); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertFull(rb); + + iter = rb.iterator(); + assertTrue(iter.hasNext()); + assertEquals(A, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(B, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(C, iter.next()); + assertFalse(iter.hasNext()); + + assertRemove(rb, 3, A); + assertAdd(rb, D, 3, B); + + iter = rb.iterator(); + assertTrue(iter.hasNext()); + assertEquals(B, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(C, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(D, iter.next()); + assertFalse(iter.hasNext()); + + assertRemove(rb, 3, B); + + iter = rb.iterator(); + assertEquals(C, iter.next()); + assertEquals(D, iter.next()); + assertFalse(iter.hasNext()); + + assertAdd(rb, E, 3, C); + assertAdd(rb, F, 4, C); + + iter = rb.iterator(); + assertTrue(iter.hasNext()); + assertEquals(C, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(D, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(E, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(F, iter.next()); + assertFalse(iter.hasNext()); + } + + public void testBack() { + CharRingBuffer rb = new CharRingBuffer(5); + + assertAdd(rb, A, 1, A); + assertEquals(rb.back(), A); + + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertEquals(rb.back(), C); + } + + public void testBackWhenEmpty() { + CharRingBuffer rb = new CharRingBuffer(5); + try { + rb.back(); + fail("expected a NoSuchElement exception"); + } catch (NoSuchElementException x) { + // expected + } + } + + public void testBackTailIsZero() { + CharRingBuffer rb = new CharRingBuffer(5); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + assertFull(rb); + + assertRemove(rb, 5, A); + assertAdd(rb, F, 5, B); + assertEquals(rb.back(), F); + } + + public void testLargeAmounts() { + CharRingBuffer rb = new CharRingBuffer(3); + + for (int i = 0; i < 100; i++) + rb.add((char)i); + + for (int i = 100; i < 200; i++) { + rb.add((char)i); + assertEquals((char)(i - 100 + 1), rb.front(1)); + assertEquals((char)(i - 100), rb.poll(SENTINEL)); + } + + for (int i = 200; i < 300; i++) { + if (i < 299) + assertEquals((char)(i - 100 + 1), rb.front(1)); + assertEquals((char)(i - 100), rb.poll(SENTINEL)); + } + } +} diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/DoubleRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/DoubleRingBufferTest.java new file mode 100644 index 00000000000..d7a2e87ce35 --- /dev/null +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/DoubleRingBufferTest.java @@ -0,0 +1,372 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit CharRingBufferTest and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +package io.deephaven.base.ringbuffer; + +import junit.framework.TestCase; + +import java.util.NoSuchElementException; + +public class DoubleRingBufferTest extends TestCase { + + final double SENTINEL = Double.MIN_VALUE; + + private void assertEmpty(DoubleRingBuffer rb) { + assertTrue(rb.isEmpty()); + assertEquals(0, rb.size()); + + assertEquals(SENTINEL, rb.peek(SENTINEL)); + try { + rb.element(); + fail("queue should be empty"); + } catch (NoSuchElementException x) { + // expected + } + + assertEquals(SENTINEL, rb.poll(SENTINEL)); + try { + rb.remove(); + fail("queue should be empty"); + } catch (NoSuchElementException x) { + // expected + } + } + + private void assertFull(DoubleRingBuffer rb) { + assertFalse(rb.isEmpty()); + assertEquals(rb.capacity(), rb.size()); + } + + private void assertNotEmpty(DoubleRingBuffer rb, int expectedSize, double expectedHead) { + assertFalse(rb.isEmpty()); + assertEquals(expectedSize, rb.size()); + + assertTrue(expectedHead == rb.peek(SENTINEL)); + try { + assertTrue(expectedHead == rb.element()); + } catch (NoSuchElementException x) { + fail("queue should not be empty"); + } + } + + private void assertAdd(DoubleRingBuffer rb, double newHead, int expectedSize, double expectedHead) { + assertTrue(rb.add(newHead)); + assertNotEmpty(rb, expectedSize, expectedHead); + } + + private void assertOffer(DoubleRingBuffer rb, double newHead, int expectedSize, double expectedHead) { + assertTrue(rb.offer(newHead)); + assertNotEmpty(rb, expectedSize, expectedHead); + } + + private void assertPoll(DoubleRingBuffer rb, int expectedSize, double expectedHead) { + assertNotEmpty(rb, expectedSize, expectedHead); + assertTrue(expectedHead == rb.poll(SENTINEL)); + } + + private void assertRemove(DoubleRingBuffer rb, int expectedSize, double expectedHead) { + assertNotEmpty(rb, expectedSize, expectedHead); + try { + assertTrue(expectedHead == rb.remove()); + } catch (NoSuchElementException x) { + fail("queue should not be empty"); + } + } + + double A = 'A'; + double B = 'B'; + double C = 'C'; + double D = 'D'; + double E = 'E'; + double F = 'F'; + + public void testAddRemove() { + + DoubleRingBuffer rb = new DoubleRingBuffer(3); + + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertFull(rb); + + assertRemove(rb, 3, A); + assertRemove(rb, 2, B); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertRemove(rb, 2, A); + assertAdd(rb, C, 2, B); + assertRemove(rb, 2, B); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertRemove(rb, 1, A); + assertEmpty(rb); + assertAdd(rb, B, 1, B); + assertRemove(rb, 1, B); + assertEmpty(rb); + assertAdd(rb, C, 1, C); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, D, 1, D); + assertRemove(rb, 1, D); + assertEmpty(rb); + assertAdd(rb, E, 1, E); + assertRemove(rb, 1, E); + assertEmpty(rb); + assertAdd(rb, F, 1, F); + assertRemove(rb, 1, F); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertFull(rb); + + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + assertAdd(rb, F, 6, A); + + assertRemove(rb, 6, A); + assertRemove(rb, 5, B); + assertRemove(rb, 4, C); + assertRemove(rb, 3, D); + assertRemove(rb, 2, E); + assertRemove(rb, 1, F); + assertEmpty(rb); + } + + public void testOfferPoll() { + DoubleRingBuffer rb = new DoubleRingBuffer(3); + + assertEmpty(rb); + + assertOffer(rb, A, 1, A); + assertOffer(rb, B, 2, A); + assertOffer(rb, C, 3, A); + + assertFull(rb); + + assertPoll(rb, 3, A); + assertPoll(rb, 2, B); + assertPoll(rb, 1, C); + assertEmpty(rb); + + assertOffer(rb, A, 1, A); + assertOffer(rb, B, 2, A); + assertPoll(rb, 2, A); + assertOffer(rb, C, 2, B); + assertPoll(rb, 2, B); + assertPoll(rb, 1, C); + assertEmpty(rb); + + assertOffer(rb, A, 1, A); + assertPoll(rb, 1, A); + assertEmpty(rb); + assertOffer(rb, B, 1, B); + assertPoll(rb, 1, B); + assertEmpty(rb); + assertOffer(rb, C, 1, C); + assertPoll(rb, 1, C); + assertEmpty(rb); + + assertOffer(rb, D, 1, D); + assertPoll(rb, 1, D); + assertEmpty(rb); + assertOffer(rb, E, 1, E); + assertPoll(rb, 1, E); + assertEmpty(rb); + assertOffer(rb, F, 1, F); + assertPoll(rb, 1, F); + assertEmpty(rb); + + assertOffer(rb, A, 1, A); + assertOffer(rb, B, 2, A); + assertOffer(rb, C, 3, A); + assertFull(rb); + + assertAdd(rb, D, 4, A); // need one add to grow it + assertOffer(rb, E, 5, A); // NOTE: assumes capacity grows by at least a factor of two + assertOffer(rb, F, 6, A); + + assertPoll(rb, 6, A); + assertPoll(rb, 5, B); + assertPoll(rb, 4, C); + assertPoll(rb, 3, D); + assertPoll(rb, 2, E); + assertPoll(rb, 1, F); + assertEmpty(rb); + } + + public void testGrowSimple() { + DoubleRingBuffer rb = new DoubleRingBuffer(5); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + assertFull(rb); + + // this will grow; the elements are in a single contiguous block + assertAdd(rb, F, 6, A); + + assertRemove(rb, 6, A); + assertRemove(rb, 5, B); + assertRemove(rb, 4, C); + assertRemove(rb, 3, D); + assertRemove(rb, 2, E); + assertRemove(rb, 1, F); + assertEmpty(rb); + } + + public void testGrowComplex() { + DoubleRingBuffer rb = new DoubleRingBuffer(5); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertRemove(rb, 3, A); + assertRemove(rb, 2, B); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + assertFull(rb); + + // this will grow; the elements are in two blocks + assertAdd(rb, F, 6, A); + + assertRemove(rb, 6, A); + assertRemove(rb, 5, B); + assertRemove(rb, 4, C); + assertRemove(rb, 3, D); + assertRemove(rb, 2, E); + assertRemove(rb, 1, F); + assertEmpty(rb); + } + + public void testIterator() { + DoubleRingBuffer rb = new DoubleRingBuffer(3); + + DoubleRingBuffer.Iterator iter = rb.iterator(); + assertFalse(iter.hasNext()); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertFull(rb); + + iter = rb.iterator(); + assertTrue(iter.hasNext()); + assertEquals(A, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(B, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(C, iter.next()); + assertFalse(iter.hasNext()); + + assertRemove(rb, 3, A); + assertAdd(rb, D, 3, B); + + iter = rb.iterator(); + assertTrue(iter.hasNext()); + assertEquals(B, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(C, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(D, iter.next()); + assertFalse(iter.hasNext()); + + assertRemove(rb, 3, B); + + iter = rb.iterator(); + assertEquals(C, iter.next()); + assertEquals(D, iter.next()); + assertFalse(iter.hasNext()); + + assertAdd(rb, E, 3, C); + assertAdd(rb, F, 4, C); + + iter = rb.iterator(); + assertTrue(iter.hasNext()); + assertEquals(C, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(D, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(E, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(F, iter.next()); + assertFalse(iter.hasNext()); + } + + public void testBack() { + DoubleRingBuffer rb = new DoubleRingBuffer(5); + + assertAdd(rb, A, 1, A); + assertEquals(rb.back(), A); + + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertEquals(rb.back(), C); + } + + public void testBackWhenEmpty() { + DoubleRingBuffer rb = new DoubleRingBuffer(5); + try { + rb.back(); + fail("expected a NoSuchElement exception"); + } catch (NoSuchElementException x) { + // expected + } + } + + public void testBackTailIsZero() { + DoubleRingBuffer rb = new DoubleRingBuffer(5); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + assertFull(rb); + + assertRemove(rb, 5, A); + assertAdd(rb, F, 5, B); + assertEquals(rb.back(), F); + } + + public void testLargeAmounts() { + DoubleRingBuffer rb = new DoubleRingBuffer(3); + + for (int i = 0; i < 100; i++) + rb.add((double)i); + + for (int i = 100; i < 200; i++) { + rb.add((double)i); + assertEquals((double)(i - 100 + 1), rb.front(1)); + assertEquals((double)(i - 100), rb.poll(SENTINEL)); + } + + for (int i = 200; i < 300; i++) { + if (i < 299) + assertEquals((double)(i - 100 + 1), rb.front(1)); + assertEquals((double)(i - 100), rb.poll(SENTINEL)); + } + } +} diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/FloatRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/FloatRingBufferTest.java new file mode 100644 index 00000000000..a77c1036f49 --- /dev/null +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/FloatRingBufferTest.java @@ -0,0 +1,372 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit CharRingBufferTest and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +package io.deephaven.base.ringbuffer; + +import junit.framework.TestCase; + +import java.util.NoSuchElementException; + +public class FloatRingBufferTest extends TestCase { + + final float SENTINEL = Float.MIN_VALUE; + + private void assertEmpty(FloatRingBuffer rb) { + assertTrue(rb.isEmpty()); + assertEquals(0, rb.size()); + + assertEquals(SENTINEL, rb.peek(SENTINEL)); + try { + rb.element(); + fail("queue should be empty"); + } catch (NoSuchElementException x) { + // expected + } + + assertEquals(SENTINEL, rb.poll(SENTINEL)); + try { + rb.remove(); + fail("queue should be empty"); + } catch (NoSuchElementException x) { + // expected + } + } + + private void assertFull(FloatRingBuffer rb) { + assertFalse(rb.isEmpty()); + assertEquals(rb.capacity(), rb.size()); + } + + private void assertNotEmpty(FloatRingBuffer rb, int expectedSize, float expectedHead) { + assertFalse(rb.isEmpty()); + assertEquals(expectedSize, rb.size()); + + assertTrue(expectedHead == rb.peek(SENTINEL)); + try { + assertTrue(expectedHead == rb.element()); + } catch (NoSuchElementException x) { + fail("queue should not be empty"); + } + } + + private void assertAdd(FloatRingBuffer rb, float newHead, int expectedSize, float expectedHead) { + assertTrue(rb.add(newHead)); + assertNotEmpty(rb, expectedSize, expectedHead); + } + + private void assertOffer(FloatRingBuffer rb, float newHead, int expectedSize, float expectedHead) { + assertTrue(rb.offer(newHead)); + assertNotEmpty(rb, expectedSize, expectedHead); + } + + private void assertPoll(FloatRingBuffer rb, int expectedSize, float expectedHead) { + assertNotEmpty(rb, expectedSize, expectedHead); + assertTrue(expectedHead == rb.poll(SENTINEL)); + } + + private void assertRemove(FloatRingBuffer rb, int expectedSize, float expectedHead) { + assertNotEmpty(rb, expectedSize, expectedHead); + try { + assertTrue(expectedHead == rb.remove()); + } catch (NoSuchElementException x) { + fail("queue should not be empty"); + } + } + + float A = 'A'; + float B = 'B'; + float C = 'C'; + float D = 'D'; + float E = 'E'; + float F = 'F'; + + public void testAddRemove() { + + FloatRingBuffer rb = new FloatRingBuffer(3); + + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertFull(rb); + + assertRemove(rb, 3, A); + assertRemove(rb, 2, B); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertRemove(rb, 2, A); + assertAdd(rb, C, 2, B); + assertRemove(rb, 2, B); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertRemove(rb, 1, A); + assertEmpty(rb); + assertAdd(rb, B, 1, B); + assertRemove(rb, 1, B); + assertEmpty(rb); + assertAdd(rb, C, 1, C); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, D, 1, D); + assertRemove(rb, 1, D); + assertEmpty(rb); + assertAdd(rb, E, 1, E); + assertRemove(rb, 1, E); + assertEmpty(rb); + assertAdd(rb, F, 1, F); + assertRemove(rb, 1, F); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertFull(rb); + + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + assertAdd(rb, F, 6, A); + + assertRemove(rb, 6, A); + assertRemove(rb, 5, B); + assertRemove(rb, 4, C); + assertRemove(rb, 3, D); + assertRemove(rb, 2, E); + assertRemove(rb, 1, F); + assertEmpty(rb); + } + + public void testOfferPoll() { + FloatRingBuffer rb = new FloatRingBuffer(3); + + assertEmpty(rb); + + assertOffer(rb, A, 1, A); + assertOffer(rb, B, 2, A); + assertOffer(rb, C, 3, A); + + assertFull(rb); + + assertPoll(rb, 3, A); + assertPoll(rb, 2, B); + assertPoll(rb, 1, C); + assertEmpty(rb); + + assertOffer(rb, A, 1, A); + assertOffer(rb, B, 2, A); + assertPoll(rb, 2, A); + assertOffer(rb, C, 2, B); + assertPoll(rb, 2, B); + assertPoll(rb, 1, C); + assertEmpty(rb); + + assertOffer(rb, A, 1, A); + assertPoll(rb, 1, A); + assertEmpty(rb); + assertOffer(rb, B, 1, B); + assertPoll(rb, 1, B); + assertEmpty(rb); + assertOffer(rb, C, 1, C); + assertPoll(rb, 1, C); + assertEmpty(rb); + + assertOffer(rb, D, 1, D); + assertPoll(rb, 1, D); + assertEmpty(rb); + assertOffer(rb, E, 1, E); + assertPoll(rb, 1, E); + assertEmpty(rb); + assertOffer(rb, F, 1, F); + assertPoll(rb, 1, F); + assertEmpty(rb); + + assertOffer(rb, A, 1, A); + assertOffer(rb, B, 2, A); + assertOffer(rb, C, 3, A); + assertFull(rb); + + assertAdd(rb, D, 4, A); // need one add to grow it + assertOffer(rb, E, 5, A); // NOTE: assumes capacity grows by at least a factor of two + assertOffer(rb, F, 6, A); + + assertPoll(rb, 6, A); + assertPoll(rb, 5, B); + assertPoll(rb, 4, C); + assertPoll(rb, 3, D); + assertPoll(rb, 2, E); + assertPoll(rb, 1, F); + assertEmpty(rb); + } + + public void testGrowSimple() { + FloatRingBuffer rb = new FloatRingBuffer(5); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + assertFull(rb); + + // this will grow; the elements are in a single contiguous block + assertAdd(rb, F, 6, A); + + assertRemove(rb, 6, A); + assertRemove(rb, 5, B); + assertRemove(rb, 4, C); + assertRemove(rb, 3, D); + assertRemove(rb, 2, E); + assertRemove(rb, 1, F); + assertEmpty(rb); + } + + public void testGrowComplex() { + FloatRingBuffer rb = new FloatRingBuffer(5); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertRemove(rb, 3, A); + assertRemove(rb, 2, B); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + assertFull(rb); + + // this will grow; the elements are in two blocks + assertAdd(rb, F, 6, A); + + assertRemove(rb, 6, A); + assertRemove(rb, 5, B); + assertRemove(rb, 4, C); + assertRemove(rb, 3, D); + assertRemove(rb, 2, E); + assertRemove(rb, 1, F); + assertEmpty(rb); + } + + public void testIterator() { + FloatRingBuffer rb = new FloatRingBuffer(3); + + FloatRingBuffer.Iterator iter = rb.iterator(); + assertFalse(iter.hasNext()); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertFull(rb); + + iter = rb.iterator(); + assertTrue(iter.hasNext()); + assertEquals(A, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(B, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(C, iter.next()); + assertFalse(iter.hasNext()); + + assertRemove(rb, 3, A); + assertAdd(rb, D, 3, B); + + iter = rb.iterator(); + assertTrue(iter.hasNext()); + assertEquals(B, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(C, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(D, iter.next()); + assertFalse(iter.hasNext()); + + assertRemove(rb, 3, B); + + iter = rb.iterator(); + assertEquals(C, iter.next()); + assertEquals(D, iter.next()); + assertFalse(iter.hasNext()); + + assertAdd(rb, E, 3, C); + assertAdd(rb, F, 4, C); + + iter = rb.iterator(); + assertTrue(iter.hasNext()); + assertEquals(C, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(D, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(E, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(F, iter.next()); + assertFalse(iter.hasNext()); + } + + public void testBack() { + FloatRingBuffer rb = new FloatRingBuffer(5); + + assertAdd(rb, A, 1, A); + assertEquals(rb.back(), A); + + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertEquals(rb.back(), C); + } + + public void testBackWhenEmpty() { + FloatRingBuffer rb = new FloatRingBuffer(5); + try { + rb.back(); + fail("expected a NoSuchElement exception"); + } catch (NoSuchElementException x) { + // expected + } + } + + public void testBackTailIsZero() { + FloatRingBuffer rb = new FloatRingBuffer(5); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + assertFull(rb); + + assertRemove(rb, 5, A); + assertAdd(rb, F, 5, B); + assertEquals(rb.back(), F); + } + + public void testLargeAmounts() { + FloatRingBuffer rb = new FloatRingBuffer(3); + + for (int i = 0; i < 100; i++) + rb.add((float)i); + + for (int i = 100; i < 200; i++) { + rb.add((float)i); + assertEquals((float)(i - 100 + 1), rb.front(1)); + assertEquals((float)(i - 100), rb.poll(SENTINEL)); + } + + for (int i = 200; i < 300; i++) { + if (i < 299) + assertEquals((float)(i - 100 + 1), rb.front(1)); + assertEquals((float)(i - 100), rb.poll(SENTINEL)); + } + } +} diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/IntRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/IntRingBufferTest.java new file mode 100644 index 00000000000..1c0815a1eda --- /dev/null +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/IntRingBufferTest.java @@ -0,0 +1,372 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit CharRingBufferTest and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +package io.deephaven.base.ringbuffer; + +import junit.framework.TestCase; + +import java.util.NoSuchElementException; + +public class IntRingBufferTest extends TestCase { + + final int SENTINEL = Integer.MIN_VALUE; + + private void assertEmpty(IntRingBuffer rb) { + assertTrue(rb.isEmpty()); + assertEquals(0, rb.size()); + + assertEquals(SENTINEL, rb.peek(SENTINEL)); + try { + rb.element(); + fail("queue should be empty"); + } catch (NoSuchElementException x) { + // expected + } + + assertEquals(SENTINEL, rb.poll(SENTINEL)); + try { + rb.remove(); + fail("queue should be empty"); + } catch (NoSuchElementException x) { + // expected + } + } + + private void assertFull(IntRingBuffer rb) { + assertFalse(rb.isEmpty()); + assertEquals(rb.capacity(), rb.size()); + } + + private void assertNotEmpty(IntRingBuffer rb, int expectedSize, int expectedHead) { + assertFalse(rb.isEmpty()); + assertEquals(expectedSize, rb.size()); + + assertTrue(expectedHead == rb.peek(SENTINEL)); + try { + assertTrue(expectedHead == rb.element()); + } catch (NoSuchElementException x) { + fail("queue should not be empty"); + } + } + + private void assertAdd(IntRingBuffer rb, int newHead, int expectedSize, int expectedHead) { + assertTrue(rb.add(newHead)); + assertNotEmpty(rb, expectedSize, expectedHead); + } + + private void assertOffer(IntRingBuffer rb, int newHead, int expectedSize, int expectedHead) { + assertTrue(rb.offer(newHead)); + assertNotEmpty(rb, expectedSize, expectedHead); + } + + private void assertPoll(IntRingBuffer rb, int expectedSize, int expectedHead) { + assertNotEmpty(rb, expectedSize, expectedHead); + assertTrue(expectedHead == rb.poll(SENTINEL)); + } + + private void assertRemove(IntRingBuffer rb, int expectedSize, int expectedHead) { + assertNotEmpty(rb, expectedSize, expectedHead); + try { + assertTrue(expectedHead == rb.remove()); + } catch (NoSuchElementException x) { + fail("queue should not be empty"); + } + } + + int A = 'A'; + int B = 'B'; + int C = 'C'; + int D = 'D'; + int E = 'E'; + int F = 'F'; + + public void testAddRemove() { + + IntRingBuffer rb = new IntRingBuffer(3); + + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertFull(rb); + + assertRemove(rb, 3, A); + assertRemove(rb, 2, B); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertRemove(rb, 2, A); + assertAdd(rb, C, 2, B); + assertRemove(rb, 2, B); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertRemove(rb, 1, A); + assertEmpty(rb); + assertAdd(rb, B, 1, B); + assertRemove(rb, 1, B); + assertEmpty(rb); + assertAdd(rb, C, 1, C); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, D, 1, D); + assertRemove(rb, 1, D); + assertEmpty(rb); + assertAdd(rb, E, 1, E); + assertRemove(rb, 1, E); + assertEmpty(rb); + assertAdd(rb, F, 1, F); + assertRemove(rb, 1, F); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertFull(rb); + + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + assertAdd(rb, F, 6, A); + + assertRemove(rb, 6, A); + assertRemove(rb, 5, B); + assertRemove(rb, 4, C); + assertRemove(rb, 3, D); + assertRemove(rb, 2, E); + assertRemove(rb, 1, F); + assertEmpty(rb); + } + + public void testOfferPoll() { + IntRingBuffer rb = new IntRingBuffer(3); + + assertEmpty(rb); + + assertOffer(rb, A, 1, A); + assertOffer(rb, B, 2, A); + assertOffer(rb, C, 3, A); + + assertFull(rb); + + assertPoll(rb, 3, A); + assertPoll(rb, 2, B); + assertPoll(rb, 1, C); + assertEmpty(rb); + + assertOffer(rb, A, 1, A); + assertOffer(rb, B, 2, A); + assertPoll(rb, 2, A); + assertOffer(rb, C, 2, B); + assertPoll(rb, 2, B); + assertPoll(rb, 1, C); + assertEmpty(rb); + + assertOffer(rb, A, 1, A); + assertPoll(rb, 1, A); + assertEmpty(rb); + assertOffer(rb, B, 1, B); + assertPoll(rb, 1, B); + assertEmpty(rb); + assertOffer(rb, C, 1, C); + assertPoll(rb, 1, C); + assertEmpty(rb); + + assertOffer(rb, D, 1, D); + assertPoll(rb, 1, D); + assertEmpty(rb); + assertOffer(rb, E, 1, E); + assertPoll(rb, 1, E); + assertEmpty(rb); + assertOffer(rb, F, 1, F); + assertPoll(rb, 1, F); + assertEmpty(rb); + + assertOffer(rb, A, 1, A); + assertOffer(rb, B, 2, A); + assertOffer(rb, C, 3, A); + assertFull(rb); + + assertAdd(rb, D, 4, A); // need one add to grow it + assertOffer(rb, E, 5, A); // NOTE: assumes capacity grows by at least a factor of two + assertOffer(rb, F, 6, A); + + assertPoll(rb, 6, A); + assertPoll(rb, 5, B); + assertPoll(rb, 4, C); + assertPoll(rb, 3, D); + assertPoll(rb, 2, E); + assertPoll(rb, 1, F); + assertEmpty(rb); + } + + public void testGrowSimple() { + IntRingBuffer rb = new IntRingBuffer(5); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + assertFull(rb); + + // this will grow; the elements are in a single contiguous block + assertAdd(rb, F, 6, A); + + assertRemove(rb, 6, A); + assertRemove(rb, 5, B); + assertRemove(rb, 4, C); + assertRemove(rb, 3, D); + assertRemove(rb, 2, E); + assertRemove(rb, 1, F); + assertEmpty(rb); + } + + public void testGrowComplex() { + IntRingBuffer rb = new IntRingBuffer(5); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertRemove(rb, 3, A); + assertRemove(rb, 2, B); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + assertFull(rb); + + // this will grow; the elements are in two blocks + assertAdd(rb, F, 6, A); + + assertRemove(rb, 6, A); + assertRemove(rb, 5, B); + assertRemove(rb, 4, C); + assertRemove(rb, 3, D); + assertRemove(rb, 2, E); + assertRemove(rb, 1, F); + assertEmpty(rb); + } + + public void testIterator() { + IntRingBuffer rb = new IntRingBuffer(3); + + IntRingBuffer.Iterator iter = rb.iterator(); + assertFalse(iter.hasNext()); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertFull(rb); + + iter = rb.iterator(); + assertTrue(iter.hasNext()); + assertEquals(A, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(B, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(C, iter.next()); + assertFalse(iter.hasNext()); + + assertRemove(rb, 3, A); + assertAdd(rb, D, 3, B); + + iter = rb.iterator(); + assertTrue(iter.hasNext()); + assertEquals(B, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(C, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(D, iter.next()); + assertFalse(iter.hasNext()); + + assertRemove(rb, 3, B); + + iter = rb.iterator(); + assertEquals(C, iter.next()); + assertEquals(D, iter.next()); + assertFalse(iter.hasNext()); + + assertAdd(rb, E, 3, C); + assertAdd(rb, F, 4, C); + + iter = rb.iterator(); + assertTrue(iter.hasNext()); + assertEquals(C, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(D, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(E, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(F, iter.next()); + assertFalse(iter.hasNext()); + } + + public void testBack() { + IntRingBuffer rb = new IntRingBuffer(5); + + assertAdd(rb, A, 1, A); + assertEquals(rb.back(), A); + + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertEquals(rb.back(), C); + } + + public void testBackWhenEmpty() { + IntRingBuffer rb = new IntRingBuffer(5); + try { + rb.back(); + fail("expected a NoSuchElement exception"); + } catch (NoSuchElementException x) { + // expected + } + } + + public void testBackTailIsZero() { + IntRingBuffer rb = new IntRingBuffer(5); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + assertFull(rb); + + assertRemove(rb, 5, A); + assertAdd(rb, F, 5, B); + assertEquals(rb.back(), F); + } + + public void testLargeAmounts() { + IntRingBuffer rb = new IntRingBuffer(3); + + for (int i = 0; i < 100; i++) + rb.add((int)i); + + for (int i = 100; i < 200; i++) { + rb.add((int)i); + assertEquals((int)(i - 100 + 1), rb.front(1)); + assertEquals((int)(i - 100), rb.poll(SENTINEL)); + } + + for (int i = 200; i < 300; i++) { + if (i < 299) + assertEquals((int)(i - 100 + 1), rb.front(1)); + assertEquals((int)(i - 100), rb.poll(SENTINEL)); + } + } +} diff --git a/Base/src/test/java/io/deephaven/base/LongRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/LongRingBufferTest.java similarity index 90% rename from Base/src/test/java/io/deephaven/base/LongRingBufferTest.java rename to Base/src/test/java/io/deephaven/base/ringbuffer/LongRingBufferTest.java index a179251cb85..f66de64e4d3 100644 --- a/Base/src/test/java/io/deephaven/base/LongRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/LongRingBufferTest.java @@ -1,7 +1,12 @@ /** * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending */ -package io.deephaven.base; +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit CharRingBufferTest and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +package io.deephaven.base.ringbuffer; import junit.framework.TestCase; @@ -9,11 +14,13 @@ public class LongRingBufferTest extends TestCase { + final long SENTINEL = Long.MIN_VALUE; + private void assertEmpty(LongRingBuffer rb) { assertTrue(rb.isEmpty()); assertEquals(0, rb.size()); - assertEquals(Long.MIN_VALUE, rb.peek(Long.MIN_VALUE)); + assertEquals(SENTINEL, rb.peek(SENTINEL)); try { rb.element(); fail("queue should be empty"); @@ -21,7 +28,7 @@ private void assertEmpty(LongRingBuffer rb) { // expected } - assertEquals(Long.MIN_VALUE, rb.poll(Long.MIN_VALUE)); + assertEquals(SENTINEL, rb.poll(SENTINEL)); try { rb.remove(); fail("queue should be empty"); @@ -39,7 +46,7 @@ private void assertNotEmpty(LongRingBuffer rb, int expectedSize, long expectedHe assertFalse(rb.isEmpty()); assertEquals(expectedSize, rb.size()); - assertTrue(expectedHead == rb.peek(Long.MIN_VALUE)); + assertTrue(expectedHead == rb.peek(SENTINEL)); try { assertTrue(expectedHead == rb.element()); } catch (NoSuchElementException x) { @@ -59,7 +66,7 @@ private void assertOffer(LongRingBuffer rb, long newHead, int expectedSize, long private void assertPoll(LongRingBuffer rb, int expectedSize, long expectedHead) { assertNotEmpty(rb, expectedSize, expectedHead); - assertTrue(expectedHead == rb.poll(Long.MIN_VALUE)); + assertTrue(expectedHead == rb.poll(SENTINEL)); } private void assertRemove(LongRingBuffer rb, int expectedSize, long expectedHead) { @@ -348,18 +355,18 @@ public void testLargeAmounts() { LongRingBuffer rb = new LongRingBuffer(3); for (int i = 0; i < 100; i++) - rb.add(i); + rb.add((long)i); for (int i = 100; i < 200; i++) { - rb.add(i); - assertEquals(i - 100 + 1, rb.front(1)); - assertEquals(i - 100, rb.poll(Long.MIN_VALUE)); + rb.add((long)i); + assertEquals((long)(i - 100 + 1), rb.front(1)); + assertEquals((long)(i - 100), rb.poll(SENTINEL)); } for (int i = 200; i < 300; i++) { if (i < 299) - assertEquals(i - 100 + 1, rb.front(1)); - assertEquals(i - 100, rb.poll(Long.MIN_VALUE)); + assertEquals((long)(i - 100 + 1), rb.front(1)); + assertEquals((long)(i - 100), rb.poll(SENTINEL)); } } } diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/ObjectRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/ObjectRingBufferTest.java new file mode 100644 index 00000000000..526152444a6 --- /dev/null +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/ObjectRingBufferTest.java @@ -0,0 +1,372 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit CharRingBufferTest and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +package io.deephaven.base.ringbuffer; + +import junit.framework.TestCase; + +import java.util.NoSuchElementException; + +public class ObjectRingBufferTest extends TestCase { + + final Object SENTINEL = "SENTINEL"; + + private void assertEmpty(ObjectRingBuffer rb) { + assertTrue(rb.isEmpty()); + assertEquals(0, rb.size()); + + assertEquals(SENTINEL, rb.peek(SENTINEL)); + try { + rb.element(); + fail("queue should be empty"); + } catch (NoSuchElementException x) { + // expected + } + + assertEquals(SENTINEL, rb.poll(SENTINEL)); + try { + rb.remove(); + fail("queue should be empty"); + } catch (NoSuchElementException x) { + // expected + } + } + + private void assertFull(ObjectRingBuffer rb) { + assertFalse(rb.isEmpty()); + assertEquals(rb.capacity(), rb.size()); + } + + private void assertNotEmpty(ObjectRingBuffer rb, int expectedSize, Object expectedHead) { + assertFalse(rb.isEmpty()); + assertEquals(expectedSize, rb.size()); + + assertTrue(expectedHead == rb.peek(SENTINEL)); + try { + assertTrue(expectedHead == rb.element()); + } catch (NoSuchElementException x) { + fail("queue should not be empty"); + } + } + + private void assertAdd(ObjectRingBuffer rb, Object newHead, int expectedSize, Object expectedHead) { + assertTrue(rb.add(newHead)); + assertNotEmpty(rb, expectedSize, expectedHead); + } + + private void assertOffer(ObjectRingBuffer rb, Object newHead, int expectedSize, Object expectedHead) { + assertTrue(rb.offer(newHead)); + assertNotEmpty(rb, expectedSize, expectedHead); + } + + private void assertPoll(ObjectRingBuffer rb, int expectedSize, Object expectedHead) { + assertNotEmpty(rb, expectedSize, expectedHead); + assertTrue(expectedHead == rb.poll(SENTINEL)); + } + + private void assertRemove(ObjectRingBuffer rb, int expectedSize, Object expectedHead) { + assertNotEmpty(rb, expectedSize, expectedHead); + try { + assertTrue(expectedHead == rb.remove()); + } catch (NoSuchElementException x) { + fail("queue should not be empty"); + } + } + + Object A = 'A'; + Object B = 'B'; + Object C = 'C'; + Object D = 'D'; + Object E = 'E'; + Object F = 'F'; + + public void testAddRemove() { + + ObjectRingBuffer rb = new ObjectRingBuffer(3); + + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertFull(rb); + + assertRemove(rb, 3, A); + assertRemove(rb, 2, B); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertRemove(rb, 2, A); + assertAdd(rb, C, 2, B); + assertRemove(rb, 2, B); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertRemove(rb, 1, A); + assertEmpty(rb); + assertAdd(rb, B, 1, B); + assertRemove(rb, 1, B); + assertEmpty(rb); + assertAdd(rb, C, 1, C); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, D, 1, D); + assertRemove(rb, 1, D); + assertEmpty(rb); + assertAdd(rb, E, 1, E); + assertRemove(rb, 1, E); + assertEmpty(rb); + assertAdd(rb, F, 1, F); + assertRemove(rb, 1, F); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertFull(rb); + + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + assertAdd(rb, F, 6, A); + + assertRemove(rb, 6, A); + assertRemove(rb, 5, B); + assertRemove(rb, 4, C); + assertRemove(rb, 3, D); + assertRemove(rb, 2, E); + assertRemove(rb, 1, F); + assertEmpty(rb); + } + + public void testOfferPoll() { + ObjectRingBuffer rb = new ObjectRingBuffer(3); + + assertEmpty(rb); + + assertOffer(rb, A, 1, A); + assertOffer(rb, B, 2, A); + assertOffer(rb, C, 3, A); + + assertFull(rb); + + assertPoll(rb, 3, A); + assertPoll(rb, 2, B); + assertPoll(rb, 1, C); + assertEmpty(rb); + + assertOffer(rb, A, 1, A); + assertOffer(rb, B, 2, A); + assertPoll(rb, 2, A); + assertOffer(rb, C, 2, B); + assertPoll(rb, 2, B); + assertPoll(rb, 1, C); + assertEmpty(rb); + + assertOffer(rb, A, 1, A); + assertPoll(rb, 1, A); + assertEmpty(rb); + assertOffer(rb, B, 1, B); + assertPoll(rb, 1, B); + assertEmpty(rb); + assertOffer(rb, C, 1, C); + assertPoll(rb, 1, C); + assertEmpty(rb); + + assertOffer(rb, D, 1, D); + assertPoll(rb, 1, D); + assertEmpty(rb); + assertOffer(rb, E, 1, E); + assertPoll(rb, 1, E); + assertEmpty(rb); + assertOffer(rb, F, 1, F); + assertPoll(rb, 1, F); + assertEmpty(rb); + + assertOffer(rb, A, 1, A); + assertOffer(rb, B, 2, A); + assertOffer(rb, C, 3, A); + assertFull(rb); + + assertAdd(rb, D, 4, A); // need one add to grow it + assertOffer(rb, E, 5, A); // NOTE: assumes capacity grows by at least a factor of two + assertOffer(rb, F, 6, A); + + assertPoll(rb, 6, A); + assertPoll(rb, 5, B); + assertPoll(rb, 4, C); + assertPoll(rb, 3, D); + assertPoll(rb, 2, E); + assertPoll(rb, 1, F); + assertEmpty(rb); + } + + public void testGrowSimple() { + ObjectRingBuffer rb = new ObjectRingBuffer(5); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + assertFull(rb); + + // this will grow; the elements are in a single contiguous block + assertAdd(rb, F, 6, A); + + assertRemove(rb, 6, A); + assertRemove(rb, 5, B); + assertRemove(rb, 4, C); + assertRemove(rb, 3, D); + assertRemove(rb, 2, E); + assertRemove(rb, 1, F); + assertEmpty(rb); + } + + public void testGrowComplex() { + ObjectRingBuffer rb = new ObjectRingBuffer(5); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertRemove(rb, 3, A); + assertRemove(rb, 2, B); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + assertFull(rb); + + // this will grow; the elements are in two blocks + assertAdd(rb, F, 6, A); + + assertRemove(rb, 6, A); + assertRemove(rb, 5, B); + assertRemove(rb, 4, C); + assertRemove(rb, 3, D); + assertRemove(rb, 2, E); + assertRemove(rb, 1, F); + assertEmpty(rb); + } + + public void testIterator() { + ObjectRingBuffer rb = new ObjectRingBuffer(3); + + ObjectRingBuffer.Iterator iter = rb.iterator(); + assertFalse(iter.hasNext()); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertFull(rb); + + iter = rb.iterator(); + assertTrue(iter.hasNext()); + assertEquals(A, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(B, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(C, iter.next()); + assertFalse(iter.hasNext()); + + assertRemove(rb, 3, A); + assertAdd(rb, D, 3, B); + + iter = rb.iterator(); + assertTrue(iter.hasNext()); + assertEquals(B, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(C, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(D, iter.next()); + assertFalse(iter.hasNext()); + + assertRemove(rb, 3, B); + + iter = rb.iterator(); + assertEquals(C, iter.next()); + assertEquals(D, iter.next()); + assertFalse(iter.hasNext()); + + assertAdd(rb, E, 3, C); + assertAdd(rb, F, 4, C); + + iter = rb.iterator(); + assertTrue(iter.hasNext()); + assertEquals(C, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(D, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(E, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(F, iter.next()); + assertFalse(iter.hasNext()); + } + + public void testBack() { + ObjectRingBuffer rb = new ObjectRingBuffer(5); + + assertAdd(rb, A, 1, A); + assertEquals(rb.back(), A); + + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertEquals(rb.back(), C); + } + + public void testBackWhenEmpty() { + ObjectRingBuffer rb = new ObjectRingBuffer(5); + try { + rb.back(); + fail("expected a NoSuchElement exception"); + } catch (NoSuchElementException x) { + // expected + } + } + + public void testBackTailIsZero() { + ObjectRingBuffer rb = new ObjectRingBuffer(5); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + assertFull(rb); + + assertRemove(rb, 5, A); + assertAdd(rb, F, 5, B); + assertEquals(rb.back(), F); + } + + public void testLargeAmounts() { + ObjectRingBuffer rb = new ObjectRingBuffer(3); + + for (int i = 0; i < 100; i++) + rb.add((Object)i); + + for (int i = 100; i < 200; i++) { + rb.add((Object)i); + assertEquals((Object)(i - 100 + 1), rb.front(1)); + assertEquals((Object)(i - 100), rb.poll(SENTINEL)); + } + + for (int i = 200; i < 300; i++) { + if (i < 299) + assertEquals((Object)(i - 100 + 1), rb.front(1)); + assertEquals((Object)(i - 100), rb.poll(SENTINEL)); + } + } +} diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/ShortRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/ShortRingBufferTest.java new file mode 100644 index 00000000000..30b980e5348 --- /dev/null +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/ShortRingBufferTest.java @@ -0,0 +1,372 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit CharRingBufferTest and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +package io.deephaven.base.ringbuffer; + +import junit.framework.TestCase; + +import java.util.NoSuchElementException; + +public class ShortRingBufferTest extends TestCase { + + final short SENTINEL = Short.MIN_VALUE; + + private void assertEmpty(ShortRingBuffer rb) { + assertTrue(rb.isEmpty()); + assertEquals(0, rb.size()); + + assertEquals(SENTINEL, rb.peek(SENTINEL)); + try { + rb.element(); + fail("queue should be empty"); + } catch (NoSuchElementException x) { + // expected + } + + assertEquals(SENTINEL, rb.poll(SENTINEL)); + try { + rb.remove(); + fail("queue should be empty"); + } catch (NoSuchElementException x) { + // expected + } + } + + private void assertFull(ShortRingBuffer rb) { + assertFalse(rb.isEmpty()); + assertEquals(rb.capacity(), rb.size()); + } + + private void assertNotEmpty(ShortRingBuffer rb, int expectedSize, short expectedHead) { + assertFalse(rb.isEmpty()); + assertEquals(expectedSize, rb.size()); + + assertTrue(expectedHead == rb.peek(SENTINEL)); + try { + assertTrue(expectedHead == rb.element()); + } catch (NoSuchElementException x) { + fail("queue should not be empty"); + } + } + + private void assertAdd(ShortRingBuffer rb, short newHead, int expectedSize, short expectedHead) { + assertTrue(rb.add(newHead)); + assertNotEmpty(rb, expectedSize, expectedHead); + } + + private void assertOffer(ShortRingBuffer rb, short newHead, int expectedSize, short expectedHead) { + assertTrue(rb.offer(newHead)); + assertNotEmpty(rb, expectedSize, expectedHead); + } + + private void assertPoll(ShortRingBuffer rb, int expectedSize, short expectedHead) { + assertNotEmpty(rb, expectedSize, expectedHead); + assertTrue(expectedHead == rb.poll(SENTINEL)); + } + + private void assertRemove(ShortRingBuffer rb, int expectedSize, short expectedHead) { + assertNotEmpty(rb, expectedSize, expectedHead); + try { + assertTrue(expectedHead == rb.remove()); + } catch (NoSuchElementException x) { + fail("queue should not be empty"); + } + } + + short A = 'A'; + short B = 'B'; + short C = 'C'; + short D = 'D'; + short E = 'E'; + short F = 'F'; + + public void testAddRemove() { + + ShortRingBuffer rb = new ShortRingBuffer(3); + + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertFull(rb); + + assertRemove(rb, 3, A); + assertRemove(rb, 2, B); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertRemove(rb, 2, A); + assertAdd(rb, C, 2, B); + assertRemove(rb, 2, B); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertRemove(rb, 1, A); + assertEmpty(rb); + assertAdd(rb, B, 1, B); + assertRemove(rb, 1, B); + assertEmpty(rb); + assertAdd(rb, C, 1, C); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, D, 1, D); + assertRemove(rb, 1, D); + assertEmpty(rb); + assertAdd(rb, E, 1, E); + assertRemove(rb, 1, E); + assertEmpty(rb); + assertAdd(rb, F, 1, F); + assertRemove(rb, 1, F); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertFull(rb); + + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + assertAdd(rb, F, 6, A); + + assertRemove(rb, 6, A); + assertRemove(rb, 5, B); + assertRemove(rb, 4, C); + assertRemove(rb, 3, D); + assertRemove(rb, 2, E); + assertRemove(rb, 1, F); + assertEmpty(rb); + } + + public void testOfferPoll() { + ShortRingBuffer rb = new ShortRingBuffer(3); + + assertEmpty(rb); + + assertOffer(rb, A, 1, A); + assertOffer(rb, B, 2, A); + assertOffer(rb, C, 3, A); + + assertFull(rb); + + assertPoll(rb, 3, A); + assertPoll(rb, 2, B); + assertPoll(rb, 1, C); + assertEmpty(rb); + + assertOffer(rb, A, 1, A); + assertOffer(rb, B, 2, A); + assertPoll(rb, 2, A); + assertOffer(rb, C, 2, B); + assertPoll(rb, 2, B); + assertPoll(rb, 1, C); + assertEmpty(rb); + + assertOffer(rb, A, 1, A); + assertPoll(rb, 1, A); + assertEmpty(rb); + assertOffer(rb, B, 1, B); + assertPoll(rb, 1, B); + assertEmpty(rb); + assertOffer(rb, C, 1, C); + assertPoll(rb, 1, C); + assertEmpty(rb); + + assertOffer(rb, D, 1, D); + assertPoll(rb, 1, D); + assertEmpty(rb); + assertOffer(rb, E, 1, E); + assertPoll(rb, 1, E); + assertEmpty(rb); + assertOffer(rb, F, 1, F); + assertPoll(rb, 1, F); + assertEmpty(rb); + + assertOffer(rb, A, 1, A); + assertOffer(rb, B, 2, A); + assertOffer(rb, C, 3, A); + assertFull(rb); + + assertAdd(rb, D, 4, A); // need one add to grow it + assertOffer(rb, E, 5, A); // NOTE: assumes capacity grows by at least a factor of two + assertOffer(rb, F, 6, A); + + assertPoll(rb, 6, A); + assertPoll(rb, 5, B); + assertPoll(rb, 4, C); + assertPoll(rb, 3, D); + assertPoll(rb, 2, E); + assertPoll(rb, 1, F); + assertEmpty(rb); + } + + public void testGrowSimple() { + ShortRingBuffer rb = new ShortRingBuffer(5); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + assertFull(rb); + + // this will grow; the elements are in a single contiguous block + assertAdd(rb, F, 6, A); + + assertRemove(rb, 6, A); + assertRemove(rb, 5, B); + assertRemove(rb, 4, C); + assertRemove(rb, 3, D); + assertRemove(rb, 2, E); + assertRemove(rb, 1, F); + assertEmpty(rb); + } + + public void testGrowComplex() { + ShortRingBuffer rb = new ShortRingBuffer(5); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertRemove(rb, 3, A); + assertRemove(rb, 2, B); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + assertFull(rb); + + // this will grow; the elements are in two blocks + assertAdd(rb, F, 6, A); + + assertRemove(rb, 6, A); + assertRemove(rb, 5, B); + assertRemove(rb, 4, C); + assertRemove(rb, 3, D); + assertRemove(rb, 2, E); + assertRemove(rb, 1, F); + assertEmpty(rb); + } + + public void testIterator() { + ShortRingBuffer rb = new ShortRingBuffer(3); + + ShortRingBuffer.Iterator iter = rb.iterator(); + assertFalse(iter.hasNext()); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertFull(rb); + + iter = rb.iterator(); + assertTrue(iter.hasNext()); + assertEquals(A, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(B, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(C, iter.next()); + assertFalse(iter.hasNext()); + + assertRemove(rb, 3, A); + assertAdd(rb, D, 3, B); + + iter = rb.iterator(); + assertTrue(iter.hasNext()); + assertEquals(B, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(C, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(D, iter.next()); + assertFalse(iter.hasNext()); + + assertRemove(rb, 3, B); + + iter = rb.iterator(); + assertEquals(C, iter.next()); + assertEquals(D, iter.next()); + assertFalse(iter.hasNext()); + + assertAdd(rb, E, 3, C); + assertAdd(rb, F, 4, C); + + iter = rb.iterator(); + assertTrue(iter.hasNext()); + assertEquals(C, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(D, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(E, iter.next()); + assertTrue(iter.hasNext()); + assertEquals(F, iter.next()); + assertFalse(iter.hasNext()); + } + + public void testBack() { + ShortRingBuffer rb = new ShortRingBuffer(5); + + assertAdd(rb, A, 1, A); + assertEquals(rb.back(), A); + + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertEquals(rb.back(), C); + } + + public void testBackWhenEmpty() { + ShortRingBuffer rb = new ShortRingBuffer(5); + try { + rb.back(); + fail("expected a NoSuchElement exception"); + } catch (NoSuchElementException x) { + // expected + } + } + + public void testBackTailIsZero() { + ShortRingBuffer rb = new ShortRingBuffer(5); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + assertFull(rb); + + assertRemove(rb, 5, A); + assertAdd(rb, F, 5, B); + assertEquals(rb.back(), F); + } + + public void testLargeAmounts() { + ShortRingBuffer rb = new ShortRingBuffer(3); + + for (int i = 0; i < 100; i++) + rb.add((short)i); + + for (int i = 100; i < 200; i++) { + rb.add((short)i); + assertEquals((short)(i - 100 + 1), rb.front(1)); + assertEquals((short)(i - 100), rb.poll(SENTINEL)); + } + + for (int i = 200; i < 300; i++) { + if (i < 299) + assertEquals((short)(i - 100 + 1), rb.front(1)); + assertEquals((short)(i - 100), rb.poll(SENTINEL)); + } + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/SortListener.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/SortListener.java index fd15134bf46..2b0ae5c3ec1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/SortListener.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/SortListener.java @@ -3,7 +3,7 @@ */ package io.deephaven.engine.table.impl; -import io.deephaven.base.LongRingBuffer; +import io.deephaven.base.ringbuffer.LongRingBuffer; import io.deephaven.base.verify.Assert; import io.deephaven.configuration.Configuration; import io.deephaven.datastructures.util.CollectionUtil; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java index dcc24e13f4c..e4a69d96aae 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java @@ -1,11 +1,10 @@ package io.deephaven.engine.table.impl; import io.deephaven.api.updateby.OperationControl; -import io.deephaven.base.LongRingBuffer; +import io.deephaven.base.ringbuffer.LongRingBuffer; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableChunk; -import io.deephaven.chunk.WritableLongChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.sized.SizedLongChunk; import io.deephaven.engine.rowset.*; @@ -43,17 +42,6 @@ public abstract class UpdateWindowedContext implements UpdateContext { public int nullCount = 0; - public LongSegmentedSortedArray getTimestampSsa() { - return timestampSsa; - } - - public RowSetBuilderSequential getModifiedBuilder() { - if (modifiedBuilder == null) { - modifiedBuilder = RowSetFactory.builderSequential(); - } - return modifiedBuilder; - } - // store a local copy of the source rowset (may not be needed) public RowSet sourceRowSet = null; @@ -76,6 +64,7 @@ public RowSetBuilderSequential getModifiedBuilder() { // for use with a ticking window protected RowSet affectedRowPositions; protected RowSet influencerPositions; + protected long currentInfluencerPosOrTimestamp; protected int currentInfluencerIndex; @@ -264,6 +253,17 @@ public RowSet getInfluencerRows() { return influencerRows; } + public LongSegmentedSortedArray getTimestampSsa() { + return timestampSsa; + } + + public RowSetBuilderSequential getModifiedBuilder() { + if (modifiedBuilder == null) { + modifiedBuilder = RowSetFactory.builderSequential(); + } + return modifiedBuilder; + } + public abstract void loadInfluencerValueChunk(); public void fillWindowTicks(UpdateWindowedContext context, long currentPos) { @@ -413,7 +413,7 @@ public void initializeFor(@NotNull final UpdateContext context, @NotNull final RowSet updateRowSet) { final UpdateWindowedContext ctx = (UpdateWindowedContext) context; - // pre=load all the influencer values this update will need + // load all the influencer values this update will need ctx.loadInfluencerValueChunk(); // load all the influencer keys @@ -427,6 +427,7 @@ public void initializeFor(@NotNull final UpdateContext context, ctx.influencerPositions.fillRowKeyChunk(ctx.influencerPosChunk.get()); ctx.currentInfluencerPosOrTimestamp = ctx.influencerPositions.firstRowKey(); } else { + // load all the influencer timestamp data ctx.influencerTimestampChunk = new SizedLongChunk<>(ctx.influencerRows.intSize()); try (final ChunkSource.FillContext fillContext = timestampColumnSource.makeFillContext(ctx.influencerRows.intSize())) { diff --git a/replication/static/build.gradle b/replication/static/build.gradle index fa0a48531f3..272936ad7e6 100644 --- a/replication/static/build.gradle +++ b/replication/static/build.gradle @@ -77,6 +77,9 @@ task replicateAllSafe { dependsOn Tasks.registerMainExecTask(project, 'replicateRingChunkSources', 'io.deephaven.replicators.ReplicateRingChunkSources') dependsOn Tasks.registerMainExecTask(project, 'replicateUpdateBy', 'io.deephaven.replicators.ReplicateUpdateBy') + + dependsOn Tasks.registerMainExecTask(project, 'replicateRingBuffers', 'io.deephaven.replicators.ReplicateRingBuffers') + } // These replicators need manual fix-up post replication and should not be run without supervision diff --git a/replication/static/src/main/java/io/deephaven/replicators/ReplicateRingBuffers.java b/replication/static/src/main/java/io/deephaven/replicators/ReplicateRingBuffers.java new file mode 100644 index 00000000000..bc81497cbf8 --- /dev/null +++ b/replication/static/src/main/java/io/deephaven/replicators/ReplicateRingBuffers.java @@ -0,0 +1,38 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.replicators; + +import io.deephaven.replication.ReplicatePrimitiveCode; +import io.deephaven.replication.ReplicationUtils; +import org.apache.commons.io.FileUtils; + +import java.io.File; +import java.io.IOException; +import java.nio.charset.Charset; +import java.util.List; + +import static io.deephaven.replication.ReplicatePrimitiveCode.*; + +public class ReplicateRingBuffers { + + public static void main(String... args) throws IOException { + // replicate ring buffers + charToAllButBoolean("Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java"); + String objectResult = ReplicatePrimitiveCode.charToObject( + "Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java"); + + // replicate the tests + charToAllButBoolean("Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java"); + objectResult = ReplicatePrimitiveCode.charToObject( + "Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java"); + replaceLines(objectResult, + "SENTINEL = Object.MIN_VALUE", "SENTINEL = \"SENTINEL\""); + } + + private static void replaceLines(String fileResult, String... replacements) throws IOException { + final File objectFile = new File(fileResult); + List lines = FileUtils.readLines(objectFile, Charset.defaultCharset()); + lines = ReplicationUtils.globalReplacements(lines, replacements); + FileUtils.writeLines(objectFile, lines); + }} From 89e51efa6d6e9d886a1fe8bd2e5bf93a89f323ca Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 16 Sep 2022 13:58:25 -0700 Subject: [PATCH 024/123] refactoring WIP, everything is broken --- .../impl/BucketedPartitionedUpdateBy.java | 7 +- .../deephaven/engine/table/impl/UpdateBy.java | 23 +- .../impl/UpdateByCumulativeOperator.java | 123 +----- .../engine/table/impl/UpdateByOperator.java | 41 +- .../table/impl/UpdateByWindowedOperator.java | 195 +-------- .../engine/table/impl/ZeroKeyUpdateBy.java | 406 +++--------------- .../table/impl/updateby/UpdateByWindow.java | 400 +++++++++++++++++ .../internal/BaseByteUpdateByOperator.java | 4 +- .../internal/BaseCharUpdateByOperator.java | 4 +- .../internal/BaseDoubleUpdateByOperator.java | 4 +- .../internal/BaseFloatUpdateByOperator.java | 4 +- .../internal/BaseIntUpdateByOperator.java | 4 +- .../internal/BaseLongUpdateByOperator.java | 4 +- .../internal/BaseObjectUpdateByOperator.java | 4 +- .../internal/BaseShortUpdateByOperator.java | 4 +- .../BigDecimalRollingSumOperator.java | 4 +- .../BigIntegerRollingSumOperator.java | 3 +- .../rollingsum/ByteRollingSumOperator.java | 4 +- .../rollingsum/DoubleRollingSumOperator.java | 4 +- .../rollingsum/FloatRollingSumOperator.java | 4 +- .../rollingsum/IntRollingSumOperator.java | 4 +- .../rollingsum/LongRollingSumOperator.java | 4 +- .../rollingsum/ShortRollingSumOperator.java | 4 +- .../replicators/ReplicateUpdateBy.java | 4 +- 24 files changed, 557 insertions(+), 705 deletions(-) create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java index 10d85035ae3..8daa50b1067 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java @@ -5,6 +5,7 @@ import io.deephaven.engine.rowset.*; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.sources.UnionSourceManager; +import io.deephaven.engine.table.impl.updateby.UpdateByWindow; import io.deephaven.engine.table.impl.util.WritableRowRedirection; import io.deephaven.util.datastructures.linked.IntrusiveDoublyLinkedNode; import org.apache.commons.lang3.mutable.MutableLong; @@ -36,6 +37,7 @@ class BucketedPartitionedUpdateBy extends UpdateBy { */ public static Table compute(@NotNull final String description, @NotNull final QueryTable source, + @NotNull final UpdateByWindow[] windows, @NotNull final UpdateByOperator[] ops, @NotNull final Map> resultSources, @NotNull final Collection byColumns, @@ -43,6 +45,7 @@ public static Table compute(@NotNull final String description, @NotNull final UpdateByControl control) { final BucketedPartitionedUpdateBy updateBy = new BucketedPartitionedUpdateBy(description, + windows, ops, source, resultSources, @@ -54,13 +57,14 @@ public static Table compute(@NotNull final String description, } protected BucketedPartitionedUpdateBy(@NotNull final String description, + @NotNull final UpdateByWindow[] windows, @NotNull final UpdateByOperator[] operators, @NotNull final QueryTable source, @NotNull final Map> resultSources, @NotNull final Collection byColumns, @NotNull final UpdateByRedirectionContext redirContext, @NotNull final UpdateByControl control) { - super(operators, source, redirContext, control); + super(windows, operators, source, redirContext, control); // create a source-listener that will listen to the source updates and apply the shifts to the output columns final QueryTable sourceListenerTable = new QueryTable(source.getRowSet(), source.getColumnSourceMap()); @@ -120,6 +124,7 @@ public void onUpdate(@NotNull final TableUpdate upstream) { Table newTable = ZeroKeyUpdateBy.compute( description, (QueryTable) t, + windows, operators, resultSources, redirContext, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java index d4244472218..2dd191f31e7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java @@ -1,5 +1,6 @@ package io.deephaven.engine.table.impl; +import gnu.trove.map.hash.TIntObjectHashMap; import gnu.trove.map.hash.TObjectIntHashMap; import io.deephaven.api.ColumnName; import io.deephaven.api.updateby.UpdateByOperation; @@ -13,6 +14,7 @@ import io.deephaven.engine.table.impl.sources.LongSparseArraySource; import io.deephaven.engine.table.impl.sources.ReinterpretUtils; import io.deephaven.engine.table.impl.sources.sparse.SparseConstants; +import io.deephaven.engine.table.impl.updateby.UpdateByWindow; import io.deephaven.engine.table.impl.util.InverseRowRedirectionImpl; import io.deephaven.engine.table.impl.util.LongColumnSourceWritableRowRedirection; import io.deephaven.engine.table.impl.util.WritableRowRedirection; @@ -30,6 +32,7 @@ public abstract class UpdateBy { protected final ChunkSource.WithPrev[] inputSources; protected final int[] inputSourceSlots; + protected final UpdateByWindow[] windows; protected final UpdateByOperator[] operators; protected final QueryTable source; @@ -123,7 +126,8 @@ public void close() { } } - protected UpdateBy(@NotNull final UpdateByOperator[] operators, + protected UpdateBy(@NotNull final UpdateByWindow[] windows, + @NotNull final UpdateByOperator[] operators, @NotNull final QueryTable source, @NotNull final UpdateByRedirectionContext redirContext, UpdateByControl control) { @@ -135,6 +139,7 @@ protected UpdateBy(@NotNull final UpdateByOperator[] operators, } this.source = source; + this.windows = windows; this.operators = operators; // noinspection unchecked this.inputSources = new ChunkSource.WithPrev[operators.length]; @@ -209,6 +214,19 @@ public static Table updateBy(@NotNull final QueryTable source, new UpdateByOperatorFactory(source, pairs, ctx, control); final Collection ops = updateByOperatorFactory.getOperators(clauses); + // build the windows for these operators + TIntObjectHashMap windowMap = new TIntObjectHashMap<>(); + ops.forEach(op -> { + UpdateByWindow newWindow = UpdateByWindow.createFromOperator(op); + final int hash = newWindow.hashCode(); + // add this if not found + if (!windowMap.containsKey(hash)) { + windowMap.put(hash, newWindow); + } + windowMap.get(hash).addOperator(op); + }); + + final StringBuilder descriptionBuilder = new StringBuilder("updateBy(ops={") .append(updateByOperatorFactory.describe(clauses)) .append("}"); @@ -231,12 +249,14 @@ public static Table updateBy(@NotNull final QueryTable source, final Map> resultSources = new LinkedHashMap<>(source.getColumnSourceMap()); resultSources.putAll(opResultSources); + final UpdateByWindow[] windowArr = windowMap.valueCollection().toArray(UpdateByOperator.ZERO_LENGTH_WINDOW_ARRAY); final UpdateByOperator[] opArr = ops.toArray(UpdateByOperator.ZERO_LENGTH_OP_ARRAY); if (pairs.length == 0) { descriptionBuilder.append(")"); Table ret = ZeroKeyUpdateBy.compute( descriptionBuilder.toString(), source, + windowArr, opArr, resultSources, ctx, @@ -270,6 +290,7 @@ public static Table updateBy(@NotNull final QueryTable source, Table ret = BucketedPartitionedUpdateBy.compute( descriptionBuilder.toString(), source, + windowArr, opArr, resultSources, byColumns, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java index 77e56be6cbd..6919eeda7bb 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java @@ -17,113 +17,10 @@ public abstract class UpdateCumulativeContext implements UpdateContext { protected RowSetBuilderSequential modifiedBuilder; protected RowSet newModified; - // store the current subset of rows that need computation - protected RowSet affectedRows; - public LongSegmentedSortedArray getTimestampSsa() { return timestampSsa; } - /** - * Find the smallest valued key that participated in the upstream {@link TableUpdate}. - * - * @param added the added rows - * @param modified the modified rows - * @param removed the removed rows - * @param shifted the shifted rows - * - * @return the smallest key that participated in any part of the update. - */ - long smallestAffectedKey(@NotNull final RowSet added, - @NotNull final RowSet modified, - @NotNull final RowSet removed, - @NotNull final RowSetShiftData shifted, - @NotNull final RowSet affectedIndex) { - - long smallestModifiedKey = Long.MAX_VALUE; - if (removed.isNonempty()) { - smallestModifiedKey = removed.firstRowKey(); - } - - if (added.isNonempty()) { - smallestModifiedKey = Math.min(smallestModifiedKey, added.firstRowKey()); - } - - if (modified.isNonempty()) { - smallestModifiedKey = Math.min(smallestModifiedKey, modified.firstRowKey()); - } - - if (shifted.nonempty()) { - final long firstModKey = modified.isEmpty() ? Long.MAX_VALUE : modified.firstRowKey(); - boolean modShiftFound = !modified.isEmpty(); - boolean affectedFound = false; - try (final RowSequence.Iterator it = affectedIndex.getRowSequenceIterator()) { - for (int shiftIdx = 0; shiftIdx < shifted.size() - && (!modShiftFound || !affectedFound); shiftIdx++) { - final long shiftStart = shifted.getBeginRange(shiftIdx); - final long shiftEnd = shifted.getEndRange(shiftIdx); - final long shiftDelta = shifted.getShiftDelta(shiftIdx); - - if (!affectedFound) { - if (it.advance(shiftStart + shiftDelta)) { - final long maybeAffectedKey = it.peekNextKey(); - if (maybeAffectedKey <= shiftEnd + shiftDelta) { - affectedFound = true; - final long keyToCompare = - shiftDelta > 0 ? maybeAffectedKey - shiftDelta : maybeAffectedKey; - smallestModifiedKey = Math.min(smallestModifiedKey, keyToCompare); - } - } else { - affectedFound = true; - } - } - - if (!modShiftFound) { - if (firstModKey <= (shiftEnd + shiftDelta)) { - modShiftFound = true; - // If the first modified key is in the range we should include it - if (firstModKey >= (shiftStart + shiftDelta)) { - smallestModifiedKey = Math.min(smallestModifiedKey, firstModKey - shiftDelta); - } else { - // Otherwise it's not included in any shifts, and since shifts can't reorder rows - // it is the smallest possible value and we've already accounted for it above. - break; - } - } - } - } - } - } - - return smallestModifiedKey; - } - - public RowSet determineAffectedRows(@NotNull final TableUpdate upstream, @NotNull final TrackingRowSet source, - final boolean initialStep) { - Assert.assertion(affectedRows == null, - "affectedRows should be null when determineAffectedRows() is called"); - if (initialStep) { - affectedRows = source.copy(); - return affectedRows; - } - - long smallestModifiedKey = smallestAffectedKey(upstream.added(), upstream.modified(), upstream.removed(), - upstream.shifted(), source); - - affectedRows = smallestModifiedKey == Long.MAX_VALUE - ? RowSetFactory.empty() - : source.subSetByKeyRange(smallestModifiedKey, source.lastRowKey()); - return affectedRows; - } - - public RowSet getAffectedRows() { - return affectedRows; - } - - public RowSet getInfluencerRows() { - return affectedRows; - } - public RowSetBuilderSequential getModifiedBuilder() { if (modifiedBuilder == null) { modifiedBuilder = RowSetFactory.builderSequential(); @@ -167,6 +64,26 @@ public String getTimestampColumnName() { return null; } + /** + * Get the value of the backward-looking window (might be nanos or ticks). + * + * @return the name of the input column + */ + @Override + public long getPrevWindowUnits() { + return 0L; + } + + /** + * Get the value of the forward-looking window (might be nanos or ticks). + * + * @return the name of the input column + */ + @Override + public long getFwdWindowUnits() { + return 0L; + } + /*** cumulative operators do not need keys */ @Override public boolean requiresKeys() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java index 88d0bd148c4..125de13b17c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java @@ -14,6 +14,7 @@ import io.deephaven.engine.table.Table; import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; +import io.deephaven.engine.table.impl.updateby.UpdateByWindow; import io.deephaven.engine.table.impl.updateby.internal.BaseCharUpdateByOperator; import io.deephaven.util.SafeCloseable; import org.jetbrains.annotations.NotNull; @@ -29,7 +30,6 @@ *
            *
          1. Reprocess *
              - *
            • {@link #resetForProcess(UpdateContext, RowSet, long)}
            • *
            • {@link #initializeFor(UpdateContext, RowSet, UpdateBy.UpdateType)}
            • *
            • {@link #reprocessChunkBucketed(UpdateContext, RowSequence, Chunk, LongChunk, IntChunk, IntChunk, IntChunk)}
            • *
            • {@link #finishFor(UpdateContext, UpdateBy.UpdateType)}
            • @@ -44,6 +44,7 @@ *

              */ public interface UpdateByOperator { + UpdateByWindow[] ZERO_LENGTH_WINDOW_ARRAY = new UpdateByWindow[0]; UpdateByOperator[] ZERO_LENGTH_OP_ARRAY = new UpdateByOperator[0]; /** @@ -65,26 +66,6 @@ static boolean isAppendOnly(@NotNull final TableUpdate update, final long lastKn * updates. */ interface UpdateContext extends SafeCloseable { - /** - * Determine all the rows affected by the {@link TableUpdate} that need to be reprocessed - * - * @param upstream the update - * @param source the rowset of the parent table (affected rows will be a subset) - */ - RowSet determineAffectedRows(@NotNull final TableUpdate upstream, @NotNull final TrackingRowSet source, - final boolean initialStep); - - /** - * Return the affected rows computed by the {@Code determineAffectedRows()} - */ - RowSet getAffectedRows(); - - /** - * Return the influencer rows computed by the {@Code determineAffectedRows()} - */ - RowSet getInfluencerRows(); - - LongSegmentedSortedArray getTimestampSsa(); } /** @@ -103,6 +84,20 @@ RowSet determineAffectedRows(@NotNull final TableUpdate upstream, @NotNull final @Nullable String getTimestampColumnName(); + /** + * Get the value of the backward-looking window (might be nanos or ticks). + * + * @return the name of the input column + */ + long getPrevWindowUnits(); + + /** + * Get the value of the forward-looking window (might be nanos or ticks). + * + * @return the name of the input column + */ + long getFwdWindowUnits(); + /** * Get an array of column names that, when modified, affect the result of this computation. * @@ -221,9 +216,7 @@ void initializeFor(@NotNull final UpdateContext context, * Apply a shift to the operation. * */ - void applyOutputShift(@NotNull final UpdateContext context, - @NotNull final RowSet subIndexToShift, - final long delta); + void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta); /** * Process a chunk of data for an updateBy table. diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java index e4a69d96aae..cefce9841c2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java @@ -24,7 +24,6 @@ public abstract class UpdateByWindowedOperator implements UpdateByOperator { protected final OperationControl control; protected final String timestampColumnName; - protected final ColumnSource timestampColumnSource; protected final long reverseTimeScaleUnits; protected final long forwardTimeScaleUnits; @@ -37,9 +36,6 @@ public abstract class UpdateByWindowedOperator implements UpdateByOperator { public abstract class UpdateWindowedContext implements UpdateContext { protected LongSegmentedSortedArray timestampSsa; - protected RowSetBuilderSequential modifiedBuilder; - protected RowSet newModified; - public int nullCount = 0; // store a local copy of the source rowset (may not be needed) @@ -72,178 +68,7 @@ public abstract class UpdateWindowedContext implements UpdateContext { protected LongRingBuffer windowPosOrTimestamp = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); public LongRingBuffer windowIndices = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); - private WritableRowSet computeAffectedRowsTime(final RowSet sourceSet, final RowSet subset, long revNanos, - long fwdNanos) { - // swap fwd/rev to get the affected windows - return computeInfluencerRowsTime(sourceSet, subset, fwdNanos, revNanos); - } - - private WritableRowSet computeInfluencerRowsTime(final RowSet sourceSet, final RowSet subset, long revNanos, - long fwdNanos) { - if (sourceSet.size() == subset.size()) { - return sourceSet.copy(); - } - - int chunkSize = (int) Math.min(subset.size(), 4096); - try (final RowSequence.Iterator it = subset.getRowSequenceIterator(); - final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(chunkSize)) { - final RowSetBuilderSequential builder = RowSetFactory.builderSequential(); - LongSegmentedSortedArray.Iterator ssaIt = timestampSsa.iterator(false, false); - while (it.hasMore() && ssaIt.hasNext()) { - final RowSequence rs = it.getNextRowSequenceWithLength(chunkSize); - LongChunk timestamps = timestampColumnSource.getChunk(context, rs).asLongChunk(); - - for (int ii = 0; ii < rs.intSize(); ii++) { - // if the timestamp of the row is null, it won't belong to any set and we can ignore it - // completely - final long ts = timestamps.get(ii); - if (ts != NULL_LONG) { - // look at every row timestamp, compute the head and tail in nanos - final long head = ts - revNanos; - final long tail = ts + fwdNanos; - - // advance the iterator to the beginning of the window - if (ssaIt.nextValue() < head) { - ssaIt.advanceToBeforeFirst(head); - if (!ssaIt.hasNext()) { - // SSA is exhausted - break; - } - } - - Assert.assertion(ssaIt.hasNext() && ssaIt.nextValue() >= head, - "SSA Iterator outside of window"); - - // step through the SSA and collect keys until outside of the window - while (ssaIt.hasNext() && ssaIt.nextValue() <= tail) { - builder.appendKey(ssaIt.nextKey()); - ssaIt.next(); - } - - if (!ssaIt.hasNext()) { - // SSA is exhausted - break; - } - } - } - } - return builder.build(); - } - } - - private WritableRowSet computeAffectedRowsTicks(final RowSet sourceSet, final RowSet subset, long revTicks, - long fwdTicks) { - // swap fwd/rev to get the influencer windows - return computeInfluencerRowsTicks(sourceSet, subset, fwdTicks, revTicks); - } - - private WritableRowSet computeInfluencerRowsTicks(final RowSet sourceSet, final RowSet subset, long revTicks, - long fwdTicks) { - if (sourceSet.size() == subset.size()) { - return sourceSet.copy(); - } - - long maxPos = sourceSet.size() - 1; - - try (final RowSet inverted = sourceSet.invert(subset)) { - final RowSetBuilderSequential builder = RowSetFactory.builderSequential(); - final MutableLong minPos = new MutableLong(0L); - - inverted.forAllRowKeyRanges((s, e) -> { - long sPos = Math.max(s - revTicks, minPos.longValue()); - long ePos = Math.min(e + fwdTicks, maxPos); - builder.appendRange(sPos, ePos); - minPos.setValue(ePos + 1); - }); - - try (final RowSet positions = builder.build()) { - return sourceSet.subSetForPositions(positions); - } - } - } - - /*** - * This function is only correct if the proper {@code source} rowset is provided. If using buckets, then the - * provided rowset must be limited to the rows in the current bucket only - * - * @param upstream the update - * @param source the rowset of the parent table (affected rows will be a subset) - * @param initialStep whether this is the initial step of building the table - */ - public RowSet determineAffectedRows(@NotNull final TableUpdate upstream, @NotNull final TrackingRowSet source, - final boolean initialStep) { - Assert.assertion(affectedRows == null, - "affectedRows should be null when determineAffectedRows() is called"); - - if (initialStep) { - // all rows are affected initially - affectedRows = source.copy(); - influencerRows = source.copy(); - - // no need to invert, just create a flat rowset - if (timestampColumnName == null) { - affectedRowPositions = RowSetFactory.flat(source.size()); - influencerPositions = RowSetFactory.flat(source.size()); - } - return affectedRows; - } - - if (source.isEmpty()) { - affectedRows = RowSetFactory.empty(); - influencerRows = RowSetFactory.empty(); - if (timestampColumnName == null) { - affectedRowPositions = RowSetFactory.empty(); - influencerPositions = RowSetFactory.empty(); - } - return affectedRows; - } - // changed rows are mods+adds - WritableRowSet changed = upstream.added().copy(); - changed.insert(upstream.modified()); - - WritableRowSet tmpAffected; - - // compute the affected rows from these changes - if (timestampColumnName == null) { - tmpAffected = computeAffectedRowsTicks(source, changed, reverseTimeScaleUnits, forwardTimeScaleUnits); - } else { - tmpAffected = computeAffectedRowsTime(source, changed, reverseTimeScaleUnits, forwardTimeScaleUnits); - } - - // add affected rows from any removes - - if (upstream.removed().isNonempty()) { - try (final RowSet prev = source.copyPrev(); - final WritableRowSet affectedByRemoves = timestampColumnName == null - ? computeAffectedRowsTicks(prev, upstream.removed(), reverseTimeScaleUnits, - forwardTimeScaleUnits) - : computeAffectedRowsTime(prev, upstream.removed(), reverseTimeScaleUnits, - forwardTimeScaleUnits)) { - // apply shifts to get back to pos-shift space - upstream.shifted().apply(affectedByRemoves); - // retain only the rows that still exist in the source - affectedByRemoves.retain(source); - tmpAffected.insert(affectedByRemoves); - } - } - - affectedRows = tmpAffected; - - // now get influencer rows for the affected - - if (timestampColumnName == null) { - influencerRows = - computeInfluencerRowsTicks(source, affectedRows, reverseTimeScaleUnits, forwardTimeScaleUnits); - // generate position data rowsets for efficiently computed position offsets - affectedRowPositions = source.invert(affectedRows); - influencerPositions = source.invert(influencerRows); - } else { - influencerRows = - computeInfluencerRowsTime(source, affectedRows, reverseTimeScaleUnits, forwardTimeScaleUnits); - } - return affectedRows; - } public RowSet getAffectedRows() { return affectedRows; @@ -480,6 +305,26 @@ public String getTimestampColumnName() { return this.timestampColumnName; } + /** + * Get the value of the backward-looking window (might be nanos or ticks). + * + * @return the name of the input column + */ + @Override + public long getPrevWindowUnits() { + return reverseTimeScaleUnits; + } + + /** + * Get the value of the forward-looking window (might be nanos or ticks). + * + * @return the name of the input column + */ + @Override + public long getFwdWindowUnits() { + return forwardTimeScaleUnits; + } + @Override public boolean requiresKeys() { return false; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java index e87647ec4a1..ee454fab794 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java @@ -3,12 +3,12 @@ import io.deephaven.api.updateby.UpdateByControl; import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.sized.SizedLongChunk; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.sources.ReinterpretUtils; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; +import io.deephaven.engine.table.impl.updateby.UpdateByWindow; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; import io.deephaven.engine.table.impl.util.UpdateSizeCalculator; import io.deephaven.util.SafeCloseable; @@ -31,7 +31,7 @@ class ZeroKeyUpdateBy extends UpdateBy { /** store timestamp data in an Ssa (if needed) */ final String timestampColumnName; final LongSegmentedSortedArray timestampSsa; - final ChunkSource.WithPrev timestampColumn; + final ColumnSource timestampColumnSource; final ModifiedColumnSet timestampColumnSet; /** @@ -39,6 +39,7 @@ class ZeroKeyUpdateBy extends UpdateBy { * * @param description the operation description * @param source the source table + * @param windows the unique windows for this updateBy call * @param ops the operations to perform * @param resultSources the result sources * @param redirContext the row redirection shared context @@ -47,13 +48,14 @@ class ZeroKeyUpdateBy extends UpdateBy { */ public static Table compute(@NotNull final String description, @NotNull final QueryTable source, + @NotNull final UpdateByWindow[] windows, @NotNull final UpdateByOperator[] ops, @NotNull final Map> resultSources, @NotNull final UpdateByRedirectionContext redirContext, @NotNull final UpdateByControl control, final boolean applyShifts) { final QueryTable result = new QueryTable(source.getRowSet(), resultSources); - final ZeroKeyUpdateBy updateBy = new ZeroKeyUpdateBy(ops, source, redirContext, control, applyShifts); + final ZeroKeyUpdateBy updateBy = new ZeroKeyUpdateBy(windows, ops, source, redirContext, control, applyShifts); updateBy.doInitialAdditions(); if (source.isRefreshing()) { @@ -65,12 +67,13 @@ public static Table compute(@NotNull final String description, return result; } - protected ZeroKeyUpdateBy(@NotNull final UpdateByOperator[] operators, + protected ZeroKeyUpdateBy(@NotNull final UpdateByWindow[] windows, + @NotNull final UpdateByOperator[] operators, @NotNull final QueryTable source, @NotNull final UpdateByRedirectionContext redirContext, @NotNull final UpdateByControl control, final boolean applyShifts) { - super(operators, source, redirContext, control); + super(windows, operators, source, redirContext, control); // do we need a timestamp SSA? this.timestampColumnName = Arrays.stream(operators) @@ -80,12 +83,12 @@ protected ZeroKeyUpdateBy(@NotNull final UpdateByOperator[] operators, if (timestampColumnName != null) { this.timestampSsa = new LongSegmentedSortedArray(4096); - this.timestampColumn = + this.timestampColumnSource = ReinterpretUtils.maybeConvertToPrimitive(source.getColumnSource(this.timestampColumnName)); this.timestampColumnSet = source.newModifiedColumnSet(timestampColumnName); } else { this.timestampSsa = null; - this.timestampColumn = null; + this.timestampColumnSource = null; this.timestampColumnSet = null; } this.applyShifts = applyShifts; @@ -114,9 +117,9 @@ private void processUpdateForSsa(TableUpdate upstream) { if (restampRemovals.isNonempty()) { final int size = (int) Math.min(restampRemovals.size(), 4096); try (final RowSequence.Iterator it = restampRemovals.getRowSequenceIterator(); - final ChunkSource.GetContext context = timestampColumn.makeGetContext(size); - final WritableLongChunk ssaValues = WritableLongChunk.makeWritableChunk(size); - final WritableLongChunk ssaKeys = WritableLongChunk.makeWritableChunk(size)) { + final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(size); + final WritableLongChunk ssaValues = WritableLongChunk.makeWritableChunk(size); + final WritableLongChunk ssaKeys = WritableLongChunk.makeWritableChunk(size)) { MutableLong lastTimestamp = new MutableLong(NULL_LONG); while (it.hasMore()) { @@ -124,7 +127,7 @@ private void processUpdateForSsa(TableUpdate upstream) { // get the chunks for values and keys LongChunk valuesChunk = - timestampColumn.getPrevChunk(context, chunkRs).asLongChunk(); + timestampColumnSource.getPrevChunk(context, chunkRs).asLongChunk(); LongChunk keysChunk = chunkRs.asRowKeyChunk(); // push only non-null values/keys into the Ssa @@ -141,7 +144,7 @@ private void processUpdateForSsa(TableUpdate upstream) { (int) upstream.shifted().getEffectiveSize()); try (final RowSet prevRowSet = source.getRowSet().copyPrev(); final RowSet withoutMods = prevRowSet.minus(upstream.getModifiedPreShift()); - final ColumnSource.GetContext getContext = timestampColumn.makeGetContext(size)) { + final ColumnSource.GetContext getContext = timestampColumnSource.makeGetContext(size)) { final RowSetShiftData.Iterator sit = upstream.shifted().applyIterator(); while (sit.hasNext()) { @@ -153,7 +156,7 @@ private void processUpdateForSsa(TableUpdate upstream) { } final LongChunk shiftValues = - timestampColumn.getPrevChunk(getContext, rowSetToShift).asLongChunk(); + timestampColumnSource.getPrevChunk(getContext, rowSetToShift).asLongChunk(); timestampSsa.applyShift(shiftValues, rowSetToShift.asRowKeyChunk(), sit.shiftDelta()); } @@ -165,16 +168,16 @@ private void processUpdateForSsa(TableUpdate upstream) { if (restampAdditions.isNonempty()) { final int size = (int) Math.min(restampAdditions.size(), 4096); try (final RowSequence.Iterator it = restampAdditions.getRowSequenceIterator(); - final ChunkSource.GetContext context = timestampColumn.makeGetContext(size); - final WritableLongChunk ssaValues = WritableLongChunk.makeWritableChunk(size); - final WritableLongChunk ssaKeys = WritableLongChunk.makeWritableChunk(size)) { + final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(size); + final WritableLongChunk ssaValues = WritableLongChunk.makeWritableChunk(size); + final WritableLongChunk ssaKeys = WritableLongChunk.makeWritableChunk(size)) { MutableLong lastTimestamp = new MutableLong(NULL_LONG); while (it.hasMore()) { RowSequence chunkRs = it.getNextRowSequenceWithLength(4096); // get the chunks for values and keys - LongChunk valuesChunk = timestampColumn.getChunk(context, chunkRs).asLongChunk(); + LongChunk valuesChunk = timestampColumnSource.getChunk(context, chunkRs).asLongChunk(); LongChunk keysChunk = chunkRs.asRowKeyChunk(); // push only non-null values/keys into the Ssa @@ -185,6 +188,9 @@ private void processUpdateForSsa(TableUpdate upstream) { } } + /** helper function to fill a LongChunk while skipping values that are NULL_LONG. Used to populate an SSA from + * a source containing null values + * */ private void fillChunkWithNonNull(LongChunk keysChunk, LongChunk valuesChunk, WritableLongChunk ssaKeys, WritableLongChunk ssaValues, MutableLong lastTimestamp) { @@ -222,11 +228,9 @@ void doInitialAdditions() { processUpdateForSsa(fakeUpdate); } + // do the processing for this fake update try (final UpdateContext ctx = new UpdateContext(fakeUpdate, null, true)) { - ctx.setAllAffected(); - - // do a reprocessing phase for operators that can't add directly - ctx.processRows(RowSetShiftData.EMPTY); + ctx.processRows(); } } @@ -241,47 +245,18 @@ private class UpdateContext implements SafeCloseable { /** An indicator of if each slot has been populated with data or not for this phase. */ boolean[] inputChunkPopulated; - /** An array of boolean denoting which operators are affected by the current update. */ - final boolean[] opAffected; - - /** true if any operator requested keys */ - boolean anyRequireKeys; - - /** true if any operator requested positions */ - boolean anyRequirePositions; - - /** An array of context objects for each underlying operator */ - final UpdateByOperator.UpdateContext[] opContext; - /** A {@link SharedContext} to be used while creating other contexts */ SharedContext sharedContext = SharedContext.makeSharedContext(); - /** An array of {@link ChunkSource.FillContext}s for each input column */ - final SizedSafeCloseable[] fillContexts; - - /** A set of chunks used to store post-shift working values */ - final SizedSafeCloseable>[] postWorkingChunks; - - /** A Chunk of longs to store the keys being updated */ - final SizedLongChunk keyChunk; + /** An array of {@link UpdateByWindow.UpdateByWindowContext}s for each input column */ + final UpdateByWindow.UpdateByWindowContext[] windowContexts; - /** A Chunk of longs to store the posiitions of the keys being updated */ - final SizedLongChunk posChunk; + /** Indicate if any of the operators in this window are affected by the update. */ + boolean[] windowAffected; /** A sharedContext to be used with previous values */ SharedContext prevSharedContext; - /** An array of {@link ChunkSource.FillContext}s for previous values */ - ChunkSource.FillContext[] prevFillContexts; - - /** An array of chunks for previous values */ - WritableChunk[] prevWorkingChunks; - - /** A Long Chunk for previous keys */ - WritableLongChunk prevKeyChunk; - - final RowSet rowsToProcess; - @SuppressWarnings("resource") UpdateContext(@NotNull final TableUpdate upstream, @Nullable final ModifiedColumnSet[] inputModifiedColumnSets, @@ -290,324 +265,53 @@ private class UpdateContext implements SafeCloseable { this.chunkSize = UpdateSizeCalculator.chunkSize(updateSize, upstream.shifted(), control.chunkCapacityOrDefault()); - this.opAffected = new boolean[operators.length]; - // noinspection unchecked - this.fillContexts = new SizedSafeCloseable[operators.length]; - this.opContext = new UpdateByOperator.UpdateContext[operators.length]; - this.keyChunk = new SizedLongChunk<>(); - this.posChunk = new SizedLongChunk<>(); - this.inputChunkPopulated = new boolean[operators.length]; - - if (upstream.shifted().nonempty()) { - this.prevKeyChunk = WritableLongChunk.makeWritableChunk(chunkSize); - } - - final boolean hasModifies = upstream.modified().isNonempty(); - if (hasModifies) { - // noinspection unchecked - this.prevWorkingChunks = new WritableChunk[operators.length]; - this.prevSharedContext = SharedContext.makeSharedContext(); - this.prevFillContexts = new ChunkSource.FillContext[operators.length]; - } - - // noinspection unchecked - this.postWorkingChunks = new SizedSafeCloseable[operators.length]; - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - opAffected[opIdx] = upstream.added().isNonempty() || - upstream.removed().isNonempty() || - upstream.shifted().nonempty() || - (upstream.modifiedColumnSet().nonempty() && (inputModifiedColumnSets == null - || upstream.modifiedColumnSet().containsAny(inputModifiedColumnSets[opIdx]))); - if (!opAffected[opIdx]) { - continue; - } - opContext[opIdx] = operators[opIdx].makeUpdateContext(chunkSize, timestampSsa); - - final int slotPosition = inputSourceSlots[opIdx]; - if (fillContexts[slotPosition] == null) { - fillContexts[slotPosition] = new SizedSafeCloseable<>( - sz -> inputSources[slotPosition].makeFillContext(sz, getSharedContext())); - fillContexts[slotPosition].ensureCapacity(chunkSize); - postWorkingChunks[slotPosition] = new SizedSafeCloseable<>( - sz -> inputSources[slotPosition].getChunkType().makeWritableChunk(sz)); - postWorkingChunks[slotPosition].ensureCapacity(chunkSize); - - // Note that these don't participate in setChunkSize() because nothing will use them. If that - // changes then these must also become SizedSafeCloseables. - if (hasModifies) { - prevFillContexts[slotPosition] = - inputSources[opIdx].makeFillContext(chunkSize, prevSharedContext); - prevWorkingChunks[slotPosition] = - inputSources[opIdx].getChunkType().makeWritableChunk(chunkSize); - } - } + this.windowContexts = new UpdateByWindow.UpdateByWindowContext[windows.length]; + this.windowAffected = new boolean[windows.length]; + + for (int winIdx = 0; winIdx < windows.length; winIdx++) { + // create a context for each window + windowContexts[winIdx] = windows[winIdx].makeWindowContext( + timestampColumnSource, + timestampSsa); + + // compute the affected/influenced operators and rowset within this window + windowAffected[winIdx] = windowContexts[winIdx].computeAffectedAndMakeContexts(upstream, + source.getRowSet(), + inputModifiedColumnSets, + chunkSize, + isInitializeStep); } - - // retrieve the affected rows from all operator update contexts - WritableRowSet tmp = RowSetFactory.empty(); - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (!opAffected[opIdx]) { - continue; - } - // trigger the operator to determine its own set of affected rows (window-specific), do not close() - // since this is managed by the operator context - final RowSet rs = - opContext[opIdx].determineAffectedRows(upstream, source.getRowSet(), isInitializeStep); - - // union the operator rowsets together to get a global set - tmp.insert(rs); - } - rowsToProcess = tmp; } public SharedContext getSharedContext() { return sharedContext; } - void setChunkSize(int newChunkSize) { - if (newChunkSize <= chunkSize) { - return; - } - - // We have to close and recreate the shared context because a .reset() is not enough to ensure that any - // cached chunks that something stuffed into there are resized. - this.sharedContext.close(); - this.sharedContext = SharedContext.makeSharedContext(); - - if (prevSharedContext != null) { - this.prevSharedContext.close(); - this.prevSharedContext = null; - } - - this.chunkSize = newChunkSize; - this.keyChunk.ensureCapacity(newChunkSize); - this.posChunk.ensureCapacity(newChunkSize); - - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (!opAffected[opIdx]) { - continue; - } - - operators[opIdx].setChunkSize(opContext[opIdx], newChunkSize); - if (fillContexts[opIdx] != null) { - fillContexts[opIdx].ensureCapacity(newChunkSize); - postWorkingChunks[opIdx].ensureCapacity(newChunkSize); - - // Note that this doesn't include the prevFillContexts or prevWorkingChunks. If they become - // needed for an op, they must be added here. - } - } - } - - void initializeFor(@NotNull final RowSet updateRowSet) { - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - operators[opIdx].initializeFor(opContext[opIdx], updateRowSet); - anyRequireKeys |= operators[opIdx].requiresKeys(); - anyRequirePositions |= operators[opIdx].requiresPositions(); - } - } - - if (anyRequireKeys) { - keyChunk.ensureCapacity(chunkSize); - } - if (anyRequirePositions) { - posChunk.ensureCapacity(chunkSize); - } - } - - void finishFor() { - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - operators[opIdx].finishFor(opContext[opIdx]); - } - } - - anyRequireKeys = false; - Arrays.fill(inputChunkPopulated, false); - } - @Override public void close() { sharedContext.close(); - keyChunk.close(); - posChunk.close(); - rowsToProcess.close(); - - if (prevKeyChunk != null) { - prevKeyChunk.close(); - } if (prevSharedContext != null) { prevSharedContext.close(); } - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opContext[opIdx] != null) { - opContext[opIdx].close(); - } - - if (fillContexts[opIdx] != null) { - fillContexts[opIdx].close(); - } - - if (postWorkingChunks[opIdx] != null) { - postWorkingChunks[opIdx].close(); - } - - if (prevFillContexts != null && prevFillContexts[opIdx] != null) { - prevFillContexts[opIdx].close(); - } - - if (prevWorkingChunks != null && prevWorkingChunks[opIdx] != null) { - prevWorkingChunks[opIdx].close(); - } - } - } - - /** - * Mark all columns as affected - */ - public void setAllAffected() { - Arrays.fill(opAffected, true); - } - - /** - * Check if any of the operators have produced additional modified rows. - * - * @return true if any operator produced more modified rows. - */ - boolean anyModified() { - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx] && operators[opIdx].anyModified(opContext[opIdx])) { - return true; + for (int winIdx = 0; winIdx < windows.length; winIdx++) { + if (windowContexts[winIdx] != null) { + windowContexts[winIdx].close(); } } - return false; - } - - void doUpdate(@NotNull final RowSet updateRowSet, - @NotNull final RowSet preShiftUpdateRowSet) { - } /** * Locate the smallest key that requires reprocessing and then replay the table from that point */ - private void processRows(RowSetShiftData shifted) { - // Get a sub-index of the source from that minimum reprocessing index and make sure we update our - // contextual chunks and FillContexts to an appropriate size for this step. - final RowSet sourceRowSet = source.getRowSet(); - - final int newChunkSize = (int) Math.min(control.chunkCapacityOrDefault(), rowsToProcess.size()); - setChunkSize(newChunkSize); - - for (int opIndex = 0; opIndex < operators.length; opIndex++) { - if (opAffected[opIndex]) { - final long keyStart = opContext[opIndex].getAffectedRows().firstRowKey(); - final long keyBefore; - try (final RowSet.SearchIterator sit = sourceRowSet.searchIterator()) { - keyBefore = sit.binarySearchValue( - (compareTo, ignored) -> Long.compare(keyStart - 1, compareTo), 1); - } - // apply a shift to keyBefore since the output column is still in prev key space- - - operators[opIndex].resetForProcess(opContext[opIndex], sourceRowSet, keyBefore); - } - } - - // Now iterate rowset to reprocess. - if (rowsToProcess.isEmpty()) { - return; - } - - initializeFor(rowsToProcess); - - try (final RowSet positionsToProcess = - anyRequirePositions ? source.getRowSet().invert(rowsToProcess) : null; - final RowSequence.Iterator keyIt = rowsToProcess.getRowSequenceIterator(); - final RowSequence.Iterator posIt = positionsToProcess == null ? null - : positionsToProcess.getRowSequenceIterator()) { - - while (keyIt.hasMore()) { - sharedContext.reset(); - if (prevSharedContext != null) { - prevSharedContext.reset(); - } - Arrays.fill(inputChunkPopulated, false); - - final RowSequence chunkOk = keyIt.getNextRowSequenceWithLength(chunkSize); - if (anyRequireKeys) { - chunkOk.fillRowKeyChunk(keyChunk.get()); - } - if (anyRequirePositions) { - posIt.getNextRowSequenceWithLength(chunkSize).fillRowKeyChunk(posChunk.get()); - } - - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (!opAffected[opIdx]) { - continue; - } - - final UpdateByOperator currentOp = operators[opIdx]; - final int slotPosition = inputSourceSlots[opIdx]; - // is this chunk relevant to this operator? If so, then intersect and process only the - // relevant rows - if (chunkOk.firstRowKey() <= opContext[opIdx].getAffectedRows().lastRowKey() - && chunkOk.lastRowKey() >= opContext[opIdx].getAffectedRows().firstRowKey()) { - try (final RowSet rs = chunkOk.asRowSet(); - final RowSet intersect = rs.intersect(opContext[opIdx].getAffectedRows())) { - - prepareValuesChunkFor(opIdx, slotPosition, false, true, intersect, intersect, - null, postWorkingChunks[slotPosition].get(), - null, fillContexts[slotPosition].get()); - currentOp.processChunk(opContext[opIdx], - intersect, - keyChunk.get(), - posChunk.get(), - postWorkingChunks[slotPosition].get(), - source.getRowSet()); - } - } - } - } - finishFor(); - } - } - - /** - * Prepare the specified chunk for use. - * - * @param opIdx the operator index - * @param usePrev if previous values should be fetched - * @param chunkOk the {@link RowSequence} for current values - * @param prevChunkOk the {@link RowSequence} for previous values. - */ - private void prepareValuesChunkFor(final int opIdx, - final int inputSlot, - final boolean usePrev, - final boolean useCurrent, - final RowSequence chunkOk, - final RowSequence prevChunkOk, - final WritableChunk prevWorkingChunk, - final WritableChunk postWorkingChunk, - final ChunkSource.FillContext prevFillContext, - final ChunkSource.FillContext postFillContext) { - if (!operators[opIdx].requiresValues(opContext[opIdx])) { - return; - } - - if (!inputChunkPopulated[inputSlot]) { - // Using opIdx below is OK, because if we are sharing an input slot, we are referring to the same - // input source. Instead of maintaining another array of sourced by slot, we'll just use the opIdx - inputChunkPopulated[inputSlot] = true; - if (usePrev) { - inputSources[opIdx].fillPrevChunk(prevFillContext, prevWorkingChunk, prevChunkOk); - } - - if (useCurrent) { - inputSources[opIdx].fillChunk(postFillContext, postWorkingChunk, chunkOk); + private void processRows() { + // this could be parallelized since the windows probably won't overlap + for (int winIdx = 0; winIdx < windows.length; winIdx++) { + if (windowAffected[winIdx]) { + // this will internally call initialize() and finish() for each operator + windowContexts[winIdx].processRows(); } } } @@ -671,8 +375,8 @@ public void onUpdate(TableUpdate upstream) { } } - // Now do the reprocessing phase. - ctx.processRows(upstream.shifted()); + // Now do the processing + ctx.processRows(); final TableUpdateImpl downstream = new TableUpdateImpl(); // copy these rowSets since TableUpdateImpl#reset will close them with the upstream update diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java new file mode 100644 index 00000000000..73b1bc817b6 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java @@ -0,0 +1,400 @@ +package io.deephaven.engine.table.impl.updateby; + +import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.*; +import io.deephaven.engine.table.ChunkSource; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.ModifiedColumnSet; +import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.UpdateByOperator; +import io.deephaven.engine.table.impl.UpdateByWindowedOperator; +import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; +import io.deephaven.util.SafeCloseable; +import org.apache.commons.lang3.mutable.MutableLong; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.util.ArrayList; + +import static io.deephaven.util.QueryConstants.NULL_LONG; + +public class UpdateByWindow { + protected final boolean windowed; + @Nullable + protected final String timestampColumnName; + protected final long prevUnits; + protected final long fwdUnits; + + protected ArrayList operators; + + public class UpdateByWindowContext implements SafeCloseable { + /** the rows affected by this update */ + protected RowSet affectedRows; + protected RowSet influencerRows; + + // for use with a ticking window + protected RowSet affectedRowPositions; + protected RowSet influencerPositions; + + // keep track of what rows were modified (we'll use a single set for all operators sharing a window) + protected RowSetBuilderSequential modifiedBuilder; + protected RowSet newModified; + + /** the column source providing the timestamp data for this window */ + @Nullable + protected ColumnSource timestampColumnSource; + + /** the timestamp SSA providing fast lookup for time windows */ + @Nullable + protected LongSegmentedSortedArray timestampSsa; + + /** An array of boolean denoting which operators are affected by the current update. */ + final boolean[] opAffected; + + /** An array of context objects for each underlying operator */ + final UpdateByOperator.UpdateContext[] opContext; + + public UpdateByWindowContext(final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa) { + this.timestampColumnSource = timestampColumnSource; + this.timestampSsa = timestampSsa; + + this.opAffected = new boolean[operators.size()]; + // noinspection unchecked +// this.fillContexts = new SizedSafeCloseable[operators.size()]; + this.opContext = new UpdateByOperator.UpdateContext[operators.size()]; + } + + public boolean computeAffectedAndMakeContexts(@NotNull final TableUpdate upstream, @NotNull final TrackingRowSet source, @Nullable final ModifiedColumnSet[] inputModifiedColumnSets, final int chunkSize, final boolean initialStep) { + // determine the affected rows for this window context + if (initialStep) { + // all rows are affected initially + affectedRows = source.copy(); + influencerRows = affectedRows; + + // no need to invert, just create a flat rowset + if (windowed && timestampColumnName == null) { + affectedRowPositions = RowSetFactory.flat(source.size()); + influencerPositions = RowSetFactory.flat(source.size()); + } + // mark all operators as affected by this update and create contexts + for (int opIdx = 0; opIdx < operators.size(); opIdx++) { + opAffected[opIdx] = true; + opContext[opIdx] = operators.get(opIdx).makeUpdateContext(chunkSize, timestampSsa); + } + return true; + } else { + // determine which operators are affected by this change + boolean anyAffected = false; + boolean allAffected = upstream.added().isNonempty() || + upstream.removed().isNonempty() || + upstream.shifted().nonempty(); + + for (int opIdx = 0; opIdx < operators.size(); opIdx++) { + opAffected[opIdx] = allAffected + || (upstream.modifiedColumnSet().nonempty() && (inputModifiedColumnSets == null + || upstream.modifiedColumnSet().containsAny(inputModifiedColumnSets[opIdx]))); + + // mark the operators affected by this update and create contexts + if (opAffected[opIdx]) { + anyAffected = true; + opContext[opIdx] = operators.get(opIdx).makeUpdateContext(chunkSize, timestampSsa); + } + } + + if (source.isEmpty() || !anyAffected) { + // no work to do for this window this cycle + return false; + } + + // handle the three major types of windows: cumulative, windowed by ticks, windowed by time + + // cumulative is simple, just find the smallest key and return the range from smallest to end + if (!windowed) { + long smallestModifiedKey = smallestAffectedKey(upstream.added(), upstream.modified(), upstream.removed(), + upstream.shifted(), source); + + affectedRows = smallestModifiedKey == Long.MAX_VALUE + ? RowSetFactory.empty() + : source.subSetByKeyRange(smallestModifiedKey, source.lastRowKey()); + influencerRows = affectedRows; + return true; + } + + // changed rows are all mods+adds + WritableRowSet changed = upstream.added().copy(); + changed.insert(upstream.modified()); + + WritableRowSet tmpAffected; + + // compute the rows affected from these changes + if (timestampColumnName == null) { + try (final WritableRowSet changedInverted = source.invert(changed)) { + tmpAffected = computeAffectedRowsTicks(source, changed, changedInverted, prevUnits, fwdUnits); + } + } else { + tmpAffected = computeAffectedRowsTime(source, changed, prevUnits, fwdUnits); + } + + // other rows can be affected by removes + if (upstream.removed().isNonempty()) { + try (final RowSet prev = source.copyPrev(); + final RowSet removedPositions = timestampColumnName == null + ? null : prev.invert(upstream.removed()); + final WritableRowSet affectedByRemoves = timestampColumnName == null + ? computeAffectedRowsTicks(prev, upstream.removed(), removedPositions, prevUnits, fwdUnits) + : computeAffectedRowsTime(prev, upstream.removed(), prevUnits, fwdUnits)) { + // apply shifts to get back to pos-shift space + upstream.shifted().apply(affectedByRemoves); + // retain only the rows that still exist in the source + affectedByRemoves.retain(source); + tmpAffected.insert(affectedByRemoves); + } + } + + affectedRows = tmpAffected; + + // now get influencer rows for the affected rows + if (timestampColumnName == null) { + // generate position data rowsets for efficiently computed position offsets + affectedRowPositions = source.invert(affectedRows); + + influencerRows = computeInfluencerRowsTicks(source, affectedRows, affectedRowPositions, prevUnits, fwdUnits); + influencerPositions = source.invert(influencerRows); + } else { + influencerRows = computeInfluencerRowsTime(source, affectedRows, prevUnits, fwdUnits); + affectedRowPositions = null; + influencerPositions = null; + } + } + return true; + } + + /** + * Find the smallest valued key that participated in the upstream {@link TableUpdate}. + * + * @param added the added rows + * @param modified the modified rows + * @param removed the removed rows + * @param shifted the shifted rows + * + * @return the smallest key that participated in any part of the update. + */ + private long smallestAffectedKey(@NotNull final RowSet added, + @NotNull final RowSet modified, + @NotNull final RowSet removed, + @NotNull final RowSetShiftData shifted, + @NotNull final RowSet affectedIndex) { + + long smallestModifiedKey = Long.MAX_VALUE; + if (removed.isNonempty()) { + smallestModifiedKey = removed.firstRowKey(); + } + + if (added.isNonempty()) { + smallestModifiedKey = Math.min(smallestModifiedKey, added.firstRowKey()); + } + + if (modified.isNonempty()) { + smallestModifiedKey = Math.min(smallestModifiedKey, modified.firstRowKey()); + } + + if (shifted.nonempty()) { + final long firstModKey = modified.isEmpty() ? Long.MAX_VALUE : modified.firstRowKey(); + boolean modShiftFound = !modified.isEmpty(); + boolean affectedFound = false; + try (final RowSequence.Iterator it = affectedIndex.getRowSequenceIterator()) { + for (int shiftIdx = 0; shiftIdx < shifted.size() + && (!modShiftFound || !affectedFound); shiftIdx++) { + final long shiftStart = shifted.getBeginRange(shiftIdx); + final long shiftEnd = shifted.getEndRange(shiftIdx); + final long shiftDelta = shifted.getShiftDelta(shiftIdx); + + if (!affectedFound) { + if (it.advance(shiftStart + shiftDelta)) { + final long maybeAffectedKey = it.peekNextKey(); + if (maybeAffectedKey <= shiftEnd + shiftDelta) { + affectedFound = true; + final long keyToCompare = + shiftDelta > 0 ? maybeAffectedKey - shiftDelta : maybeAffectedKey; + smallestModifiedKey = Math.min(smallestModifiedKey, keyToCompare); + } + } else { + affectedFound = true; + } + } + + if (!modShiftFound) { + if (firstModKey <= (shiftEnd + shiftDelta)) { + modShiftFound = true; + // If the first modified key is in the range we should include it + if (firstModKey >= (shiftStart + shiftDelta)) { + smallestModifiedKey = Math.min(smallestModifiedKey, firstModKey - shiftDelta); + } else { + // Otherwise it's not included in any shifts, and since shifts can't reorder rows + // it is the smallest possible value and we've already accounted for it above. + break; + } + } + } + } + } + } + + return smallestModifiedKey; + } + + private WritableRowSet computeAffectedRowsTime(final RowSet sourceSet, final RowSet subset, long revNanos, + long fwdNanos) { + // swap fwd/rev to get the affected windows + return computeInfluencerRowsTime(sourceSet, subset, fwdNanos, revNanos); + } + + private WritableRowSet computeInfluencerRowsTime(final RowSet sourceSet, final RowSet subset, long revNanos, + long fwdNanos) { + if (sourceSet.size() == subset.size()) { + return sourceSet.copy(); + } + + int chunkSize = (int) Math.min(subset.size(), 4096); + try (final RowSequence.Iterator it = subset.getRowSequenceIterator(); + final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(chunkSize)) { + final RowSetBuilderSequential builder = RowSetFactory.builderSequential(); + LongSegmentedSortedArray.Iterator ssaIt = timestampSsa.iterator(false, false); + while (it.hasMore() && ssaIt.hasNext()) { + final RowSequence rs = it.getNextRowSequenceWithLength(chunkSize); + LongChunk timestamps = timestampColumnSource.getChunk(context, rs).asLongChunk(); + + for (int ii = 0; ii < rs.intSize(); ii++) { + // if the timestamp of the row is null, it won't belong to any set and we can ignore it + // completely + final long ts = timestamps.get(ii); + if (ts != NULL_LONG) { + // look at every row timestamp, compute the head and tail in nanos + final long head = ts - revNanos; + final long tail = ts + fwdNanos; + + // advance the iterator to the beginning of the window + if (ssaIt.nextValue() < head) { + ssaIt.advanceToBeforeFirst(head); + if (!ssaIt.hasNext()) { + // SSA is exhausted + break; + } + } + + Assert.assertion(ssaIt.hasNext() && ssaIt.nextValue() >= head, + "SSA Iterator outside of window"); + + // step through the SSA and collect keys until outside of the window + while (ssaIt.hasNext() && ssaIt.nextValue() <= tail) { + builder.appendKey(ssaIt.nextKey()); + ssaIt.next(); + } + + if (!ssaIt.hasNext()) { + // SSA is exhausted + break; + } + } + } + } + return builder.build(); + } + } + + private WritableRowSet computeAffectedRowsTicks(final RowSet sourceSet, final RowSet subset, final RowSet invertedSubSet, long revTicks, long fwdTicks, @Nullable final WritableRowSet ) { + // swap fwd/rev to get the influencer windows + return computeInfluencerRowsTicks(sourceSet, subset, invertedSubSet, fwdTicks, revTicks); + } + + private WritableRowSet computeInfluencerRowsTicks(final RowSet sourceSet, final RowSet subset, final RowSet invertedSubSet, long revTicks, + long fwdTicks) { + if (sourceSet.size() == subset.size()) { + return sourceSet.copy(); + } + + long maxPos = sourceSet.size() - 1; + + final RowSetBuilderSequential builder = RowSetFactory.builderSequential(); + final MutableLong minPos = new MutableLong(0L); + + invertedSubSet.forAllRowKeyRanges((s, e) -> { + long sPos = Math.max(s - revTicks, minPos.longValue()); + long ePos = Math.min(e + fwdTicks, maxPos); + builder.appendRange(sPos, ePos); + minPos.setValue(ePos + 1); + }); + + try (final RowSet positions = builder.build()) { + return sourceSet.subSetForPositions(positions); + } + } + + public RowSet getAffectedRows() { + return affectedRows; + } + + public RowSet getInfluencerRows() { + if (!windowed) { + return affectedRows; + } + return influencerRows; + } + + @Override + public void close() { + if (influencerRows != null && influencerRows != affectedRows) { + influencerRows.close(); + } + if (influencerPositions != null && influencerPositions != affectedRowPositions) { + influencerPositions.close(); + } + try (final RowSet ignoredRs1 = affectedRows; + final RowSet ignoredRs2 = affectedRowPositions; + final RowSet ignoredRs3 = newModified) { + } + } + } + + public UpdateByWindowContext makeWindowContext(final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa) { + return new UpdateByWindowContext(timestampColumnSource, timestampSsa); + } + + private UpdateByWindow(boolean windowed, @Nullable String timestampColumnName, long prevUnits, long fwdUnits) { + this.windowed = windowed; + this.timestampColumnName = timestampColumnName; + this.prevUnits = prevUnits; + this.fwdUnits = fwdUnits; + + this.operators = new ArrayList<>(); + } + + public static UpdateByWindow createFromOperator(final UpdateByOperator op) { + return new UpdateByWindow(op instanceof UpdateByWindowedOperator, + op.getTimestampColumnName(), + op.getPrevWindowUnits(), + op.getPrevWindowUnits()); + } + + public void addOperator(UpdateByOperator op) { + operators.add(op); + } + + @Nullable + public String getTimestampColumnName() { + return timestampColumnName; + } + + @Override + public int hashCode() { + int hash = Boolean.hashCode(windowed); + if (timestampColumnName != null) { + hash = 31 * hash + timestampColumnName.hashCode(); + } + hash = 31 * hash + Long.hashCode(prevUnits); + hash = 31 * hash + Long.hashCode(fwdUnits); + return hash; + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java index c9b9c31644e..6fbe777afad 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java @@ -173,9 +173,7 @@ protected abstract void doProcessChunk(@NotNull final Context ctx, // region Shifts @Override - public void applyOutputShift(@NotNull final UpdateContext context, - @NotNull final RowSet subIndexToShift, - final long delta) { + public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { if (outputSource instanceof BooleanSparseArraySource.ReinterpretedAsByte) { ((BooleanSparseArraySource.ReinterpretedAsByte)outputSource).shift(subIndexToShift, delta); } else { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java index 0fe5c19a4b0..817a91f2359 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java @@ -150,9 +150,7 @@ protected abstract void doProcessChunk(@NotNull final Context ctx, // region Shifts @Override - public void applyOutputShift(@NotNull final UpdateContext context, - @NotNull final RowSet subIndexToShift, - final long delta) { + public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { ((CharacterSparseArraySource)outputSource).shift(subIndexToShift, delta); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java index 6963f208584..62d723cc54b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java @@ -144,9 +144,7 @@ protected abstract void doProcessChunk(@NotNull final Context ctx, // region Shifts @Override - public void applyOutputShift(@NotNull final UpdateContext context, - @NotNull final RowSet subRowSetToShift, - final long delta) { + public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long delta) { ((DoubleSparseArraySource)outputSource).shift(subRowSetToShift, delta); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java index 78978009810..d5cbd89809a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java @@ -139,9 +139,7 @@ protected abstract void doProcessChunk(@NotNull final Context ctx, // region Shifts @Override - public void applyOutputShift(@NotNull final UpdateContext context, - @NotNull final RowSet subRowSetToShift, - final long delta) { + public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long delta) { ((FloatSparseArraySource)outputSource).shift(subRowSetToShift, delta); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java index 89bf7134535..012691a9f09 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java @@ -155,9 +155,7 @@ protected abstract void doProcessChunk(@NotNull final Context ctx, // region Shifts @Override - public void applyOutputShift(@NotNull final UpdateContext context, - @NotNull final RowSet subIndexToShift, - final long delta) { + public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { ((IntegerSparseArraySource)outputSource).shift(subIndexToShift, delta); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java index 999a09ce517..3706d57e9f7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java @@ -155,9 +155,7 @@ protected abstract void doProcessChunk(@NotNull final Context ctx, // region Shifts @Override - public void applyOutputShift(@NotNull final UpdateContext context, - @NotNull final RowSet subIndexToShift, - final long delta) { + public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { ((LongSparseArraySource)outputSource).shift(subIndexToShift, delta); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java index 39319af9d2d..45991523d0e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java @@ -158,9 +158,7 @@ protected abstract void doProcessChunk(@NotNull final Context ctx, // region Shifts @Override - public void applyOutputShift(@NotNull final UpdateContext context, - @NotNull final RowSet subIndexToShift, - final long delta) { + public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { ((ObjectSparseArraySource)outputSource).shift(subIndexToShift, delta); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java index 8be05b14229..3f27959b997 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java @@ -155,9 +155,7 @@ protected abstract void doProcessChunk(@NotNull final Context ctx, // region Shifts @Override - public void applyOutputShift(@NotNull final UpdateContext context, - @NotNull final RowSet subIndexToShift, - final long delta) { + public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { ((ShortSparseArraySource)outputSource).shift(subIndexToShift, delta); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java index 92b527a7073..a230c8adaab 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java @@ -197,9 +197,7 @@ public void startTrackingPrev() { } @Override - public void applyOutputShift(@NotNull final UpdateContext context, - @NotNull final RowSet subIndexToShift, - final long delta) { + public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { ((ObjectSparseArraySource) outputSource).shift(subIndexToShift, delta); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java index cd631d52863..189060bbb1e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java @@ -190,8 +190,7 @@ public void startTrackingPrev() { } @Override - public void applyOutputShift(@NotNull final UpdateContext context, - @NotNull final RowSet subIndexToShift, + public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { ((ObjectSparseArraySource)outputSource).shift(subIndexToShift, delta); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java index 219a70d4e0e..b1787862a07 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java @@ -200,9 +200,7 @@ public void startTrackingPrev() { } @Override - public void applyOutputShift(@NotNull final UpdateContext context, - @NotNull final RowSet subIndexToShift, - final long delta) { + public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { ((LongSparseArraySource)outputSource).shift(subIndexToShift, delta); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java index ed8ad72fcec..640cf45b833 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java @@ -218,9 +218,7 @@ public void startTrackingPrev() { } @Override - public void applyOutputShift(@NotNull final UpdateContext context, - @NotNull final RowSet subIndexToShift, - final long delta) { + public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { ((DoubleSparseArraySource)outputSource).shift(subIndexToShift, delta); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java index eadf635c01f..5d20daffd35 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java @@ -213,9 +213,7 @@ public void startTrackingPrev() { } @Override - public void applyOutputShift(@NotNull final UpdateContext context, - @NotNull final RowSet subIndexToShift, - final long delta) { + public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { ((FloatSparseArraySource)outputSource).shift(subIndexToShift, delta); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java index 02b5963b515..35125f3faa6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java @@ -197,9 +197,7 @@ public void startTrackingPrev() { } @Override - public void applyOutputShift(@NotNull final UpdateContext context, - @NotNull final RowSet subIndexToShift, - final long delta) { + public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { ((LongSparseArraySource)outputSource).shift(subIndexToShift, delta); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java index 79fbf0df7d1..a5ed7d7bbe1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java @@ -197,9 +197,7 @@ public void startTrackingPrev() { } @Override - public void applyOutputShift(@NotNull final UpdateContext context, - @NotNull final RowSet subIndexToShift, - final long delta) { + public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { ((LongSparseArraySource)outputSource).shift(subIndexToShift, delta); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java index add2b323dcb..fb2b8a60d70 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java @@ -192,9 +192,7 @@ public void startTrackingPrev() { } @Override - public void applyOutputShift(@NotNull final UpdateContext context, - @NotNull final RowSet subIndexToShift, - final long delta) { + public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { ((LongSparseArraySource)outputSource).shift(subIndexToShift, delta); } } diff --git a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java index 204b440a188..45da27436d1 100644 --- a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java +++ b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java @@ -151,9 +151,7 @@ private static void fixupByteBase(String byteResult) throws IOException { lines = replaceRegion(lines, "Shifts", Collections.singletonList( " @Override\n" + - " public void applyOutputShift(@NotNull final UpdateContext context,\n" + - " @NotNull final RowSet subIndexToShift,\n" + - " final long delta) {\n" + + " public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) {\n" + " if (outputSource instanceof BooleanSparseArraySource.ReinterpretedAsByte) {\n" + " ((BooleanSparseArraySource.ReinterpretedAsByte)outputSource).shift(subIndexToShift, delta);\n" From 4405b3fd6310d52db56776b1e414e2c3573025e3 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 23 Sep 2022 13:02:11 -0700 Subject: [PATCH 025/123] huge refactor almost complete, most tests passing --- .../impl/BucketedPartitionedUpdateBy.java | 8 +- .../deephaven/engine/table/impl/UpdateBy.java | 65 +- .../impl/UpdateByCumulativeOperator.java | 113 ++- .../engine/table/impl/UpdateByOperator.java | 145 +--- .../table/impl/UpdateByOperatorFactory.java | 92 +-- .../table/impl/UpdateByWindowedOperator.java | 411 ++++------ .../engine/table/impl/ZeroKeyUpdateBy.java | 100 +-- .../table/impl/updateby/UpdateByWindow.java | 733 ++++++++++++------ .../ema/BasePrimitiveEMAOperator.java | 136 +--- .../updateby/ema/BigDecimalEMAOperator.java | 70 +- .../updateby/ema/BigIntegerEMAOperator.java | 78 +- .../updateby/ema/BigNumberEMAOperator.java | 144 ++-- .../impl/updateby/ema/ByteEMAOperator.java | 64 +- .../impl/updateby/ema/DoubleEMAOperator.java | 98 +-- .../impl/updateby/ema/FloatEMAOperator.java | 98 +-- .../impl/updateby/ema/IntEMAOperator.java | 64 +- .../impl/updateby/ema/LongEMAOperator.java | 64 +- .../impl/updateby/ema/ShortEMAOperator.java | 64 +- .../updateby/fill/BooleanFillByOperator.java | 59 +- .../updateby/fill/ByteFillByOperator.java | 53 +- .../updateby/fill/CharFillByOperator.java | 53 +- .../updateby/fill/DoubleFillByOperator.java | 53 +- .../updateby/fill/FloatFillByOperator.java | 53 +- .../impl/updateby/fill/IntFillByOperator.java | 53 +- .../updateby/fill/LongFillByOperator.java | 59 +- .../updateby/fill/ObjectFillByOperator.java | 53 +- .../updateby/fill/ShortFillByOperator.java | 53 +- .../internal/BaseByteUpdateByOperator.java | 138 ++-- .../internal/BaseCharUpdateByOperator.java | 138 ++-- .../internal/BaseDoubleUpdateByOperator.java | 151 ++-- .../internal/BaseFloatUpdateByOperator.java | 151 ++-- .../internal/BaseIntUpdateByOperator.java | 138 ++-- .../internal/BaseLongUpdateByOperator.java | 138 ++-- .../internal/BaseObjectBinaryOperator.java | 76 +- .../internal/BaseObjectUpdateByOperator.java | 136 ++-- .../internal/BaseShortUpdateByOperator.java | 138 ++-- .../BaseWindowedByteUpdateByOperator.java | 131 ++-- .../BaseWindowedCharUpdateByOperator.java | 127 +-- .../BaseWindowedDoubleUpdateByOperator.java | 133 ++-- .../BaseWindowedFloatUpdateByOperator.java | 133 ++-- .../BaseWindowedIntUpdateByOperator.java | 127 +-- .../BaseWindowedLongUpdateByOperator.java | 127 +-- .../BaseWindowedObjectUpdateByOperator.java | 127 +-- .../BaseWindowedShortUpdateByOperator.java | 127 +-- .../LongRecordingUpdateByOperator.java | 154 ---- .../internal/PairwiseDoubleRingBuffer.java | 7 +- .../internal/PairwiseFloatRingBuffer.java | 7 +- .../minmax/ByteCumMinMaxOperator.java | 76 +- .../minmax/DoubleCumMinMaxOperator.java | 75 +- .../minmax/FloatCumMinMaxOperator.java | 75 +- .../updateby/minmax/IntCumMinMaxOperator.java | 76 +- .../minmax/LongCumMinMaxOperator.java | 86 +- .../minmax/ShortCumMinMaxOperator.java | 76 +- .../prod/BigDecimalCumProdOperator.java | 46 +- .../prod/BigIntegerCumProdOperator.java | 46 +- .../updateby/prod/ByteCumProdOperator.java | 72 +- .../updateby/prod/DoubleCumProdOperator.java | 71 +- .../updateby/prod/FloatCumProdOperator.java | 71 +- .../updateby/prod/IntCumProdOperator.java | 72 +- .../updateby/prod/LongCumProdOperator.java | 72 +- .../updateby/prod/ShortCumProdOperator.java | 72 +- .../BigDecimalRollingSumOperator.java | 154 +--- .../BigIntegerRollingSumOperator.java | 142 +--- .../rollingsum/ByteRollingSumOperator.java | 158 +--- .../rollingsum/DoubleRollingSumOperator.java | 145 +--- .../rollingsum/FloatRollingSumOperator.java | 145 +--- .../rollingsum/IntRollingSumOperator.java | 158 +--- .../rollingsum/LongRollingSumOperator.java | 154 +--- .../rollingsum/ShortRollingSumOperator.java | 158 +--- .../sum/BigDecimalCumSumOperator.java | 46 +- .../sum/BigIntegerCumSumOperator.java | 45 +- .../impl/updateby/sum/ByteCumSumOperator.java | 60 +- .../updateby/sum/DoubleCumSumOperator.java | 63 +- .../updateby/sum/FloatCumSumOperator.java | 65 +- .../impl/updateby/sum/IntCumSumOperator.java | 60 +- .../impl/updateby/sum/LongCumSumOperator.java | 60 +- .../updateby/sum/ShortCumSumOperator.java | 60 +- .../replicators/ReplicateUpdateBy.java | 6 +- 78 files changed, 3511 insertions(+), 4594 deletions(-) delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/LongRecordingUpdateByOperator.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java index 8daa50b1067..ebac3a92e85 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java @@ -37,7 +37,6 @@ class BucketedPartitionedUpdateBy extends UpdateBy { */ public static Table compute(@NotNull final String description, @NotNull final QueryTable source, - @NotNull final UpdateByWindow[] windows, @NotNull final UpdateByOperator[] ops, @NotNull final Map> resultSources, @NotNull final Collection byColumns, @@ -45,7 +44,6 @@ public static Table compute(@NotNull final String description, @NotNull final UpdateByControl control) { final BucketedPartitionedUpdateBy updateBy = new BucketedPartitionedUpdateBy(description, - windows, ops, source, resultSources, @@ -57,14 +55,13 @@ public static Table compute(@NotNull final String description, } protected BucketedPartitionedUpdateBy(@NotNull final String description, - @NotNull final UpdateByWindow[] windows, @NotNull final UpdateByOperator[] operators, @NotNull final QueryTable source, @NotNull final Map> resultSources, @NotNull final Collection byColumns, @NotNull final UpdateByRedirectionContext redirContext, @NotNull final UpdateByControl control) { - super(windows, operators, source, redirContext, control); + super(operators, source, redirContext, control); // create a source-listener that will listen to the source updates and apply the shifts to the output columns final QueryTable sourceListenerTable = new QueryTable(source.getRowSet(), source.getColumnSourceMap()); @@ -83,7 +80,7 @@ public void onUpdate(@NotNull final TableUpdate upstream) { upstream.shifted().apply((begin, end, delta) -> { try (final RowSet subRowSet = prevIdx.subSetByKeyRange(begin, end)) { for (int opIdx = 0; opIdx < operators.length; opIdx++) { - operators[opIdx].applyOutputShift(null, subRowSet, delta); + operators[opIdx].applyOutputShift(subRowSet, delta); } } }); @@ -124,7 +121,6 @@ public void onUpdate(@NotNull final TableUpdate upstream) { Table newTable = ZeroKeyUpdateBy.compute( description, (QueryTable) t, - windows, operators, resultSources, redirContext, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java index 2dd191f31e7..a26967fb417 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java @@ -1,5 +1,7 @@ package io.deephaven.engine.table.impl; +import gnu.trove.list.array.TIntArrayList; +import gnu.trove.map.TIntIntMap; import gnu.trove.map.hash.TIntObjectHashMap; import gnu.trove.map.hash.TObjectIntHashMap; import io.deephaven.api.ColumnName; @@ -30,7 +32,7 @@ * The core of the {@link Table#updateBy(UpdateByControl, Collection, Collection)} operation. */ public abstract class UpdateBy { - protected final ChunkSource.WithPrev[] inputSources; + protected final ChunkSource[] inputSources; protected final int[] inputSourceSlots; protected final UpdateByWindow[] windows; protected final UpdateByOperator[] operators; @@ -126,8 +128,7 @@ public void close() { } } - protected UpdateBy(@NotNull final UpdateByWindow[] windows, - @NotNull final UpdateByOperator[] operators, + protected UpdateBy(@NotNull final UpdateByOperator[] operators, @NotNull final QueryTable source, @NotNull final UpdateByRedirectionContext redirContext, UpdateByControl control) { @@ -139,13 +140,14 @@ protected UpdateBy(@NotNull final UpdateByWindow[] windows, } this.source = source; - this.windows = windows; this.operators = operators; - // noinspection unchecked - this.inputSources = new ChunkSource.WithPrev[operators.length]; + // the next bit is complicated but the goal is simple. We don't want to have duplicate input column sources, so + // we will store each one only once in inputSources and setup some mapping from the opIdx to the input column. + // noinspection unchecked + inputSources = new ChunkSource[operators.length]; + inputSourceSlots = new int[operators.length]; final TObjectIntHashMap> sourceToSlotMap = new TObjectIntHashMap<>(); - this.inputSourceSlots = new int[operators.length]; for (int opIdx = 0; opIdx < operators.length; opIdx++) { final ColumnSource input = source.getColumnSource(operators[opIdx].getInputColumnName()); final int maybeExistingSlot = sourceToSlotMap.get(input); @@ -157,6 +159,36 @@ protected UpdateBy(@NotNull final UpdateByWindow[] windows, inputSourceSlots[opIdx] = maybeExistingSlot; } } + + // now we want to divide the operators into similar windows for efficient processing + TIntObjectHashMap windowHashToObjectMap = new TIntObjectHashMap<>(); + TIntObjectHashMap windowHashToOperatorIndicesMap = new TIntObjectHashMap<>(); + + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + UpdateByWindow newWindow = UpdateByWindow.createFromOperator(operators[opIdx]); + final int hash = newWindow.hashCode(); + + // add this if not found + if (!windowHashToObjectMap.containsKey(hash)) { + windowHashToObjectMap.put(hash, newWindow); + windowHashToOperatorIndicesMap.put(hash, new TIntArrayList()); + } + windowHashToOperatorIndicesMap.get(hash).add(opIdx); + } + + // store the operator information into the windows + windowHashToObjectMap.valueCollection().forEach(window -> { + final int hash = window.hashCode(); + final int[] opIndices = windowHashToOperatorIndicesMap.get(hash).toArray(); + + final UpdateByOperator[] ops = + Arrays.stream(opIndices).mapToObj(idx -> operators[idx]).toArray(UpdateByOperator[]::new); + final int[] opInputSourceSlots = Arrays.stream(opIndices).map(idx -> inputSourceSlots[idx]).toArray(); + + window.setOperators(ops, opInputSourceSlots); + }); + + this.windows = windowHashToObjectMap.valueCollection().toArray(UpdateByOperator.ZERO_LENGTH_WINDOW_ARRAY); } // region UpdateBy implementation @@ -214,19 +246,6 @@ public static Table updateBy(@NotNull final QueryTable source, new UpdateByOperatorFactory(source, pairs, ctx, control); final Collection ops = updateByOperatorFactory.getOperators(clauses); - // build the windows for these operators - TIntObjectHashMap windowMap = new TIntObjectHashMap<>(); - ops.forEach(op -> { - UpdateByWindow newWindow = UpdateByWindow.createFromOperator(op); - final int hash = newWindow.hashCode(); - // add this if not found - if (!windowMap.containsKey(hash)) { - windowMap.put(hash, newWindow); - } - windowMap.get(hash).addOperator(op); - }); - - final StringBuilder descriptionBuilder = new StringBuilder("updateBy(ops={") .append(updateByOperatorFactory.describe(clauses)) .append("}"); @@ -245,18 +264,17 @@ public static Table updateBy(@NotNull final QueryTable source, String.join(", ", problems) + "}"); } + final UpdateByOperator[] opArr = ops.toArray(UpdateByOperator.ZERO_LENGTH_OP_ARRAY); + // noinspection rawtypes final Map> resultSources = new LinkedHashMap<>(source.getColumnSourceMap()); resultSources.putAll(opResultSources); - final UpdateByWindow[] windowArr = windowMap.valueCollection().toArray(UpdateByOperator.ZERO_LENGTH_WINDOW_ARRAY); - final UpdateByOperator[] opArr = ops.toArray(UpdateByOperator.ZERO_LENGTH_OP_ARRAY); if (pairs.length == 0) { descriptionBuilder.append(")"); Table ret = ZeroKeyUpdateBy.compute( descriptionBuilder.toString(), source, - windowArr, opArr, resultSources, ctx, @@ -290,7 +308,6 @@ public static Table updateBy(@NotNull final QueryTable source, Table ret = BucketedPartitionedUpdateBy.compute( descriptionBuilder.toString(), source, - windowArr, opArr, resultSources, byColumns, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java index 6919eeda7bb..6072bf1d610 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java @@ -1,104 +1,95 @@ package io.deephaven.engine.table.impl; -import io.deephaven.base.verify.Assert; -import io.deephaven.engine.rowset.*; -import io.deephaven.engine.table.TableUpdate; -import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; -import io.deephaven.engine.table.impl.updateby.internal.BaseCharUpdateByOperator; -import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedByteUpdateByOperator; -import io.deephaven.tablelogger.Row; +import io.deephaven.api.updateby.OperationControl; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.chunk.sized.SizedLongChunk; +import io.deephaven.engine.table.ChunkSink; +import io.deephaven.engine.table.MatchPair; import org.jetbrains.annotations.NotNull; +import static io.deephaven.util.QueryConstants.NULL_LONG; + public abstract class UpdateByCumulativeOperator implements UpdateByOperator { + protected final MatchPair pair; + protected final String[] affectingColumns; - public abstract class UpdateCumulativeContext implements UpdateContext { - protected LongSegmentedSortedArray timestampSsa; + protected final UpdateBy.UpdateByRedirectionContext redirContext; - protected RowSetBuilderSequential modifiedBuilder; - protected RowSet newModified; + // these will be used by the timestamp-aware operators (EMA for example) + protected OperationControl control; + protected long timeScaleUnits; + protected String timestampColumnName; - public LongSegmentedSortedArray getTimestampSsa() { - return timestampSsa; - } + protected class Context implements UpdateContext { + public long curTimestamp; - public RowSetBuilderSequential getModifiedBuilder() { - if (modifiedBuilder == null) { - modifiedBuilder = RowSetFactory.builderSequential(); - } - return modifiedBuilder; + protected Context(final int chunkSize) { + curTimestamp = NULL_LONG; } + @Override - public void close() { - try (final RowSet ignored = affectedRows; - final RowSet ignored2 = newModified) { - } - } + public void close() {} } - @Override - public void initializeFor(@NotNull final UpdateContext context, - @NotNull final RowSet updateRowSet) {} + public UpdateByCumulativeOperator(@NotNull final MatchPair pair, + @NotNull final String[] affectingColumns, + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { + this.pair = pair; + this.affectingColumns = affectingColumns; + this.redirContext = redirContext; - @Override - public void finishFor(@NotNull final UpdateContext context) { - UpdateCumulativeContext ctx = (UpdateCumulativeContext) context; - ctx.newModified = ctx.getModifiedBuilder().build(); + this.timeScaleUnits = 0L; + this.timestampColumnName = null; } - @NotNull - final public RowSet getAdditionalModifications(@NotNull final UpdateContext context) { - UpdateCumulativeContext ctx = (UpdateCumulativeContext) context; - return ctx.newModified; + abstract public void initializeUpdate(@NotNull final UpdateContext context, final long firstUnmodifiedKey, + long firstUnmodifiedTimestamp); + + public boolean isValueValid(long atKey) { + throw new UnsupportedOperationException("isValueValid() must be overriden by time-aware cumulative operators"); } @Override - final public boolean anyModified(@NotNull final UpdateContext context) { - UpdateCumulativeContext ctx = (UpdateCumulativeContext) context; - return ctx.newModified != null && ctx.newModified.isNonempty(); - } + public void finishUpdate(@NotNull final UpdateContext context) {} - /*** nearly all cumulative operators will not reference a timestamp column, exceptions are Ema */ @Override public String getTimestampColumnName() { - return null; + return timestampColumnName; } - /** - * Get the value of the backward-looking window (might be nanos or ticks). - * - * @return the name of the input column - */ @Override public long getPrevWindowUnits() { - return 0L; + return timeScaleUnits; } - /** - * Get the value of the forward-looking window (might be nanos or ticks). - * - * @return the name of the input column - */ + /** cumulative operators do not have a forward-looking window */ @Override public long getFwdWindowUnits() { return 0L; } - /*** cumulative operators do not need keys */ + @NotNull + @Override + public String getInputColumnName() { + return pair.rightColumn; + } + + @NotNull @Override - public boolean requiresKeys() { - return false; + public String[] getAffectingColumnNames() { + return affectingColumns; } - /*** cumulative operators do not need position data */ + @NotNull @Override - public boolean requiresPositions() { - return false; + public String[] getOutputColumnNames() { + return new String[] {pair.leftColumn}; } - /*** cumulative operators always need values */ @Override - public boolean requiresValues(@NotNull final UpdateContext context) { - return true; + public void pop(UpdateContext context) { + throw new UnsupportedOperationException("Cumulative operators should never call pop()"); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java index 125de13b17c..3c1e4d2e434 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java @@ -7,15 +7,10 @@ import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.rowset.RowSetShiftData; -import io.deephaven.engine.rowset.TrackingRowSet; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.Table; -import io.deephaven.engine.table.TableUpdate; -import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.updateby.UpdateByWindow; -import io.deephaven.engine.table.impl.updateby.internal.BaseCharUpdateByOperator; import io.deephaven.util.SafeCloseable; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -47,20 +42,6 @@ public interface UpdateByOperator { UpdateByWindow[] ZERO_LENGTH_WINDOW_ARRAY = new UpdateByWindow[0]; UpdateByOperator[] ZERO_LENGTH_OP_ARRAY = new UpdateByOperator[0]; - /** - * Check if the specified {@link TableUpdate} was append-only given the last known key within some other index. - * - * @param update the update to check - * @param lastKnownKey the last known key from some other index. - * @return if the update was append-only given the last known key - */ - static boolean isAppendOnly(@NotNull final TableUpdate update, final long lastKnownKey) { - return update.removed().isEmpty() && - update.modified().isEmpty() && - update.shifted().empty() && - update.added().firstRowKey() > lastKnownKey; - } - /** * A context item for use with {@link Table#updateBy(UpdateByControl, Collection, String...)} for non-bucketed * updates. @@ -68,6 +49,33 @@ static boolean isAppendOnly(@NotNull final TableUpdate update, final long lastKn interface UpdateContext extends SafeCloseable { } + /** + * Add a value to the operators current data set + * + * @param context the operator context for this action + * @param key the row key associated with the value + * @param pos the index in the associated chunk where this value can be found. Depending on the usage, might be a + * values chunk (for cumulative operators) or an influencer values chunk (for windowed). It is the task of + * the operator to pull the data from the chunk and use it properly + */ + void push(UpdateContext context, long key, int pos); + + /** + * Remove a value from the operators current data set. This is only valid for windowed operators since cumulative + * operators only append values + * + * @param context the operator context for this action + */ + void pop(UpdateContext context); + + /** + * Reset the operator data values to a known state. This may occur during initialization or when a windowed operator + * has an empty window + * + * @param context the operator context for this action + */ + void reset(UpdateContext context); + /** * Get the name of the input column this operator depends on. * @@ -131,86 +139,18 @@ interface UpdateContext extends SafeCloseable { * Make an {@link UpdateContext} suitable for use with non-bucketed updates. * * @param chunkSize The expected size of chunks that will be provided during the update, - * @param timestampSsa The timestamp SSA to use for time-based operations (null if using ticks) * @return a new context */ @NotNull - UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa); - - /** - *

              - * Initialize the context for the specified stage of the update process. This will always be followed by a call to - * {@link #finishFor(UpdateContext)} at the end of each successful update. - *

              - * - * @param context the context object - * @param updateRowSet the index of rows associated with the update. - */ - void initializeFor(@NotNull final UpdateContext context, - @NotNull final RowSet updateRowSet); + UpdateContext makeUpdateContext(final int chunkSize); /** * Perform and bookkeeping required at the end of a single part of the update. This is always preceded with a call - * to - * - * {@link #initializeFor(UpdateContext, RowSet)} + * to {@code #initializeUpdate(UpdateContext)} (specialized for each type of operator) * * @param context the context object */ - void finishFor(@NotNull final UpdateContext context); - - /** - * Get an index of rows that were modified beyond the input set of modifications from the upstream. This is invoked - * once at the end of a complete update cycle (that is, after all adds, removes, modifies and shifts have been - * processed) if {@link #anyModified(UpdateContext)} has returned true. - * - * @param context the context object - * @return a {@link RowSet index} of additional rows that were modified - */ - @NotNull - RowSet getAdditionalModifications(@NotNull final UpdateContext context); - - /** - * Check if the update has modified any rows for this operator. This is invoked once at the end of a complete update - * cycle (that is, after all adds, removes, modifies and shifts have been processed). - * - * @param context the context object - * @return true if the update modified any rows. - */ - boolean anyModified(@NotNull final UpdateContext context); - - /** - * Query if the operator requires key values for the current stage. This method will always be invoked after an - * appropriate invocation of {@link #initializeFor(UpdateContext, RowSet)} - * - * @return true if the operator requires keys for this operation - */ - boolean requiresKeys(); - - /** - * Query if the operator requires position values for the current stage. This method will always be invoked after an - * appropriate invocation of {@link #initializeFor(UpdateContext, RowSet)} - * - * @return true if the operator requires position indices for this operation - */ - boolean requiresPositions(); - - /** - * Query if the operator requires values for the current stage. - * - * @param context the context object - * @return true if values are required for compuitation - */ - boolean requiresValues(@NotNull final UpdateContext context); - - /** - * Set the chunk size to be used for operations. This is used during the processing phase when the chunks allocated - * during the normal processing phase may not be large enough. - * - * @param context the context object - * @param chunkSize the new chunk size - */ - void setChunkSize(@NotNull final UpdateContext context, final int chunkSize); + void finishUpdate(@NotNull final UpdateContext context); /** * Apply a shift to the operation. @@ -220,33 +160,18 @@ void initializeFor(@NotNull final UpdateContext context, /** * Process a chunk of data for an updateBy table. - * + * * @param context the context object * @param inputKeys the keys contained in the chunk - * @param keyChunk a {@link LongChunk} containing the keys if requested by {@link #requiresKeys()} or null. - * @param posChunk a {@link LongChunk} containing the positions if requested by {@link #requiresPositions()} or - * null. + * @param keyChunk a {@link LongChunk} containing the keys if requested + * @param posChunk a {@link LongChunk} containing the positions if requested * @param valuesChunk the current chunk of working values. - * @param postUpdateSourceIndex the resulting source index af + * @param timestampValuesChunk a {@link LongChunk} containing the working timestamps if requested */ void processChunk(@NotNull final UpdateContext context, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, @Nullable final LongChunk posChunk, @Nullable final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex); - - /** - * Reset the operator to the state at the `firstModifiedKey` for non-bucketed operation. This is invoked immediately - * prior to calls to {@link #resetForProcess(UpdateContext, RowSet, long)}.
              - *
              - * A `firstUnmodifiedKey` of {@link RowSet#NULL_ROW_KEY} indicates that the entire table needs to be recomputed. - * - * @param context the context object - * @param sourceIndex the current index of the source table - * @param firstUnmodifiedKey the first unmodified key after which we will reprocess rows. - */ - void resetForProcess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceIndex, - final long firstUnmodifiedKey); + @Nullable final LongChunk timestampValuesChunk); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java index 1181d0ed131..ef340be5298 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java @@ -2,31 +2,25 @@ import io.deephaven.api.agg.Pair; import io.deephaven.api.updateby.ColumnUpdateOperation; -import io.deephaven.api.updateby.UpdateByOperation; import io.deephaven.api.updateby.UpdateByControl; +import io.deephaven.api.updateby.UpdateByOperation; import io.deephaven.api.updateby.spec.*; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.Table; -import io.deephaven.engine.table.impl.sources.ReinterpretUtils; import io.deephaven.engine.table.impl.updateby.ema.*; import io.deephaven.engine.table.impl.updateby.fill.*; -import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; import io.deephaven.engine.table.impl.updateby.minmax.*; import io.deephaven.engine.table.impl.updateby.prod.*; import io.deephaven.engine.table.impl.updateby.rollingsum.*; import io.deephaven.engine.table.impl.updateby.sum.*; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; import io.deephaven.time.DateTime; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import java.math.BigDecimal; import java.math.BigInteger; -import java.time.Instant; import java.util.*; import java.util.stream.Collectors; -import java.util.stream.Stream; import static io.deephaven.util.BooleanUtils.NULL_BOOLEAN_AS_BYTE; import static io.deephaven.util.QueryConstants.NULL_BYTE; @@ -154,22 +148,13 @@ public Void visit(@NotNull final ColumnUpdateOperation clause) { @Override public Void visit(@NotNull final EmaSpec ema) { - final LongRecordingUpdateByOperator timeStampRecorder; final boolean isTimeBased = ema.timeScale().isTimeBased(); final String timestampCol = ema.timeScale().timestampCol(); - if (isTimeBased) { - timeStampRecorder = makeLongRecordingOperator(source, timestampCol); - ops.add(timeStampRecorder); - } else { - timeStampRecorder = null; - } - Arrays.stream(pairs) .filter(p -> !isTimeBased || !p.rightColumn().equals(timestampCol)) .map(fc -> makeEmaOperator(fc, source, - timeStampRecorder, ema)) .forEach(ops::add); return null; @@ -224,14 +209,13 @@ public Void visit(@NotNull final RollingSumSpec rs) { @SuppressWarnings("unchecked") private UpdateByOperator makeEmaOperator(@NotNull final MatchPair pair, @NotNull final TableDefaults source, - @Nullable final LongRecordingUpdateByOperator recorder, @NotNull final EmaSpec ema) { // noinspection rawtypes final ColumnSource columnSource = source.getColumnSource(pair.rightColumn); final Class csType = columnSource.getType(); final String[] affectingColumns; - if (recorder == null) { + if (ema.timeScale().timestampCol() == null) { affectingColumns = new String[] {pair.rightColumn}; } else { affectingColumns = new String[] {ema.timeScale().timestampCol(), pair.rightColumn}; @@ -241,35 +225,35 @@ private UpdateByOperator makeEmaOperator(@NotNull final MatchPair pair, final long timeScaleUnits = ema.timeScale().timescaleUnits(); if (csType == byte.class || csType == Byte.class) { - return new ByteEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, + return new ByteEMAOperator(pair, affectingColumns, ema.controlOrDefault(), ema.timeScale().timestampCol(), timeScaleUnits, columnSource, redirContext); } else if (csType == short.class || csType == Short.class) { - return new ShortEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, + return new ShortEMAOperator(pair, affectingColumns, ema.controlOrDefault(), ema.timeScale().timestampCol(), timeScaleUnits, columnSource, redirContext); } else if (csType == int.class || csType == Integer.class) { - return new IntEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, + return new IntEMAOperator(pair, affectingColumns, ema.controlOrDefault(), ema.timeScale().timestampCol(), timeScaleUnits, columnSource, redirContext); } else if (csType == long.class || csType == Long.class) { - return new LongEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, + return new LongEMAOperator(pair, affectingColumns, ema.controlOrDefault(), ema.timeScale().timestampCol(), timeScaleUnits, columnSource, redirContext); } else if (csType == float.class || csType == Float.class) { - return new FloatEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, + return new FloatEMAOperator(pair, affectingColumns, ema.controlOrDefault(), ema.timeScale().timestampCol(), timeScaleUnits, columnSource, redirContext); } else if (csType == double.class || csType == Double.class) { - return new DoubleEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, + return new DoubleEMAOperator(pair, affectingColumns, ema.controlOrDefault(), ema.timeScale().timestampCol(), timeScaleUnits, columnSource, redirContext); } else if (csType == BigDecimal.class) { - return new BigDecimalEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, + return new BigDecimalEMAOperator(pair, affectingColumns, ema.controlOrDefault(), ema.timeScale().timestampCol(), timeScaleUnits, columnSource, redirContext); } else if (csType == BigInteger.class) { - return new BigIntegerEMAOperator(pair, affectingColumns, ema.controlOrDefault(), recorder, + return new BigIntegerEMAOperator(pair, affectingColumns, ema.controlOrDefault(), ema.timeScale().timestampCol(), timeScaleUnits, columnSource, redirContext); } @@ -277,23 +261,6 @@ private UpdateByOperator makeEmaOperator(@NotNull final MatchPair pair, throw new IllegalArgumentException("Can not perform EMA on type " + csType); } - private LongRecordingUpdateByOperator makeLongRecordingOperator(TableDefaults source, String colName) { - final ColumnSource columnSource = source.getColumnSource(colName); - final Class colType = columnSource.getType(); - if (colType != long.class && - colType != Long.class && - colType != DateTime.class && - colType != Instant.class && - !columnSource.allowsReinterpret(long.class)) { - throw new IllegalArgumentException("Column " + colName + " cannot be interpreted as a long"); - } - - final String[] inputColumns = Stream.concat(Stream.of(colName), - Arrays.stream(pairs).map(MatchPair::rightColumn)).toArray(String[]::new); - - return new LongRecordingUpdateByOperator(colName, inputColumns, columnSource); - } - private UpdateByOperator makeCumProdOperator(MatchPair fc, TableDefaults source) { final Class csType = source.getColumnSource(fc.rightColumn).getType(); if (csType == byte.class || csType == Byte.class) { @@ -394,16 +361,13 @@ private UpdateByOperator makeRollingSumOperator(@NotNull final MatchPair pair, @NotNull final RollingSumSpec rs) { // noinspection rawtypes final ColumnSource columnSource = source.getColumnSource(pair.rightColumn); - final ColumnSource timestampColumnSource; final Class csType = columnSource.getType(); final String[] affectingColumns; if (rs.prevTimeScale().timestampCol() == null) { affectingColumns = new String[] {pair.rightColumn}; - timestampColumnSource = null; } else { affectingColumns = new String[] {rs.prevTimeScale().timestampCol(), pair.rightColumn}; - timestampColumnSource = source.getColumnSource(rs.prevTimeScale().timestampCol()); } final long prevTimeScaleUnits = rs.prevTimeScale().timescaleUnits(); @@ -411,41 +375,41 @@ private UpdateByOperator makeRollingSumOperator(@NotNull final MatchPair pair, if (csType == Boolean.class || csType == boolean.class) { return new ByteRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), - rs.prevTimeScale().timestampCol(), timestampColumnSource, prevTimeScaleUnits, fwdTimeScaleUnits, - redirContext, ReinterpretUtils.booleanToByteSource(columnSource), NULL_BOOLEAN_AS_BYTE); + rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, + redirContext, NULL_BOOLEAN_AS_BYTE); } else if (csType == byte.class || csType == Byte.class) { return new ByteRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), - rs.prevTimeScale().timestampCol(), timestampColumnSource, prevTimeScaleUnits, fwdTimeScaleUnits, - redirContext, columnSource, NULL_BYTE); + rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, + redirContext, NULL_BYTE); } else if (csType == short.class || csType == Short.class) { return new ShortRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), - rs.prevTimeScale().timestampCol(), timestampColumnSource, prevTimeScaleUnits, fwdTimeScaleUnits, - redirContext, columnSource); + rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, + redirContext); } else if (csType == int.class || csType == Integer.class) { return new IntRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), - rs.prevTimeScale().timestampCol(), timestampColumnSource, prevTimeScaleUnits, fwdTimeScaleUnits, - redirContext, columnSource); + rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, + redirContext); } else if (csType == long.class || csType == Long.class) { return new LongRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), - rs.prevTimeScale().timestampCol(), timestampColumnSource, prevTimeScaleUnits, fwdTimeScaleUnits, - redirContext, columnSource); + rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, + redirContext); } else if (csType == float.class || csType == Float.class) { return new FloatRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), - rs.prevTimeScale().timestampCol(), timestampColumnSource, prevTimeScaleUnits, fwdTimeScaleUnits, - redirContext, columnSource); + rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, + redirContext); } else if (csType == double.class || csType == Double.class) { return new DoubleRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), - rs.prevTimeScale().timestampCol(), timestampColumnSource, - prevTimeScaleUnits, fwdTimeScaleUnits, redirContext, columnSource); + rs.prevTimeScale().timestampCol(), + prevTimeScaleUnits, fwdTimeScaleUnits, redirContext); } else if (csType == BigDecimal.class) { return new BigDecimalRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), - rs.prevTimeScale().timestampCol(), timestampColumnSource, - prevTimeScaleUnits, fwdTimeScaleUnits, redirContext, columnSource, + rs.prevTimeScale().timestampCol(), + prevTimeScaleUnits, fwdTimeScaleUnits, redirContext, control.mathContextOrDefault()); } else if (csType == BigInteger.class) { return new BigIntegerRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), - rs.prevTimeScale().timestampCol(), timestampColumnSource, - prevTimeScaleUnits, fwdTimeScaleUnits, redirContext, columnSource); + rs.prevTimeScale().timestampCol(), + prevTimeScaleUnits, fwdTimeScaleUnits, redirContext); } throw new IllegalArgumentException("Can not perform RollingSum on type " + csType); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java index cefce9841c2..99d476900a2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java @@ -1,8 +1,10 @@ package io.deephaven.engine.table.impl; import io.deephaven.api.updateby.OperationControl; +import io.deephaven.base.ringbuffer.IntRingBuffer; import io.deephaven.base.ringbuffer.LongRingBuffer; import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; @@ -22,180 +24,148 @@ import static io.deephaven.util.QueryConstants.NULL_LONG; public abstract class UpdateByWindowedOperator implements UpdateByOperator { + protected final MatchPair pair; + protected final String[] affectingColumns; + + protected UpdateBy.UpdateByRedirectionContext redirContext; + protected final OperationControl control; protected final String timestampColumnName; protected final long reverseTimeScaleUnits; protected final long forwardTimeScaleUnits; - protected final MatchPair pair; - protected final String[] affectingColumns; - - protected UpdateBy.UpdateByRedirectionContext redirContext; - public abstract class UpdateWindowedContext implements UpdateContext { - protected LongSegmentedSortedArray timestampSsa; + public void storeInfluencerValuesChunk(@NotNull final Chunk influencerValuesChunk) {} public int nullCount = 0; - - // store a local copy of the source rowset (may not be needed) - public RowSet sourceRowSet = null; - - // there are two sets of rows we will be tracking. `affected` rows need to be recomputed because of this - // update and `influencer` rows contain the data that will be used to compute the new values for the `affected` - // items. Because the windows are user-configurable, there may be no overlap between these two sets and we - // don't need values for the `affected` rows at all - protected RowSet affectedRows; - protected RowSet influencerRows; - protected long currentInfluencerKey; - + // + // protected long currentInfluencerKey; + // // candidate data for the window public final int WINDOW_CHUNK_SIZE = 4096; - - // allocate some chunks for holding the key, position and timestamp data - protected SizedLongChunk influencerKeyChunk; - protected SizedLongChunk influencerPosChunk; - protected SizedLongChunk influencerTimestampChunk; - - // for use with a ticking window - protected RowSet affectedRowPositions; - protected RowSet influencerPositions; - - protected long currentInfluencerPosOrTimestamp; - protected int currentInfluencerIndex; + // + // // allocate some chunks for holding the key, position and timestamp data + // protected SizedLongChunk influencerKeyChunk; + // protected SizedLongChunk influencerPosChunk; + // protected SizedLongChunk influencerTimestampChunk; + // + // // for use with a ticking window + // protected RowSet affectedRowPositions; + // protected RowSet influencerPositions; + // + // protected long currentInfluencerPosOrTimestamp; + // protected int currentInfluencerIndex; protected LongRingBuffer windowKeys = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); protected LongRingBuffer windowPosOrTimestamp = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); - public LongRingBuffer windowIndices = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); - - - - public RowSet getAffectedRows() { - return affectedRows; - } - - public RowSet getInfluencerRows() { - return influencerRows; - } - - public LongSegmentedSortedArray getTimestampSsa() { - return timestampSsa; - } - - public RowSetBuilderSequential getModifiedBuilder() { - if (modifiedBuilder == null) { - modifiedBuilder = RowSetFactory.builderSequential(); - } - return modifiedBuilder; - } - - public abstract void loadInfluencerValueChunk(); - - public void fillWindowTicks(UpdateWindowedContext context, long currentPos) { - // compute the head and tail positions (inclusive) - final long head = Math.max(0, currentPos - reverseTimeScaleUnits + 1); - final long tail = Math.min(sourceRowSet.size() - 1, currentPos + forwardTimeScaleUnits); - - // pop out all values from the current window that are not in the new window - while (!windowPosOrTimestamp.isEmpty() && windowPosOrTimestamp.front() < head) { - pop(context, windowKeys.remove(), (int) windowIndices.remove()); - windowPosOrTimestamp.remove(); - } - - // if the window is empty or completly filled with null, call reset() - if (windowPosOrTimestamp.isEmpty() || context.nullCount == windowPosOrTimestamp.size()) { - reset(context); - } - - // skip values until they match the window - while (currentInfluencerPosOrTimestamp < head) { - currentInfluencerIndex++; - - if (currentInfluencerIndex < influencerPosChunk.get().size()) { - currentInfluencerPosOrTimestamp = influencerPosChunk.get().get(currentInfluencerIndex); - currentInfluencerKey = influencerKeyChunk.get().get(currentInfluencerIndex); - } else { - currentInfluencerPosOrTimestamp = Long.MAX_VALUE; - currentInfluencerKey = Long.MAX_VALUE; - } - } - - // push matching values - while (currentInfluencerPosOrTimestamp <= tail) { - push(context, currentInfluencerKey, currentInfluencerIndex); - windowKeys.add(currentInfluencerKey); - windowPosOrTimestamp.add(currentInfluencerPosOrTimestamp); - windowIndices.add(currentInfluencerIndex); - currentInfluencerIndex++; - - if (currentInfluencerIndex < influencerPosChunk.get().size()) { - currentInfluencerPosOrTimestamp = influencerPosChunk.get().get(currentInfluencerIndex); - currentInfluencerKey = influencerKeyChunk.get().get(currentInfluencerIndex); - } else { - currentInfluencerPosOrTimestamp = Long.MAX_VALUE; - currentInfluencerKey = Long.MAX_VALUE; - } - } - } - - public void fillWindowTime(UpdateWindowedContext context, long currentTimestamp) { - // compute the head and tail positions (inclusive) - final long head = currentTimestamp - reverseTimeScaleUnits; - final long tail = currentTimestamp + forwardTimeScaleUnits; - - // pop out all values from the current window that are not in the new window - while (!windowPosOrTimestamp.isEmpty() && windowPosOrTimestamp.front() < head) { - pop(context, windowKeys.remove(), (int) windowIndices.remove()); - windowPosOrTimestamp.remove(); - } - - // if the window is empty or completly filled with null, call reset() - if (windowPosOrTimestamp.isEmpty() || context.nullCount == windowPosOrTimestamp.size()) { - reset(context); - } - - // skip values until they match the window - while (currentInfluencerPosOrTimestamp < head) { - currentInfluencerIndex++; - - if (currentInfluencerIndex < influencerTimestampChunk.get().size()) { - currentInfluencerPosOrTimestamp = influencerTimestampChunk.get().get(currentInfluencerIndex); - currentInfluencerKey = influencerKeyChunk.get().get(currentInfluencerIndex); - } else { - currentInfluencerPosOrTimestamp = Long.MAX_VALUE; - currentInfluencerKey = Long.MAX_VALUE; - } - } - - // push matching values - while (currentInfluencerPosOrTimestamp <= tail) { - push(context, currentInfluencerKey, currentInfluencerIndex); - windowKeys.add(currentInfluencerKey); - windowPosOrTimestamp.add(currentInfluencerPosOrTimestamp); - windowIndices.add(currentInfluencerIndex); - currentInfluencerIndex++; - - if (currentInfluencerIndex < influencerTimestampChunk.get().size()) { - currentInfluencerPosOrTimestamp = influencerTimestampChunk.get().get(currentInfluencerIndex); - currentInfluencerKey = influencerKeyChunk.get().get(currentInfluencerIndex); - } else { - currentInfluencerPosOrTimestamp = Long.MAX_VALUE; - currentInfluencerKey = Long.MAX_VALUE; - } - } - } + public IntRingBuffer windowIndices = new IntRingBuffer(WINDOW_CHUNK_SIZE, true); + + // public abstract void loadInfluencerValueChunk(); + + // public void fillWindowTicks(UpdateWindowedContext context, long currentPos) { + // // compute the head and tail positions (inclusive) + // final long head = Math.max(0, currentPos - reverseTimeScaleUnits + 1); + // final long tail = Math.min(sourceRowSet.size() - 1, currentPos + forwardTimeScaleUnits); + // + // // pop out all values from the current window that are not in the new window + // while (!windowPosOrTimestamp.isEmpty() && windowPosOrTimestamp.front() < head) { + // pop(context, windowKeys.remove(), (int) windowIndices.remove()); + // windowPosOrTimestamp.remove(); + // } + // + // // if the window is empty or completly filled with null, call reset() + // if (windowPosOrTimestamp.isEmpty() || context.nullCount == windowPosOrTimestamp.size()) { + // reset(context); + // } + // + // // skip values until they match the window + // while (currentInfluencerPosOrTimestamp < head) { + // currentInfluencerIndex++; + // + // if (currentInfluencerIndex < influencerPosChunk.get().size()) { + // currentInfluencerPosOrTimestamp = influencerPosChunk.get().get(currentInfluencerIndex); + // currentInfluencerKey = influencerKeyChunk.get().get(currentInfluencerIndex); + // } else { + // currentInfluencerPosOrTimestamp = Long.MAX_VALUE; + // currentInfluencerKey = Long.MAX_VALUE; + // } + // } + // + // // push matching values + // while (currentInfluencerPosOrTimestamp <= tail) { + // push(context, currentInfluencerKey, currentInfluencerIndex); + // windowKeys.add(currentInfluencerKey); + // windowPosOrTimestamp.add(currentInfluencerPosOrTimestamp); + // windowIndices.add(currentInfluencerIndex); + // currentInfluencerIndex++; + // + // if (currentInfluencerIndex < influencerPosChunk.get().size()) { + // currentInfluencerPosOrTimestamp = influencerPosChunk.get().get(currentInfluencerIndex); + // currentInfluencerKey = influencerKeyChunk.get().get(currentInfluencerIndex); + // } else { + // currentInfluencerPosOrTimestamp = Long.MAX_VALUE; + // currentInfluencerKey = Long.MAX_VALUE; + // } + // } + // } + // + // public void fillWindowTime(UpdateWindowedContext context, long currentTimestamp) { + // // compute the head and tail positions (inclusive) + // final long head = currentTimestamp - reverseTimeScaleUnits; + // final long tail = currentTimestamp + forwardTimeScaleUnits; + // + // // pop out all values from the current window that are not in the new window + // while (!windowPosOrTimestamp.isEmpty() && windowPosOrTimestamp.front() < head) { + // pop(context, windowKeys.remove(), (int) windowIndices.remove()); + // windowPosOrTimestamp.remove(); + // } + // + // // if the window is empty or completly filled with null, call reset() + // if (windowPosOrTimestamp.isEmpty() || context.nullCount == windowPosOrTimestamp.size()) { + // reset(context); + // } + // + // // skip values until they match the window + // while (currentInfluencerPosOrTimestamp < head) { + // currentInfluencerIndex++; + // + // if (currentInfluencerIndex < influencerTimestampChunk.get().size()) { + // currentInfluencerPosOrTimestamp = influencerTimestampChunk.get().get(currentInfluencerIndex); + // currentInfluencerKey = influencerKeyChunk.get().get(currentInfluencerIndex); + // } else { + // currentInfluencerPosOrTimestamp = Long.MAX_VALUE; + // currentInfluencerKey = Long.MAX_VALUE; + // } + // } + // + // // push matching values + // while (currentInfluencerPosOrTimestamp <= tail) { + // push(context, currentInfluencerKey, currentInfluencerIndex); + // windowKeys.add(currentInfluencerKey); + // windowPosOrTimestamp.add(currentInfluencerPosOrTimestamp); + // windowIndices.add(currentInfluencerIndex); + // currentInfluencerIndex++; + // + // if (currentInfluencerIndex < influencerTimestampChunk.get().size()) { + // currentInfluencerPosOrTimestamp = influencerTimestampChunk.get().get(currentInfluencerIndex); + // currentInfluencerKey = influencerKeyChunk.get().get(currentInfluencerIndex); + // } else { + // currentInfluencerPosOrTimestamp = Long.MAX_VALUE; + // currentInfluencerKey = Long.MAX_VALUE; + // } + // } + // } @Override public void close() { - try (final SizedLongChunk ignoredChk1 = influencerKeyChunk; - final SizedLongChunk ignoredChk2 = influencerPosChunk; - final SizedLongChunk ignoredChk3 = influencerTimestampChunk; - final RowSet ignoredRs1 = affectedRows; - final RowSet ignoredRs2 = influencerRows; - final RowSet ignoredRs3 = affectedRowPositions; - final RowSet ignoredRs4 = influencerPositions; - final RowSet ignoredRs5 = newModified) { - } + // try (final SizedLongChunk ignoredChk1 = influencerKeyChunk; + // final SizedLongChunk ignoredChk2 = influencerPosChunk; + // final SizedLongChunk ignoredChk3 = influencerTimestampChunk; + // final RowSet ignoredRs3 = affectedRowPositions; + // final RowSet ignoredRs4 = influencerPositions; + // ) { + // } } } @@ -212,7 +182,6 @@ public UpdateByWindowedOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final String timestampColumnName, - @Nullable final ColumnSource timestampColumnSource, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { @@ -220,126 +189,74 @@ public UpdateByWindowedOperator(@NotNull final MatchPair pair, this.affectingColumns = affectingColumns; this.control = control; this.timestampColumnName = timestampColumnName; - this.timestampColumnSource = - timestampColumnSource == null ? null : ReinterpretUtils.maybeConvertToPrimitive(timestampColumnSource); this.reverseTimeScaleUnits = reverseTimeScaleUnits; this.forwardTimeScaleUnits = forwardTimeScaleUnits; this.redirContext = redirContext; } - public abstract void push(UpdateContext context, long key, int pos); - - public abstract void pop(UpdateContext context, long key, int pos); - - public abstract void reset(UpdateContext context); - - @Override - public void initializeFor(@NotNull final UpdateContext context, - @NotNull final RowSet updateRowSet) { + public void initializeUpdate(@NotNull final UpdateContext context) { final UpdateWindowedContext ctx = (UpdateWindowedContext) context; - - // load all the influencer values this update will need - ctx.loadInfluencerValueChunk(); - - // load all the influencer keys - ctx.influencerKeyChunk = new SizedLongChunk<>(ctx.influencerRows.intSize()); - ctx.influencerRows.fillRowKeyChunk(ctx.influencerKeyChunk.get()); - ctx.currentInfluencerKey = ctx.influencerRows.firstRowKey(); - - if (timestampColumnName == null) { - // load all the influencer positions - ctx.influencerPosChunk = new SizedLongChunk<>(ctx.influencerRows.intSize()); - ctx.influencerPositions.fillRowKeyChunk(ctx.influencerPosChunk.get()); - ctx.currentInfluencerPosOrTimestamp = ctx.influencerPositions.firstRowKey(); - } else { - // load all the influencer timestamp data - ctx.influencerTimestampChunk = new SizedLongChunk<>(ctx.influencerRows.intSize()); - try (final ChunkSource.FillContext fillContext = - timestampColumnSource.makeFillContext(ctx.influencerRows.intSize())) { - timestampColumnSource.fillChunk(fillContext, - (WritableChunk) ctx.influencerTimestampChunk.get(), ctx.influencerRows); - } - ctx.currentInfluencerPosOrTimestamp = ctx.influencerTimestampChunk.get().get(0); - } - ctx.currentInfluencerIndex = 0; + // // load all the influencer values this update will need + // ctx.loadInfluencerValueChunk(); + // + // // load all the influencer keys + // ctx.influencerKeyChunk = new SizedLongChunk<>(ctx.influencerRows.intSize()); + // ctx.influencerRows.fillRowKeyChunk(ctx.influencerKeyChunk.get()); + // ctx.currentInfluencerKey = ctx.influencerRows.firstRowKey(); + // + // if (timestampColumnName == null) { + // // load all the influencer positions + // ctx.influencerPosChunk = new SizedLongChunk<>(ctx.influencerRows.intSize()); + // ctx.influencerPositions.fillRowKeyChunk(ctx.influencerPosChunk.get()); + // ctx.currentInfluencerPosOrTimestamp = ctx.influencerPositions.firstRowKey(); + // } else { + // // load all the influencer timestamp data + // ctx.influencerTimestampChunk = new SizedLongChunk<>(ctx.influencerRows.intSize()); + // try (final ChunkSource.FillContext fillContext = + // timestampColumnSource.makeFillContext(ctx.influencerRows.intSize())) { + // timestampColumnSource.fillChunk(fillContext, + // (WritableChunk) ctx.influencerTimestampChunk.get(), ctx.influencerRows); + // } + // ctx.currentInfluencerPosOrTimestamp = ctx.influencerTimestampChunk.get().get(0); + // } + // ctx.currentInfluencerIndex = 0; } @Override - public void finishFor(@NotNull final UpdateContext context) { - UpdateWindowedContext ctx = (UpdateWindowedContext) context; - ctx.newModified = ctx.getModifiedBuilder().build(); - } - - @NotNull - final public RowSet getAdditionalModifications(@NotNull final UpdateContext context) { - UpdateWindowedContext ctx = (UpdateWindowedContext) context; - return ctx.newModified; - } - - @Override - final public boolean anyModified(@NotNull final UpdateContext context) { - UpdateWindowedContext ctx = (UpdateWindowedContext) context; - return ctx.newModified != null && ctx.newModified.isNonempty(); - } - - @NotNull - @Override - public String getInputColumnName() { - return pair.rightColumn; - } - - @NotNull - @Override - public String[] getAffectingColumnNames() { - return affectingColumns; - } - - @NotNull - @Override - public String[] getOutputColumnNames() { - return new String[] {pair.leftColumn}; - } + public void finishUpdate(@NotNull final UpdateContext context) {} @Override public String getTimestampColumnName() { return this.timestampColumnName; } - /** - * Get the value of the backward-looking window (might be nanos or ticks). - * - * @return the name of the input column - */ + /*** Get the value of the backward-looking window (might be nanos or ticks) */ @Override public long getPrevWindowUnits() { return reverseTimeScaleUnits; } - /** - * Get the value of the forward-looking window (might be nanos or ticks). - * - * @return the name of the input column - */ + /*** Get the value of the forward-looking window (might be nanos or ticks) */ @Override public long getFwdWindowUnits() { return forwardTimeScaleUnits; } + @NotNull @Override - public boolean requiresKeys() { - return false; + public String getInputColumnName() { + return pair.rightColumn; } - /*** windowed operators need position data when computing ticks */ + @NotNull @Override - public boolean requiresPositions() { - return this.timestampColumnName == null; + public String[] getAffectingColumnNames() { + return affectingColumns; } + @NotNull @Override - public boolean requiresValues(@NotNull final UpdateContext context) { - // windowed operators don't need current values supplied to them, they only care about windowed values which - // may or may not intersect with the column values - return false; + public String[] getOutputColumnNames() { + return new String[] {pair.leftColumn}; } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java index ee454fab794..e56c1eb52a6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java @@ -9,7 +9,6 @@ import io.deephaven.engine.table.impl.sources.ReinterpretUtils; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.updateby.UpdateByWindow; -import io.deephaven.engine.table.impl.util.SizedSafeCloseable; import io.deephaven.engine.table.impl.util.UpdateSizeCalculator; import io.deephaven.util.SafeCloseable; import org.apache.commons.lang3.mutable.MutableLong; @@ -39,7 +38,6 @@ class ZeroKeyUpdateBy extends UpdateBy { * * @param description the operation description * @param source the source table - * @param windows the unique windows for this updateBy call * @param ops the operations to perform * @param resultSources the result sources * @param redirContext the row redirection shared context @@ -48,14 +46,13 @@ class ZeroKeyUpdateBy extends UpdateBy { */ public static Table compute(@NotNull final String description, @NotNull final QueryTable source, - @NotNull final UpdateByWindow[] windows, @NotNull final UpdateByOperator[] ops, @NotNull final Map> resultSources, @NotNull final UpdateByRedirectionContext redirContext, @NotNull final UpdateByControl control, final boolean applyShifts) { final QueryTable result = new QueryTable(source.getRowSet(), resultSources); - final ZeroKeyUpdateBy updateBy = new ZeroKeyUpdateBy(windows, ops, source, redirContext, control, applyShifts); + final ZeroKeyUpdateBy updateBy = new ZeroKeyUpdateBy(ops, source, redirContext, control, applyShifts); updateBy.doInitialAdditions(); if (source.isRefreshing()) { @@ -67,13 +64,12 @@ public static Table compute(@NotNull final String description, return result; } - protected ZeroKeyUpdateBy(@NotNull final UpdateByWindow[] windows, - @NotNull final UpdateByOperator[] operators, + protected ZeroKeyUpdateBy(@NotNull final UpdateByOperator[] operators, @NotNull final QueryTable source, @NotNull final UpdateByRedirectionContext redirContext, @NotNull final UpdateByControl control, final boolean applyShifts) { - super(windows, operators, source, redirContext, control); + super(operators, source, redirContext, control); // do we need a timestamp SSA? this.timestampColumnName = Arrays.stream(operators) @@ -117,9 +113,9 @@ private void processUpdateForSsa(TableUpdate upstream) { if (restampRemovals.isNonempty()) { final int size = (int) Math.min(restampRemovals.size(), 4096); try (final RowSequence.Iterator it = restampRemovals.getRowSequenceIterator(); - final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(size); - final WritableLongChunk ssaValues = WritableLongChunk.makeWritableChunk(size); - final WritableLongChunk ssaKeys = WritableLongChunk.makeWritableChunk(size)) { + final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(size); + final WritableLongChunk ssaValues = WritableLongChunk.makeWritableChunk(size); + final WritableLongChunk ssaKeys = WritableLongChunk.makeWritableChunk(size)) { MutableLong lastTimestamp = new MutableLong(NULL_LONG); while (it.hasMore()) { @@ -168,16 +164,17 @@ private void processUpdateForSsa(TableUpdate upstream) { if (restampAdditions.isNonempty()) { final int size = (int) Math.min(restampAdditions.size(), 4096); try (final RowSequence.Iterator it = restampAdditions.getRowSequenceIterator(); - final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(size); - final WritableLongChunk ssaValues = WritableLongChunk.makeWritableChunk(size); - final WritableLongChunk ssaKeys = WritableLongChunk.makeWritableChunk(size)) { + final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(size); + final WritableLongChunk ssaValues = WritableLongChunk.makeWritableChunk(size); + final WritableLongChunk ssaKeys = WritableLongChunk.makeWritableChunk(size)) { MutableLong lastTimestamp = new MutableLong(NULL_LONG); while (it.hasMore()) { RowSequence chunkRs = it.getNextRowSequenceWithLength(4096); // get the chunks for values and keys - LongChunk valuesChunk = timestampColumnSource.getChunk(context, chunkRs).asLongChunk(); + LongChunk valuesChunk = + timestampColumnSource.getChunk(context, chunkRs).asLongChunk(); LongChunk keysChunk = chunkRs.asRowKeyChunk(); // push only non-null values/keys into the Ssa @@ -188,9 +185,10 @@ private void processUpdateForSsa(TableUpdate upstream) { } } - /** helper function to fill a LongChunk while skipping values that are NULL_LONG. Used to populate an SSA from - * a source containing null values - * */ + /** + * helper function to fill a LongChunk while skipping values that are NULL_LONG. Used to populate an SSA from a + * source containing null values + */ private void fillChunkWithNonNull(LongChunk keysChunk, LongChunk valuesChunk, WritableLongChunk ssaKeys, WritableLongChunk ssaValues, MutableLong lastTimestamp) { @@ -242,9 +240,6 @@ private class UpdateContext implements SafeCloseable { /** The expected size of chunks to the various update stages */ int chunkSize; - /** An indicator of if each slot has been populated with data or not for this phase. */ - boolean[] inputChunkPopulated; - /** A {@link SharedContext} to be used while creating other contexts */ SharedContext sharedContext = SharedContext.makeSharedContext(); @@ -272,15 +267,16 @@ private class UpdateContext implements SafeCloseable { for (int winIdx = 0; winIdx < windows.length; winIdx++) { // create a context for each window windowContexts[winIdx] = windows[winIdx].makeWindowContext( - timestampColumnSource, - timestampSsa); - - // compute the affected/influenced operators and rowset within this window - windowAffected[winIdx] = windowContexts[winIdx].computeAffectedAndMakeContexts(upstream, source.getRowSet(), - inputModifiedColumnSets, + inputSources, + timestampColumnSource, + timestampSsa, chunkSize, isInitializeStep); + + // compute the affected/influenced operators and rowset within this window + windowAffected[winIdx] = windowContexts[winIdx].computeAffectedRowsAndOperators(upstream, + inputModifiedColumnSets[winIdx]); } } @@ -288,6 +284,15 @@ public SharedContext getSharedContext() { return sharedContext; } + public boolean anyModified() { + for (int winIdx = 0; winIdx < windows.length; winIdx++) { + if (windowContexts[winIdx].anyModified()) { + return true; + } + } + return false; + } + @Override public void close() { sharedContext.close(); @@ -303,11 +308,8 @@ public void close() { } } - /** - * Locate the smallest key that requires reprocessing and then replay the table from that point - */ private void processRows() { - // this could be parallelized since the windows probably won't overlap + // this might be parallelized if there are multiple windows for (int winIdx = 0; winIdx < windows.length; winIdx++) { if (windowAffected[winIdx]) { // this will internally call initialize() and finish() for each operator @@ -331,13 +333,12 @@ public ZeroKeyUpdateByListener(@Nullable String description, @NotNull final QueryTable result) { super(description, source, false); this.result = result; - this.inputModifiedColumnSets = new ModifiedColumnSet[operators.length]; - this.outputModifiedColumnSets = new ModifiedColumnSet[operators.length]; + this.inputModifiedColumnSets = new ModifiedColumnSet[windows.length]; + this.outputModifiedColumnSets = new ModifiedColumnSet[windows.length]; - for (int ii = 0; ii < operators.length; ii++) { - final String[] outputColumnNames = operators[ii].getOutputColumnNames(); - inputModifiedColumnSets[ii] = source.newModifiedColumnSet(operators[ii].getAffectingColumnNames()); - outputModifiedColumnSets[ii] = result.newModifiedColumnSet(outputColumnNames); + for (int ii = 0; ii < windows.length; ii++) { + inputModifiedColumnSets[ii] = source.newModifiedColumnSet(windows[ii].getAffectingColumnNames()); + outputModifiedColumnSets[ii] = result.newModifiedColumnSet(windows[ii].getOutputColumnNames()); } this.transformer = @@ -352,8 +353,6 @@ public void onUpdate(TableUpdate upstream) { } try (final UpdateContext ctx = new UpdateContext(upstream, inputModifiedColumnSets, false)) { - - if (applyShifts) { if (redirContext.isRedirected()) { redirContext.processUpdateForRedirection(upstream, source.getRowSet()); @@ -366,7 +365,7 @@ public void onUpdate(TableUpdate upstream) { upstream.shifted().apply((begin, end, delta) -> { try (final RowSet subRowSet = prevIdx.subSetByKeyRange(begin, end)) { for (int opIdx = 0; opIdx < operators.length; opIdx++) { - operators[opIdx].applyOutputShift(ctx.opContext[opIdx], subRowSet, delta); + operators[opIdx].applyOutputShift(subRowSet, delta); } } }); @@ -387,7 +386,9 @@ public void onUpdate(TableUpdate upstream) { downstream.modifiedColumnSet = result.getModifiedColumnSetForUpdates(); downstream.modifiedColumnSet.clear(); - if (upstream.modified().isNonempty() || ctx.anyModified()) { + final boolean windowsModified = ctx.anyModified(); + + if (upstream.modified().isNonempty() || windowsModified) { WritableRowSet modifiedRowSet = RowSetFactory.empty(); downstream.modified = modifiedRowSet; if (upstream.modified().isNonempty()) { @@ -396,16 +397,16 @@ public void onUpdate(TableUpdate upstream) { modifiedRowSet.insert(upstream.modified()); } - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (ctx.opAffected[opIdx]) { - if (operators[opIdx].anyModified(ctx.opContext[opIdx])) { - modifiedRowSet - .insert(operators[opIdx].getAdditionalModifications(ctx.opContext[opIdx])); + // retrieve the modified rowsets from the windows + for (int winIdx = 0; winIdx < windows.length; winIdx++) { + if (ctx.windowAffected[winIdx]) { + if (ctx.windowContexts[winIdx].anyModified()) { + modifiedRowSet.insert(ctx.windowContexts[winIdx].getAdditionalModifications()); } } } - if (ctx.anyModified()) { + if (windowsModified) { modifiedRowSet.remove(upstream.added()); } } else { @@ -413,9 +414,10 @@ public void onUpdate(TableUpdate upstream) { } // set the modified columns if any operators made changes (add/rem/modify) - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (ctx.opAffected[opIdx]) { - downstream.modifiedColumnSet.setAll(outputModifiedColumnSets[opIdx]); + for (int winIdx = 0; winIdx < windows.length; winIdx++) { + if (ctx.windowAffected[winIdx]) { + // TODO: need to add only the affected column sets from the window, not all + downstream.modifiedColumnSet.setAll(outputModifiedColumnSets[winIdx]); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java index 73b1bc817b6..2e9399c9c28 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java @@ -2,12 +2,14 @@ import io.deephaven.base.verify.Assert; import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.*; import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.ModifiedColumnSet; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; @@ -16,7 +18,7 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.util.ArrayList; +import java.util.*; import static io.deephaven.util.QueryConstants.NULL_LONG; @@ -27,339 +29,569 @@ public class UpdateByWindow { protected final long prevUnits; protected final long fwdUnits; - protected ArrayList operators; + // store the operators for this window + protected UpdateByOperator[] operators; + // store the index in the {@link UpdateBy.inputSources} + protected int[] operatorSourceSlots; public class UpdateByWindowContext implements SafeCloseable { - /** the rows affected by this update */ - protected RowSet affectedRows; - protected RowSet influencerRows; - - // for use with a ticking window - protected RowSet affectedRowPositions; - protected RowSet influencerPositions; - - // keep track of what rows were modified (we'll use a single set for all operators sharing a window) - protected RowSetBuilderSequential modifiedBuilder; - protected RowSet newModified; + /** store a reference to the source rowset */ + final TrackingRowSet sourceRowSet; - /** the column source providing the timestamp data for this window */ + /** the column source providing the timestamp data for this window */ @Nullable - protected ColumnSource timestampColumnSource; + final ColumnSource timestampColumnSource; /** the timestamp SSA providing fast lookup for time windows */ @Nullable - protected LongSegmentedSortedArray timestampSsa; + final LongSegmentedSortedArray timestampSsa; /** An array of boolean denoting which operators are affected by the current update. */ final boolean[] opAffected; /** An array of context objects for each underlying operator */ final UpdateByOperator.UpdateContext[] opContext; - - public UpdateByWindowContext(final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa) { + + /** An array of ColumnSources for each underlying operator */ + final ChunkSource[] inputSource; + + /** An array of {@link ChunkSource.FillContext}s for each input column */ + final ChunkSource.FillContext[] inputSourceFillContexts; + + /** A set of chunks used to store working values */ + final WritableChunk[] inputSourceChunks; + + /** An indicator of if each slot has been populated with data or not for this phase. */ + final boolean[] inputSourceChunkPopulated; + + /** the rows affected by this update */ + RowSet affectedRows; + /** the rows that contain values used to compute affected row values */ + RowSet influencerRows; + + /** for use with a ticking window */ + RowSet affectedRowPositions; + RowSet influencerPositions; + + /** keep track of what rows were modified (we'll use a single set for all operators sharing a window) */ + RowSetBuilderSequential modifiedBuilder; + RowSet newModified; + + final int chunkSize; + final boolean initialStep; + + public UpdateByWindowContext(final TrackingRowSet sourceRowSet, final ChunkSource[] opInputSource, + @Nullable final ColumnSource timestampColumnSource, + @Nullable final LongSegmentedSortedArray timestampSsa, final int chunkSize, final boolean initialStep) { + this.sourceRowSet = sourceRowSet; + this.inputSource = opInputSource; this.timestampColumnSource = timestampColumnSource; this.timestampSsa = timestampSsa; - this.opAffected = new boolean[operators.size()]; + this.opAffected = new boolean[operators.length]; + this.opContext = new UpdateByOperator.UpdateContext[operators.length]; + this.inputSourceFillContexts = new ChunkSource.FillContext[operators.length]; + this.inputSourceChunkPopulated = new boolean[operators.length]; // noinspection unchecked -// this.fillContexts = new SizedSafeCloseable[operators.size()]; - this.opContext = new UpdateByOperator.UpdateContext[operators.size()]; + this.inputSourceChunks = new WritableChunk[operators.length]; + + this.chunkSize = chunkSize; + this.initialStep = initialStep; } - public boolean computeAffectedAndMakeContexts(@NotNull final TableUpdate upstream, @NotNull final TrackingRowSet source, @Nullable final ModifiedColumnSet[] inputModifiedColumnSets, final int chunkSize, final boolean initialStep) { - // determine the affected rows for this window context + public boolean computeAffectedRowsAndOperators(@NotNull final TableUpdate upstream, + @Nullable final ModifiedColumnSet inputModifiedColumnSets) { + // all rows are affected on the initial step if (initialStep) { - // all rows are affected initially - affectedRows = source.copy(); + affectedRows = sourceRowSet.copy(); influencerRows = affectedRows; // no need to invert, just create a flat rowset if (windowed && timestampColumnName == null) { - affectedRowPositions = RowSetFactory.flat(source.size()); - influencerPositions = RowSetFactory.flat(source.size()); - } - // mark all operators as affected by this update and create contexts - for (int opIdx = 0; opIdx < operators.size(); opIdx++) { - opAffected[opIdx] = true; - opContext[opIdx] = operators.get(opIdx).makeUpdateContext(chunkSize, timestampSsa); + affectedRowPositions = RowSetFactory.flat(sourceRowSet.size()); + influencerPositions = RowSetFactory.flat(sourceRowSet.size()); } + // mark all operators as affected by this update + Arrays.fill(opAffected, true); + makeOperatorContexts(); return true; - } else { - // determine which operators are affected by this change - boolean anyAffected = false; - boolean allAffected = upstream.added().isNonempty() || - upstream.removed().isNonempty() || - upstream.shifted().nonempty(); - - for (int opIdx = 0; opIdx < operators.size(); opIdx++) { - opAffected[opIdx] = allAffected - || (upstream.modifiedColumnSet().nonempty() && (inputModifiedColumnSets == null - || upstream.modifiedColumnSet().containsAny(inputModifiedColumnSets[opIdx]))); + } - // mark the operators affected by this update and create contexts - if (opAffected[opIdx]) { - anyAffected = true; - opContext[opIdx] = operators.get(opIdx).makeUpdateContext(chunkSize, timestampSsa); - } - } + // determine which operators are affected by this update + boolean anyAffected = false; + boolean allAffected = upstream.added().isNonempty() || + upstream.removed().isNonempty() || + upstream.shifted().nonempty(); - if (source.isEmpty() || !anyAffected) { - // no work to do for this window this cycle - return false; + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + opAffected[opIdx] = allAffected + || (upstream.modifiedColumnSet().nonempty() && (inputModifiedColumnSets == null + || upstream.modifiedColumnSet().containsAny(inputModifiedColumnSets))); + if (opAffected[opIdx]) { + anyAffected = true; } + } - // handle the three major types of windows: cumulative, windowed by ticks, windowed by time + if (sourceRowSet.isEmpty() || !anyAffected) { + // no work to do for this window this cycle + return false; + } - // cumulative is simple, just find the smallest key and return the range from smallest to end - if (!windowed) { - long smallestModifiedKey = smallestAffectedKey(upstream.added(), upstream.modified(), upstream.removed(), - upstream.shifted(), source); + if (!windowed) { + computeCumulativeRowsAffected(upstream); + } else { + computeWindowedRowsAffected(upstream); + } - affectedRows = smallestModifiedKey == Long.MAX_VALUE - ? RowSetFactory.empty() - : source.subSetByKeyRange(smallestModifiedKey, source.lastRowKey()); - influencerRows = affectedRows; - return true; - } + makeOperatorContexts(); + return true; + } - // changed rows are all mods+adds - WritableRowSet changed = upstream.added().copy(); - changed.insert(upstream.modified()); + // cumulative windows is simple, just find the smallest key and return the range from smallest to end + private void computeCumulativeRowsAffected(@NotNull TableUpdate upstream) { + long smallestModifiedKey = smallestAffectedKey(upstream.added(), upstream.modified(), upstream.removed(), + upstream.shifted(), sourceRowSet); - WritableRowSet tmpAffected; + affectedRows = smallestModifiedKey == Long.MAX_VALUE + ? RowSetFactory.empty() + : sourceRowSet.subSetByKeyRange(smallestModifiedKey, sourceRowSet.lastRowKey()); + influencerRows = affectedRows; + } - // compute the rows affected from these changes - if (timestampColumnName == null) { - try (final WritableRowSet changedInverted = source.invert(changed)) { - tmpAffected = computeAffectedRowsTicks(source, changed, changedInverted, prevUnits, fwdUnits); - } - } else { - tmpAffected = computeAffectedRowsTime(source, changed, prevUnits, fwdUnits); + // windowed by time/ticks is more complex to compute: find all the changed rows and the rows that would + // be affected by the changes (includes newly added rows) and need to be recomputed. Then include all + // the rows that are affected by deletions (if any). After the affected rows have been identified, + // determine which rows will be needed to compute new values for the affected rows (influencer rows) + private void computeWindowedRowsAffected(@NotNull TableUpdate upstream) { + // changed rows are all mods+adds + WritableRowSet changed = upstream.added().copy(); + changed.insert(upstream.modified()); + + // need a writable rowset + WritableRowSet tmpAffected; + + // compute the rows affected from these changes + if (timestampColumnName == null) { + try (final WritableRowSet changedInverted = sourceRowSet.invert(changed)) { + tmpAffected = computeAffectedRowsTicks(sourceRowSet, changed, changedInverted, prevUnits, fwdUnits); } + } else { + tmpAffected = computeAffectedRowsTime(sourceRowSet, changed, prevUnits, fwdUnits, timestampColumnSource, + timestampSsa); + } - // other rows can be affected by removes - if (upstream.removed().isNonempty()) { - try (final RowSet prev = source.copyPrev(); - final RowSet removedPositions = timestampColumnName == null - ? null : prev.invert(upstream.removed()); - final WritableRowSet affectedByRemoves = timestampColumnName == null - ? computeAffectedRowsTicks(prev, upstream.removed(), removedPositions, prevUnits, fwdUnits) - : computeAffectedRowsTime(prev, upstream.removed(), prevUnits, fwdUnits)) { - // apply shifts to get back to pos-shift space - upstream.shifted().apply(affectedByRemoves); - // retain only the rows that still exist in the source - affectedByRemoves.retain(source); - tmpAffected.insert(affectedByRemoves); - } + // other rows can be affected by removes + if (upstream.removed().isNonempty()) { + try (final RowSet prev = sourceRowSet.copyPrev(); + final RowSet removedPositions = timestampColumnName == null + ? null + : prev.invert(upstream.removed()); + final WritableRowSet affectedByRemoves = timestampColumnName == null + ? computeAffectedRowsTicks(prev, upstream.removed(), removedPositions, prevUnits, + fwdUnits) + : computeAffectedRowsTime(prev, upstream.removed(), prevUnits, fwdUnits, + timestampColumnSource, timestampSsa)) { + // apply shifts to get back to pos-shift space + upstream.shifted().apply(affectedByRemoves); + // retain only the rows that still exist in the sourceRowSet + affectedByRemoves.retain(sourceRowSet); + tmpAffected.insert(affectedByRemoves); } + } + + affectedRows = tmpAffected; - affectedRows = tmpAffected; + // now get influencer rows for the affected rows + if (timestampColumnName == null) { + // generate position data rowsets for efficiently computed position offsets + affectedRowPositions = sourceRowSet.invert(affectedRows); - // now get influencer rows for the affected rows - if (timestampColumnName == null) { - // generate position data rowsets for efficiently computed position offsets - affectedRowPositions = source.invert(affectedRows); + influencerRows = computeInfluencerRowsTicks(sourceRowSet, affectedRows, affectedRowPositions, prevUnits, + fwdUnits); + influencerPositions = sourceRowSet.invert(influencerRows); + } else { + influencerRows = computeInfluencerRowsTime(sourceRowSet, affectedRows, prevUnits, fwdUnits, + timestampColumnSource, timestampSsa); + } + } - influencerRows = computeInfluencerRowsTicks(source, affectedRows, affectedRowPositions, prevUnits, fwdUnits); - influencerPositions = source.invert(influencerRows); - } else { - influencerRows = computeInfluencerRowsTime(source, affectedRows, prevUnits, fwdUnits); - affectedRowPositions = null; - influencerPositions = null; + private void makeOperatorContexts() { + // use this to make which input sources are initialized + Arrays.fill(inputSourceChunkPopulated, false); + + // create contexts for the affected operators + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (opAffected[opIdx]) { + opContext[opIdx] = operators[opIdx].makeUpdateContext(chunkSize); + + // create the fill contexts and + int sourceSlot = operatorSourceSlots[opIdx]; + if (!inputSourceChunkPopulated[sourceSlot]) { + inputSourceChunks[sourceSlot] = + inputSource[sourceSlot].getChunkType().makeWritableChunk(chunkSize); + inputSourceFillContexts[sourceSlot] = inputSource[sourceSlot].makeFillContext(chunkSize); + inputSourceChunkPopulated[sourceSlot] = true; + } } } - return true; + + } + + public boolean anyModified() { + return newModified != null && newModified.isNonempty(); } - /** - * Find the smallest valued key that participated in the upstream {@link TableUpdate}. - * - * @param added the added rows - * @param modified the modified rows - * @param removed the removed rows - * @param shifted the shifted rows - * - * @return the smallest key that participated in any part of the update. - */ - private long smallestAffectedKey(@NotNull final RowSet added, - @NotNull final RowSet modified, - @NotNull final RowSet removed, - @NotNull final RowSetShiftData shifted, - @NotNull final RowSet affectedIndex) { - - long smallestModifiedKey = Long.MAX_VALUE; - if (removed.isNonempty()) { - smallestModifiedKey = removed.firstRowKey(); + public RowSet getAdditionalModifications() { + return newModified; + } + + public RowSet getAffectedRows() { + return affectedRows; + } + + public RowSet getInfluencerRows() { + return influencerRows; + } + + public void processRows() { + modifiedBuilder = RowSetFactory.builderSequential(); + + // these could be nested and/or simplified but this is most readable + if (!windowed && timestampColumnName == null) { + processRowsCumulative(); + } else if (!windowed && timestampColumnName != null) { + processRowsCumulativeTimestamp(); + } else if (windowed && timestampColumnName == null) { + processRowsWindowedTicks(); + } else { + processRowsWindowedTime(); } - if (added.isNonempty()) { - smallestModifiedKey = Math.min(smallestModifiedKey, added.firstRowKey()); + newModified = modifiedBuilder.build(); + } + + private void prepareValuesChunkForSource(final int srcIdx, final RowSequence rs) { + if (!inputSourceChunkPopulated[srcIdx]) { + inputSource[srcIdx].fillChunk( + inputSourceFillContexts[srcIdx], + inputSourceChunks[srcIdx], + rs); + inputSourceChunkPopulated[srcIdx] = true; } + } - if (modified.isNonempty()) { - smallestModifiedKey = Math.min(smallestModifiedKey, modified.firstRowKey()); + private void processRowsCumulative() { + // find the key before the first affected row and preload that data for these operators + final long keyBefore; + try (final RowSet.SearchIterator sit = sourceRowSet.searchIterator()) { + keyBefore = sit.binarySearchValue( + (compareTo, ignored) -> Long.compare(affectedRows.firstRowKey() - 1, compareTo), 1); } - if (shifted.nonempty()) { - final long firstModKey = modified.isEmpty() ? Long.MAX_VALUE : modified.firstRowKey(); - boolean modShiftFound = !modified.isEmpty(); - boolean affectedFound = false; - try (final RowSequence.Iterator it = affectedIndex.getRowSequenceIterator()) { - for (int shiftIdx = 0; shiftIdx < shifted.size() - && (!modShiftFound || !affectedFound); shiftIdx++) { - final long shiftStart = shifted.getBeginRange(shiftIdx); - final long shiftEnd = shifted.getEndRange(shiftIdx); - final long shiftDelta = shifted.getShiftDelta(shiftIdx); - - if (!affectedFound) { - if (it.advance(shiftStart + shiftDelta)) { - final long maybeAffectedKey = it.peekNextKey(); - if (maybeAffectedKey <= shiftEnd + shiftDelta) { - affectedFound = true; - final long keyToCompare = - shiftDelta > 0 ? maybeAffectedKey - shiftDelta : maybeAffectedKey; - smallestModifiedKey = Math.min(smallestModifiedKey, keyToCompare); - } - } else { - affectedFound = true; - } - } + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (opAffected[opIdx]) { + // call the specialized version of `intializeUpdate()` for these operators + // TODO: make sure the time-based cumulative oerators are starting from a valid value and timestamp + ((UpdateByCumulativeOperator) operators[opIdx]).initializeUpdate(opContext[opIdx], keyBefore, + NULL_LONG); + } + } - if (!modShiftFound) { - if (firstModKey <= (shiftEnd + shiftDelta)) { - modShiftFound = true; - // If the first modified key is in the range we should include it - if (firstModKey >= (shiftStart + shiftDelta)) { - smallestModifiedKey = Math.min(smallestModifiedKey, firstModKey - shiftDelta); - } else { - // Otherwise it's not included in any shifts, and since shifts can't reorder rows - // it is the smallest possible value and we've already accounted for it above. - break; - } - } + try (final RowSequence.Iterator it = affectedRows.getRowSequenceIterator()) { + while (it.hasMore()) { + final RowSequence rs = it.getNextRowSequenceWithLength(chunkSize); + Arrays.fill(inputSourceChunkPopulated, false); + + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (opAffected[opIdx]) { + final int srcIdx = operatorSourceSlots[opIdx]; + + // get the values this operator needs + prepareValuesChunkForSource(srcIdx, rs); + + // process the chunk + operators[opIdx].processChunk( + opContext[opIdx], + rs, + null, + null, + inputSourceChunks[srcIdx], + null); } } + // all these rows were modified + modifiedBuilder.appendRowSequence(rs); + } + } + + // call the generic `finishUpdate()` function for each operator + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (opAffected[opIdx]) { + operators[opIdx].finishUpdate(opContext[opIdx]); } } - return smallestModifiedKey; } - private WritableRowSet computeAffectedRowsTime(final RowSet sourceSet, final RowSet subset, long revNanos, - long fwdNanos) { - // swap fwd/rev to get the affected windows - return computeInfluencerRowsTime(sourceSet, subset, fwdNanos, revNanos); + private void processRowsCumulativeTimestamp() { + // find the key before the first affected row (that has a valid timestamp) and preload + // that data for these operators + } + + private void processRowsWindowedTicks() { + // start loading the window for these operators using position data + } + + private void processRowsWindowedTime() { + // start loading the window for these operators using timestamp data } - private WritableRowSet computeInfluencerRowsTime(final RowSet sourceSet, final RowSet subset, long revNanos, - long fwdNanos) { - if (sourceSet.size() == subset.size()) { - return sourceSet.copy(); + @Override + public void close() { + if (influencerRows != null && influencerRows != affectedRows) { + influencerRows.close(); + } + if (influencerPositions != null && influencerPositions != affectedRowPositions) { + influencerPositions.close(); + } + try (final RowSet ignoredRs1 = affectedRows; + final RowSet ignoredRs2 = affectedRowPositions; + final RowSet ignoredRs3 = newModified) { + } + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (opAffected[opIdx]) { + final int srcIdx = operatorSourceSlots[opIdx]; + if (inputSourceChunks[srcIdx] != null) { + + inputSourceChunks[srcIdx].close(); + inputSourceChunks[srcIdx] = null; + + inputSourceFillContexts[srcIdx].close(); + inputSourceFillContexts[srcIdx] = null; + } + opContext[opIdx].close(); + } } + } + } - int chunkSize = (int) Math.min(subset.size(), 4096); - try (final RowSequence.Iterator it = subset.getRowSequenceIterator(); - final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(chunkSize)) { - final RowSetBuilderSequential builder = RowSetFactory.builderSequential(); - LongSegmentedSortedArray.Iterator ssaIt = timestampSsa.iterator(false, false); - while (it.hasMore() && ssaIt.hasNext()) { - final RowSequence rs = it.getNextRowSequenceWithLength(chunkSize); - LongChunk timestamps = timestampColumnSource.getChunk(context, rs).asLongChunk(); - - for (int ii = 0; ii < rs.intSize(); ii++) { - // if the timestamp of the row is null, it won't belong to any set and we can ignore it - // completely - final long ts = timestamps.get(ii); - if (ts != NULL_LONG) { - // look at every row timestamp, compute the head and tail in nanos - final long head = ts - revNanos; - final long tail = ts + fwdNanos; - - // advance the iterator to the beginning of the window - if (ssaIt.nextValue() < head) { - ssaIt.advanceToBeforeFirst(head); - if (!ssaIt.hasNext()) { - // SSA is exhausted - break; - } - } + public UpdateByWindowContext makeWindowContext(final TrackingRowSet sourceRowSet, + final ChunkSource[] inputSources, + final ColumnSource timestampColumnSource, + final LongSegmentedSortedArray timestampSsa, + final int chunkSize, + final boolean isInitializeStep) { + return new UpdateByWindowContext(sourceRowSet, inputSources, timestampColumnSource, timestampSsa, chunkSize, + isInitializeStep); + } + + public void setOperators(final UpdateByOperator[] operators, final int[] operatorSourceSlots) { + this.operators = operators; + this.operatorSourceSlots = operatorSourceSlots; + } + + + @NotNull + public String[] getAffectingColumnNames() { + Set columns = new TreeSet<>(); + for (UpdateByOperator operator : operators) { + columns.addAll(Arrays.asList(operator.getAffectingColumnNames())); + } + return columns.toArray(new String[0]); + } + + @NotNull + public String[] getOutputColumnNames() { + // we can use a list since we have previously checked for duplicates + List columns = new ArrayList<>(); + for (UpdateByOperator operator : operators) { + columns.addAll(Arrays.asList(operator.getOutputColumnNames())); + } + return columns.toArray(new String[0]); + } + + /** + * Find the smallest valued key that participated in the upstream {@link TableUpdate}. + * + * @param added the added rows + * @param modified the modified rows + * @param removed the removed rows + * @param shifted the shifted rows + * + * @return the smallest key that participated in any part of the update. + */ + private static long smallestAffectedKey(@NotNull final RowSet added, + @NotNull final RowSet modified, + @NotNull final RowSet removed, + @NotNull final RowSetShiftData shifted, + @NotNull final RowSet affectedIndex) { + + long smallestModifiedKey = Long.MAX_VALUE; + if (removed.isNonempty()) { + smallestModifiedKey = removed.firstRowKey(); + } - Assert.assertion(ssaIt.hasNext() && ssaIt.nextValue() >= head, - "SSA Iterator outside of window"); + if (added.isNonempty()) { + smallestModifiedKey = Math.min(smallestModifiedKey, added.firstRowKey()); + } - // step through the SSA and collect keys until outside of the window - while (ssaIt.hasNext() && ssaIt.nextValue() <= tail) { - builder.appendKey(ssaIt.nextKey()); - ssaIt.next(); + if (modified.isNonempty()) { + smallestModifiedKey = Math.min(smallestModifiedKey, modified.firstRowKey()); + } + + if (shifted.nonempty()) { + final long firstModKey = modified.isEmpty() ? Long.MAX_VALUE : modified.firstRowKey(); + boolean modShiftFound = !modified.isEmpty(); + boolean affectedFound = false; + try (final RowSequence.Iterator it = affectedIndex.getRowSequenceIterator()) { + for (int shiftIdx = 0; shiftIdx < shifted.size() + && (!modShiftFound || !affectedFound); shiftIdx++) { + final long shiftStart = shifted.getBeginRange(shiftIdx); + final long shiftEnd = shifted.getEndRange(shiftIdx); + final long shiftDelta = shifted.getShiftDelta(shiftIdx); + + if (!affectedFound) { + if (it.advance(shiftStart + shiftDelta)) { + final long maybeAffectedKey = it.peekNextKey(); + if (maybeAffectedKey <= shiftEnd + shiftDelta) { + affectedFound = true; + final long keyToCompare = + shiftDelta > 0 ? maybeAffectedKey - shiftDelta : maybeAffectedKey; + smallestModifiedKey = Math.min(smallestModifiedKey, keyToCompare); } + } else { + affectedFound = true; + } + } - if (!ssaIt.hasNext()) { - // SSA is exhausted + if (!modShiftFound) { + if (firstModKey <= (shiftEnd + shiftDelta)) { + modShiftFound = true; + // If the first modified key is in the range we should include it + if (firstModKey >= (shiftStart + shiftDelta)) { + smallestModifiedKey = Math.min(smallestModifiedKey, firstModKey - shiftDelta); + } else { + // Otherwise it's not included in any shifts, and since shifts can't reorder rows + // it is the smallest possible value and we've already accounted for it above. break; } } } } - return builder.build(); } } - private WritableRowSet computeAffectedRowsTicks(final RowSet sourceSet, final RowSet subset, final RowSet invertedSubSet, long revTicks, long fwdTicks, @Nullable final WritableRowSet ) { - // swap fwd/rev to get the influencer windows - return computeInfluencerRowsTicks(sourceSet, subset, invertedSubSet, fwdTicks, revTicks); - } + return smallestModifiedKey; + } - private WritableRowSet computeInfluencerRowsTicks(final RowSet sourceSet, final RowSet subset, final RowSet invertedSubSet, long revTicks, - long fwdTicks) { - if (sourceSet.size() == subset.size()) { - return sourceSet.copy(); - } + private static WritableRowSet computeAffectedRowsTime(final RowSet sourceSet, final RowSet subset, long revNanos, + long fwdNanos, final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa) { + // swap fwd/rev to get the affected windows + return computeInfluencerRowsTime(sourceSet, subset, fwdNanos, revNanos, timestampColumnSource, timestampSsa); + } - long maxPos = sourceSet.size() - 1; + private static WritableRowSet computeInfluencerRowsTime(final RowSet sourceSet, final RowSet subset, long revNanos, + long fwdNanos, final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa) { + if (sourceSet.size() == subset.size()) { + return sourceSet.copy(); + } + int chunkSize = (int) Math.min(subset.size(), 4096); + try (final RowSequence.Iterator it = subset.getRowSequenceIterator(); + final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(chunkSize)) { final RowSetBuilderSequential builder = RowSetFactory.builderSequential(); - final MutableLong minPos = new MutableLong(0L); + LongSegmentedSortedArray.Iterator ssaIt = timestampSsa.iterator(false, false); + while (it.hasMore() && ssaIt.hasNext()) { + final RowSequence rs = it.getNextRowSequenceWithLength(chunkSize); + LongChunk timestamps = timestampColumnSource.getChunk(context, rs).asLongChunk(); + + for (int ii = 0; ii < rs.intSize(); ii++) { + // if the timestamp of the row is null, it won't belong to any set and we can ignore it + // completely + final long ts = timestamps.get(ii); + if (ts != NULL_LONG) { + // look at every row timestamp, compute the head and tail in nanos + final long head = ts - revNanos; + final long tail = ts + fwdNanos; + + // advance the iterator to the beginning of the window + if (ssaIt.nextValue() < head) { + ssaIt.advanceToBeforeFirst(head); + if (!ssaIt.hasNext()) { + // SSA is exhausted + break; + } + } - invertedSubSet.forAllRowKeyRanges((s, e) -> { - long sPos = Math.max(s - revTicks, minPos.longValue()); - long ePos = Math.min(e + fwdTicks, maxPos); - builder.appendRange(sPos, ePos); - minPos.setValue(ePos + 1); - }); + Assert.assertion(ssaIt.hasNext() && ssaIt.nextValue() >= head, + "SSA Iterator outside of window"); - try (final RowSet positions = builder.build()) { - return sourceSet.subSetForPositions(positions); + // step through the SSA and collect keys until outside of the window + while (ssaIt.hasNext() && ssaIt.nextValue() <= tail) { + builder.appendKey(ssaIt.nextKey()); + ssaIt.next(); + } + + if (!ssaIt.hasNext()) { + // SSA is exhausted + break; + } + } + } } + return builder.build(); } + } - public RowSet getAffectedRows() { - return affectedRows; + private static WritableRowSet computeAffectedRowsTicks(final RowSet sourceSet, final RowSet subset, + final RowSet invertedSubSet, long revTicks, long fwdTicks) { + // swap fwd/rev to get the influencer windows + return computeInfluencerRowsTicks(sourceSet, subset, invertedSubSet, fwdTicks, revTicks); + } + + private static WritableRowSet computeInfluencerRowsTicks(final RowSet sourceSet, final RowSet subset, + final RowSet invertedSubSet, long revTicks, long fwdTicks) { + if (sourceSet.size() == subset.size()) { + return sourceSet.copy(); } - public RowSet getInfluencerRows() { - if (!windowed) { - return affectedRows; - } - return influencerRows; + long maxPos = sourceSet.size() - 1; + + final RowSetBuilderSequential builder = RowSetFactory.builderSequential(); + final MutableLong minPos = new MutableLong(0L); + + invertedSubSet.forAllRowKeyRanges((s, e) -> { + long sPos = Math.max(s - revTicks, minPos.longValue()); + long ePos = Math.min(e + fwdTicks, maxPos); + builder.appendRange(sPos, ePos); + minPos.setValue(ePos + 1); + }); + + try (final RowSet positions = builder.build()) { + return sourceSet.subSetForPositions(positions); } + } - @Override - public void close() { - if (influencerRows != null && influencerRows != affectedRows) { - influencerRows.close(); - } - if (influencerPositions != null && influencerPositions != affectedRowPositions) { - influencerPositions.close(); - } - try (final RowSet ignoredRs1 = affectedRows; - final RowSet ignoredRs2 = affectedRowPositions; - final RowSet ignoredRs3 = newModified) { + private static long locatePreviousTimestamp(final RowSet sourceSet, final ColumnSource timestampColumnSource, + final long firstUnmodifiedKey) { + long potentialResetTimestamp = timestampColumnSource.getLong(firstUnmodifiedKey); + if (potentialResetTimestamp != NULL_LONG) { + return potentialResetTimestamp; + } + + try (final RowSet.SearchIterator rIt = sourceSet.reverseIterator()) { + if (rIt.advance(firstUnmodifiedKey)) { + while (rIt.hasNext()) { + final long nextKey = rIt.nextLong(); + potentialResetTimestamp = timestampColumnSource.getLong(nextKey); + if (potentialResetTimestamp != NULL_LONG) { + return potentialResetTimestamp; + } + } } } - } - public UpdateByWindowContext makeWindowContext(final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa) { - return new UpdateByWindowContext(timestampColumnSource, timestampSsa); + return NULL_LONG; } private UpdateByWindow(boolean windowed, @Nullable String timestampColumnName, long prevUnits, long fwdUnits) { @@ -367,8 +599,6 @@ private UpdateByWindow(boolean windowed, @Nullable String timestampColumnName, l this.timestampColumnName = timestampColumnName; this.prevUnits = prevUnits; this.fwdUnits = fwdUnits; - - this.operators = new ArrayList<>(); } public static UpdateByWindow createFromOperator(final UpdateByOperator op) { @@ -378,15 +608,20 @@ public static UpdateByWindow createFromOperator(final UpdateByOperator op) { op.getPrevWindowUnits()); } - public void addOperator(UpdateByOperator op) { - operators.add(op); - } - @Nullable public String getTimestampColumnName() { return timestampColumnName; } + @NotNull + final public RowSet getAdditionalModifications(@NotNull final UpdateByWindowContext context) { + return context.newModified; + } + + final public boolean anyModified(@NotNull final UpdateByWindowContext context) { + return context.newModified != null && context.newModified.isNonempty(); + } + @Override public int hashCode() { int hash = Boolean.hashCode(windowed); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java index 3d034460249..fb0994b3540 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java @@ -1,18 +1,17 @@ package io.deephaven.engine.table.impl.updateby.ema; +import io.deephaven.api.updateby.BadDataBehavior; import io.deephaven.api.updateby.OperationControl; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.api.updateby.BadDataBehavior; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.locations.TableDataException; -import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.updateby.internal.BaseDoubleUpdateByOperator; -import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -22,19 +21,18 @@ public abstract class BasePrimitiveEMAOperator extends BaseDoubleUpdateByOperator { protected final OperationControl control; - protected final LongRecordingUpdateByOperator timeRecorder; protected final String timestampColumnName; protected final double timeScaleUnits; + protected final double alpha; + protected double oneMinusAlpha; + + class Context extends BaseDoubleUpdateByOperator.Context { + public LongChunk timestampValueChunk; - class EmaContext extends Context { - double alpha; - double oneMinusAlpha; long lastStamp = NULL_LONG; - EmaContext(final double timeScaleUnits, final int chunkSize, final LongSegmentedSortedArray timestampSsa) { - super(chunkSize, timestampSsa); - this.alpha = Math.exp(-1 / timeScaleUnits); - this.oneMinusAlpha = 1 - alpha; + Context(final int chunkSize) { + super(chunkSize); } } @@ -44,115 +42,63 @@ class EmaContext extends Context { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this ema * @param control the control parameters for EMA - * @param timeRecorder an optional recorder for a timestamp column. If this is null, it will be assumed time is - * measured in integer ticks. - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timeRecorder} is provided, this is measured - * in ticks, otherwise it is measured in nanoseconds. + * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in + * integer ticks. + * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is + * measured in ticks, otherwise it is measured in nanoseconds. * @param redirContext the row redirection context to use for the EMA */ public BasePrimitiveEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, - @Nullable final LongRecordingUpdateByOperator timeRecorder, @Nullable final String timestampColumnName, final long timeScaleUnits, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { super(pair, affectingColumns, redirContext); this.control = control; - this.timeRecorder = timeRecorder; this.timestampColumnName = timestampColumnName; - this.timeScaleUnits = timeScaleUnits; - } - - @NotNull - @Override - public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize, - final LongSegmentedSortedArray timestampSsa) { - return new EmaContext(timeScaleUnits, chunkSize, timestampSsa); - } + this.timeScaleUnits = (double) timeScaleUnits; - @Override - public void initializeFor(@NotNull final UpdateByOperator.UpdateContext updateContext, - @NotNull final RowSet updateRowSet) { - super.initializeFor(updateContext, updateRowSet); - } + alpha = Math.exp(-1.0 / (double) timeScaleUnits); + oneMinusAlpha = 1 - alpha; - @Override - protected void doProcessChunk(@NotNull final Context context, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { - final EmaContext ctx = (EmaContext) context; - if (timeRecorder == null) { - computeWithTicks(ctx, workingChunk, 0, inputKeys.intSize()); - } else { - computeWithTime(ctx, workingChunk, 0, inputKeys.intSize()); - } - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); } @Override - public void resetForProcess(@NotNull final UpdateByOperator.UpdateContext context, - @NotNull final RowSet sourceIndex, - final long firstUnmodifiedKey) { - super.resetForProcess(context, sourceIndex, firstUnmodifiedKey); + public void initializeUpdate(@NotNull final UpdateContext updateContext, + @NotNull final long firstUnmodifiedKey, long firstUnmodifiedTimestamp) { + super.initializeUpdate(updateContext, firstUnmodifiedKey, firstUnmodifiedTimestamp); - if (timeRecorder == null) { - return; - } - - final EmaContext ctx = (EmaContext) context; + final Context ctx = (Context) updateContext; // If we set the last state to null, then we know it was a reset state and the timestamp must also // have been reset. if (ctx.curVal == NULL_DOUBLE || (firstUnmodifiedKey == NULL_ROW_KEY)) { ctx.lastStamp = NULL_LONG; } else { - // If it hasn't been reset to null, then it's possible that the value at that position was null, in - // which case we must have ignored it, and so we have to actually keep looking backwards until we find - // something not null. - ctx.lastStamp = locateFirstValidPreviousTimestamp(sourceIndex, firstUnmodifiedKey); + // rely on the caller to validate this is a valid timestamp (or NULL_LONG when appropriate) + ctx.lastStamp = firstUnmodifiedTimestamp; } } - private long locateFirstValidPreviousTimestamp(@NotNull final RowSet indexToSearch, - final long firstUnmodifiedKey) { - long potentialResetTimestamp = timeRecorder.getCurrentLong(firstUnmodifiedKey); - if (potentialResetTimestamp != NULL_LONG && isValueValid(firstUnmodifiedKey)) { - return potentialResetTimestamp; - } - - try (final RowSet.SearchIterator rIt = indexToSearch.reverseIterator()) { - if (rIt.advance(firstUnmodifiedKey)) { - while (rIt.hasNext()) { - final long nextKey = rIt.nextLong(); - potentialResetTimestamp = timeRecorder.getCurrentLong(nextKey); - if (potentialResetTimestamp != NULL_LONG && isValueValid(nextKey)) { - return potentialResetTimestamp; - } - } - } - } - - return NULL_LONG; - } - @Override - public String getTimestampColumnName() { - return this.timestampColumnName; + public void processChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @Nullable final LongChunk posChunk, + @Nullable final Chunk valuesChunk, + @Nullable final LongChunk timestampValuesChunk) { + Assert.notEquals(valuesChunk, "valuesChunk must not be null for a cumulative operator", null); + final Context ctx = (Context) updateContext; + ctx.storeValuesChunk(valuesChunk); + ctx.timestampValueChunk = timestampValuesChunk; + for (int ii = 0; ii < valuesChunk.size(); ii++) { + push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); + ctx.outputValues.set(ii, ctx.curVal); + } + outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues, inputKeys); } - abstract boolean isValueValid(final long atKey); - - abstract void computeWithTicks(final EmaContext ctx, - final Chunk valueChunk, - final int chunkStart, - final int chunkEnd); - - abstract void computeWithTime(final EmaContext ctx, - final Chunk valueChunk, - final int chunkStart, - final int chunkEnd); - - void handleBadData(@NotNull final EmaContext ctx, + void handleBadData(@NotNull final Context ctx, final boolean isNull, final boolean isNan, final boolean isNullTime) { @@ -185,7 +131,7 @@ void handleBadData(@NotNull final EmaContext ctx, } } - void handleBadTime(@NotNull final EmaContext ctx, final long dt) { + void handleBadTime(@NotNull final Context ctx, final long dt) { boolean doReset = false; if (dt == 0) { if (control.onZeroDeltaTimeOrDefault() == BadDataBehavior.THROW) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java index dfd1afe5f4d..7596ae6d1d2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java @@ -1,14 +1,9 @@ package io.deephaven.engine.table.impl.updateby.ema; -import io.deephaven.chunk.ObjectChunk; -import io.deephaven.chunk.WritableObjectChunk; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.api.updateby.OperationControl; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -24,61 +19,43 @@ public class BigDecimalEMAOperator extends BigNumberEMAOperator { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this ema * @param control defines how to handle {@code null} input values. - * @param timeRecorder an optional recorder for a timestamp column. If this is null, it will be assumed time is - * measured in integer ticks. + * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in + * integer ticks. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timeRecorder} is provided, this is measured * in ticks, otherwise it is measured in nanoseconds * @param valueSource the input column source. Used when determining reset positions for reprocessing + * @param redirContext the row redirection context to use for the EMA */ public BigDecimalEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, - @Nullable final LongRecordingUpdateByOperator timeRecorder, @Nullable final String timestampColumnName, final long timeScaleUnits, @NotNull final ColumnSource valueSource, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext - // region extra-constructor-args - // endregion extra-constructor-args - ) { - super(pair, affectingColumns, control, timeRecorder, timestampColumnName, timeScaleUnits, valueSource, - redirContext); - // region constructor - // endregion constructor + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { + super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, valueSource, redirContext); } - void computeWithTicks(final EmaContext ctx, - final ObjectChunk valueChunk, - final int chunkStart, - final int chunkEnd) { - final WritableObjectChunk localOutputChunk = ctx.outputValues.get(); - for (int ii = chunkStart; ii < chunkEnd; ii++) { - final BigDecimal input = valueChunk.get(ii); + @Override + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + final BigDecimal input = ctx.objectValueChunk.get(pos); + if (timestampColumnName == null) { + // compute with ticks if (input == null) { handleBadData(ctx, true, false); } else { if (ctx.curVal == null) { ctx.curVal = input; } else { - ctx.curVal = ctx.curVal.multiply(ctx.alpha, control.bigValueContextOrDefault()) - .add(input.multiply(ctx.oneMinusAlpha, control.bigValueContextOrDefault()), + ctx.curVal = ctx.curVal.multiply(alpha, control.bigValueContextOrDefault()) + .add(input.multiply(oneMinusAlpha, control.bigValueContextOrDefault()), control.bigValueContextOrDefault()); } } - - localOutputChunk.set(ii, ctx.curVal); - } - } - - void computeWithTime(final EmaContext ctx, - final ObjectChunk valueChunk, - final int chunkStart, - final int chunkEnd) { - final WritableObjectChunk localOutputChunk = ctx.outputValues.get(); - for (int ii = chunkStart; ii < chunkEnd; ii++) { - final BigDecimal input = valueChunk.get(ii); - // noinspection ConstantConditions - final long timestamp = timeRecorder.getLong(ii); + } else { + // compute with time + final long timestamp = ctx.timestampValueChunk.get(pos); final boolean isNull = input == null; final boolean isNullTime = timestamp == NULL_LONG; if (isNull || isNullTime) { @@ -92,18 +69,17 @@ void computeWithTime(final EmaContext ctx, if (dt <= 0) { handleBadTime(ctx, dt); } else { - ctx.alpha = BigDecimal.valueOf(Math.exp(-dt / timeScaleUnits)); - ctx.curVal = ctx.curVal.multiply(ctx.alpha, control.bigValueContextOrDefault()) - .add(input.multiply( - BigDecimal.ONE.subtract(ctx.alpha, control.bigValueContextOrDefault()), - control.bigValueContextOrDefault()), + // alpha is dynamic, based on time + BigDecimal alpha = BigDecimal.valueOf(Math.exp(-dt / timeScaleUnits)); + BigDecimal oneMinusAlpha = BigDecimal.ONE.subtract(alpha, control.bigValueContextOrDefault()); + + ctx.curVal = ctx.curVal.multiply(alpha, control.bigValueContextOrDefault()) + .add(input.multiply(oneMinusAlpha, control.bigValueContextOrDefault()), control.bigValueContextOrDefault()); ctx.lastStamp = timestamp; } } } - - localOutputChunk.set(ii, ctx.curVal); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java index b73a646de41..be019d8dc25 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java @@ -1,14 +1,9 @@ package io.deephaven.engine.table.impl.updateby.ema; import io.deephaven.api.updateby.OperationControl; -import io.deephaven.chunk.ObjectChunk; -import io.deephaven.chunk.WritableObjectChunk; -import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -19,73 +14,53 @@ public class BigIntegerEMAOperator extends BigNumberEMAOperator { /** - * An operator that computes an EMA from an int column using an exponential decay function. + * An operator that computes an EMA from a int column using an exponential decay function. * * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this ema - * @param control defines how to handle {@code null} input values. - * @param timeRecorder an optional recorder for a timestamp column. If this is null, it will be assumed time is - * measured in integer ticks. + * @param control defines how to handle {@code null} input values. + * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in + * integer ticks. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timeRecorder} is provided, this is measured - * in ticks, otherwise it is measured in nanoseconds - * @param valueSource the input column source. Used when determining reset positions for reprocessing + * in ticks, otherwise it is measured in nanoseconds + * @param valueSource the input column source. Used when determining reset positions for reprocessing + * @param redirContext the row redirection context to use for the EMA */ public BigIntegerEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, - @Nullable final LongRecordingUpdateByOperator timeRecorder, @Nullable final String timestampColumnName, final long timeScaleUnits, @NotNull final ColumnSource valueSource, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext - // region extra-constructor-args - // endregion extra-constructor-args - ) { - super(pair, affectingColumns, control, timeRecorder, timestampColumnName, timeScaleUnits, valueSource, redirContext); - // region constructor - // endregion constructor + ) { + super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, valueSource, redirContext); } @Override - void computeWithTicks(final EmaContext ctx, - final ObjectChunk valueChunk, - final int chunkStart, - final int chunkEnd) { - final WritableObjectChunk localOutputChunk = ctx.outputValues.get(); - for (int ii = chunkStart; ii < chunkEnd; ii++) { - final BigInteger input = valueChunk.get(ii); - if(input == null) { + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + final BigInteger input = ctx.objectValueChunk.get(pos); + if (timestampColumnName == null) { + // compute with ticks + if (input == null) { handleBadData(ctx, true, false); } else { final BigDecimal decimalInput = new BigDecimal(input, control.bigValueContextOrDefault()); if(ctx.curVal == null) { ctx.curVal = decimalInput; } else { - ctx.curVal = ctx.curVal.multiply(ctx.alpha, control.bigValueContextOrDefault()) - .add(decimalInput.multiply( - BigDecimal.ONE.subtract(ctx.alpha, control.bigValueContextOrDefault()), - control.bigValueContextOrDefault()), + ctx.curVal = ctx.curVal.multiply(alpha, control.bigValueContextOrDefault()) + .add(decimalInput.multiply(oneMinusAlpha, control.bigValueContextOrDefault()), control.bigValueContextOrDefault()); } } - - localOutputChunk.set(ii, ctx.curVal); - } - } - - @Override - void computeWithTime(final EmaContext ctx, - final ObjectChunk valueChunk, - final int chunkStart, - final int chunkEnd) { - final WritableObjectChunk localOutputChunk = ctx.outputValues.get(); - for (int ii = chunkStart; ii < chunkEnd; ii++) { - final BigInteger input = valueChunk.get(ii); - //noinspection ConstantConditions - final long timestamp = timeRecorder.getLong(ii); + } else { + // compute with time + final long timestamp = ctx.timestampValueChunk.get(pos); final boolean isNull = input == null; final boolean isNullTime = timestamp == NULL_LONG; - if(isNull || isNullTime) { + if (isNull || isNullTime) { handleBadData(ctx, isNull, isNullTime); } else { final BigDecimal decimalInput = new BigDecimal(input, control.bigValueContextOrDefault()); @@ -97,16 +72,17 @@ void computeWithTime(final EmaContext ctx, if(dt <= 0) { handleBadTime(ctx, dt); } else { - ctx.alpha = BigDecimal.valueOf(Math.exp(-dt / timeScaleUnits)); - ctx.curVal = ctx.curVal.multiply(ctx.alpha, control.bigValueContextOrDefault()) - .add(decimalInput.multiply(BigDecimal.ONE.subtract(ctx.alpha, control.bigValueContextOrDefault()), control.bigValueContextOrDefault()), + // alpha is dynamic, based on time + BigDecimal alpha = BigDecimal.valueOf(Math.exp(-dt / timeScaleUnits)); + BigDecimal oneMinusAlpha = BigDecimal.ONE.subtract(alpha, control.bigValueContextOrDefault()); + + ctx.curVal = ctx.curVal.multiply(alpha, control.bigValueContextOrDefault()) + .add(decimalInput.multiply(oneMinusAlpha, control.bigValueContextOrDefault()), control.bigValueContextOrDefault()); ctx.lastStamp = timestamp; } } } - - localOutputChunk.set(ii, ctx.curVal); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java index c538051cb68..33fbe250758 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java @@ -2,42 +2,49 @@ import io.deephaven.api.updateby.BadDataBehavior; import io.deephaven.api.updateby.OperationControl; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.locations.TableDataException; -import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; -import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import java.math.BigDecimal; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; -import static io.deephaven.util.QueryConstants.NULL_LONG; +import static io.deephaven.util.QueryConstants.*; public abstract class BigNumberEMAOperator extends BaseObjectUpdateByOperator { protected final ColumnSource valueSource; protected final OperationControl control; - protected final LongRecordingUpdateByOperator timeRecorder; protected final String timestampColumnName; protected final double timeScaleUnits; + protected final BigDecimal alpha; + protected final BigDecimal oneMinusAlpha; + + + class Context extends BaseObjectUpdateByOperator.Context { + public LongChunk timestampValueChunk; + public ObjectChunk objectValueChunk; - class EmaContext extends Context { - BigDecimal alpha = BigDecimal.valueOf(Math.exp(-1 / timeScaleUnits)); - BigDecimal oneMinusAlpha = - timeRecorder == null ? BigDecimal.ONE.subtract(alpha, control.bigValueContextOrDefault()) : null; long lastStamp = NULL_LONG; - EmaContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { - super(chunkSize, timestampSsa); + Context(final int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + objectValueChunk = valuesChunk.asObjectChunk(); } } @@ -47,8 +54,6 @@ class EmaContext extends Context { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this ema * @param control defines how to handle {@code null} input values. - * @param timeRecorder an optional recorder for a timestamp column. If this is null, it will be assumed time is - * measured in integer ticks. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timeRecorder} is provided, this is measured * in ticks, otherwise it is measured in nanoseconds * @param valueSource the input column source. Used when determining reset positions for reprocessing @@ -56,107 +61,63 @@ class EmaContext extends Context { public BigNumberEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, - @Nullable final LongRecordingUpdateByOperator timeRecorder, @Nullable final String timestampColumnName, final long timeScaleUnits, @NotNull ColumnSource valueSource, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext - // region extra-constructor-args - // endregion extra-constructor-args - ) { + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { super(pair, affectingColumns, redirContext, BigDecimal.class); + this.control = control; - this.timeRecorder = timeRecorder; this.timestampColumnName = timestampColumnName; - this.timeScaleUnits = timeScaleUnits; + this.timeScaleUnits = (double) timeScaleUnits; this.valueSource = valueSource; - // region constructor - // endregion constructor - } - @NotNull - @Override - public UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { - return new EmaContext(chunkSize, timestampSsa); + alpha = BigDecimal.valueOf(Math.exp(-1.0 / (double) timeScaleUnits)); + oneMinusAlpha = + timestampColumnName == null ? BigDecimal.ONE.subtract(alpha, control.bigValueContextOrDefault()) : null; } + @NotNull @Override - protected void doProcessChunk(@NotNull final Context updateContext, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { - final ObjectChunk asObjects = workingChunk.asObjectChunk(); - final EmaContext ctx = (EmaContext) updateContext; - - if (timeRecorder == null) { - computeWithTicks(ctx, asObjects, 0, inputKeys.intSize()); - } else { - computeWithTime(ctx, asObjects, 0, inputKeys.intSize()); - } - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); + public UpdateContext makeUpdateContext(final int chunkSize) { + return new Context(chunkSize); } - @Override - public void resetForProcess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceIndex, - final long firstUnmodifiedKey) { - super.resetForProcess(context, sourceIndex, firstUnmodifiedKey); - - if (timeRecorder == null) { - return; - } - - final EmaContext ctx = (EmaContext) context; + public void initializeUpdate(@NotNull final UpdateContext updateContext, + @NotNull final long firstUnmodifiedKey, long firstUnmodifiedTimestamp) { + super.initializeUpdate(updateContext, firstUnmodifiedKey, firstUnmodifiedTimestamp); + final Context ctx = (Context) updateContext; // If we set the last state to null, then we know it was a reset state and the timestamp must also // have been reset. if (ctx.curVal == null || (firstUnmodifiedKey == NULL_ROW_KEY)) { ctx.lastStamp = NULL_LONG; } else { - // If it hasn't been reset to null, then it's possible that the value at that position was null, in - // which case we must have ignored it, and so we have to actually keep looking backwards until we find - // something not null. - ctx.lastStamp = locateFirstValidPreviousTimestamp(sourceIndex, firstUnmodifiedKey); + // rely on the caller to validate this is a valid timestamp (or NULL_LONG when appropriate) + ctx.lastStamp = firstUnmodifiedTimestamp; } } - private long locateFirstValidPreviousTimestamp(@NotNull final RowSet indexToSearch, - final long firstUnmodifiedKey) { - long potentialResetTimestamp = timeRecorder.getCurrentLong(firstUnmodifiedKey); - if (potentialResetTimestamp != NULL_LONG && isValueValid(firstUnmodifiedKey)) { - return potentialResetTimestamp; - } - - try (final RowSet.SearchIterator rIt = indexToSearch.reverseIterator()) { - if (rIt.advance(firstUnmodifiedKey)) { - while (rIt.hasNext()) { - final long nextKey = rIt.nextLong(); - potentialResetTimestamp = timeRecorder.getCurrentLong(nextKey); - if (potentialResetTimestamp != NULL_LONG && isValueValid(nextKey)) { - return potentialResetTimestamp; - } - } - } + @Override + public void processChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @Nullable final LongChunk posChunk, + @Nullable final Chunk valuesChunk, + @Nullable final LongChunk timestampValuesChunk) { + Assert.notEquals(valuesChunk, "valuesChunk must not be null for a cumulative operator", null); + final Context ctx = (Context) updateContext; + ctx.storeValuesChunk(valuesChunk); + ctx.timestampValueChunk = timestampValuesChunk; + for (int ii = 0; ii < valuesChunk.size(); ii++) { + push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); + ctx.outputValues.set(ii, ctx.curVal); } - - return NULL_LONG; - } - - private boolean isValueValid(final long atKey) { - return valueSource.get(atKey) != null; + outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues, inputKeys); } - abstract void computeWithTicks(final EmaContext ctx, - final ObjectChunk valueChunk, - final int chunkStart, - final int chunkEnd); - - abstract void computeWithTime(final EmaContext ctx, - final ObjectChunk valueChunk, - final int chunkStart, - final int chunkEnd); - - void handleBadData(@NotNull final EmaContext ctx, + void handleBadData(@NotNull final Context ctx, final boolean isNull, final boolean isNullTime) { boolean doReset = false; @@ -180,7 +141,7 @@ void handleBadData(@NotNull final EmaContext ctx, } } - void handleBadTime(@NotNull final EmaContext ctx, final long dt) { + void handleBadTime(@NotNull final Context ctx, final long dt) { boolean doReset = false; if (dt == 0) { if (control.onZeroDeltaTimeOrDefault() == BadDataBehavior.THROW) { @@ -199,4 +160,9 @@ void handleBadTime(@NotNull final EmaContext ctx, final long dt) { ctx.lastStamp = NULL_LONG; } } + + @Override + public boolean isValueValid(long atKey) { + return valueSource.get(atKey) != null; + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java index 5775a07d9f6..f5730d8af25 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java @@ -8,13 +8,10 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ByteChunk; -import io.deephaven.chunk.WritableDoubleChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -23,14 +20,25 @@ public class ByteEMAOperator extends BasePrimitiveEMAOperator { private final ColumnSource valueSource; + protected class Context extends BasePrimitiveEMAOperator.Context { + public ByteChunk byteValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + byteValueChunk = valuesChunk.asByteChunk(); + } + } + /** * An operator that computes an EMA from a byte column using an exponential decay function. * * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this ema * @param control defines how to handle {@code null} input values. - * @param timeRecorder an optional recorder for a timestamp column. If this is null, it will be assumed time is - * measured in integer ticks. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timeRecorder} is provided, this is measured * in ticks, otherwise it is measured in nanoseconds * @param valueSource the input column source. Used when determining reset positions for reprocessing @@ -38,7 +46,6 @@ public class ByteEMAOperator extends BasePrimitiveEMAOperator { public ByteEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, - @Nullable final LongRecordingUpdateByOperator timeRecorder, @Nullable final String timestampColumnName, final long timeScaleUnits, @NotNull final ColumnSource valueSource, @@ -46,45 +53,38 @@ public ByteEMAOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, timestampColumnName, timeScaleUnits, redirContext); + super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits,redirContext); this.valueSource = valueSource; // region constructor // endregion constructor } + @NotNull + @Override + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); + } + @Override - void computeWithTicks(final EmaContext ctx, - final Chunk valueChunk, - final int chunkStart, - final int chunkEnd) { - final ByteChunk asBytes = valueChunk.asByteChunk(); - final WritableDoubleChunk localOutputChunk = ctx.outputValues.get(); - for (int ii = chunkStart; ii < chunkEnd; ii++) { - final byte input = asBytes.get(ii); + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + final byte input = ctx.byteValueChunk.get(pos); + if (timestampColumnName == null) { + // compute with ticks if(input == NULL_BYTE) { handleBadData(ctx, true, false, false); } else { if(ctx.curVal == NULL_DOUBLE) { ctx.curVal = input; } else { - ctx.curVal = ctx.alpha * ctx.curVal + (ctx.oneMinusAlpha * input); + ctx.curVal = alpha * ctx.curVal + (oneMinusAlpha * input); } } - localOutputChunk.set(ii, ctx.curVal); - } - } - - @Override - void computeWithTime(final EmaContext ctx, - final Chunk valueChunk, - final int chunkStart, - final int chunkEnd) { - final ByteChunk asBytes = valueChunk.asByteChunk(); - final WritableDoubleChunk localOutputChunk = ctx.outputValues.get(); - for (int ii = chunkStart; ii < chunkEnd; ii++) { - final byte input = asBytes.get(ii); + } else { + // compute with time + final long timestamp = ctx.timestampValueChunk.get(pos); //noinspection ConstantConditions - final long timestamp = timeRecorder.getLong(ii); final boolean isNull = input == NULL_BYTE; final boolean isNullTime = timestamp == NULL_LONG; if(isNull || isNullTime) { @@ -98,18 +98,18 @@ void computeWithTime(final EmaContext ctx, if(dt <= 0) { handleBadTime(ctx, dt); } else { + // alpha is dynamic, based on time final double alpha = Math.exp(-dt / timeScaleUnits); ctx.curVal = alpha * ctx.curVal + ((1 - alpha) * input); ctx.lastStamp = timestamp; } } } - localOutputChunk.set(ii, ctx.curVal); } } @Override - boolean isValueValid(long atKey) { + public boolean isValueValid(long atKey) { return valueSource.getByte(atKey) != NULL_BYTE; } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java index 33ddb59351b..f47fb81d7c9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java @@ -5,17 +5,15 @@ */ package io.deephaven.engine.table.impl.updateby.ema; +import io.deephaven.api.updateby.BadDataBehavior; import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.DoubleChunk; -import io.deephaven.chunk.WritableDoubleChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.api.updateby.BadDataBehavior; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; +import io.deephaven.engine.table.impl.updateby.prod.ShortCumProdOperator; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -24,14 +22,25 @@ public class DoubleEMAOperator extends BasePrimitiveEMAOperator { private final ColumnSource valueSource; + protected class Context extends BasePrimitiveEMAOperator.Context { + public DoubleChunk doubleValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + doubleValueChunk = valuesChunk.asDoubleChunk(); + } + } + /** * An operator that computes an EMA from a double column using an exponential decay function. * * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this ema * @param control defines how to handle {@code null} input values. - * @param timeRecorder an optional recorder for a timestamp column. If this is null, it will be assumed time is - * measured in integer ticks. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timeRecorder} is provided, this is measured * in ticks, otherwise it is measured in nanoseconds * @param valueSource the input column source. Used when determining reset positions for reprocessing @@ -39,7 +48,6 @@ public class DoubleEMAOperator extends BasePrimitiveEMAOperator { public DoubleEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, - @Nullable final LongRecordingUpdateByOperator timeRecorder, @Nullable final String timestampColumnName, final long timeScaleUnits, @NotNull final ColumnSource valueSource, @@ -47,82 +55,62 @@ public DoubleEMAOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, timestampColumnName, timeScaleUnits, redirContext); + super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor // endregion constructor } + @NotNull @Override - void computeWithTicks(final EmaContext ctx, - final Chunk valueChunk, - final int chunkStart, - final int chunkEnd) { - final DoubleChunk asDoubles = valueChunk.asDoubleChunk(); - final WritableDoubleChunk localOutputChunk = ctx.outputValues.get(); - for (int ii = chunkStart; ii < chunkEnd; ii++) { - final double input = asDoubles.get(ii); - final boolean isNull = input == NULL_DOUBLE; - final boolean isNan = Double.isNaN(input); - if(isNull || isNan) { - handleBadData(ctx, isNull, isNan, false); - } else if(!Double.isNaN(ctx.curVal)) { - if (ctx.curVal == NULL_DOUBLE) { + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); + } + + @Override + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + final double input = ctx.doubleValueChunk.get(pos); + if (timestampColumnName == null) { + // compute with ticks + if(input == NULL_DOUBLE) { + handleBadData(ctx, true, false, false); + } else { + if(ctx.curVal == NULL_DOUBLE) { ctx.curVal = input; } else { - ctx.curVal = ctx.alpha * ctx.curVal + (ctx.oneMinusAlpha * input); + ctx.curVal = alpha * ctx.curVal + (oneMinusAlpha * input); } } - - localOutputChunk.set(ii, ctx.curVal); - } - } - - @Override - void computeWithTime(final EmaContext ctx, - final Chunk valueChunk, - final int chunkStart, - final int chunkEnd) { - final DoubleChunk asDoubles = valueChunk.asDoubleChunk(); - final WritableDoubleChunk localOutputChunk = ctx.outputValues.get(); - for (int ii = chunkStart; ii < chunkEnd; ii++) { - final double input = asDoubles.get(ii); - final long timestamp = timeRecorder.getLong(ii); + } else { + // compute with time + final long timestamp = ctx.timestampValueChunk.get(pos); + //noinspection ConstantConditions final boolean isNull = input == NULL_DOUBLE; - final boolean isNan = Double.isNaN(input); final boolean isNullTime = timestamp == NULL_LONG; - - // Handle bad data first - if(isNull || isNan || isNullTime) { - handleBadData(ctx, isNull, isNan, isNullTime); - } else if(ctx.curVal == NULL_DOUBLE) { - // If the data looks good, and we have a null ema, just accept the current value - ctx.curVal = input; - ctx.lastStamp = timestamp; + if(isNull || isNullTime) { + handleBadData(ctx, isNull, false, isNullTime); } else { - final boolean currentPoisoned = Double.isNaN(ctx.curVal); - if(currentPoisoned && ctx.lastStamp == NULL_LONG) { - // If the current EMA was a NaN, we should accept the first good timestamp so that - // we can handle reset behavior properly in the following else + if(ctx.curVal == NULL_DOUBLE) { + ctx.curVal = input; ctx.lastStamp = timestamp; } else { final long dt = timestamp - ctx.lastStamp; if(dt <= 0) { handleBadTime(ctx, dt); - } else if(!currentPoisoned) { + } else { final double alpha = Math.exp(-dt / timeScaleUnits); ctx.curVal = alpha * ctx.curVal + ((1 - alpha) * input); ctx.lastStamp = timestamp; } } } - - localOutputChunk.set(ii, ctx.curVal); } } @Override - boolean isValueValid(long atKey) { + public boolean isValueValid(long atKey) { final double value = valueSource.getDouble(atKey); if(value == NULL_DOUBLE) { return false; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java index a68e61fcd96..2962bd8d4cd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java @@ -1,16 +1,14 @@ package io.deephaven.engine.table.impl.updateby.ema; +import io.deephaven.api.updateby.BadDataBehavior; import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.FloatChunk; -import io.deephaven.chunk.WritableDoubleChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.api.updateby.BadDataBehavior; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; +import io.deephaven.engine.table.impl.updateby.prod.ShortCumProdOperator; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -19,14 +17,25 @@ public class FloatEMAOperator extends BasePrimitiveEMAOperator { private final ColumnSource valueSource; + protected class Context extends BasePrimitiveEMAOperator.Context { + public FloatChunk floatValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + floatValueChunk = valuesChunk.asFloatChunk(); + } + } + /** * An operator that computes an EMA from a float column using an exponential decay function. * * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this ema * @param control defines how to handle {@code null} input values. - * @param timeRecorder an optional recorder for a timestamp column. If this is null, it will be assumed time is - * measured in integer ticks. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timeRecorder} is provided, this is measured * in ticks, otherwise it is measured in nanoseconds * @param valueSource the input column source. Used when determining reset positions for reprocessing @@ -34,7 +43,6 @@ public class FloatEMAOperator extends BasePrimitiveEMAOperator { public FloatEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, - @Nullable final LongRecordingUpdateByOperator timeRecorder, @Nullable final String timestampColumnName, final long timeScaleUnits, @NotNull final ColumnSource valueSource, @@ -42,82 +50,62 @@ public FloatEMAOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, timestampColumnName, timeScaleUnits, redirContext); + super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, redirContext); this.valueSource = valueSource; // region constructor // endregion constructor } + @NotNull @Override - void computeWithTicks(final EmaContext ctx, - final Chunk valueChunk, - final int chunkStart, - final int chunkEnd) { - final FloatChunk asFloats = valueChunk.asFloatChunk(); - final WritableDoubleChunk localOutputChunk = ctx.outputValues.get(); - for (int ii = chunkStart; ii < chunkEnd; ii++) { - final float input = asFloats.get(ii); - final boolean isNull = input == NULL_FLOAT; - final boolean isNan = Float.isNaN(input); - if(isNull || isNan) { - handleBadData(ctx, isNull, isNan, false); - } else if(!Double.isNaN(ctx.curVal)) { - if (ctx.curVal == NULL_DOUBLE) { + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); + } + + @Override + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + final float input = ctx.floatValueChunk.get(pos); + if (timestampColumnName == null) { + // compute with ticks + if(input == NULL_FLOAT) { + handleBadData(ctx, true, false, false); + } else { + if(ctx.curVal == NULL_DOUBLE) { ctx.curVal = input; } else { - ctx.curVal = ctx.alpha * ctx.curVal + (ctx.oneMinusAlpha * input); + ctx.curVal = alpha * ctx.curVal + (oneMinusAlpha * input); } } - - localOutputChunk.set(ii, ctx.curVal); - } - } - - @Override - void computeWithTime(final EmaContext ctx, - final Chunk valueChunk, - final int chunkStart, - final int chunkEnd) { - final FloatChunk asFloats = valueChunk.asFloatChunk(); - final WritableDoubleChunk localOutputChunk = ctx.outputValues.get(); - for (int ii = chunkStart; ii < chunkEnd; ii++) { - final float input = asFloats.get(ii); - final long timestamp = timeRecorder.getLong(ii); + } else { + // compute with time + final long timestamp = ctx.timestampValueChunk.get(pos); + //noinspection ConstantConditions final boolean isNull = input == NULL_FLOAT; - final boolean isNan = Float.isNaN(input); final boolean isNullTime = timestamp == NULL_LONG; - - // Handle bad data first - if(isNull || isNan || isNullTime) { - handleBadData(ctx, isNull, isNan, isNullTime); - } else if(ctx.curVal == NULL_DOUBLE) { - // If the data looks good, and we have a null ema, just accept the current value - ctx.curVal = input; - ctx.lastStamp = timestamp; + if(isNull || isNullTime) { + handleBadData(ctx, isNull, false, isNullTime); } else { - final boolean currentPoisoned = Double.isNaN(ctx.curVal); - if(currentPoisoned && ctx.lastStamp == NULL_LONG) { - // If the current EMA was a NaN, we should accept the first good timestamp so that - // we can handle reset behavior properly in the following else + if(ctx.curVal == NULL_DOUBLE) { + ctx.curVal = input; ctx.lastStamp = timestamp; } else { final long dt = timestamp - ctx.lastStamp; if(dt <= 0) { handleBadTime(ctx, dt); - } else if(!currentPoisoned) { + } else { final double alpha = Math.exp(-dt / timeScaleUnits); ctx.curVal = alpha * ctx.curVal + ((1 - alpha) * input); ctx.lastStamp = timestamp; } } } - - localOutputChunk.set(ii, ctx.curVal); } } @Override - boolean isValueValid(long atKey) { + public boolean isValueValid(long atKey) { final float value = valueSource.getFloat(atKey); if(value == NULL_FLOAT) { return false; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java index ab584fd87ca..805df4d30a5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java @@ -8,13 +8,10 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.IntChunk; -import io.deephaven.chunk.WritableDoubleChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -23,14 +20,25 @@ public class IntEMAOperator extends BasePrimitiveEMAOperator { private final ColumnSource valueSource; + protected class Context extends BasePrimitiveEMAOperator.Context { + public IntChunk intValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + intValueChunk = valuesChunk.asIntChunk(); + } + } + /** * An operator that computes an EMA from a int column using an exponential decay function. * * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this ema * @param control defines how to handle {@code null} input values. - * @param timeRecorder an optional recorder for a timestamp column. If this is null, it will be assumed time is - * measured in integer ticks. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timeRecorder} is provided, this is measured * in ticks, otherwise it is measured in nanoseconds * @param valueSource the input column source. Used when determining reset positions for reprocessing @@ -38,7 +46,6 @@ public class IntEMAOperator extends BasePrimitiveEMAOperator { public IntEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, - @Nullable final LongRecordingUpdateByOperator timeRecorder, @Nullable final String timestampColumnName, final long timeScaleUnits, @NotNull final ColumnSource valueSource, @@ -46,45 +53,38 @@ public IntEMAOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, timestampColumnName, timeScaleUnits, redirContext); + super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits,redirContext); this.valueSource = valueSource; // region constructor // endregion constructor } + @NotNull + @Override + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); + } + @Override - void computeWithTicks(final EmaContext ctx, - final Chunk valueChunk, - final int chunkStart, - final int chunkEnd) { - final IntChunk asIntegers = valueChunk.asIntChunk(); - final WritableDoubleChunk localOutputChunk = ctx.outputValues.get(); - for (int ii = chunkStart; ii < chunkEnd; ii++) { - final int input = asIntegers.get(ii); + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + final int input = ctx.intValueChunk.get(pos); + if (timestampColumnName == null) { + // compute with ticks if(input == NULL_INT) { handleBadData(ctx, true, false, false); } else { if(ctx.curVal == NULL_DOUBLE) { ctx.curVal = input; } else { - ctx.curVal = ctx.alpha * ctx.curVal + (ctx.oneMinusAlpha * input); + ctx.curVal = alpha * ctx.curVal + (oneMinusAlpha * input); } } - localOutputChunk.set(ii, ctx.curVal); - } - } - - @Override - void computeWithTime(final EmaContext ctx, - final Chunk valueChunk, - final int chunkStart, - final int chunkEnd) { - final IntChunk asIntegers = valueChunk.asIntChunk(); - final WritableDoubleChunk localOutputChunk = ctx.outputValues.get(); - for (int ii = chunkStart; ii < chunkEnd; ii++) { - final int input = asIntegers.get(ii); + } else { + // compute with time + final long timestamp = ctx.timestampValueChunk.get(pos); //noinspection ConstantConditions - final long timestamp = timeRecorder.getLong(ii); final boolean isNull = input == NULL_INT; final boolean isNullTime = timestamp == NULL_LONG; if(isNull || isNullTime) { @@ -98,18 +98,18 @@ void computeWithTime(final EmaContext ctx, if(dt <= 0) { handleBadTime(ctx, dt); } else { + // alpha is dynamic, based on time final double alpha = Math.exp(-dt / timeScaleUnits); ctx.curVal = alpha * ctx.curVal + ((1 - alpha) * input); ctx.lastStamp = timestamp; } } } - localOutputChunk.set(ii, ctx.curVal); } } @Override - boolean isValueValid(long atKey) { + public boolean isValueValid(long atKey) { return valueSource.getInt(atKey) != NULL_INT; } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java index 97794db3111..3593e0c26d0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java @@ -8,13 +8,10 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.WritableDoubleChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -23,14 +20,25 @@ public class LongEMAOperator extends BasePrimitiveEMAOperator { private final ColumnSource valueSource; + protected class Context extends BasePrimitiveEMAOperator.Context { + public LongChunk longValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + longValueChunk = valuesChunk.asLongChunk(); + } + } + /** * An operator that computes an EMA from a long column using an exponential decay function. * * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this ema * @param control defines how to handle {@code null} input values. - * @param timeRecorder an optional recorder for a timestamp column. If this is null, it will be assumed time is - * measured in integer ticks. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timeRecorder} is provided, this is measured * in ticks, otherwise it is measured in nanoseconds * @param valueSource the input column source. Used when determining reset positions for reprocessing @@ -38,7 +46,6 @@ public class LongEMAOperator extends BasePrimitiveEMAOperator { public LongEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, - @Nullable final LongRecordingUpdateByOperator timeRecorder, @Nullable final String timestampColumnName, final long timeScaleUnits, @NotNull final ColumnSource valueSource, @@ -46,45 +53,38 @@ public LongEMAOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, timestampColumnName, timeScaleUnits, redirContext); + super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits,redirContext); this.valueSource = valueSource; // region constructor // endregion constructor } + @NotNull + @Override + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); + } + @Override - void computeWithTicks(final EmaContext ctx, - final Chunk valueChunk, - final int chunkStart, - final int chunkEnd) { - final LongChunk asLongs = valueChunk.asLongChunk(); - final WritableDoubleChunk localOutputChunk = ctx.outputValues.get(); - for (int ii = chunkStart; ii < chunkEnd; ii++) { - final long input = asLongs.get(ii); + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + final long input = ctx.longValueChunk.get(pos); + if (timestampColumnName == null) { + // compute with ticks if(input == NULL_LONG) { handleBadData(ctx, true, false, false); } else { if(ctx.curVal == NULL_DOUBLE) { ctx.curVal = input; } else { - ctx.curVal = ctx.alpha * ctx.curVal + (ctx.oneMinusAlpha * input); + ctx.curVal = alpha * ctx.curVal + (oneMinusAlpha * input); } } - localOutputChunk.set(ii, ctx.curVal); - } - } - - @Override - void computeWithTime(final EmaContext ctx, - final Chunk valueChunk, - final int chunkStart, - final int chunkEnd) { - final LongChunk asLongs = valueChunk.asLongChunk(); - final WritableDoubleChunk localOutputChunk = ctx.outputValues.get(); - for (int ii = chunkStart; ii < chunkEnd; ii++) { - final long input = asLongs.get(ii); + } else { + // compute with time + final long timestamp = ctx.timestampValueChunk.get(pos); //noinspection ConstantConditions - final long timestamp = timeRecorder.getLong(ii); final boolean isNull = input == NULL_LONG; final boolean isNullTime = timestamp == NULL_LONG; if(isNull || isNullTime) { @@ -98,18 +98,18 @@ void computeWithTime(final EmaContext ctx, if(dt <= 0) { handleBadTime(ctx, dt); } else { + // alpha is dynamic, based on time final double alpha = Math.exp(-dt / timeScaleUnits); ctx.curVal = alpha * ctx.curVal + ((1 - alpha) * input); ctx.lastStamp = timestamp; } } } - localOutputChunk.set(ii, ctx.curVal); } } @Override - boolean isValueValid(long atKey) { + public boolean isValueValid(long atKey) { return valueSource.getLong(atKey) != NULL_LONG; } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java index d964c80c348..f1259e16618 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java @@ -3,13 +3,10 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ShortChunk; -import io.deephaven.chunk.WritableDoubleChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.updateby.internal.LongRecordingUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -18,14 +15,25 @@ public class ShortEMAOperator extends BasePrimitiveEMAOperator { private final ColumnSource valueSource; + protected class Context extends BasePrimitiveEMAOperator.Context { + public ShortChunk shortValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + shortValueChunk = valuesChunk.asShortChunk(); + } + } + /** * An operator that computes an EMA from a short column using an exponential decay function. * * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this ema * @param control defines how to handle {@code null} input values. - * @param timeRecorder an optional recorder for a timestamp column. If this is null, it will be assumed time is - * measured in integer ticks. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timeRecorder} is provided, this is measured * in ticks, otherwise it is measured in nanoseconds * @param valueSource the input column source. Used when determining reset positions for reprocessing @@ -33,7 +41,6 @@ public class ShortEMAOperator extends BasePrimitiveEMAOperator { public ShortEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, - @Nullable final LongRecordingUpdateByOperator timeRecorder, @Nullable final String timestampColumnName, final long timeScaleUnits, @NotNull final ColumnSource valueSource, @@ -41,45 +48,38 @@ public ShortEMAOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timeRecorder, timestampColumnName, timeScaleUnits, redirContext); + super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits,redirContext); this.valueSource = valueSource; // region constructor // endregion constructor } + @NotNull + @Override + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); + } + @Override - void computeWithTicks(final EmaContext ctx, - final Chunk valueChunk, - final int chunkStart, - final int chunkEnd) { - final ShortChunk asShorts = valueChunk.asShortChunk(); - final WritableDoubleChunk localOutputChunk = ctx.outputValues.get(); - for (int ii = chunkStart; ii < chunkEnd; ii++) { - final short input = asShorts.get(ii); + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + final short input = ctx.shortValueChunk.get(pos); + if (timestampColumnName == null) { + // compute with ticks if(input == NULL_SHORT) { handleBadData(ctx, true, false, false); } else { if(ctx.curVal == NULL_DOUBLE) { ctx.curVal = input; } else { - ctx.curVal = ctx.alpha * ctx.curVal + (ctx.oneMinusAlpha * input); + ctx.curVal = alpha * ctx.curVal + (oneMinusAlpha * input); } } - localOutputChunk.set(ii, ctx.curVal); - } - } - - @Override - void computeWithTime(final EmaContext ctx, - final Chunk valueChunk, - final int chunkStart, - final int chunkEnd) { - final ShortChunk asShorts = valueChunk.asShortChunk(); - final WritableDoubleChunk localOutputChunk = ctx.outputValues.get(); - for (int ii = chunkStart; ii < chunkEnd; ii++) { - final short input = asShorts.get(ii); + } else { + // compute with time + final long timestamp = ctx.timestampValueChunk.get(pos); //noinspection ConstantConditions - final long timestamp = timeRecorder.getLong(ii); final boolean isNull = input == NULL_SHORT; final boolean isNullTime = timestamp == NULL_LONG; if(isNull || isNullTime) { @@ -93,18 +93,18 @@ void computeWithTime(final EmaContext ctx, if(dt <= 0) { handleBadTime(ctx, dt); } else { + // alpha is dynamic, based on time final double alpha = Math.exp(-dt / timeScaleUnits); ctx.curVal = alpha * ctx.curVal + ((1 - alpha) * input); ctx.lastStamp = timestamp; } } } - localOutputChunk.set(ii, ctx.curVal); } } @Override - boolean isValueValid(long atKey) { + public boolean isValueValid(long atKey) { return valueSource.getShort(atKey) != NULL_SHORT; } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java index 71fac368a3b..e8bf4a703b5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java @@ -12,18 +12,13 @@ import io.deephaven.engine.table.impl.sources.BooleanSparseArraySource; import io.deephaven.engine.table.WritableColumnSource; -import io.deephaven.chunk.*; -import io.deephaven.chunk.attributes.ChunkLengths; -import io.deephaven.chunk.attributes.ChunkPositions; +import io.deephaven.chunk.ByteChunk; +import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseByteUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import static io.deephaven.util.BooleanUtils.NULL_BOOLEAN_AS_BYTE; @@ -31,6 +26,19 @@ public class BooleanFillByOperator extends BaseByteUpdateByOperator { // region extra-fields // endregion extra-fields + protected class Context extends BaseByteUpdateByOperator.Context { + public ByteChunk booleanValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + booleanValueChunk = valuesChunk.asByteChunk(); + } + } + public BooleanFillByOperator(@NotNull final MatchPair fillPair, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args @@ -41,6 +49,21 @@ public BooleanFillByOperator(@NotNull final MatchPair fillPair, // endregion constructor } + @NotNull + @Override + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); + } + + @Override + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + if(ctx.curVal == NULL_BOOLEAN_AS_BYTE) { + ctx.curVal = ctx.booleanValueChunk.get(pos); + } + } + // region extra-methods @Override protected byte getNullValue() { @@ -62,26 +85,4 @@ public Map> getOutputColumns() { return Collections.singletonMap(pair.leftColumn, outputSource.reinterpret(Boolean.class)); } // endregion extra-methods - - @Override - protected void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { - accumulate(workingChunk.asByteChunk(), ctx, 0, workingChunk.size()); - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); - } - - private void accumulate(@NotNull final ByteChunk asBooleans, - @NotNull final Context ctx, - final int runStart, - final int runLength) { - final WritableByteChunk localOutputValues = ctx.outputValues.get(); - for (int ii = runStart; ii < runStart + runLength; ii++) { - final byte currentVal = asBooleans.get(ii); - if(currentVal != NULL_BOOLEAN_AS_BYTE) { - ctx.curVal = currentVal; - } - localOutputValues.set(ii, ctx.curVal); - } - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java index 7ac31e69e93..671f8cd4268 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java @@ -5,18 +5,13 @@ */ package io.deephaven.engine.table.impl.updateby.fill; -import io.deephaven.chunk.*; -import io.deephaven.chunk.attributes.ChunkLengths; -import io.deephaven.chunk.attributes.ChunkPositions; +import io.deephaven.chunk.ByteChunk; +import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseByteUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.NULL_BYTE; @@ -24,6 +19,19 @@ public class ByteFillByOperator extends BaseByteUpdateByOperator { // region extra-fields // endregion extra-fields + protected class Context extends BaseByteUpdateByOperator.Context { + public ByteChunk byteValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + byteValueChunk = valuesChunk.asByteChunk(); + } + } + public ByteFillByOperator(@NotNull final MatchPair fillPair, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args @@ -34,28 +42,21 @@ public ByteFillByOperator(@NotNull final MatchPair fillPair, // endregion constructor } - // region extra-methods - // endregion extra-methods - + @NotNull @Override - protected void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { - accumulate(workingChunk.asByteChunk(), ctx, 0, workingChunk.size()); - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); } - private void accumulate(@NotNull final ByteChunk asBytes, - @NotNull final Context ctx, - final int runStart, - final int runLength) { - final WritableByteChunk localOutputValues = ctx.outputValues.get(); - for (int ii = runStart; ii < runStart + runLength; ii++) { - final byte currentVal = asBytes.get(ii); - if(currentVal != NULL_BYTE) { - ctx.curVal = currentVal; - } - localOutputValues.set(ii, ctx.curVal); + @Override + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + if(ctx.curVal == NULL_BYTE) { + ctx.curVal = ctx.byteValueChunk.get(pos); } } + + // region extra-methods + // endregion extra-methods } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java index 887b6a2ed8e..14a8a438e2b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java @@ -1,17 +1,12 @@ package io.deephaven.engine.table.impl.updateby.fill; -import io.deephaven.chunk.*; -import io.deephaven.chunk.attributes.ChunkLengths; -import io.deephaven.chunk.attributes.ChunkPositions; +import io.deephaven.chunk.CharChunk; +import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseCharUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.NULL_CHAR; @@ -19,6 +14,19 @@ public class CharFillByOperator extends BaseCharUpdateByOperator { // region extra-fields // endregion extra-fields + protected class Context extends BaseCharUpdateByOperator.Context { + public CharChunk charValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + charValueChunk = valuesChunk.asCharChunk(); + } + } + public CharFillByOperator(@NotNull final MatchPair fillPair, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args @@ -29,28 +37,21 @@ public CharFillByOperator(@NotNull final MatchPair fillPair, // endregion constructor } - // region extra-methods - // endregion extra-methods - + @NotNull @Override - protected void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { - accumulate(workingChunk.asCharChunk(), ctx, 0, workingChunk.size()); - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); } - private void accumulate(@NotNull final CharChunk asChars, - @NotNull final Context ctx, - final int runStart, - final int runLength) { - final WritableCharChunk localOutputValues = ctx.outputValues.get(); - for (int ii = runStart; ii < runStart + runLength; ii++) { - final char currentVal = asChars.get(ii); - if(currentVal != NULL_CHAR) { - ctx.curVal = currentVal; - } - localOutputValues.set(ii, ctx.curVal); + @Override + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + if(ctx.curVal == NULL_CHAR) { + ctx.curVal = ctx.charValueChunk.get(pos); } } + + // region extra-methods + // endregion extra-methods } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java index 760c784a396..9e9fb8ef8d8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java @@ -5,18 +5,13 @@ */ package io.deephaven.engine.table.impl.updateby.fill; -import io.deephaven.chunk.*; -import io.deephaven.chunk.attributes.ChunkLengths; -import io.deephaven.chunk.attributes.ChunkPositions; +import io.deephaven.chunk.DoubleChunk; +import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseDoubleUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.NULL_DOUBLE; @@ -24,6 +19,19 @@ public class DoubleFillByOperator extends BaseDoubleUpdateByOperator { // region extra-fields // endregion extra-fields + protected class Context extends BaseDoubleUpdateByOperator.Context { + public DoubleChunk doubleValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + doubleValueChunk = valuesChunk.asDoubleChunk(); + } + } + public DoubleFillByOperator(@NotNull final MatchPair fillPair, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args @@ -34,28 +42,21 @@ public DoubleFillByOperator(@NotNull final MatchPair fillPair, // endregion constructor } - // region extra-methods - // endregion extra-methods - + @NotNull @Override - protected void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { - accumulate(workingChunk.asDoubleChunk(), ctx, 0, workingChunk.size()); - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); } - private void accumulate(@NotNull final DoubleChunk asDoubles, - @NotNull final Context ctx, - final int runStart, - final int runLength) { - final WritableDoubleChunk localOutputValues = ctx.outputValues.get(); - for (int ii = runStart; ii < runStart + runLength; ii++) { - final double currentVal = asDoubles.get(ii); - if(currentVal != NULL_DOUBLE) { - ctx.curVal = currentVal; - } - localOutputValues.set(ii, ctx.curVal); + @Override + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + if(ctx.curVal == NULL_DOUBLE) { + ctx.curVal = ctx.doubleValueChunk.get(pos); } } + + // region extra-methods + // endregion extra-methods } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java index bdd0d9dd040..cfd2a329a14 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java @@ -5,18 +5,13 @@ */ package io.deephaven.engine.table.impl.updateby.fill; -import io.deephaven.chunk.*; -import io.deephaven.chunk.attributes.ChunkLengths; -import io.deephaven.chunk.attributes.ChunkPositions; +import io.deephaven.chunk.FloatChunk; +import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseFloatUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.NULL_FLOAT; @@ -24,6 +19,19 @@ public class FloatFillByOperator extends BaseFloatUpdateByOperator { // region extra-fields // endregion extra-fields + protected class Context extends BaseFloatUpdateByOperator.Context { + public FloatChunk floatValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + floatValueChunk = valuesChunk.asFloatChunk(); + } + } + public FloatFillByOperator(@NotNull final MatchPair fillPair, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args @@ -34,28 +42,21 @@ public FloatFillByOperator(@NotNull final MatchPair fillPair, // endregion constructor } - // region extra-methods - // endregion extra-methods - + @NotNull @Override - protected void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { - accumulate(workingChunk.asFloatChunk(), ctx, 0, workingChunk.size()); - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); } - private void accumulate(@NotNull final FloatChunk asFloats, - @NotNull final Context ctx, - final int runStart, - final int runLength) { - final WritableFloatChunk localOutputValues = ctx.outputValues.get(); - for (int ii = runStart; ii < runStart + runLength; ii++) { - final float currentVal = asFloats.get(ii); - if(currentVal != NULL_FLOAT) { - ctx.curVal = currentVal; - } - localOutputValues.set(ii, ctx.curVal); + @Override + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + if(ctx.curVal == NULL_FLOAT) { + ctx.curVal = ctx.floatValueChunk.get(pos); } } + + // region extra-methods + // endregion extra-methods } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java index 0f50f86223e..c0775271095 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java @@ -5,18 +5,13 @@ */ package io.deephaven.engine.table.impl.updateby.fill; -import io.deephaven.chunk.*; -import io.deephaven.chunk.attributes.ChunkLengths; -import io.deephaven.chunk.attributes.ChunkPositions; +import io.deephaven.chunk.IntChunk; +import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseIntUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.NULL_INT; @@ -24,6 +19,19 @@ public class IntFillByOperator extends BaseIntUpdateByOperator { // region extra-fields // endregion extra-fields + protected class Context extends BaseIntUpdateByOperator.Context { + public IntChunk intValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + intValueChunk = valuesChunk.asIntChunk(); + } + } + public IntFillByOperator(@NotNull final MatchPair fillPair, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args @@ -34,28 +42,21 @@ public IntFillByOperator(@NotNull final MatchPair fillPair, // endregion constructor } - // region extra-methods - // endregion extra-methods - + @NotNull @Override - protected void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { - accumulate(workingChunk.asIntChunk(), ctx, 0, workingChunk.size()); - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); } - private void accumulate(@NotNull final IntChunk asInts, - @NotNull final Context ctx, - final int runStart, - final int runLength) { - final WritableIntChunk localOutputValues = ctx.outputValues.get(); - for (int ii = runStart; ii < runStart + runLength; ii++) { - final int currentVal = asInts.get(ii); - if(currentVal != NULL_INT) { - ctx.curVal = currentVal; - } - localOutputValues.set(ii, ctx.curVal); + @Override + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + if(ctx.curVal == NULL_INT) { + ctx.curVal = ctx.intValueChunk.get(pos); } } + + // region extra-methods + // endregion extra-methods } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java index 4a6e60616fe..981e68c0b2d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java @@ -12,18 +12,13 @@ import java.time.Instant; import io.deephaven.engine.table.impl.sources.ReinterpretUtils; -import io.deephaven.chunk.*; -import io.deephaven.chunk.attributes.ChunkLengths; -import io.deephaven.chunk.attributes.ChunkPositions; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.NULL_LONG; @@ -32,6 +27,19 @@ public class LongFillByOperator extends BaseLongUpdateByOperator { private final Class type; // endregion extra-fields + protected class Context extends BaseLongUpdateByOperator.Context { + public LongChunk longValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + longValueChunk = valuesChunk.asLongChunk(); + } + } + public LongFillByOperator(@NotNull final MatchPair fillPair, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args @@ -44,6 +52,21 @@ public LongFillByOperator(@NotNull final MatchPair fillPair, // endregion constructor } + @NotNull + @Override + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); + } + + @Override + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + if(ctx.curVal == NULL_LONG) { + ctx.curVal = ctx.longValueChunk.get(pos); + } + } + // region extra-methods @NotNull @Override @@ -57,26 +80,4 @@ public Map> getOutputColumns() { return Collections.singletonMap(pair.leftColumn, actualOutput); } // endregion extra-methods - - @Override - protected void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { - accumulate(workingChunk.asLongChunk(), ctx, 0, workingChunk.size()); - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); - } - - private void accumulate(@NotNull final LongChunk asLongs, - @NotNull final Context ctx, - final int runStart, - final int runLength) { - final WritableLongChunk localOutputValues = ctx.outputValues.get(); - for (int ii = runStart; ii < runStart + runLength; ii++) { - final long currentVal = asLongs.get(ii); - if(currentVal != NULL_LONG) { - ctx.curVal = currentVal; - } - localOutputValues.set(ii, ctx.curVal); - } - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java index b35dd7e9a13..46a3790a237 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java @@ -5,24 +5,32 @@ */ package io.deephaven.engine.table.impl.updateby.fill; -import io.deephaven.chunk.*; -import io.deephaven.chunk.attributes.ChunkLengths; -import io.deephaven.chunk.attributes.ChunkPositions; +import io.deephaven.chunk.ObjectChunk; +import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; public class ObjectFillByOperator extends BaseObjectUpdateByOperator { // region extra-fields // endregion extra-fields + protected class Context extends BaseObjectUpdateByOperator.Context { + public ObjectChunk ObjectValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + ObjectValueChunk = valuesChunk.asObjectChunk(); + } + } + public ObjectFillByOperator(@NotNull final MatchPair fillPair, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args @@ -34,28 +42,21 @@ public ObjectFillByOperator(@NotNull final MatchPair fillPair, // endregion constructor } - // region extra-methods - // endregion extra-methods - + @NotNull @Override - protected void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { - accumulate(workingChunk.asObjectChunk(), ctx, 0, workingChunk.size()); - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); } - private void accumulate(@NotNull final ObjectChunk asObjects, - @NotNull final Context ctx, - final int runStart, - final int runLength) { - final WritableObjectChunk localOutputValues = ctx.outputValues.get(); - for (int ii = runStart; ii < runStart + runLength; ii++) { - final T currentVal = asObjects.get(ii); - if(currentVal != null) { - ctx.curVal = currentVal; - } - localOutputValues.set(ii, ctx.curVal); + @Override + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + if(ctx.curVal == null) { + ctx.curVal = ctx.ObjectValueChunk.get(pos); } } + + // region extra-methods + // endregion extra-methods } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java index 94b47dfe27d..2ef8be83796 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java @@ -5,18 +5,13 @@ */ package io.deephaven.engine.table.impl.updateby.fill; -import io.deephaven.chunk.*; -import io.deephaven.chunk.attributes.ChunkLengths; -import io.deephaven.chunk.attributes.ChunkPositions; +import io.deephaven.chunk.ShortChunk; +import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseShortUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.NULL_SHORT; @@ -24,6 +19,19 @@ public class ShortFillByOperator extends BaseShortUpdateByOperator { // region extra-fields // endregion extra-fields + protected class Context extends BaseShortUpdateByOperator.Context { + public ShortChunk shortValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + shortValueChunk = valuesChunk.asShortChunk(); + } + } + public ShortFillByOperator(@NotNull final MatchPair fillPair, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args @@ -34,28 +42,21 @@ public ShortFillByOperator(@NotNull final MatchPair fillPair, // endregion constructor } - // region extra-methods - // endregion extra-methods - + @NotNull @Override - protected void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { - accumulate(workingChunk.asShortChunk(), ctx, 0, workingChunk.size()); - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); } - private void accumulate(@NotNull final ShortChunk asShorts, - @NotNull final Context ctx, - final int runStart, - final int runLength) { - final WritableShortChunk localOutputValues = ctx.outputValues.get(); - for (int ii = runStart; ii < runStart + runLength; ii++) { - final short currentVal = asShorts.get(ii); - if(currentVal != NULL_SHORT) { - ctx.curVal = currentVal; - } - localOutputValues.set(ii, ctx.curVal); + @Override + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + if(ctx.curVal == NULL_SHORT) { + ctx.curVal = ctx.shortValueChunk.get(pos); } } + + // region extra-methods + // endregion extra-methods } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java index 6fbe777afad..745d414c7c2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java @@ -10,8 +10,10 @@ import io.deephaven.engine.table.impl.sources.ByteSparseArraySource; import io.deephaven.engine.table.WritableColumnSource; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.WritableByteChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.sized.SizedByteChunk; import io.deephaven.engine.rowset.*; @@ -20,9 +22,6 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; -import io.deephaven.engine.table.impl.util.SizedSafeCloseable; -import io.deephaven.util.QueryConstants; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -30,36 +29,33 @@ import java.util.Map; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; +import static io.deephaven.util.QueryConstants.*; public abstract class BaseByteUpdateByOperator extends UpdateByCumulativeOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; - protected final MatchPair pair; - protected final String[] affectingColumns; - - private UpdateBy.UpdateByRedirectionContext redirContext; - // region extra-fields final byte nullValue; // endregion extra-fields - protected class Context extends UpdateCumulativeContext { - public final SizedSafeCloseable fillContext; - public final SizedByteChunk outputValues; + protected class Context extends UpdateByCumulativeOperator.Context { + public final ChunkSink.FillFromContext fillContext; + public final WritableByteChunk outputValues; - public byte curVal = nullValue; + public byte curVal = NULL_BYTE; - protected Context(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { - this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); - this.fillContext.ensureCapacity(chunkSize); - this.outputValues = new SizedByteChunk<>(chunkSize); + protected Context(final int chunkSize) { + super(chunkSize); + this.fillContext = outputSource.makeFillFromContext(chunkSize); + this.outputValues = WritableByteChunk.makeWritableChunk(chunkSize); } + public void storeValuesChunk(@NotNull final Chunk valuesChunk) {} + @Override public void close() { super.close(); - outputValues.close(); fillContext.close(); } @@ -79,9 +75,7 @@ public BaseByteUpdateByOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - this.pair = pair; - this.affectingColumns = affectingColumns; - this.redirContext = redirContext; + super(pair, affectingColumns, redirContext); if(this.redirContext.isRedirected()) { // region create-dense this.maybeInnerSource = makeDenseSource(); @@ -99,6 +93,12 @@ public BaseByteUpdateByOperator(@NotNull final MatchPair pair, // endregion constructor } + @Override + public void reset(UpdateContext context) { + final Context ctx = (Context)context; + ctx.curVal = NULL_BYTE; + } + // region extra-methods protected byte getNullValue() { return QueryConstants.NULL_BYTE; @@ -115,39 +115,21 @@ protected WritableColumnSource makeDenseSource() { // endregion extra-methods @Override - public void setChunkSize(@NotNull final UpdateContext context, final int chunkSize) { - ((Context)context).outputValues.ensureCapacity(chunkSize); - ((Context)context).fillContext.ensureCapacity(chunkSize); - } - - @NotNull - @Override - public String getInputColumnName() { - return pair.rightColumn; - } - - @NotNull - @Override - public String[] getAffectingColumnNames() { - return affectingColumns; - } - - @NotNull - @Override - public String[] getOutputColumnNames() { - return new String[] { pair.leftColumn }; - } - - @NotNull - @Override - public Map> getOutputColumns() { - return Collections.singletonMap(pair.leftColumn, outputSource); - } + public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifiedKey, long firstUnmodifiedTimestamp) { + Context ctx = (Context) context; + if (firstUnmodifiedKey != NULL_ROW_KEY) { + ctx.curVal = outputSource.getByte(firstUnmodifiedKey); + } else { + reset(ctx); + } - @NotNull - @Override - public UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { - return new Context(chunkSize, timestampSsa); + // If we're redirected we have to make sure we tell the output source it's actual size, or we're going + // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not + // need to do anything with capacity. + if(redirContext.isRedirected()) { + // The redirection index does not use the 0th index for some reason. + outputSource.ensureCapacity(redirContext.requiredCapacity()); + } } @Override @@ -158,19 +140,6 @@ public void startTrackingPrev() { } } - /** - * Perform the processing for a chunk of values by the operator. - * - * @param ctx the context object - * @param inputKeys the input keys for the chunk - * @param workingChunk the chunk of values - */ - protected abstract void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk); - - // endregion - // region Shifts @Override public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { @@ -182,35 +151,28 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d } // endregion Shifts - // region Reprocessing - - public void resetForProcess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceIndex, - long firstUnmodifiedKey) { - - // If we're redirected we have to make sure we tell the output source it's actual size, or we're going - // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not - // need to do anything with capacity. - if(redirContext.isRedirected()) { - // The redirection index does not use the 0th index for some reason. - outputSource.ensureCapacity(redirContext.requiredCapacity()); - } - - final Context ctx = (Context) context; - ctx.curVal = firstUnmodifiedKey == NULL_ROW_KEY ? nullValue : outputSource.getByte(firstUnmodifiedKey); - } - + // region Processing @Override public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, @Nullable final LongChunk posChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex) { + @Nullable final Chunk valuesChunk, + @Nullable final LongChunk timestampValuesChunk) { + Assert.neqNull(valuesChunk, "valuesChunk must not be null for a cumulative operator"); final Context ctx = (Context) updateContext; - doProcessChunk(ctx, inputKeys, valuesChunk); - ctx.getModifiedBuilder().appendRowSequence(inputKeys); + ctx.storeValuesChunk(valuesChunk); + for (int ii = 0; ii < valuesChunk.size(); ii++) { + push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); + ctx.outputValues.set(ii, ctx.curVal); + } + outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues, inputKeys); } - // endregion + + @NotNull + @Override + public Map> getOutputColumns() { + return Collections.singletonMap(pair.leftColumn, outputSource); + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java index 817a91f2359..cb2af8012c3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java @@ -1,7 +1,9 @@ package io.deephaven.engine.table.impl.updateby.internal; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.WritableCharChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.sized.SizedCharChunk; import io.deephaven.engine.rowset.*; @@ -10,9 +12,6 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; -import io.deephaven.engine.table.impl.util.SizedSafeCloseable; -import io.deephaven.util.QueryConstants; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -20,35 +19,32 @@ import java.util.Map; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; +import static io.deephaven.util.QueryConstants.*; public abstract class BaseCharUpdateByOperator extends UpdateByCumulativeOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; - protected final MatchPair pair; - protected final String[] affectingColumns; - - private UpdateBy.UpdateByRedirectionContext redirContext; - // region extra-fields // endregion extra-fields - protected class Context extends UpdateCumulativeContext { - public final SizedSafeCloseable fillContext; - public final SizedCharChunk outputValues; + protected class Context extends UpdateByCumulativeOperator.Context { + public final ChunkSink.FillFromContext fillContext; + public final WritableCharChunk outputValues; - public char curVal = QueryConstants.NULL_CHAR; + public char curVal = NULL_CHAR; - protected Context(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { - this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); - this.fillContext.ensureCapacity(chunkSize); - this.outputValues = new SizedCharChunk<>(chunkSize); + protected Context(final int chunkSize) { + super(chunkSize); + this.fillContext = outputSource.makeFillFromContext(chunkSize); + this.outputValues = WritableCharChunk.makeWritableChunk(chunkSize); } + public void storeValuesChunk(@NotNull final Chunk valuesChunk) {} + @Override public void close() { super.close(); - outputValues.close(); fillContext.close(); } @@ -68,9 +64,7 @@ public BaseCharUpdateByOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - this.pair = pair; - this.affectingColumns = affectingColumns; - this.redirContext = redirContext; + super(pair, affectingColumns, redirContext); if(this.redirContext.isRedirected()) { // region create-dense this.maybeInnerSource = new CharacterArraySource(); @@ -87,43 +81,31 @@ public BaseCharUpdateByOperator(@NotNull final MatchPair pair, // endregion constructor } - // region extra-methods - // endregion extra-methods - - @Override - public void setChunkSize(@NotNull final UpdateContext context, final int chunkSize) { - ((Context)context).outputValues.ensureCapacity(chunkSize); - ((Context)context).fillContext.ensureCapacity(chunkSize); - } - - @NotNull - @Override - public String getInputColumnName() { - return pair.rightColumn; - } - - @NotNull @Override - public String[] getAffectingColumnNames() { - return affectingColumns; + public void reset(UpdateContext context) { + final Context ctx = (Context)context; + ctx.curVal = NULL_CHAR; } - @NotNull - @Override - public String[] getOutputColumnNames() { - return new String[] { pair.leftColumn }; - } + // region extra-methods + // endregion extra-methods - @NotNull @Override - public Map> getOutputColumns() { - return Collections.singletonMap(pair.leftColumn, outputSource); - } + public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifiedKey, long firstUnmodifiedTimestamp) { + Context ctx = (Context) context; + if (firstUnmodifiedKey != NULL_ROW_KEY) { + ctx.curVal = outputSource.getChar(firstUnmodifiedKey); + } else { + reset(ctx); + } - @NotNull - @Override - public UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { - return new Context(chunkSize, timestampSsa); + // If we're redirected we have to make sure we tell the output source it's actual size, or we're going + // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not + // need to do anything with capacity. + if(redirContext.isRedirected()) { + // The redirection index does not use the 0th index for some reason. + outputSource.ensureCapacity(redirContext.requiredCapacity()); + } } @Override @@ -134,57 +116,35 @@ public void startTrackingPrev() { } } - /** - * Perform the processing for a chunk of values by the operator. - * - * @param ctx the context object - * @param inputKeys the input keys for the chunk - * @param workingChunk the chunk of values - */ - protected abstract void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk); - - // endregion - // region Shifts - @Override public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { ((CharacterSparseArraySource)outputSource).shift(subIndexToShift, delta); } - // endregion Shifts - // region Reprocessing - - public void resetForProcess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceIndex, - long firstUnmodifiedKey) { - - // If we're redirected we have to make sure we tell the output source it's actual size, or we're going - // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not - // need to do anything with capacity. - if(redirContext.isRedirected()) { - // The redirection index does not use the 0th index for some reason. - outputSource.ensureCapacity(redirContext.requiredCapacity()); - } - - final Context ctx = (Context) context; - ctx.curVal = firstUnmodifiedKey == NULL_ROW_KEY ? QueryConstants.NULL_CHAR : outputSource.getChar(firstUnmodifiedKey); - } - + // region Processing @Override public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, @Nullable final LongChunk posChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex) { + @Nullable final Chunk valuesChunk, + @Nullable final LongChunk timestampValuesChunk) { + Assert.neqNull(valuesChunk, "valuesChunk must not be null for a cumulative operator"); final Context ctx = (Context) updateContext; - doProcessChunk(ctx, inputKeys, valuesChunk); - ctx.getModifiedBuilder().appendRowSequence(inputKeys); + ctx.storeValuesChunk(valuesChunk); + for (int ii = 0; ii < valuesChunk.size(); ii++) { + push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); + ctx.outputValues.set(ii, ctx.curVal); + } + outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues, inputKeys); } - // endregion + + @NotNull + @Override + public Map> getOutputColumns() { + return Collections.singletonMap(pair.leftColumn, outputSource); + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java index 62d723cc54b..04944ed4469 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java @@ -5,8 +5,11 @@ */ package io.deephaven.engine.table.impl.updateby.internal; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.WritableCharChunk; +import io.deephaven.chunk.WritableDoubleChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.sized.SizedDoubleChunk; import io.deephaven.engine.rowset.*; @@ -14,11 +17,7 @@ import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; -import io.deephaven.engine.table.impl.sources.DoubleArraySource; -import io.deephaven.engine.table.impl.sources.DoubleSparseArraySource; -import io.deephaven.engine.table.impl.sources.WritableRedirectedColumnSource; -import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; -import io.deephaven.engine.table.impl.util.SizedSafeCloseable; +import io.deephaven.engine.table.impl.sources.*; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -26,44 +25,33 @@ import java.util.Map; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; +import static io.deephaven.util.QueryConstants.NULL_CHAR; import static io.deephaven.util.QueryConstants.NULL_DOUBLE; public abstract class BaseDoubleUpdateByOperator extends UpdateByCumulativeOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; - private final MatchPair pair; - private final String[] affectingColumns; + // region extra-fields + // endregion extra-fields - private UpdateBy.UpdateByRedirectionContext redirContext; + protected class Context extends UpdateByCumulativeOperator.Context { + public final ChunkSink.FillFromContext fillContext; + public final WritableDoubleChunk outputValues; - protected class Context extends UpdateCumulativeContext { - public final SizedSafeCloseable fillContext; - public final SizedDoubleChunk outputValues; + public double curVal = NULL_CHAR; - public RowSetBuilderSequential modifiedBuilder; - public RowSet newModified; - - public double curVal = NULL_DOUBLE; - - public boolean filledWithPermanentValue = false; - - protected Context(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { - this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); - this.fillContext.ensureCapacity(chunkSize); - this.outputValues = new SizedDoubleChunk<>(chunkSize); - this.timestampSsa = timestampSsa; + protected Context(final int chunkSize) { + super(chunkSize); + this.fillContext = outputSource.makeFillFromContext(chunkSize); + this.outputValues = WritableDoubleChunk.makeWritableChunk(chunkSize); } - public RowSetBuilderSequential getModifiedBuilder() { - if(modifiedBuilder == null) { - modifiedBuilder = RowSetFactory.builderSequential(); - } - return modifiedBuilder; - } + public void storeValuesChunk(@NotNull final Chunk valuesChunk) {} @Override public void close() { + super.close(); outputValues.close(); fillContext.close(); } @@ -79,53 +67,52 @@ public void close() { */ public BaseDoubleUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { - this.pair = pair; - this.affectingColumns = affectingColumns; - this.redirContext = redirContext; + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + // region extra-constructor-args + // endregion extra-constructor-args + ) { + super(pair, affectingColumns, redirContext); if(this.redirContext.isRedirected()) { + // region create-dense this.maybeInnerSource = new DoubleArraySource(); + // endregion create-dense this.outputSource = new WritableRedirectedColumnSource(this.redirContext.getRowRedirection(), maybeInnerSource, 0); } else { this.maybeInnerSource = null; + // region create-sparse this.outputSource = new DoubleSparseArraySource(); + // endregion create-sparse } - } - @Override - public void setChunkSize(@NotNull final UpdateContext context, final int chunkSize) { - ((Context)context).outputValues.ensureCapacity(chunkSize); - ((Context)context).fillContext.ensureCapacity(chunkSize); + // region constructor + // endregion constructor } - @NotNull @Override - public String getInputColumnName() { - return pair.rightColumn; + public void reset(UpdateContext context) { + final Context ctx = (Context)context; + ctx.curVal = NULL_DOUBLE; } - @NotNull - @Override - public String[] getAffectingColumnNames() { - return affectingColumns; - } + // region extra-methods + // endregion extra-methods - @NotNull @Override - public String[] getOutputColumnNames() { - return new String[] { pair.leftColumn }; - } - - @NotNull - @Override - public Map> getOutputColumns() { - return Collections.singletonMap(pair.leftColumn, outputSource); - } + public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifiedKey, long firstUnmodifiedTimestamp) { + Context ctx = (Context) context; + if (firstUnmodifiedKey != NULL_ROW_KEY) { + ctx.curVal = outputSource.getDouble(firstUnmodifiedKey); + } else { + reset(ctx); + } - @NotNull - @Override - public UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { - return new Context(chunkSize, timestampSsa); + // If we're redirected we have to make sure we tell the output source it's actual size, or we're going + // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not + // need to do anything with capacity. + if(redirContext.isRedirected()) { + // The redirection index does not use the 0th index for some reason. + outputSource.ensureCapacity(redirContext.requiredCapacity()); + } } @Override @@ -136,47 +123,35 @@ public void startTrackingPrev() { } } - protected abstract void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk); - // endregion - // region Shifts - @Override public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long delta) { ((DoubleSparseArraySource)outputSource).shift(subRowSetToShift, delta); } - // endregion - // region Reprocessing - - public void resetForProcess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceRowSet, - final long firstUnmodifiedKey) { - final Context ctx = (Context) context; - - // If we're redirected we have to make sure we tell the output source it's actual size, or we're going - // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not - // need to do anything with capacity. - if(redirContext.isRedirected()) { - outputSource.ensureCapacity(redirContext.requiredCapacity()); - } - - ctx.curVal = firstUnmodifiedKey == NULL_ROW_KEY ? NULL_DOUBLE : outputSource.getDouble(firstUnmodifiedKey); - } - + // region Processing @Override public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, @Nullable final LongChunk posChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceRowSet) { + @Nullable final Chunk valuesChunk, + @Nullable final LongChunk timestampValuesChunk) { + Assert.neqNull(valuesChunk, "valuesChunk must not be null for a cumulative operator"); final Context ctx = (Context) updateContext; - doProcessChunk(ctx, inputKeys, valuesChunk); - ctx.getModifiedBuilder().appendRowSequence(inputKeys); + ctx.storeValuesChunk(valuesChunk); + for (int ii = 0; ii < valuesChunk.size(); ii++) { + push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); + ctx.outputValues.set(ii, ctx.curVal); + } + outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues, inputKeys); } // endregion + + @NotNull + @Override + public Map> getOutputColumns() { + return Collections.singletonMap(pair.leftColumn, outputSource); + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java index d5cbd89809a..7db73c690dd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java @@ -1,7 +1,10 @@ package io.deephaven.engine.table.impl.updateby.internal; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.WritableCharChunk; +import io.deephaven.chunk.WritableFloatChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.sized.SizedFloatChunk; import io.deephaven.engine.rowset.*; @@ -9,11 +12,7 @@ import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; -import io.deephaven.engine.table.impl.sources.FloatArraySource; -import io.deephaven.engine.table.impl.sources.FloatSparseArraySource; -import io.deephaven.engine.table.impl.sources.WritableRedirectedColumnSource; -import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; -import io.deephaven.engine.table.impl.util.SizedSafeCloseable; +import io.deephaven.engine.table.impl.sources.*; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -21,44 +20,33 @@ import java.util.Map; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; +import static io.deephaven.util.QueryConstants.NULL_CHAR; import static io.deephaven.util.QueryConstants.NULL_FLOAT; public abstract class BaseFloatUpdateByOperator extends UpdateByCumulativeOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; - private final MatchPair pair; - private final String[] affectingColumns; + // region extra-fields + // endregion extra-fields - private UpdateBy.UpdateByRedirectionContext redirContext; + protected class Context extends UpdateByCumulativeOperator.Context { + public final ChunkSink.FillFromContext fillContext; + public final WritableFloatChunk outputValues; - protected class Context extends UpdateCumulativeContext { - public final SizedSafeCloseable fillContext; - public final SizedFloatChunk outputValues; + public float curVal = NULL_CHAR; - public RowSetBuilderSequential modifiedBuilder; - public RowSet newModified; - - public float curVal = NULL_FLOAT; - - public boolean filledWithPermanentValue = false; - - protected Context(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { - this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); - this.fillContext.ensureCapacity(chunkSize); - this.outputValues = new SizedFloatChunk<>(chunkSize); - this.timestampSsa = timestampSsa; + protected Context(final int chunkSize) { + super(chunkSize); + this.fillContext = outputSource.makeFillFromContext(chunkSize); + this.outputValues = WritableFloatChunk.makeWritableChunk(chunkSize); } - public RowSetBuilderSequential getModifiedBuilder() { - if(modifiedBuilder == null) { - modifiedBuilder = RowSetFactory.builderSequential(); - } - return modifiedBuilder; - } + public void storeValuesChunk(@NotNull final Chunk valuesChunk) {} @Override public void close() { + super.close(); outputValues.close(); fillContext.close(); } @@ -74,53 +62,52 @@ public void close() { */ public BaseFloatUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { - this.pair = pair; - this.affectingColumns = affectingColumns; - this.redirContext = redirContext; + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + // region extra-constructor-args + // endregion extra-constructor-args + ) { + super(pair, affectingColumns, redirContext); if(this.redirContext.isRedirected()) { + // region create-dense this.maybeInnerSource = new FloatArraySource(); + // endregion create-dense this.outputSource = new WritableRedirectedColumnSource(this.redirContext.getRowRedirection(), maybeInnerSource, 0); } else { this.maybeInnerSource = null; + // region create-sparse this.outputSource = new FloatSparseArraySource(); + // endregion create-sparse } - } - @Override - public void setChunkSize(@NotNull final UpdateContext context, final int chunkSize) { - ((Context)context).outputValues.ensureCapacity(chunkSize); - ((Context)context).fillContext.ensureCapacity(chunkSize); + // region constructor + // endregion constructor } - @NotNull @Override - public String getInputColumnName() { - return pair.rightColumn; + public void reset(UpdateContext context) { + final Context ctx = (Context)context; + ctx.curVal = NULL_FLOAT; } - @NotNull - @Override - public String[] getAffectingColumnNames() { - return affectingColumns; - } + // region extra-methods + // endregion extra-methods - @NotNull @Override - public String[] getOutputColumnNames() { - return new String[] { pair.leftColumn }; - } - - @NotNull - @Override - public Map> getOutputColumns() { - return Collections.singletonMap(pair.leftColumn, outputSource); - } + public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifiedKey, long firstUnmodifiedTimestamp) { + Context ctx = (Context) context; + if (firstUnmodifiedKey != NULL_ROW_KEY) { + ctx.curVal = outputSource.getFloat(firstUnmodifiedKey); + } else { + reset(ctx); + } - @NotNull - @Override - public UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { - return new Context(chunkSize, timestampSsa); + // If we're redirected we have to make sure we tell the output source it's actual size, or we're going + // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not + // need to do anything with capacity. + if(redirContext.isRedirected()) { + // The redirection index does not use the 0th index for some reason. + outputSource.ensureCapacity(redirContext.requiredCapacity()); + } } @Override @@ -131,47 +118,35 @@ public void startTrackingPrev() { } } - protected abstract void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk); - // endregion - // region Shifts - @Override public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long delta) { ((FloatSparseArraySource)outputSource).shift(subRowSetToShift, delta); } - // endregion - // region Reprocessing - - public void resetForProcess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceRowSet, - final long firstUnmodifiedKey) { - final Context ctx = (Context) context; - - // If we're redirected we have to make sure we tell the output source it's actual size, or we're going - // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not - // need to do anything with capacity. - if(redirContext.isRedirected()) { - outputSource.ensureCapacity(redirContext.requiredCapacity()); - } - - ctx.curVal = firstUnmodifiedKey == NULL_ROW_KEY ? NULL_FLOAT : outputSource.getFloat(firstUnmodifiedKey); - } - + // region Processing @Override public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, @Nullable final LongChunk posChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceRowSet) { + @Nullable final Chunk valuesChunk, + @Nullable final LongChunk timestampValuesChunk) { + Assert.neqNull(valuesChunk, "valuesChunk must not be null for a cumulative operator"); final Context ctx = (Context) updateContext; - doProcessChunk(ctx, inputKeys, valuesChunk); - ctx.getModifiedBuilder().appendRowSequence(inputKeys); + ctx.storeValuesChunk(valuesChunk); + for (int ii = 0; ii < valuesChunk.size(); ii++) { + push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); + ctx.outputValues.set(ii, ctx.curVal); + } + outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues, inputKeys); } // endregion + + @NotNull + @Override + public Map> getOutputColumns() { + return Collections.singletonMap(pair.leftColumn, outputSource); + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java index 012691a9f09..14f47729480 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java @@ -5,8 +5,10 @@ */ package io.deephaven.engine.table.impl.updateby.internal; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.sized.SizedIntChunk; import io.deephaven.engine.rowset.*; @@ -15,9 +17,6 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; -import io.deephaven.engine.table.impl.util.SizedSafeCloseable; -import io.deephaven.util.QueryConstants; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -25,35 +24,32 @@ import java.util.Map; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; +import static io.deephaven.util.QueryConstants.*; public abstract class BaseIntUpdateByOperator extends UpdateByCumulativeOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; - protected final MatchPair pair; - protected final String[] affectingColumns; - - private UpdateBy.UpdateByRedirectionContext redirContext; - // region extra-fields // endregion extra-fields - protected class Context extends UpdateCumulativeContext { - public final SizedSafeCloseable fillContext; - public final SizedIntChunk outputValues; + protected class Context extends UpdateByCumulativeOperator.Context { + public final ChunkSink.FillFromContext fillContext; + public final WritableIntChunk outputValues; - public int curVal = QueryConstants.NULL_INT; + public int curVal = NULL_INT; - protected Context(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { - this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); - this.fillContext.ensureCapacity(chunkSize); - this.outputValues = new SizedIntChunk<>(chunkSize); + protected Context(final int chunkSize) { + super(chunkSize); + this.fillContext = outputSource.makeFillFromContext(chunkSize); + this.outputValues = WritableIntChunk.makeWritableChunk(chunkSize); } + public void storeValuesChunk(@NotNull final Chunk valuesChunk) {} + @Override public void close() { super.close(); - outputValues.close(); fillContext.close(); } @@ -73,9 +69,7 @@ public BaseIntUpdateByOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - this.pair = pair; - this.affectingColumns = affectingColumns; - this.redirContext = redirContext; + super(pair, affectingColumns, redirContext); if(this.redirContext.isRedirected()) { // region create-dense this.maybeInnerSource = new IntegerArraySource(); @@ -92,43 +86,31 @@ public BaseIntUpdateByOperator(@NotNull final MatchPair pair, // endregion constructor } - // region extra-methods - // endregion extra-methods - - @Override - public void setChunkSize(@NotNull final UpdateContext context, final int chunkSize) { - ((Context)context).outputValues.ensureCapacity(chunkSize); - ((Context)context).fillContext.ensureCapacity(chunkSize); - } - - @NotNull - @Override - public String getInputColumnName() { - return pair.rightColumn; - } - - @NotNull @Override - public String[] getAffectingColumnNames() { - return affectingColumns; + public void reset(UpdateContext context) { + final Context ctx = (Context)context; + ctx.curVal = NULL_INT; } - @NotNull - @Override - public String[] getOutputColumnNames() { - return new String[] { pair.leftColumn }; - } + // region extra-methods + // endregion extra-methods - @NotNull @Override - public Map> getOutputColumns() { - return Collections.singletonMap(pair.leftColumn, outputSource); - } + public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifiedKey, long firstUnmodifiedTimestamp) { + Context ctx = (Context) context; + if (firstUnmodifiedKey != NULL_ROW_KEY) { + ctx.curVal = outputSource.getInt(firstUnmodifiedKey); + } else { + reset(ctx); + } - @NotNull - @Override - public UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { - return new Context(chunkSize, timestampSsa); + // If we're redirected we have to make sure we tell the output source it's actual size, or we're going + // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not + // need to do anything with capacity. + if(redirContext.isRedirected()) { + // The redirection index does not use the 0th index for some reason. + outputSource.ensureCapacity(redirContext.requiredCapacity()); + } } @Override @@ -139,57 +121,35 @@ public void startTrackingPrev() { } } - /** - * Perform the processing for a chunk of values by the operator. - * - * @param ctx the context object - * @param inputKeys the input keys for the chunk - * @param workingChunk the chunk of values - */ - protected abstract void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk); - - // endregion - // region Shifts - @Override public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { ((IntegerSparseArraySource)outputSource).shift(subIndexToShift, delta); } - // endregion Shifts - // region Reprocessing - - public void resetForProcess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceIndex, - long firstUnmodifiedKey) { - - // If we're redirected we have to make sure we tell the output source it's actual size, or we're going - // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not - // need to do anything with capacity. - if(redirContext.isRedirected()) { - // The redirection index does not use the 0th index for some reason. - outputSource.ensureCapacity(redirContext.requiredCapacity()); - } - - final Context ctx = (Context) context; - ctx.curVal = firstUnmodifiedKey == NULL_ROW_KEY ? QueryConstants.NULL_INT : outputSource.getInt(firstUnmodifiedKey); - } - + // region Processing @Override public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, @Nullable final LongChunk posChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex) { + @Nullable final Chunk valuesChunk, + @Nullable final LongChunk timestampValuesChunk) { + Assert.neqNull(valuesChunk, "valuesChunk must not be null for a cumulative operator"); final Context ctx = (Context) updateContext; - doProcessChunk(ctx, inputKeys, valuesChunk); - ctx.getModifiedBuilder().appendRowSequence(inputKeys); + ctx.storeValuesChunk(valuesChunk); + for (int ii = 0; ii < valuesChunk.size(); ii++) { + push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); + ctx.outputValues.set(ii, ctx.curVal); + } + outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues, inputKeys); } - // endregion + + @NotNull + @Override + public Map> getOutputColumns() { + return Collections.singletonMap(pair.leftColumn, outputSource); + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java index 3706d57e9f7..fc20a35e843 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java @@ -5,8 +5,10 @@ */ package io.deephaven.engine.table.impl.updateby.internal; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.WritableLongChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.sized.SizedLongChunk; import io.deephaven.engine.rowset.*; @@ -15,9 +17,6 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; -import io.deephaven.engine.table.impl.util.SizedSafeCloseable; -import io.deephaven.util.QueryConstants; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -25,35 +24,32 @@ import java.util.Map; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; +import static io.deephaven.util.QueryConstants.*; public abstract class BaseLongUpdateByOperator extends UpdateByCumulativeOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; - protected final MatchPair pair; - protected final String[] affectingColumns; - - private UpdateBy.UpdateByRedirectionContext redirContext; - // region extra-fields // endregion extra-fields - protected class Context extends UpdateCumulativeContext { - public final SizedSafeCloseable fillContext; - public final SizedLongChunk outputValues; + protected class Context extends UpdateByCumulativeOperator.Context { + public final ChunkSink.FillFromContext fillContext; + public final WritableLongChunk outputValues; - public long curVal = QueryConstants.NULL_LONG; + public long curVal = NULL_LONG; - protected Context(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { - this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); - this.fillContext.ensureCapacity(chunkSize); - this.outputValues = new SizedLongChunk<>(chunkSize); + protected Context(final int chunkSize) { + super(chunkSize); + this.fillContext = outputSource.makeFillFromContext(chunkSize); + this.outputValues = WritableLongChunk.makeWritableChunk(chunkSize); } + public void storeValuesChunk(@NotNull final Chunk valuesChunk) {} + @Override public void close() { super.close(); - outputValues.close(); fillContext.close(); } @@ -73,9 +69,7 @@ public BaseLongUpdateByOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - this.pair = pair; - this.affectingColumns = affectingColumns; - this.redirContext = redirContext; + super(pair, affectingColumns, redirContext); if(this.redirContext.isRedirected()) { // region create-dense this.maybeInnerSource = new LongArraySource(); @@ -92,43 +86,31 @@ public BaseLongUpdateByOperator(@NotNull final MatchPair pair, // endregion constructor } - // region extra-methods - // endregion extra-methods - - @Override - public void setChunkSize(@NotNull final UpdateContext context, final int chunkSize) { - ((Context)context).outputValues.ensureCapacity(chunkSize); - ((Context)context).fillContext.ensureCapacity(chunkSize); - } - - @NotNull - @Override - public String getInputColumnName() { - return pair.rightColumn; - } - - @NotNull @Override - public String[] getAffectingColumnNames() { - return affectingColumns; + public void reset(UpdateContext context) { + final Context ctx = (Context)context; + ctx.curVal = NULL_LONG; } - @NotNull - @Override - public String[] getOutputColumnNames() { - return new String[] { pair.leftColumn }; - } + // region extra-methods + // endregion extra-methods - @NotNull @Override - public Map> getOutputColumns() { - return Collections.singletonMap(pair.leftColumn, outputSource); - } + public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifiedKey, long firstUnmodifiedTimestamp) { + Context ctx = (Context) context; + if (firstUnmodifiedKey != NULL_ROW_KEY) { + ctx.curVal = outputSource.getLong(firstUnmodifiedKey); + } else { + reset(ctx); + } - @NotNull - @Override - public UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { - return new Context(chunkSize, timestampSsa); + // If we're redirected we have to make sure we tell the output source it's actual size, or we're going + // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not + // need to do anything with capacity. + if(redirContext.isRedirected()) { + // The redirection index does not use the 0th index for some reason. + outputSource.ensureCapacity(redirContext.requiredCapacity()); + } } @Override @@ -139,57 +121,35 @@ public void startTrackingPrev() { } } - /** - * Perform the processing for a chunk of values by the operator. - * - * @param ctx the context object - * @param inputKeys the input keys for the chunk - * @param workingChunk the chunk of values - */ - protected abstract void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk); - - // endregion - // region Shifts - @Override public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { ((LongSparseArraySource)outputSource).shift(subIndexToShift, delta); } - // endregion Shifts - // region Reprocessing - - public void resetForProcess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceIndex, - long firstUnmodifiedKey) { - - // If we're redirected we have to make sure we tell the output source it's actual size, or we're going - // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not - // need to do anything with capacity. - if(redirContext.isRedirected()) { - // The redirection index does not use the 0th index for some reason. - outputSource.ensureCapacity(redirContext.requiredCapacity()); - } - - final Context ctx = (Context) context; - ctx.curVal = firstUnmodifiedKey == NULL_ROW_KEY ? QueryConstants.NULL_LONG : outputSource.getLong(firstUnmodifiedKey); - } - + // region Processing @Override public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, @Nullable final LongChunk posChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex) { + @Nullable final Chunk valuesChunk, + @Nullable final LongChunk timestampValuesChunk) { + Assert.neqNull(valuesChunk, "valuesChunk must not be null for a cumulative operator"); final Context ctx = (Context) updateContext; - doProcessChunk(ctx, inputKeys, valuesChunk); - ctx.getModifiedBuilder().appendRowSequence(inputKeys); + ctx.storeValuesChunk(valuesChunk); + for (int ii = 0; ii < valuesChunk.size(); ii++) { + push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); + ctx.outputValues.set(ii, ctx.curVal); + } + outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues, inputKeys); } - // endregion + + @NotNull + @Override + public Map> getOutputColumns() { + return Collections.singletonMap(pair.leftColumn, outputSource); + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java index d346892a370..d26108c5f68 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java @@ -1,18 +1,33 @@ package io.deephaven.engine.table.impl.updateby.internal; -import io.deephaven.chunk.*; -import io.deephaven.chunk.attributes.ChunkLengths; -import io.deephaven.chunk.attributes.ChunkPositions; +import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; + public abstract class BaseObjectBinaryOperator extends BaseObjectUpdateByOperator { + protected class Context extends BaseObjectUpdateByOperator.Context { + public ObjectChunk objectValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + objectValueChunk = valuesChunk.asObjectChunk(); + } + } + public BaseObjectBinaryOperator(@NotNull final Class type, @NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @@ -22,32 +37,41 @@ public BaseObjectBinaryOperator(@NotNull final Class type, protected abstract T doOperation(T bucketCurVal, T chunkCurVal); - // region Addition - - + @NotNull @Override - protected void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { - accumulate(workingChunk.asObjectChunk(), ctx, 0, inputKeys.intSize()); - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); } + // region Processing + @Override + public void processChunk(@NotNull final UpdateContext updateContext, + @NotNull final RowSequence inputKeys, + @Nullable final LongChunk keyChunk, + @Nullable final LongChunk posChunk, + @Nullable final Chunk valuesChunk, + @Nullable final LongChunk timestampValuesChunk) { + Assert.neqNull(valuesChunk, "valuesChunk must not be null for a cumulative operator"); + final Context ctx = (Context) updateContext; + ctx.storeValuesChunk(valuesChunk); + for (int ii = 0; ii < valuesChunk.size(); ii++) { + push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); + ctx.outputValues.set(ii, ctx.curVal); + } + outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues, inputKeys); + } // endregion - private void accumulate(@NotNull final ObjectChunk asObject, - @NotNull final Context ctx, - final int runStart, - final int runLength) { - final WritableObjectChunk localOutputValues = ctx.outputValues.get(); - for (int ii = runStart; ii < runStart + runLength; ii++) { - final T currentVal = asObject.get(ii); - if(ctx.curVal == null) { - ctx.curVal = currentVal; - } else if(currentVal != null) { - ctx.curVal = doOperation(ctx.curVal, currentVal); - } - localOutputValues.set(ii, ctx.curVal); + @Override + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + // read the value from the values chunk + final T currentVal = ctx.objectValueChunk.get(pos); + if(ctx.curVal == null) { + ctx.curVal = currentVal; + } else if(currentVal != null) { + ctx.curVal = doOperation(ctx.curVal, currentVal); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java index 45991523d0e..47c8affde87 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java @@ -5,8 +5,10 @@ */ package io.deephaven.engine.table.impl.updateby.internal; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.sized.SizedObjectChunk; import io.deephaven.engine.rowset.*; @@ -15,9 +17,6 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; -import io.deephaven.engine.table.impl.util.SizedSafeCloseable; -import io.deephaven.util.QueryConstants; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -25,36 +24,33 @@ import java.util.Map; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; +import static io.deephaven.util.QueryConstants.*; public abstract class BaseObjectUpdateByOperator extends UpdateByCumulativeOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; - protected final MatchPair pair; - protected final String[] affectingColumns; - - private UpdateBy.UpdateByRedirectionContext redirContext; - // region extra-fields private final Class colType; // endregion extra-fields - protected class Context extends UpdateCumulativeContext { - public final SizedSafeCloseable fillContext; - public final SizedObjectChunk outputValues; + protected class Context extends UpdateByCumulativeOperator.Context { + public final ChunkSink.FillFromContext fillContext; + public final WritableObjectChunk outputValues; public T curVal = null; - protected Context(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { - this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); - this.fillContext.ensureCapacity(chunkSize); - this.outputValues = new SizedObjectChunk<>(chunkSize); + protected Context(final int chunkSize) { + super(chunkSize); + this.fillContext = outputSource.makeFillFromContext(chunkSize); + this.outputValues = WritableObjectChunk.makeWritableChunk(chunkSize); } + public void storeValuesChunk(@NotNull final Chunk valuesChunk) {} + @Override public void close() { super.close(); - outputValues.close(); fillContext.close(); } @@ -75,9 +71,7 @@ public BaseObjectUpdateByOperator(@NotNull final MatchPair pair, , final Class colType // endregion extra-constructor-args ) { - this.pair = pair; - this.affectingColumns = affectingColumns; - this.redirContext = redirContext; + super(pair, affectingColumns, redirContext); if(this.redirContext.isRedirected()) { // region create-dense this.maybeInnerSource = new ObjectArraySource(colType); @@ -95,43 +89,31 @@ public BaseObjectUpdateByOperator(@NotNull final MatchPair pair, // endregion constructor } - // region extra-methods - // endregion extra-methods - - @Override - public void setChunkSize(@NotNull final UpdateContext context, final int chunkSize) { - ((Context)context).outputValues.ensureCapacity(chunkSize); - ((Context)context).fillContext.ensureCapacity(chunkSize); - } - - @NotNull - @Override - public String getInputColumnName() { - return pair.rightColumn; - } - - @NotNull @Override - public String[] getAffectingColumnNames() { - return affectingColumns; + public void reset(UpdateContext context) { + final Context ctx = (Context)context; + ctx.curVal = null; } - @NotNull - @Override - public String[] getOutputColumnNames() { - return new String[] { pair.leftColumn }; - } + // region extra-methods + // endregion extra-methods - @NotNull @Override - public Map> getOutputColumns() { - return Collections.singletonMap(pair.leftColumn, outputSource); - } + public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifiedKey, long firstUnmodifiedTimestamp) { + Context ctx = (Context) context; + if (firstUnmodifiedKey != NULL_ROW_KEY) { + ctx.curVal = outputSource.get(firstUnmodifiedKey); + } else { + reset(ctx); + } - @NotNull - @Override - public UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { - return new Context(chunkSize, timestampSsa); + // If we're redirected we have to make sure we tell the output source it's actual size, or we're going + // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not + // need to do anything with capacity. + if(redirContext.isRedirected()) { + // The redirection index does not use the 0th index for some reason. + outputSource.ensureCapacity(redirContext.requiredCapacity()); + } } @Override @@ -142,57 +124,35 @@ public void startTrackingPrev() { } } - /** - * Perform the processing for a chunk of values by the operator. - * - * @param ctx the context object - * @param inputKeys the input keys for the chunk - * @param workingChunk the chunk of values - */ - protected abstract void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk); - - // endregion - // region Shifts - @Override public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { ((ObjectSparseArraySource)outputSource).shift(subIndexToShift, delta); } - // endregion Shifts - // region Reprocessing - - public void resetForProcess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceIndex, - long firstUnmodifiedKey) { - - // If we're redirected we have to make sure we tell the output source it's actual size, or we're going - // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not - // need to do anything with capacity. - if(redirContext.isRedirected()) { - // The redirection index does not use the 0th index for some reason. - outputSource.ensureCapacity(redirContext.requiredCapacity()); - } - - final Context ctx = (Context) context; - ctx.curVal = firstUnmodifiedKey == NULL_ROW_KEY ? null : outputSource.get(firstUnmodifiedKey); - } - + // region Processing @Override public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, @Nullable final LongChunk posChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex) { + @Nullable final Chunk valuesChunk, + @Nullable final LongChunk timestampValuesChunk) { + Assert.neqNull(valuesChunk, "valuesChunk must not be null for a cumulative operator"); final Context ctx = (Context) updateContext; - doProcessChunk(ctx, inputKeys, valuesChunk); - ctx.getModifiedBuilder().appendRowSequence(inputKeys); + ctx.storeValuesChunk(valuesChunk); + for (int ii = 0; ii < valuesChunk.size(); ii++) { + push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); + ctx.outputValues.set(ii, ctx.curVal); + } + outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues, inputKeys); } - // endregion + + @NotNull + @Override + public Map> getOutputColumns() { + return Collections.singletonMap(pair.leftColumn, outputSource); + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java index 3f27959b997..17157fa0760 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java @@ -5,8 +5,10 @@ */ package io.deephaven.engine.table.impl.updateby.internal; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.WritableShortChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.sized.SizedShortChunk; import io.deephaven.engine.rowset.*; @@ -15,9 +17,6 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; -import io.deephaven.engine.table.impl.util.SizedSafeCloseable; -import io.deephaven.util.QueryConstants; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -25,35 +24,32 @@ import java.util.Map; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; +import static io.deephaven.util.QueryConstants.*; public abstract class BaseShortUpdateByOperator extends UpdateByCumulativeOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; - protected final MatchPair pair; - protected final String[] affectingColumns; - - private UpdateBy.UpdateByRedirectionContext redirContext; - // region extra-fields // endregion extra-fields - protected class Context extends UpdateCumulativeContext { - public final SizedSafeCloseable fillContext; - public final SizedShortChunk outputValues; + protected class Context extends UpdateByCumulativeOperator.Context { + public final ChunkSink.FillFromContext fillContext; + public final WritableShortChunk outputValues; - public short curVal = QueryConstants.NULL_SHORT; + public short curVal = NULL_SHORT; - protected Context(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { - this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); - this.fillContext.ensureCapacity(chunkSize); - this.outputValues = new SizedShortChunk<>(chunkSize); + protected Context(final int chunkSize) { + super(chunkSize); + this.fillContext = outputSource.makeFillFromContext(chunkSize); + this.outputValues = WritableShortChunk.makeWritableChunk(chunkSize); } + public void storeValuesChunk(@NotNull final Chunk valuesChunk) {} + @Override public void close() { super.close(); - outputValues.close(); fillContext.close(); } @@ -73,9 +69,7 @@ public BaseShortUpdateByOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - this.pair = pair; - this.affectingColumns = affectingColumns; - this.redirContext = redirContext; + super(pair, affectingColumns, redirContext); if(this.redirContext.isRedirected()) { // region create-dense this.maybeInnerSource = new ShortArraySource(); @@ -92,43 +86,31 @@ public BaseShortUpdateByOperator(@NotNull final MatchPair pair, // endregion constructor } - // region extra-methods - // endregion extra-methods - - @Override - public void setChunkSize(@NotNull final UpdateContext context, final int chunkSize) { - ((Context)context).outputValues.ensureCapacity(chunkSize); - ((Context)context).fillContext.ensureCapacity(chunkSize); - } - - @NotNull - @Override - public String getInputColumnName() { - return pair.rightColumn; - } - - @NotNull @Override - public String[] getAffectingColumnNames() { - return affectingColumns; + public void reset(UpdateContext context) { + final Context ctx = (Context)context; + ctx.curVal = NULL_SHORT; } - @NotNull - @Override - public String[] getOutputColumnNames() { - return new String[] { pair.leftColumn }; - } + // region extra-methods + // endregion extra-methods - @NotNull @Override - public Map> getOutputColumns() { - return Collections.singletonMap(pair.leftColumn, outputSource); - } + public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifiedKey, long firstUnmodifiedTimestamp) { + Context ctx = (Context) context; + if (firstUnmodifiedKey != NULL_ROW_KEY) { + ctx.curVal = outputSource.getShort(firstUnmodifiedKey); + } else { + reset(ctx); + } - @NotNull - @Override - public UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { - return new Context(chunkSize, timestampSsa); + // If we're redirected we have to make sure we tell the output source it's actual size, or we're going + // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not + // need to do anything with capacity. + if(redirContext.isRedirected()) { + // The redirection index does not use the 0th index for some reason. + outputSource.ensureCapacity(redirContext.requiredCapacity()); + } } @Override @@ -139,57 +121,35 @@ public void startTrackingPrev() { } } - /** - * Perform the processing for a chunk of values by the operator. - * - * @param ctx the context object - * @param inputKeys the input keys for the chunk - * @param workingChunk the chunk of values - */ - protected abstract void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk); - - // endregion - // region Shifts - @Override public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { ((ShortSparseArraySource)outputSource).shift(subIndexToShift, delta); } - // endregion Shifts - // region Reprocessing - - public void resetForProcess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceIndex, - long firstUnmodifiedKey) { - - // If we're redirected we have to make sure we tell the output source it's actual size, or we're going - // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not - // need to do anything with capacity. - if(redirContext.isRedirected()) { - // The redirection index does not use the 0th index for some reason. - outputSource.ensureCapacity(redirContext.requiredCapacity()); - } - - final Context ctx = (Context) context; - ctx.curVal = firstUnmodifiedKey == NULL_ROW_KEY ? QueryConstants.NULL_SHORT : outputSource.getShort(firstUnmodifiedKey); - } - + // region Processing @Override public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, @Nullable final LongChunk posChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex) { + @Nullable final Chunk valuesChunk, + @Nullable final LongChunk timestampValuesChunk) { + Assert.neqNull(valuesChunk, "valuesChunk must not be null for a cumulative operator"); final Context ctx = (Context) updateContext; - doProcessChunk(ctx, inputKeys, valuesChunk); - ctx.getModifiedBuilder().appendRowSequence(inputKeys); + ctx.storeValuesChunk(valuesChunk); + for (int ii = 0; ii < valuesChunk.size(); ii++) { + push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); + ctx.outputValues.set(ii, ctx.curVal); + } + outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues, inputKeys); } - // endregion + + @NotNull + @Override + public Map> getOutputColumns() { + return Collections.singletonMap(pair.leftColumn, outputSource); + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java index 11ce039fcf6..914dda218d6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java @@ -11,47 +11,48 @@ import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.api.updateby.OperationControl; -import io.deephaven.chunk.*; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.chunk.sized.SizedByteChunk; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.table.ChunkSource; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; +import io.deephaven.engine.table.impl.sources.*; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import java.util.Collections; +import java.util.Map; + +import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; +import static io.deephaven.util.QueryConstants.*; + public abstract class BaseWindowedByteUpdateByOperator extends UpdateByWindowedOperator { - protected final ColumnSource valueSource; + protected final WritableColumnSource outputSource; + protected final WritableColumnSource maybeInnerSource; // region extra-fields final byte nullValue; // endregion extra-fields protected class Context extends UpdateWindowedContext { - public WritableByteChunk candidateValuesChunk; + public final ChunkSink.FillFromContext fillContext; + public final SizedByteChunk outputValues; - @Override - public void close() { - super.close(); - if (candidateValuesChunk != null) { - candidateValuesChunk.close(); - candidateValuesChunk = null; - } + public byte curVal = NULL_BYTE; + + protected Context(final int chunkSize) { + this.fillContext = outputSource.makeFillFromContext(chunkSize); + this.outputValues = new SizedByteChunk<>(chunkSize); } @Override - public void loadInfluencerValueChunk() { - int size = influencerRows.intSize(); - // fill the window values chunk - if (candidateValuesChunk == null) { - candidateValuesChunk = WritableByteChunk.makeWritableChunk(size); - } - try (ChunkSource.FillContext fc = valueSource.makeFillContext(size)){ - valueSource.fillChunk(fc, candidateValuesChunk, influencerRows); - } + public void close() { + outputValues.close(); + fillContext.close(); } } @@ -59,21 +60,37 @@ public BaseWindowedByteUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final String timestampColumnName, - @Nullable final ColumnSource timestampColumnSource, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, - @NotNull final ColumnSource valueSource + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, timestampColumnSource, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); - this.valueSource = valueSource; + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); + if(this.redirContext.isRedirected()) { + // region create-dense + this.maybeInnerSource = makeDenseSource(); + // endregion create-dense + this.outputSource = new WritableRedirectedColumnSource(this.redirContext.getRowRedirection(), maybeInnerSource, 0); + } else { + this.maybeInnerSource = null; + // region create-sparse + this.outputSource = makeSparseSource(); + // endregion create-sparse + } + // region constructor this.nullValue = getNullValue(); // endregion constructor } + @Override + public void reset(UpdateContext context) { + final Context ctx = (Context)context; + ctx.curVal = NULL_BYTE; + ctx.nullCount = 0; + } + // region extra-methods protected byte getNullValue() { return QueryConstants.NULL_BYTE; @@ -89,42 +106,52 @@ protected WritableColumnSource makeDenseSource() { } // endregion extra-methods - // region Addition - /** - * Add a chunk of values to the operator. - * - * @param ctx the context object - * @param inputKeys the input keys for the chunk - * @param workingChunk the chunk of values - */ - protected abstract void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @NotNull final Chunk workingChunk); - // endregion + @NotNull + @Override + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); + } - // region Reprocessing + @Override + public void startTrackingPrev() { + outputSource.startTrackingPrevValues(); + if (redirContext.isRedirected()) { + maybeInnerSource.startTrackingPrevValues(); + } + } - public void resetForProcess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceRowSet, - long firstUnmodifiedKey) { - final Context ctx = (Context) context; - ctx.sourceRowSet = sourceRowSet; + // region Shifts + @Override + public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { + if (outputSource instanceof BooleanSparseArraySource.ReinterpretedAsByte) { + ((BooleanSparseArraySource.ReinterpretedAsByte)outputSource).shift(subIndexToShift, delta); + } else { + ((ByteSparseArraySource)outputSource).shift(subIndexToShift, delta); + } } + // endregion Shifts + // region Processing @Override public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, @Nullable final LongChunk posChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex) { + @Nullable final Chunk valuesChunk, + @Nullable final LongChunk timestampValuesChunk) { final Context ctx = (Context) updateContext; - doProcessChunk(ctx, inputKeys, keyChunk, posChunk, valuesChunk); - ctx.getModifiedBuilder().appendRowSequence(inputKeys); + for (int ii = 0; ii < valuesChunk.size(); ii++) { + push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); + ctx.outputValues.get().set(ii, ctx.curVal); + } + outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues.get(), inputKeys); } - // endregion + + @NotNull + @Override + public Map> getOutputColumns() { + return Collections.singletonMap(pair.leftColumn, outputSource); + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java index dbb4ac41952..2563e20c316 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java @@ -1,46 +1,47 @@ package io.deephaven.engine.table.impl.updateby.internal; import io.deephaven.api.updateby.OperationControl; -import io.deephaven.chunk.*; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.chunk.sized.SizedCharChunk; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.table.ChunkSource; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; +import io.deephaven.engine.table.impl.sources.*; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import java.util.Collections; +import java.util.Map; + +import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; +import static io.deephaven.util.QueryConstants.*; + public abstract class BaseWindowedCharUpdateByOperator extends UpdateByWindowedOperator { - protected final ColumnSource valueSource; + protected final WritableColumnSource outputSource; + protected final WritableColumnSource maybeInnerSource; // region extra-fields // endregion extra-fields protected class Context extends UpdateWindowedContext { - public WritableCharChunk candidateValuesChunk; + public final ChunkSink.FillFromContext fillContext; + public final SizedCharChunk outputValues; - @Override - public void close() { - super.close(); - if (candidateValuesChunk != null) { - candidateValuesChunk.close(); - candidateValuesChunk = null; - } + public char curVal = NULL_CHAR; + + protected Context(final int chunkSize) { + this.fillContext = outputSource.makeFillFromContext(chunkSize); + this.outputValues = new SizedCharChunk<>(chunkSize); } @Override - public void loadInfluencerValueChunk() { - int size = influencerRows.intSize(); - // fill the window values chunk - if (candidateValuesChunk == null) { - candidateValuesChunk = WritableCharChunk.makeWritableChunk(size); - } - try (ChunkSource.FillContext fc = valueSource.makeFillContext(size)){ - valueSource.fillChunk(fc, candidateValuesChunk, influencerRows); - } + public void close() { + outputValues.close(); + fillContext.close(); } } @@ -48,59 +49,81 @@ public BaseWindowedCharUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final String timestampColumnName, - @Nullable final ColumnSource timestampColumnSource, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, - @NotNull final ColumnSource valueSource + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, timestampColumnSource, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); - this.valueSource = valueSource; + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); + if(this.redirContext.isRedirected()) { + // region create-dense + this.maybeInnerSource = new CharacterArraySource(); + // endregion create-dense + this.outputSource = new WritableRedirectedColumnSource(this.redirContext.getRowRedirection(), maybeInnerSource, 0); + } else { + this.maybeInnerSource = null; + // region create-sparse + this.outputSource = new CharacterSparseArraySource(); + // endregion create-sparse + } + // region constructor // endregion constructor } + @Override + public void reset(UpdateContext context) { + final Context ctx = (Context)context; + ctx.curVal = NULL_CHAR; + ctx.nullCount = 0; + } + // region extra-methods // endregion extra-methods - // region Addition - /** - * Add a chunk of values to the operator. - * - * @param ctx the context object - * @param inputKeys the input keys for the chunk - * @param workingChunk the chunk of values - */ - protected abstract void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @NotNull final Chunk workingChunk); - // endregion + @NotNull + @Override + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); + } - // region Reprocessing + @Override + public void startTrackingPrev() { + outputSource.startTrackingPrevValues(); + if (redirContext.isRedirected()) { + maybeInnerSource.startTrackingPrevValues(); + } + } - public void resetForProcess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceRowSet, - long firstUnmodifiedKey) { - final Context ctx = (Context) context; - ctx.sourceRowSet = sourceRowSet; + // region Shifts + @Override + public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { + ((CharacterSparseArraySource)outputSource).shift(subIndexToShift, delta); } + // endregion Shifts + // region Processing @Override public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, @Nullable final LongChunk posChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex) { + @Nullable final Chunk valuesChunk, + @Nullable final LongChunk timestampValuesChunk) { final Context ctx = (Context) updateContext; - doProcessChunk(ctx, inputKeys, keyChunk, posChunk, valuesChunk); - ctx.getModifiedBuilder().appendRowSequence(inputKeys); + for (int ii = 0; ii < valuesChunk.size(); ii++) { + push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); + ctx.outputValues.get().set(ii, ctx.curVal); + } + outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues.get(), inputKeys); } - // endregion + + @NotNull + @Override + public Map> getOutputColumns() { + return Collections.singletonMap(pair.leftColumn, outputSource); + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java index 63bc0023daa..825ebffdaee 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java @@ -8,110 +8,117 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; +import io.deephaven.chunk.sized.SizedDoubleChunk; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.table.ChunkSource; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; +import io.deephaven.engine.table.impl.sources.DoubleArraySource; +import io.deephaven.engine.table.impl.sources.DoubleSparseArraySource; +import io.deephaven.engine.table.impl.sources.WritableRedirectedColumnSource; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; + public abstract class BaseWindowedDoubleUpdateByOperator extends UpdateByWindowedOperator { - protected final ColumnSource valueSource; + protected final WritableColumnSource outputSource; + protected final WritableColumnSource maybeInnerSource; // region extra-fields // endregion extra-fields protected class Context extends UpdateWindowedContext { - public WritableDoubleChunk candidateValuesChunk; + public final ChunkSink.FillFromContext fillContext; + public final SizedDoubleChunk outputValues; - @Override - public void close() { - super.close(); - if (candidateValuesChunk != null) { - candidateValuesChunk.close(); - candidateValuesChunk = null; - } + protected Context(final int chunkSize) { + this.fillContext = outputSource.makeFillFromContext(chunkSize); + this.outputValues = new SizedDoubleChunk<>(chunkSize); } + public void storeWorkingChunk(@NotNull final Chunk valuesChunk) {} + @Override - public void loadInfluencerValueChunk() { - int size = influencerRows.intSize(); - // fill the window values chunk - if (candidateValuesChunk == null) { - candidateValuesChunk = WritableDoubleChunk.makeWritableChunk(size); - } - try (ChunkSource.FillContext fc = valueSource.makeFillContext(size)){ - valueSource.fillChunk(fc, candidateValuesChunk, influencerRows); - } + public void close() { + outputValues.close(); + fillContext.close(); } } public BaseWindowedDoubleUpdateByOperator(@NotNull final MatchPair pair, - @NotNull final String[] affectingColumns, - @NotNull final OperationControl control, - @Nullable final String timestampColumnName, - @Nullable final ColumnSource timestampColumnSource, - final long reverseTimeScaleUnits, - final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, - @NotNull final ColumnSource valueSource - // region extra-constructor-args - // endregion extra-constructor-args - ) { - super(pair, affectingColumns, control, timestampColumnName, timestampColumnSource, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); - this.valueSource = valueSource; + @NotNull final String[] affectingColumns, + @NotNull final OperationControl control, + @Nullable final String timestampColumnName, + final long reverseTimeScaleUnits, + final long forwardTimeScaleUnits, + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + // region extra-constructor-args + // endregion extra-constructor-args + ) { + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); + if(this.redirContext.isRedirected()) { + // region create-dense + this.maybeInnerSource = new DoubleArraySource(); + // endregion create-dense + this.outputSource = new WritableRedirectedColumnSource(this.redirContext.getRowRedirection(), maybeInnerSource, 0); + } else { + this.maybeInnerSource = null; + // region create-sparse + this.outputSource = new DoubleSparseArraySource(); + // endregion create-sparse + } + // region constructor // endregion constructor } + //*** for doubleing point operators, we want a computed result */ + public abstract double result(UpdateContext context); + // region extra-methods // endregion extra-methods + @NotNull @Override - public boolean requiresValues(@NotNull final UpdateContext context) { - // windowed operators don't need current values supplied to them, they only care about windowed values which - // may or may not intersect with the column values - return false; + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); } - // region Addition - /** - * Add a chunk of values to the operator. - * - * @param ctx the context object - * @param inputKeys the input keys for the chunk - * @param workingChunk the chunk of values - */ - protected abstract void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @NotNull final Chunk workingChunk); - - // endregion + @Override + public void startTrackingPrev() { + outputSource.startTrackingPrevValues(); + if (redirContext.isRedirected()) { + maybeInnerSource.startTrackingPrevValues(); + } + } - // region Reprocessing + // region Shifts - public void resetForProcess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceRowSet, - long firstUnmodifiedKey) { - final Context ctx = (Context) context; - ctx.sourceRowSet = sourceRowSet; + @Override + public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { + ((DoubleSparseArraySource)outputSource).shift(subIndexToShift, delta); } + // endregion Shifts + + // region Processing + @Override public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, @Nullable final LongChunk posChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex) { + @Nullable final Chunk valuesChunk, + @Nullable final LongChunk timestampValuesChunk) { final Context ctx = (Context) updateContext; - doProcessChunk(ctx, inputKeys, keyChunk, posChunk, valuesChunk); - ctx.getModifiedBuilder().appendRowSequence(inputKeys); + ctx.storeWorkingChunk(valuesChunk); + for (int ii = 0; ii < valuesChunk.size(); ii++) { + push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); + ctx.outputValues.get().set(ii, result(ctx)); + } + outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues.get(), inputKeys); } // endregion diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java index 4c7b7343d59..4547ca38e14 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java @@ -3,110 +3,117 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; +import io.deephaven.chunk.sized.SizedFloatChunk; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.table.ChunkSource; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; +import io.deephaven.engine.table.impl.sources.FloatArraySource; +import io.deephaven.engine.table.impl.sources.FloatSparseArraySource; +import io.deephaven.engine.table.impl.sources.WritableRedirectedColumnSource; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; + public abstract class BaseWindowedFloatUpdateByOperator extends UpdateByWindowedOperator { - protected final ColumnSource valueSource; + protected final WritableColumnSource outputSource; + protected final WritableColumnSource maybeInnerSource; // region extra-fields // endregion extra-fields protected class Context extends UpdateWindowedContext { - public WritableFloatChunk candidateValuesChunk; + public final ChunkSink.FillFromContext fillContext; + public final SizedFloatChunk outputValues; - @Override - public void close() { - super.close(); - if (candidateValuesChunk != null) { - candidateValuesChunk.close(); - candidateValuesChunk = null; - } + protected Context(final int chunkSize) { + this.fillContext = outputSource.makeFillFromContext(chunkSize); + this.outputValues = new SizedFloatChunk<>(chunkSize); } + public void storeWorkingChunk(@NotNull final Chunk valuesChunk) {} + @Override - public void loadInfluencerValueChunk() { - int size = influencerRows.intSize(); - // fill the window values chunk - if (candidateValuesChunk == null) { - candidateValuesChunk = WritableFloatChunk.makeWritableChunk(size); - } - try (ChunkSource.FillContext fc = valueSource.makeFillContext(size)){ - valueSource.fillChunk(fc, candidateValuesChunk, influencerRows); - } + public void close() { + outputValues.close(); + fillContext.close(); } } public BaseWindowedFloatUpdateByOperator(@NotNull final MatchPair pair, - @NotNull final String[] affectingColumns, - @NotNull final OperationControl control, - @Nullable final String timestampColumnName, - @Nullable final ColumnSource timestampColumnSource, - final long reverseTimeScaleUnits, - final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, - @NotNull final ColumnSource valueSource - // region extra-constructor-args - // endregion extra-constructor-args - ) { - super(pair, affectingColumns, control, timestampColumnName, timestampColumnSource, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); - this.valueSource = valueSource; + @NotNull final String[] affectingColumns, + @NotNull final OperationControl control, + @Nullable final String timestampColumnName, + final long reverseTimeScaleUnits, + final long forwardTimeScaleUnits, + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + // region extra-constructor-args + // endregion extra-constructor-args + ) { + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); + if(this.redirContext.isRedirected()) { + // region create-dense + this.maybeInnerSource = new FloatArraySource(); + // endregion create-dense + this.outputSource = new WritableRedirectedColumnSource(this.redirContext.getRowRedirection(), maybeInnerSource, 0); + } else { + this.maybeInnerSource = null; + // region create-sparse + this.outputSource = new FloatSparseArraySource(); + // endregion create-sparse + } + // region constructor // endregion constructor } + //*** for floating point operators, we want a computed result */ + public abstract float result(UpdateContext context); + // region extra-methods // endregion extra-methods + @NotNull @Override - public boolean requiresValues(@NotNull final UpdateContext context) { - // windowed operators don't need current values supplied to them, they only care about windowed values which - // may or may not intersect with the column values - return false; + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); } - // region Addition - /** - * Add a chunk of values to the operator. - * - * @param ctx the context object - * @param inputKeys the input keys for the chunk - * @param workingChunk the chunk of values - */ - protected abstract void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @NotNull final Chunk workingChunk); - - // endregion + @Override + public void startTrackingPrev() { + outputSource.startTrackingPrevValues(); + if (redirContext.isRedirected()) { + maybeInnerSource.startTrackingPrevValues(); + } + } - // region Reprocessing + // region Shifts - public void resetForProcess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceRowSet, - long firstUnmodifiedKey) { - final Context ctx = (Context) context; - ctx.sourceRowSet = sourceRowSet; + @Override + public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { + ((FloatSparseArraySource)outputSource).shift(subIndexToShift, delta); } + // endregion Shifts + + // region Processing + @Override public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, @Nullable final LongChunk posChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex) { + @Nullable final Chunk valuesChunk, + @Nullable final LongChunk timestampValuesChunk) { final Context ctx = (Context) updateContext; - doProcessChunk(ctx, inputKeys, keyChunk, posChunk, valuesChunk); - ctx.getModifiedBuilder().appendRowSequence(inputKeys); + ctx.storeWorkingChunk(valuesChunk); + for (int ii = 0; ii < valuesChunk.size(); ii++) { + push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); + ctx.outputValues.get().set(ii, result(ctx)); + } + outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues.get(), inputKeys); } // endregion diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java index 5b911543f64..b8df5b2c86f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java @@ -6,46 +6,47 @@ package io.deephaven.engine.table.impl.updateby.internal; import io.deephaven.api.updateby.OperationControl; -import io.deephaven.chunk.*; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.chunk.sized.SizedIntChunk; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.table.ChunkSource; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; +import io.deephaven.engine.table.impl.sources.*; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import java.util.Collections; +import java.util.Map; + +import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; +import static io.deephaven.util.QueryConstants.*; + public abstract class BaseWindowedIntUpdateByOperator extends UpdateByWindowedOperator { - protected final ColumnSource valueSource; + protected final WritableColumnSource outputSource; + protected final WritableColumnSource maybeInnerSource; // region extra-fields // endregion extra-fields protected class Context extends UpdateWindowedContext { - public WritableIntChunk candidateValuesChunk; + public final ChunkSink.FillFromContext fillContext; + public final SizedIntChunk outputValues; - @Override - public void close() { - super.close(); - if (candidateValuesChunk != null) { - candidateValuesChunk.close(); - candidateValuesChunk = null; - } + public int curVal = NULL_INT; + + protected Context(final int chunkSize) { + this.fillContext = outputSource.makeFillFromContext(chunkSize); + this.outputValues = new SizedIntChunk<>(chunkSize); } @Override - public void loadInfluencerValueChunk() { - int size = influencerRows.intSize(); - // fill the window values chunk - if (candidateValuesChunk == null) { - candidateValuesChunk = WritableIntChunk.makeWritableChunk(size); - } - try (ChunkSource.FillContext fc = valueSource.makeFillContext(size)){ - valueSource.fillChunk(fc, candidateValuesChunk, influencerRows); - } + public void close() { + outputValues.close(); + fillContext.close(); } } @@ -53,59 +54,81 @@ public BaseWindowedIntUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final String timestampColumnName, - @Nullable final ColumnSource timestampColumnSource, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, - @NotNull final ColumnSource valueSource + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, timestampColumnSource, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); - this.valueSource = valueSource; + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); + if(this.redirContext.isRedirected()) { + // region create-dense + this.maybeInnerSource = new IntegerArraySource(); + // endregion create-dense + this.outputSource = new WritableRedirectedColumnSource(this.redirContext.getRowRedirection(), maybeInnerSource, 0); + } else { + this.maybeInnerSource = null; + // region create-sparse + this.outputSource = new IntegerSparseArraySource(); + // endregion create-sparse + } + // region constructor // endregion constructor } + @Override + public void reset(UpdateContext context) { + final Context ctx = (Context)context; + ctx.curVal = NULL_INT; + ctx.nullCount = 0; + } + // region extra-methods // endregion extra-methods - // region Addition - /** - * Add a chunk of values to the operator. - * - * @param ctx the context object - * @param inputKeys the input keys for the chunk - * @param workingChunk the chunk of values - */ - protected abstract void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @NotNull final Chunk workingChunk); - // endregion + @NotNull + @Override + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); + } - // region Reprocessing + @Override + public void startTrackingPrev() { + outputSource.startTrackingPrevValues(); + if (redirContext.isRedirected()) { + maybeInnerSource.startTrackingPrevValues(); + } + } - public void resetForProcess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceRowSet, - long firstUnmodifiedKey) { - final Context ctx = (Context) context; - ctx.sourceRowSet = sourceRowSet; + // region Shifts + @Override + public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { + ((IntegerSparseArraySource)outputSource).shift(subIndexToShift, delta); } + // endregion Shifts + // region Processing @Override public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, @Nullable final LongChunk posChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex) { + @Nullable final Chunk valuesChunk, + @Nullable final LongChunk timestampValuesChunk) { final Context ctx = (Context) updateContext; - doProcessChunk(ctx, inputKeys, keyChunk, posChunk, valuesChunk); - ctx.getModifiedBuilder().appendRowSequence(inputKeys); + for (int ii = 0; ii < valuesChunk.size(); ii++) { + push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); + ctx.outputValues.get().set(ii, ctx.curVal); + } + outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues.get(), inputKeys); } - // endregion + + @NotNull + @Override + public Map> getOutputColumns() { + return Collections.singletonMap(pair.leftColumn, outputSource); + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java index f588a4483d8..824f09ac90d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java @@ -6,46 +6,47 @@ package io.deephaven.engine.table.impl.updateby.internal; import io.deephaven.api.updateby.OperationControl; -import io.deephaven.chunk.*; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.chunk.sized.SizedLongChunk; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.table.ChunkSource; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; +import io.deephaven.engine.table.impl.sources.*; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import java.util.Collections; +import java.util.Map; + +import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; +import static io.deephaven.util.QueryConstants.*; + public abstract class BaseWindowedLongUpdateByOperator extends UpdateByWindowedOperator { - protected final ColumnSource valueSource; + protected final WritableColumnSource outputSource; + protected final WritableColumnSource maybeInnerSource; // region extra-fields // endregion extra-fields protected class Context extends UpdateWindowedContext { - public WritableLongChunk candidateValuesChunk; + public final ChunkSink.FillFromContext fillContext; + public final SizedLongChunk outputValues; - @Override - public void close() { - super.close(); - if (candidateValuesChunk != null) { - candidateValuesChunk.close(); - candidateValuesChunk = null; - } + public long curVal = NULL_LONG; + + protected Context(final int chunkSize) { + this.fillContext = outputSource.makeFillFromContext(chunkSize); + this.outputValues = new SizedLongChunk<>(chunkSize); } @Override - public void loadInfluencerValueChunk() { - int size = influencerRows.intSize(); - // fill the window values chunk - if (candidateValuesChunk == null) { - candidateValuesChunk = WritableLongChunk.makeWritableChunk(size); - } - try (ChunkSource.FillContext fc = valueSource.makeFillContext(size)){ - valueSource.fillChunk(fc, candidateValuesChunk, influencerRows); - } + public void close() { + outputValues.close(); + fillContext.close(); } } @@ -53,59 +54,81 @@ public BaseWindowedLongUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final String timestampColumnName, - @Nullable final ColumnSource timestampColumnSource, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, - @NotNull final ColumnSource valueSource + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, timestampColumnSource, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); - this.valueSource = valueSource; + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); + if(this.redirContext.isRedirected()) { + // region create-dense + this.maybeInnerSource = new LongArraySource(); + // endregion create-dense + this.outputSource = new WritableRedirectedColumnSource(this.redirContext.getRowRedirection(), maybeInnerSource, 0); + } else { + this.maybeInnerSource = null; + // region create-sparse + this.outputSource = new LongSparseArraySource(); + // endregion create-sparse + } + // region constructor // endregion constructor } + @Override + public void reset(UpdateContext context) { + final Context ctx = (Context)context; + ctx.curVal = NULL_LONG; + ctx.nullCount = 0; + } + // region extra-methods // endregion extra-methods - // region Addition - /** - * Add a chunk of values to the operator. - * - * @param ctx the context object - * @param inputKeys the input keys for the chunk - * @param workingChunk the chunk of values - */ - protected abstract void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @NotNull final Chunk workingChunk); - // endregion + @NotNull + @Override + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); + } - // region Reprocessing + @Override + public void startTrackingPrev() { + outputSource.startTrackingPrevValues(); + if (redirContext.isRedirected()) { + maybeInnerSource.startTrackingPrevValues(); + } + } - public void resetForProcess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceRowSet, - long firstUnmodifiedKey) { - final Context ctx = (Context) context; - ctx.sourceRowSet = sourceRowSet; + // region Shifts + @Override + public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { + ((LongSparseArraySource)outputSource).shift(subIndexToShift, delta); } + // endregion Shifts + // region Processing @Override public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, @Nullable final LongChunk posChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex) { + @Nullable final Chunk valuesChunk, + @Nullable final LongChunk timestampValuesChunk) { final Context ctx = (Context) updateContext; - doProcessChunk(ctx, inputKeys, keyChunk, posChunk, valuesChunk); - ctx.getModifiedBuilder().appendRowSequence(inputKeys); + for (int ii = 0; ii < valuesChunk.size(); ii++) { + push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); + ctx.outputValues.get().set(ii, ctx.curVal); + } + outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues.get(), inputKeys); } - // endregion + + @NotNull + @Override + public Map> getOutputColumns() { + return Collections.singletonMap(pair.leftColumn, outputSource); + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java index c604d5012d2..84398f8a537 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java @@ -6,47 +6,48 @@ package io.deephaven.engine.table.impl.updateby.internal; import io.deephaven.api.updateby.OperationControl; -import io.deephaven.chunk.*; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.chunk.sized.SizedObjectChunk; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.table.ChunkSource; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; +import io.deephaven.engine.table.impl.sources.*; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import java.util.Collections; +import java.util.Map; + +import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; +import static io.deephaven.util.QueryConstants.*; + public abstract class BaseWindowedObjectUpdateByOperator extends UpdateByWindowedOperator { - protected final ColumnSource valueSource; + protected final WritableColumnSource outputSource; + protected final WritableColumnSource maybeInnerSource; // region extra-fields private final Class colType; // endregion extra-fields protected class Context extends UpdateWindowedContext { - public WritableObjectChunk candidateValuesChunk; + public final ChunkSink.FillFromContext fillContext; + public final SizedObjectChunk outputValues; - @Override - public void close() { - super.close(); - if (candidateValuesChunk != null) { - candidateValuesChunk.close(); - candidateValuesChunk = null; - } + public T curVal = null; + + protected Context(final int chunkSize) { + this.fillContext = outputSource.makeFillFromContext(chunkSize); + this.outputValues = new SizedObjectChunk<>(chunkSize); } @Override - public void loadInfluencerValueChunk() { - int size = influencerRows.intSize(); - // fill the window values chunk - if (candidateValuesChunk == null) { - candidateValuesChunk = WritableObjectChunk.makeWritableChunk(size); - } - try (ChunkSource.FillContext fc = valueSource.makeFillContext(size)){ - valueSource.fillChunk(fc, candidateValuesChunk, influencerRows); - } + public void close() { + outputValues.close(); + fillContext.close(); } } @@ -54,61 +55,83 @@ public BaseWindowedObjectUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final String timestampColumnName, - @Nullable final ColumnSource timestampColumnSource, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, - @NotNull final ColumnSource valueSource + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args , final Class colType // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, timestampColumnSource, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); - this.valueSource = valueSource; + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); + if(this.redirContext.isRedirected()) { + // region create-dense + this.maybeInnerSource = new ObjectArraySource(colType); + // endregion create-dense + this.outputSource = new WritableRedirectedColumnSource(this.redirContext.getRowRedirection(), maybeInnerSource, 0); + } else { + this.maybeInnerSource = null; + // region create-sparse + this.outputSource = new ObjectSparseArraySource<>(colType); + // endregion create-sparse + } + // region constructor this.colType = colType; // endregion constructor } + @Override + public void reset(UpdateContext context) { + final Context ctx = (Context)context; + ctx.curVal = null; + ctx.nullCount = 0; + } + // region extra-methods // endregion extra-methods - // region Addition - /** - * Add a chunk of values to the operator. - * - * @param ctx the context object - * @param inputKeys the input keys for the chunk - * @param workingChunk the chunk of values - */ - protected abstract void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @NotNull final Chunk workingChunk); - // endregion + @NotNull + @Override + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); + } - // region Reprocessing + @Override + public void startTrackingPrev() { + outputSource.startTrackingPrevValues(); + if (redirContext.isRedirected()) { + maybeInnerSource.startTrackingPrevValues(); + } + } - public void resetForProcess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceRowSet, - long firstUnmodifiedKey) { - final Context ctx = (Context) context; - ctx.sourceRowSet = sourceRowSet; + // region Shifts + @Override + public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { + ((ObjectSparseArraySource)outputSource).shift(subIndexToShift, delta); } + // endregion Shifts + // region Processing @Override public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, @Nullable final LongChunk posChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex) { + @Nullable final Chunk valuesChunk, + @Nullable final LongChunk timestampValuesChunk) { final Context ctx = (Context) updateContext; - doProcessChunk(ctx, inputKeys, keyChunk, posChunk, valuesChunk); - ctx.getModifiedBuilder().appendRowSequence(inputKeys); + for (int ii = 0; ii < valuesChunk.size(); ii++) { + push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); + ctx.outputValues.get().set(ii, ctx.curVal); + } + outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues.get(), inputKeys); } - // endregion + + @NotNull + @Override + public Map> getOutputColumns() { + return Collections.singletonMap(pair.leftColumn, outputSource); + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java index 92843d51b95..98e4a42e599 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java @@ -6,46 +6,47 @@ package io.deephaven.engine.table.impl.updateby.internal; import io.deephaven.api.updateby.OperationControl; -import io.deephaven.chunk.*; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.chunk.sized.SizedShortChunk; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.table.ChunkSource; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; +import io.deephaven.engine.table.impl.sources.*; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import java.util.Collections; +import java.util.Map; + +import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; +import static io.deephaven.util.QueryConstants.*; + public abstract class BaseWindowedShortUpdateByOperator extends UpdateByWindowedOperator { - protected final ColumnSource valueSource; + protected final WritableColumnSource outputSource; + protected final WritableColumnSource maybeInnerSource; // region extra-fields // endregion extra-fields protected class Context extends UpdateWindowedContext { - public WritableShortChunk candidateValuesChunk; + public final ChunkSink.FillFromContext fillContext; + public final SizedShortChunk outputValues; - @Override - public void close() { - super.close(); - if (candidateValuesChunk != null) { - candidateValuesChunk.close(); - candidateValuesChunk = null; - } + public short curVal = NULL_SHORT; + + protected Context(final int chunkSize) { + this.fillContext = outputSource.makeFillFromContext(chunkSize); + this.outputValues = new SizedShortChunk<>(chunkSize); } @Override - public void loadInfluencerValueChunk() { - int size = influencerRows.intSize(); - // fill the window values chunk - if (candidateValuesChunk == null) { - candidateValuesChunk = WritableShortChunk.makeWritableChunk(size); - } - try (ChunkSource.FillContext fc = valueSource.makeFillContext(size)){ - valueSource.fillChunk(fc, candidateValuesChunk, influencerRows); - } + public void close() { + outputValues.close(); + fillContext.close(); } } @@ -53,59 +54,81 @@ public BaseWindowedShortUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final String timestampColumnName, - @Nullable final ColumnSource timestampColumnSource, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, - @NotNull final ColumnSource valueSource + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, timestampColumnSource, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); - this.valueSource = valueSource; + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); + if(this.redirContext.isRedirected()) { + // region create-dense + this.maybeInnerSource = new ShortArraySource(); + // endregion create-dense + this.outputSource = new WritableRedirectedColumnSource(this.redirContext.getRowRedirection(), maybeInnerSource, 0); + } else { + this.maybeInnerSource = null; + // region create-sparse + this.outputSource = new ShortSparseArraySource(); + // endregion create-sparse + } + // region constructor // endregion constructor } + @Override + public void reset(UpdateContext context) { + final Context ctx = (Context)context; + ctx.curVal = NULL_SHORT; + ctx.nullCount = 0; + } + // region extra-methods // endregion extra-methods - // region Addition - /** - * Add a chunk of values to the operator. - * - * @param ctx the context object - * @param inputKeys the input keys for the chunk - * @param workingChunk the chunk of values - */ - protected abstract void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @NotNull final Chunk workingChunk); - // endregion + @NotNull + @Override + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); + } - // region Reprocessing + @Override + public void startTrackingPrev() { + outputSource.startTrackingPrevValues(); + if (redirContext.isRedirected()) { + maybeInnerSource.startTrackingPrevValues(); + } + } - public void resetForProcess(@NotNull final UpdateContext context, - @NotNull final RowSet sourceRowSet, - long firstUnmodifiedKey) { - final Context ctx = (Context) context; - ctx.sourceRowSet = sourceRowSet; + // region Shifts + @Override + public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { + ((ShortSparseArraySource)outputSource).shift(subIndexToShift, delta); } + // endregion Shifts + // region Processing @Override public void processChunk(@NotNull final UpdateContext updateContext, @NotNull final RowSequence inputKeys, @Nullable final LongChunk keyChunk, @Nullable final LongChunk posChunk, - @NotNull final Chunk valuesChunk, - @NotNull final RowSet postUpdateSourceIndex) { + @Nullable final Chunk valuesChunk, + @Nullable final LongChunk timestampValuesChunk) { final Context ctx = (Context) updateContext; - doProcessChunk(ctx, inputKeys, keyChunk, posChunk, valuesChunk); - ctx.getModifiedBuilder().appendRowSequence(inputKeys); + for (int ii = 0; ii < valuesChunk.size(); ii++) { + push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); + ctx.outputValues.get().set(ii, ctx.curVal); + } + outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues.get(), inputKeys); } - // endregion + + @NotNull + @Override + public Map> getOutputColumns() { + return Collections.singletonMap(pair.leftColumn, outputSource); + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/LongRecordingUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/LongRecordingUpdateByOperator.java deleted file mode 100644 index a71f9cd4ebe..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/LongRecordingUpdateByOperator.java +++ /dev/null @@ -1,154 +0,0 @@ -package io.deephaven.engine.table.impl.updateby.internal; - -import io.deephaven.base.verify.Require; -import io.deephaven.chunk.Chunk; -import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.datastructures.util.CollectionUtil; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; -import io.deephaven.engine.table.impl.sources.ReinterpretUtils; -import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; -import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; - -import java.util.Collections; -import java.util.Map; - -/** - * An operator that simply remembers the current chunks during the add and reprocess phases. - */ -public class LongRecordingUpdateByOperator extends UpdateByCumulativeOperator { - private final String inputColumnName; - private final String[] affectingColumns; - private final ColumnSource columnSource; - private RecordingContext currentContext = null; - - public LongRecordingUpdateByOperator(@NotNull final String inputColumnName, - @NotNull final String[] affectingColumns, - @NotNull final ColumnSource columnSource) { - this.inputColumnName = inputColumnName; - this.affectingColumns = affectingColumns; - this.columnSource = ReinterpretUtils.maybeConvertToPrimitive(columnSource); - } - - private class RecordingContext extends UpdateCumulativeContext { - private LongChunk addedChunk; - - @Override - public void close() { - addedChunk = null; - currentContext = null; - } - } - - /** - * Get the long value at the specified position in the currently retained chunk. - * - * @param chunkPosition the position - * @return the value at the position - */ - public long getLong(final int chunkPosition) { - return Require.neqNull(currentContext, "context").addedChunk.get(chunkPosition); - } - - /** - * Get the current long value within the underlying {@link ColumnSource}. - * - * @param key the key - * @return the current value at the key within the column source. - */ - public long getCurrentLong(final long key) { - return columnSource.getLong(key); - } - - /** - * Get the pervious long value within the underlying {@link ColumnSource}. - * - * @param key the key - * @return the previous value at the key within the column source. - */ - public long getPrevLong(final long key) { - return columnSource.getPrevLong(key); - } - - /** - * Get the current underlying {@link ColumnSource}. - * - * @return the current value at the key within the column source. - */ - public ColumnSource getColumnSource() { - return columnSource; - } - - @Override - public void processChunk(@NotNull UpdateContext updateContext, @NotNull RowSequence inputKeys, - @Nullable LongChunk keyChunk, - @Nullable LongChunk posChunk, - @NotNull Chunk valuesChunk, @NotNull RowSet postUpdateSourceIndex) { - currentContext.addedChunk = valuesChunk.asLongChunk(); - } - - @NotNull - @Override - public UpdateContext makeUpdateContext(int chunkSize, LongSegmentedSortedArray timestampSsa) { - return this.currentContext = new RecordingContext(); - } - - // region Unused methods - - @NotNull - @Override - public String getInputColumnName() { - return inputColumnName; - } - - @NotNull - @Override - public String[] getAffectingColumnNames() { - return affectingColumns; - } - - @NotNull - @Override - public String[] getOutputColumnNames() { - return CollectionUtil.ZERO_LENGTH_STRING_ARRAY; - } - - @NotNull - @Override - public Map> getOutputColumns() { - return Collections.emptyMap(); - } - - @Override - public void startTrackingPrev() { - } - - @Override - public boolean requiresKeys() { - return false; - } - - @Override - public boolean requiresValues(@NotNull UpdateContext ctx) { - return true; - } - - @Override - public void setChunkSize(@NotNull UpdateContext context, int chunkSize) { - } - - @Override - public void applyOutputShift(@NotNull UpdateContext context, @NotNull RowSet subIndexToShift, long delta) { - } - - @Override - public void resetForProcess(@NotNull UpdateContext context, @NotNull RowSet sourceIndex, long firstUnmodifiedKey) { - } - - // endregion -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java index 4aed616b19e..3cc1b07cf7e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java @@ -62,7 +62,6 @@ public PairwiseDoubleRingBuffer(int initialSize, double emptyVal, DoubleFunction this.capacity = Integer.highestOneBit(initialSize) * 2; this.chunkSize = capacity * 2; this.storageChunk = WritableDoubleChunk.makeWritableChunk(chunkSize); -// this.dirtyBits = new BitSet(chunkSize); this.dirtyIndices = new TIntArrayList(chunkSize); this.pairwiseFunction = pairwiseFunction; this.emptyVal = emptyVal; @@ -283,6 +282,12 @@ public int remaining() { return capacity() - size(); } + public void clear() { + head = tail = 0; + dirtyIndices.clear(); + allDirty = false; + } + @Override public void close() { try (final WritableDoubleChunk ignoredChunk = storageChunk) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java index 647a86dd7ad..f686788da53 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java @@ -57,7 +57,6 @@ public PairwiseFloatRingBuffer(int initialSize, float emptyVal, FloatFunction pa this.capacity = Integer.highestOneBit(initialSize) * 2; this.chunkSize = capacity * 2; this.storageChunk = WritableFloatChunk.makeWritableChunk(chunkSize); -// this.dirtyBits = new BitSet(chunkSize); this.dirtyIndices = new TIntArrayList(chunkSize); this.pairwiseFunction = pairwiseFunction; this.emptyVal = emptyVal; @@ -278,6 +277,12 @@ public int remaining() { return capacity() - size(); } + public void clear() { + head = tail = 0; + dirtyIndices.clear(); + allDirty = false; + } + @Override public void close() { try (final WritableFloatChunk ignoredChunk = storageChunk) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java index 5fc850e5bf9..070dfacb145 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java @@ -5,20 +5,16 @@ */ package io.deephaven.engine.table.impl.updateby.minmax; -import io.deephaven.chunk.*; -import io.deephaven.chunk.attributes.ChunkLengths; -import io.deephaven.chunk.attributes.ChunkPositions; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.ByteChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseByteUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.NULL_BYTE; +import static io.deephaven.util.QueryConstants.*; public class ByteCumMinMaxOperator extends BaseByteUpdateByOperator { private final boolean isMax; @@ -26,45 +22,53 @@ public class ByteCumMinMaxOperator extends BaseByteUpdateByOperator { // region extra-fields // endregion extra-fields - public ByteCumMinMaxOperator(@NotNull final MatchPair inputPair, + protected class Context extends BaseByteUpdateByOperator.Context { + public ByteChunk byteValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + byteValueChunk = valuesChunk.asByteChunk(); + } + } + + public ByteCumMinMaxOperator(@NotNull final MatchPair pair, final boolean isMax, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext - // region extra-constructor-args - // endregion extra-constructor-args - ) { - super(inputPair, new String[] { inputPair.rightColumn }, redirContext); + // region extra-constructor-args + // endregion extra-constructor-args + ) { + super(pair, new String[] { pair.rightColumn }, redirContext); this.isMax = isMax; - // region constructorÏ + // region constructor // endregion constructor } - // region extra-methods - // endregion extra-methods - + @NotNull @Override - protected void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { - accumulate(workingChunk.asByteChunk(), ctx, 0, workingChunk.size()); - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); } - private void accumulate(@NotNull final ByteChunk asBytes, - @NotNull final Context ctx, - final int runStart, - final int runLength) { - final WritableByteChunk localOutputValues = ctx.outputValues.get(); - for (int ii = runStart; ii < runStart + runLength; ii++) { - final byte currentVal = asBytes.get(ii); - if(ctx.curVal == NULL_BYTE) { + @Override + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + // read the value from the values chunk + final byte currentVal = ctx.byteValueChunk.get(pos); + + if (ctx.curVal == NULL_BYTE) { + ctx.curVal = currentVal; + } else if (currentVal != NULL_BYTE) { + if ((isMax && currentVal > ctx.curVal) || + (!isMax && currentVal < ctx.curVal)) { ctx.curVal = currentVal; - } else if(currentVal != NULL_BYTE) { - if((isMax && currentVal > ctx.curVal) || - (!isMax && currentVal < ctx.curVal) ) { - ctx.curVal = currentVal; - } } - localOutputValues.set(ii, ctx.curVal); } } + // region extra-methods + // endregion extra-methods } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java index ceaeeb1e338..edcab22e88f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java @@ -5,67 +5,70 @@ */ package io.deephaven.engine.table.impl.updateby.minmax; -import io.deephaven.chunk.*; -import io.deephaven.chunk.attributes.ChunkLengths; -import io.deephaven.chunk.attributes.ChunkPositions; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.DoubleChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseDoubleUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.NULL_DOUBLE; +import static io.deephaven.util.QueryConstants.NULL_LONG; public class DoubleCumMinMaxOperator extends BaseDoubleUpdateByOperator { private final boolean isMax; - public DoubleCumMinMaxOperator(@NotNull final MatchPair inputPair, + // region extra-fields + // endregion extra-fields + + protected class Context extends BaseDoubleUpdateByOperator.Context { + public DoubleChunk doubleValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + doubleValueChunk = valuesChunk.asDoubleChunk(); + } + } + + public DoubleCumMinMaxOperator(@NotNull final MatchPair pair, final boolean isMax, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args - ) { - super(inputPair, new String[] { inputPair.rightColumn }, redirContext); + ) { + super(pair, new String[] { pair.rightColumn }, redirContext); this.isMax = isMax; // region constructor // endregion constructor } + @NotNull @Override - protected void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { - if(Double.isNaN(ctx.curVal) || Double.isInfinite(ctx.curVal)) { - if(!ctx.filledWithPermanentValue) { - ctx.filledWithPermanentValue = true; - ctx.outputValues.get().fillWithValue(0, ctx.outputValues.get().capacity(), ctx.curVal); - } - } else { - accumulateMinMax(workingChunk.asDoubleChunk(), ctx, 0, workingChunk.size()); - } - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); } - private void accumulateMinMax(@NotNull final DoubleChunk asDoubles, - @NotNull final Context ctx, - final int runStart, - final int runLength) { - final WritableDoubleChunk localOutputChunk = ctx.outputValues.get(); - for (int ii = runStart; ii < runStart + runLength; ii++) { - final double currentVal = asDoubles.get(ii); - if (ctx.curVal == NULL_DOUBLE) { + @Override + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + // read the value from the values chunk + final double currentVal = ctx.doubleValueChunk.get(pos); + + if(ctx.curVal == NULL_DOUBLE) { + ctx.curVal = currentVal; + } else if(currentVal != NULL_DOUBLE) { + if ((isMax && currentVal > ctx.curVal) || + (!isMax && currentVal < ctx.curVal)) { ctx.curVal = currentVal; - } else if (currentVal != NULL_DOUBLE) { - if ((isMax && currentVal > ctx.curVal) || - (!isMax && currentVal < ctx.curVal)) { - ctx.curVal = currentVal; - } } - localOutputChunk.set(ii, ctx.curVal); } } + // region extra-methods + // endregion extra-methods } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java index 6c1d5dbf2f7..2b4aa2fb4fe 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java @@ -1,66 +1,69 @@ package io.deephaven.engine.table.impl.updateby.minmax; -import io.deephaven.chunk.*; -import io.deephaven.chunk.attributes.ChunkLengths; -import io.deephaven.chunk.attributes.ChunkPositions; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.FloatChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseFloatUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.NULL_FLOAT; +import static io.deephaven.util.QueryConstants.NULL_LONG; public class FloatCumMinMaxOperator extends BaseFloatUpdateByOperator { private final boolean isMax; - public FloatCumMinMaxOperator(@NotNull final MatchPair inputPair, + // region extra-fields + // endregion extra-fields + + protected class Context extends BaseFloatUpdateByOperator.Context { + public FloatChunk floatValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + floatValueChunk = valuesChunk.asFloatChunk(); + } + } + + public FloatCumMinMaxOperator(@NotNull final MatchPair pair, final boolean isMax, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args - ) { - super(inputPair, new String[] { inputPair.rightColumn }, redirContext); + ) { + super(pair, new String[] { pair.rightColumn }, redirContext); this.isMax = isMax; // region constructor // endregion constructor } + @NotNull @Override - protected void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { - if(Float.isNaN(ctx.curVal) || Float.isInfinite(ctx.curVal)) { - if(!ctx.filledWithPermanentValue) { - ctx.filledWithPermanentValue = true; - ctx.outputValues.get().fillWithValue(0, ctx.outputValues.get().capacity(), ctx.curVal); - } - } else { - accumulateMinMax(workingChunk.asFloatChunk(), ctx, 0, workingChunk.size()); - } - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); } - private void accumulateMinMax(@NotNull final FloatChunk asFloats, - @NotNull final Context ctx, - final int runStart, - final int runLength) { - final WritableFloatChunk localOutputChunk = ctx.outputValues.get(); - for (int ii = runStart; ii < runStart + runLength; ii++) { - final float currentVal = asFloats.get(ii); - if (ctx.curVal == NULL_FLOAT) { + @Override + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + // read the value from the values chunk + final float currentVal = ctx.floatValueChunk.get(pos); + + if(ctx.curVal == NULL_FLOAT) { + ctx.curVal = currentVal; + } else if(currentVal != NULL_FLOAT) { + if ((isMax && currentVal > ctx.curVal) || + (!isMax && currentVal < ctx.curVal)) { ctx.curVal = currentVal; - } else if (currentVal != NULL_FLOAT) { - if ((isMax && currentVal > ctx.curVal) || - (!isMax && currentVal < ctx.curVal)) { - ctx.curVal = currentVal; - } } - localOutputChunk.set(ii, ctx.curVal); } } + // region extra-methods + // endregion extra-methods } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java index 2ae3b10136b..442910db0ef 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java @@ -5,20 +5,16 @@ */ package io.deephaven.engine.table.impl.updateby.minmax; -import io.deephaven.chunk.*; -import io.deephaven.chunk.attributes.ChunkLengths; -import io.deephaven.chunk.attributes.ChunkPositions; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseIntUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.NULL_INT; +import static io.deephaven.util.QueryConstants.*; public class IntCumMinMaxOperator extends BaseIntUpdateByOperator { private final boolean isMax; @@ -26,45 +22,53 @@ public class IntCumMinMaxOperator extends BaseIntUpdateByOperator { // region extra-fields // endregion extra-fields - public IntCumMinMaxOperator(@NotNull final MatchPair inputPair, + protected class Context extends BaseIntUpdateByOperator.Context { + public IntChunk intValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + intValueChunk = valuesChunk.asIntChunk(); + } + } + + public IntCumMinMaxOperator(@NotNull final MatchPair pair, final boolean isMax, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext - // region extra-constructor-args - // endregion extra-constructor-args - ) { - super(inputPair, new String[] { inputPair.rightColumn }, redirContext); + // region extra-constructor-args + // endregion extra-constructor-args + ) { + super(pair, new String[] { pair.rightColumn }, redirContext); this.isMax = isMax; - // region constructorÏ + // region constructor // endregion constructor } - // region extra-methods - // endregion extra-methods - + @NotNull @Override - protected void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { - accumulate(workingChunk.asIntChunk(), ctx, 0, workingChunk.size()); - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); } - private void accumulate(@NotNull final IntChunk asIntegers, - @NotNull final Context ctx, - final int runStart, - final int runLength) { - final WritableIntChunk localOutputValues = ctx.outputValues.get(); - for (int ii = runStart; ii < runStart + runLength; ii++) { - final int currentVal = asIntegers.get(ii); - if(ctx.curVal == NULL_INT) { + @Override + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + // read the value from the values chunk + final int currentVal = ctx.intValueChunk.get(pos); + + if (ctx.curVal == NULL_INT) { + ctx.curVal = currentVal; + } else if (currentVal != NULL_INT) { + if ((isMax && currentVal > ctx.curVal) || + (!isMax && currentVal < ctx.curVal)) { ctx.curVal = currentVal; - } else if(currentVal != NULL_INT) { - if((isMax && currentVal > ctx.curVal) || - (!isMax && currentVal < ctx.curVal) ) { - ctx.curVal = currentVal; - } } - localOutputValues.set(ii, ctx.curVal); } } + // region extra-methods + // endregion extra-methods } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java index 05c98ff630c..660d0f20113 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java @@ -12,20 +12,16 @@ import java.time.Instant; import io.deephaven.engine.table.impl.sources.ReinterpretUtils; -import io.deephaven.chunk.*; -import io.deephaven.chunk.attributes.ChunkLengths; -import io.deephaven.chunk.attributes.ChunkPositions; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; +import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.NULL_LONG; +import static io.deephaven.util.QueryConstants.*; public class LongCumMinMaxOperator extends BaseLongUpdateByOperator { private final boolean isMax; @@ -34,20 +30,55 @@ public class LongCumMinMaxOperator extends BaseLongUpdateByOperator { private final Class type; // endregion extra-fields - public LongCumMinMaxOperator(@NotNull final MatchPair inputPair, + protected class Context extends BaseLongUpdateByOperator.Context { + public LongChunk longValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + longValueChunk = valuesChunk.asLongChunk(); + } + } + + public LongCumMinMaxOperator(@NotNull final MatchPair pair, final boolean isMax, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext - // region extra-constructor-args + // region extra-constructor-args ,@NotNull final Class type - // endregion extra-constructor-args - ) { - super(inputPair, new String[] { inputPair.rightColumn }, redirContext); + // endregion extra-constructor-args + ) { + super(pair, new String[] { pair.rightColumn }, redirContext); this.isMax = isMax; - // region constructorÏ + // region constructor this.type = type; // endregion constructor } + @NotNull + @Override + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); + } + + @Override + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + // read the value from the values chunk + final long currentVal = ctx.longValueChunk.get(pos); + + if (ctx.curVal == NULL_LONG) { + ctx.curVal = currentVal; + } else if (currentVal != NULL_LONG) { + if ((isMax && currentVal > ctx.curVal) || + (!isMax && currentVal < ctx.curVal)) { + ctx.curVal = currentVal; + } + } + } // region extra-methods @NotNull @Override @@ -61,31 +92,4 @@ public Map> getOutputColumns() { return Collections.singletonMap(pair.leftColumn, actualOutput); } // endregion extra-methods - - @Override - protected void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { - accumulate(workingChunk.asLongChunk(), ctx, 0, workingChunk.size()); - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); - } - - private void accumulate(@NotNull final LongChunk asLongs, - @NotNull final Context ctx, - final int runStart, - final int runLength) { - final WritableLongChunk localOutputValues = ctx.outputValues.get(); - for (int ii = runStart; ii < runStart + runLength; ii++) { - final long currentVal = asLongs.get(ii); - if(ctx.curVal == NULL_LONG) { - ctx.curVal = currentVal; - } else if(currentVal != NULL_LONG) { - if((isMax && currentVal > ctx.curVal) || - (!isMax && currentVal < ctx.curVal) ) { - ctx.curVal = currentVal; - } - } - localOutputValues.set(ii, ctx.curVal); - } - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java index 83b1a9f1fe3..17613b46cc9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java @@ -1,19 +1,15 @@ package io.deephaven.engine.table.impl.updateby.minmax; -import io.deephaven.chunk.*; -import io.deephaven.chunk.attributes.ChunkLengths; -import io.deephaven.chunk.attributes.ChunkPositions; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.ShortChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseShortUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.NULL_SHORT; +import static io.deephaven.util.QueryConstants.*; public class ShortCumMinMaxOperator extends BaseShortUpdateByOperator { private final boolean isMax; @@ -21,45 +17,53 @@ public class ShortCumMinMaxOperator extends BaseShortUpdateByOperator { // region extra-fields // endregion extra-fields - public ShortCumMinMaxOperator(@NotNull final MatchPair inputPair, + protected class Context extends BaseShortUpdateByOperator.Context { + public ShortChunk shortValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + shortValueChunk = valuesChunk.asShortChunk(); + } + } + + public ShortCumMinMaxOperator(@NotNull final MatchPair pair, final boolean isMax, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext - // region extra-constructor-args - // endregion extra-constructor-args - ) { - super(inputPair, new String[] { inputPair.rightColumn }, redirContext); + // region extra-constructor-args + // endregion extra-constructor-args + ) { + super(pair, new String[] { pair.rightColumn }, redirContext); this.isMax = isMax; - // region constructorÏ + // region constructor // endregion constructor } - // region extra-methods - // endregion extra-methods - + @NotNull @Override - protected void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { - accumulate(workingChunk.asShortChunk(), ctx, 0, workingChunk.size()); - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); } - private void accumulate(@NotNull final ShortChunk asShorts, - @NotNull final Context ctx, - final int runStart, - final int runLength) { - final WritableShortChunk localOutputValues = ctx.outputValues.get(); - for (int ii = runStart; ii < runStart + runLength; ii++) { - final short currentVal = asShorts.get(ii); - if(ctx.curVal == NULL_SHORT) { + @Override + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + // read the value from the values chunk + final short currentVal = ctx.shortValueChunk.get(pos); + + if (ctx.curVal == NULL_SHORT) { + ctx.curVal = currentVal; + } else if (currentVal != NULL_SHORT) { + if ((isMax && currentVal > ctx.curVal) || + (!isMax && currentVal < ctx.curVal)) { ctx.curVal = currentVal; - } else if(currentVal != NULL_SHORT) { - if((isMax && currentVal > ctx.curVal) || - (!isMax && currentVal < ctx.curVal) ) { - ctx.curVal = currentVal; - } } - localOutputValues.set(ii, ctx.curVal); } } + // region extra-methods + // endregion extra-methods } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java index bd04dc77270..a13e2dc5cdf 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java @@ -1,28 +1,60 @@ package io.deephaven.engine.table.impl.updateby.prod; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.ObjectChunk; +import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.updateby.internal.BaseObjectBinaryOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; +import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import java.math.BigDecimal; import java.math.MathContext; -public final class BigDecimalCumProdOperator extends BaseObjectBinaryOperator { +public final class BigDecimalCumProdOperator extends BaseObjectUpdateByOperator { @NotNull private final MathContext mathContext; + protected class Context extends BaseObjectUpdateByOperator.Context { + public ObjectChunk objectValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + objectValueChunk = valuesChunk.asObjectChunk(); + } + } + public BigDecimalCumProdOperator(@NotNull final MatchPair inputPair, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, @NotNull final MathContext mathContext) { - super(BigDecimal.class, inputPair, new String[] {inputPair.rightColumn}, redirContext); + super(inputPair, new String[] {inputPair.rightColumn}, redirContext, BigDecimal.class); this.mathContext = mathContext; } + @NotNull + @Override + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); + } + @Override - protected BigDecimal doOperation(BigDecimal bucketCurVal, BigDecimal chunkCurVal) { - return bucketCurVal.multiply(chunkCurVal, mathContext); + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + // read the value from the values chunk + final BigDecimal currentVal = ctx.objectValueChunk.get(pos); + + final boolean isCurrentNull = currentVal == null; + if (ctx.curVal == null) { + ctx.curVal = isCurrentNull ? null : currentVal; + } else { + if (!isCurrentNull) { + ctx.curVal = ctx.curVal.multiply(ctx.objectValueChunk.get(pos), mathContext); + } + } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java index 7b6fc90ffb1..105f018a49f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java @@ -1,22 +1,54 @@ package io.deephaven.engine.table.impl.updateby.prod; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.ObjectChunk; +import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.updateby.internal.BaseObjectBinaryOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; +import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import java.math.BigInteger; -public final class BigIntegerCumProdOperator extends BaseObjectBinaryOperator { +public final class BigIntegerCumProdOperator extends BaseObjectUpdateByOperator { + protected class Context extends BaseObjectUpdateByOperator.Context { + public ObjectChunk objectValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + objectValueChunk = valuesChunk.asObjectChunk(); + } + } + public BigIntegerCumProdOperator(@NotNull final MatchPair inputPair, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { - super(BigInteger.class, inputPair, new String[] { inputPair.rightColumn }, redirContext); + super(inputPair, new String[] {inputPair.rightColumn}, redirContext, BigInteger.class); } + @NotNull @Override - protected BigInteger doOperation(BigInteger bucketCurVal, BigInteger chunkCurVal) { - return bucketCurVal.multiply(chunkCurVal); + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); + } + + @Override + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + // read the value from the values chunk + final BigInteger currentVal = ctx.objectValueChunk.get(pos); + + final boolean isCurrentNull = currentVal == null; + if(ctx.curVal == null) { + ctx.curVal = isCurrentNull ? null : currentVal; + } else { + if(!isCurrentNull) { + ctx.curVal = ctx.curVal.multiply(ctx.objectValueChunk.get(pos)); + } + } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java index d202ee5741e..122a99dc946 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java @@ -5,61 +5,61 @@ */ package io.deephaven.engine.table.impl.updateby.prod; -import io.deephaven.chunk.*; -import io.deephaven.chunk.attributes.ChunkLengths; -import io.deephaven.chunk.attributes.ChunkPositions; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.ByteChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.NULL_LONG; import static io.deephaven.util.QueryConstants.NULL_BYTE; public class ByteCumProdOperator extends BaseLongUpdateByOperator { + // region extra-fields + // endregion extra-fields - public ByteCumProdOperator(final @NotNull MatchPair inputPair, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext - // region extra-constructor-args - // endregion extra-constructor-args + protected class Context extends BaseLongUpdateByOperator.Context { + public ByteChunk byteValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + byteValueChunk = valuesChunk.asByteChunk(); + } + } + + public ByteCumProdOperator(@NotNull final MatchPair pair, + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + // region extra-constructor-args + // endregion extra-constructor-args ) { - super(inputPair, new String[]{ inputPair.rightColumn }, redirContext); + super(pair, new String[] { pair.rightColumn }, redirContext); // region constructor // endregion constructor } + @NotNull @Override - protected void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { -// ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_LONG; - accumulate(workingChunk.asByteChunk(), ctx, 0, workingChunk.size()); -// singletonGroup = groupPosition; -// singletonVal = ctx.curVal; - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); } - private void accumulate(@NotNull final ByteChunk asBytes, - @NotNull final Context ctx, - final int runStart, - final int runLength) { - final WritableLongChunk localOutputChunk = ctx.outputValues.get(); - for (int ii = runStart; ii < runStart + runLength; ii++) { - final byte currentVal = asBytes.get(ii); - final boolean isCurrentNull = currentVal == NULL_BYTE;; - if(ctx.curVal == NULL_LONG) { - ctx.curVal = isCurrentNull ? NULL_LONG : currentVal; - } else { - if(!isCurrentNull) { - ctx.curVal *= currentVal; - } - } - localOutputChunk.set(ii, ctx.curVal); + @Override + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + // read the value from the values chunk + final byte currentVal = ctx.byteValueChunk.get(pos); + + if (ctx.curVal == NULL_LONG) { + ctx.curVal = currentVal == NULL_BYTE ? NULL_LONG : currentVal; + } else if (currentVal != NULL_BYTE) { + ctx.curVal *= currentVal; } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java index 3892429924d..81dea1f128b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java @@ -5,62 +5,61 @@ */ package io.deephaven.engine.table.impl.updateby.prod; -import io.deephaven.chunk.*; -import io.deephaven.chunk.attributes.ChunkLengths; -import io.deephaven.chunk.attributes.ChunkPositions; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.DoubleChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseDoubleUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.NULL_DOUBLE; +import static io.deephaven.util.QueryConstants.NULL_LONG; public class DoubleCumProdOperator extends BaseDoubleUpdateByOperator { + // region extra-fields + // endregion extra-fields - public DoubleCumProdOperator(@NotNull final MatchPair inputPair, + protected class Context extends BaseDoubleUpdateByOperator.Context { + public DoubleChunk doubleValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + doubleValueChunk = valuesChunk.asDoubleChunk(); + } + } + + public DoubleCumProdOperator(@NotNull final MatchPair pair, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(inputPair, new String[]{inputPair.rightColumn}, redirContext); + super(pair, new String[] { pair.rightColumn }, redirContext); // region constructor // endregion constructor } - protected void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { - if(Double.isNaN(ctx.curVal)) { - if(!ctx.filledWithPermanentValue) { - ctx.filledWithPermanentValue = true; - ctx.outputValues.get().fillWithValue(0, ctx.outputValues.get().capacity(), Double.NaN); - } - } else { - accumulate(workingChunk.asDoubleChunk(), ctx, 0, workingChunk.size()); - } - - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); + @NotNull + @Override + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); } - private void accumulate(@NotNull final Chunk values, - @NotNull final Context ctx, - final int runStart, - final int runLength) { - final DoubleChunk asDoubles = values.asDoubleChunk(); - final WritableDoubleChunk localOutputValues = ctx.outputValues.get(); - for (int ii = runStart; ii < runStart + runLength; ii++) { - final double currentVal = asDoubles.get(ii); - if (ctx.curVal == NULL_DOUBLE) { - ctx.curVal = currentVal; - } else if (currentVal != NULL_DOUBLE) { - ctx.curVal *= currentVal; - } - localOutputValues.set(ii, ctx.curVal); + @Override + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + // read the value from the values chunk + final double currentVal = ctx.doubleValueChunk.get(pos); + + if (ctx.curVal == NULL_DOUBLE) { + ctx.curVal = currentVal; + } else if (currentVal != NULL_DOUBLE) { + ctx.curVal *= currentVal; } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java index 6d21b12c9aa..72d288e0892 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java @@ -1,61 +1,60 @@ package io.deephaven.engine.table.impl.updateby.prod; -import io.deephaven.chunk.*; -import io.deephaven.chunk.attributes.ChunkLengths; -import io.deephaven.chunk.attributes.ChunkPositions; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.FloatChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseFloatUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.NULL_FLOAT; +import static io.deephaven.util.QueryConstants.NULL_LONG; public class FloatCumProdOperator extends BaseFloatUpdateByOperator { + // region extra-fields + // endregion extra-fields - public FloatCumProdOperator(@NotNull final MatchPair inputPair, + protected class Context extends BaseFloatUpdateByOperator.Context { + public FloatChunk floatValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + floatValueChunk = valuesChunk.asFloatChunk(); + } + } + + public FloatCumProdOperator(@NotNull final MatchPair pair, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(inputPair, new String[]{inputPair.rightColumn}, redirContext); + super(pair, new String[] { pair.rightColumn }, redirContext); // region constructor // endregion constructor } - protected void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { - if(Float.isNaN(ctx.curVal)) { - if(!ctx.filledWithPermanentValue) { - ctx.filledWithPermanentValue = true; - ctx.outputValues.get().fillWithValue(0, ctx.outputValues.get().capacity(), Float.NaN); - } - } else { - accumulate(workingChunk.asFloatChunk(), ctx, 0, workingChunk.size()); - } - - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); + @NotNull + @Override + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); } - private void accumulate(@NotNull final Chunk values, - @NotNull final Context ctx, - final int runStart, - final int runLength) { - final FloatChunk asFloats = values.asFloatChunk(); - final WritableFloatChunk localOutputValues = ctx.outputValues.get(); - for (int ii = runStart; ii < runStart + runLength; ii++) { - final float currentVal = asFloats.get(ii); - if (ctx.curVal == NULL_FLOAT) { - ctx.curVal = currentVal; - } else if (currentVal != NULL_FLOAT) { - ctx.curVal *= currentVal; - } - localOutputValues.set(ii, ctx.curVal); + @Override + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + // read the value from the values chunk + final float currentVal = ctx.floatValueChunk.get(pos); + + if (ctx.curVal == NULL_FLOAT) { + ctx.curVal = currentVal; + } else if (currentVal != NULL_FLOAT) { + ctx.curVal *= currentVal; } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java index 6fd417057f1..d13ec4d7b77 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java @@ -5,61 +5,61 @@ */ package io.deephaven.engine.table.impl.updateby.prod; -import io.deephaven.chunk.*; -import io.deephaven.chunk.attributes.ChunkLengths; -import io.deephaven.chunk.attributes.ChunkPositions; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.NULL_LONG; import static io.deephaven.util.QueryConstants.NULL_INT; public class IntCumProdOperator extends BaseLongUpdateByOperator { + // region extra-fields + // endregion extra-fields - public IntCumProdOperator(final @NotNull MatchPair inputPair, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext - // region extra-constructor-args - // endregion extra-constructor-args + protected class Context extends BaseLongUpdateByOperator.Context { + public IntChunk intValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + intValueChunk = valuesChunk.asIntChunk(); + } + } + + public IntCumProdOperator(@NotNull final MatchPair pair, + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + // region extra-constructor-args + // endregion extra-constructor-args ) { - super(inputPair, new String[]{ inputPair.rightColumn }, redirContext); + super(pair, new String[] { pair.rightColumn }, redirContext); // region constructor // endregion constructor } + @NotNull @Override - protected void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { -// ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_LONG; - accumulate(workingChunk.asIntChunk(), ctx, 0, workingChunk.size()); -// singletonGroup = groupPosition; -// singletonVal = ctx.curVal; - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); } - private void accumulate(@NotNull final IntChunk asIntegers, - @NotNull final Context ctx, - final int runStart, - final int runLength) { - final WritableLongChunk localOutputChunk = ctx.outputValues.get(); - for (int ii = runStart; ii < runStart + runLength; ii++) { - final int currentVal = asIntegers.get(ii); - final boolean isCurrentNull = currentVal == NULL_INT;; - if(ctx.curVal == NULL_LONG) { - ctx.curVal = isCurrentNull ? NULL_LONG : currentVal; - } else { - if(!isCurrentNull) { - ctx.curVal *= currentVal; - } - } - localOutputChunk.set(ii, ctx.curVal); + @Override + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + // read the value from the values chunk + final int currentVal = ctx.intValueChunk.get(pos); + + if (ctx.curVal == NULL_LONG) { + ctx.curVal = currentVal == NULL_INT ? NULL_LONG : currentVal; + } else if (currentVal != NULL_INT) { + ctx.curVal *= currentVal; } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java index ad00bde1bc1..a9550f56e7a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java @@ -5,61 +5,61 @@ */ package io.deephaven.engine.table.impl.updateby.prod; -import io.deephaven.chunk.*; -import io.deephaven.chunk.attributes.ChunkLengths; -import io.deephaven.chunk.attributes.ChunkPositions; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.NULL_LONG; import static io.deephaven.util.QueryConstants.NULL_LONG; public class LongCumProdOperator extends BaseLongUpdateByOperator { + // region extra-fields + // endregion extra-fields - public LongCumProdOperator(final @NotNull MatchPair inputPair, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext - // region extra-constructor-args - // endregion extra-constructor-args + protected class Context extends BaseLongUpdateByOperator.Context { + public LongChunk longValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + longValueChunk = valuesChunk.asLongChunk(); + } + } + + public LongCumProdOperator(@NotNull final MatchPair pair, + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + // region extra-constructor-args + // endregion extra-constructor-args ) { - super(inputPair, new String[]{ inputPair.rightColumn }, redirContext); + super(pair, new String[] { pair.rightColumn }, redirContext); // region constructor // endregion constructor } + @NotNull @Override - protected void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { -// ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_LONG; - accumulate(workingChunk.asLongChunk(), ctx, 0, workingChunk.size()); -// singletonGroup = groupPosition; -// singletonVal = ctx.curVal; - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); } - private void accumulate(@NotNull final LongChunk asLongs, - @NotNull final Context ctx, - final int runStart, - final int runLength) { - final WritableLongChunk localOutputChunk = ctx.outputValues.get(); - for (int ii = runStart; ii < runStart + runLength; ii++) { - final long currentVal = asLongs.get(ii); - final boolean isCurrentNull = currentVal == NULL_LONG;; - if(ctx.curVal == NULL_LONG) { - ctx.curVal = isCurrentNull ? NULL_LONG : currentVal; - } else { - if(!isCurrentNull) { - ctx.curVal *= currentVal; - } - } - localOutputChunk.set(ii, ctx.curVal); + @Override + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + // read the value from the values chunk + final long currentVal = ctx.longValueChunk.get(pos); + + if (ctx.curVal == NULL_LONG) { + ctx.curVal = currentVal == NULL_LONG ? NULL_LONG : currentVal; + } else if (currentVal != NULL_LONG) { + ctx.curVal *= currentVal; } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java index f18852c3ab4..303c8a70d32 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java @@ -1,60 +1,60 @@ package io.deephaven.engine.table.impl.updateby.prod; -import io.deephaven.chunk.*; -import io.deephaven.chunk.attributes.ChunkLengths; -import io.deephaven.chunk.attributes.ChunkPositions; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.ShortChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.NULL_LONG; import static io.deephaven.util.QueryConstants.NULL_SHORT; public class ShortCumProdOperator extends BaseLongUpdateByOperator { + // region extra-fields + // endregion extra-fields - public ShortCumProdOperator(final @NotNull MatchPair inputPair, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext - // region extra-constructor-args - // endregion extra-constructor-args + protected class Context extends BaseLongUpdateByOperator.Context { + public ShortChunk shortValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + shortValueChunk = valuesChunk.asShortChunk(); + } + } + + public ShortCumProdOperator(@NotNull final MatchPair pair, + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + // region extra-constructor-args + // endregion extra-constructor-args ) { - super(inputPair, new String[]{ inputPair.rightColumn }, redirContext); + super(pair, new String[] { pair.rightColumn }, redirContext); // region constructor // endregion constructor } + @NotNull @Override - protected void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { -// ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_LONG; - accumulate(workingChunk.asShortChunk(), ctx, 0, workingChunk.size()); -// singletonGroup = groupPosition; -// singletonVal = ctx.curVal; - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); } - private void accumulate(@NotNull final ShortChunk asShorts, - @NotNull final Context ctx, - final int runStart, - final int runLength) { - final WritableLongChunk localOutputChunk = ctx.outputValues.get(); - for (int ii = runStart; ii < runStart + runLength; ii++) { - final short currentVal = asShorts.get(ii); - final boolean isCurrentNull = currentVal == NULL_SHORT;; - if(ctx.curVal == NULL_LONG) { - ctx.curVal = isCurrentNull ? NULL_LONG : currentVal; - } else { - if(!isCurrentNull) { - ctx.curVal *= currentVal; - } - } - localOutputChunk.set(ii, ctx.curVal); + @Override + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + // read the value from the values chunk + final short currentVal = ctx.shortValueChunk.get(pos); + + if (ctx.curVal == NULL_LONG) { + ctx.curVal = currentVal == NULL_SHORT ? NULL_LONG : currentVal; + } else if (currentVal != NULL_SHORT) { + ctx.curVal *= currentVal; } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java index a230c8adaab..2ef1c33879a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java @@ -1,9 +1,7 @@ package io.deephaven.engine.table.impl.updateby.rollingsum; import io.deephaven.api.updateby.OperationControl; -import io.deephaven.chunk.Chunk; -import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.WritableObjectChunk; +import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.sized.SizedObjectChunk; import io.deephaven.engine.rowset.RowSequence; @@ -12,7 +10,9 @@ import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByOperator; -import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.engine.table.impl.sources.ObjectArraySource; +import io.deephaven.engine.table.impl.sources.ObjectSparseArraySource; +import io.deephaven.engine.table.impl.sources.WritableRedirectedColumnSource; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedObjectUpdateByOperator; import io.deephaven.engine.table.impl.util.SizedSafeCloseable; @@ -28,88 +28,49 @@ public final class BigDecimalRollingSumOperator extends BaseWindowedObjectUpdate @NotNull private final MathContext mathContext; - private final WritableColumnSource outputSource; - private final WritableColumnSource maybeInnerSource; - protected class Context extends BaseWindowedObjectUpdateByOperator.Context { - public final SizedSafeCloseable fillContext; - public final SizedObjectChunk outputValues; - - public BigDecimal currentVal = null; + public ObjectChunk objectInfluencerValuesChunk; - protected Context(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { - this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); - this.fillContext.ensureCapacity(chunkSize); - this.outputValues = new SizedObjectChunk<>(chunkSize); - this.timestampSsa = timestampSsa; + protected Context(final int chunkSize) { + super(chunkSize); } @Override - public void close() { - super.close(); - outputValues.close(); - fillContext.close(); + public void storeInfluencerValuesChunk(@NotNull final Chunk influencerValuesChunk) { + objectInfluencerValuesChunk = influencerValuesChunk.asObjectChunk(); } } @NotNull @Override - public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize, - final LongSegmentedSortedArray timestampSsa) { - return new Context(chunkSize, timestampSsa); - } - - @Override - public void setChunkSize(@NotNull UpdateContext context, int chunkSize) { - final Context ctx = (Context) context; - ctx.outputValues.ensureCapacity(chunkSize); - ctx.fillContext.ensureCapacity(chunkSize); + public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize) { + return new Context(chunkSize); } - public BigDecimalRollingSumOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final String timestampColumnName, - @Nullable final ColumnSource timestampColumnSource, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, - @NotNull final ColumnSource valueSource - // region extra-constructor-args - // endregion extra-constructor-args - , @NotNull final MathContext mathContext) { - super(pair, affectingColumns, control, timestampColumnName, timestampColumnSource, reverseTimeScaleUnits, - forwardTimeScaleUnits, redirContext, valueSource, BigDecimal.class); + @NotNull final MathContext mathContext) { + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, + redirContext, BigDecimal.class); this.mathContext = mathContext; - if (redirContext.isRedirected()) { - // region create-dense - this.maybeInnerSource = new ObjectArraySource(BigDecimal.class); - // endregion create-dense - this.outputSource = - new WritableRedirectedColumnSource(redirContext.getRowRedirection(), maybeInnerSource, 0); - } else { - this.maybeInnerSource = null; - // region create-sparse - this.outputSource = new ObjectSparseArraySource(BigDecimal.class); - // endregion create-sparse - } - - // region constructor - // endregion constructor } @Override public void push(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; - BigDecimal val = (BigDecimal) ctx.candidateValuesChunk.get(pos); + BigDecimal val = ctx.objectInfluencerValuesChunk.get(pos); // increase the running sum if (val != null) { - if (ctx.currentVal == null) { - ctx.currentVal = val; + if (ctx.curVal == null) { + ctx.curVal = val; } else { - ctx.currentVal = ctx.currentVal.add(val, mathContext); + ctx.curVal = ctx.curVal.add(val, mathContext); } } else { ctx.nullCount++; @@ -117,87 +78,16 @@ public void push(UpdateContext context, long key, int pos) { } @Override - public void pop(UpdateContext context, long key, int pos) { + public void pop(UpdateContext context) { final Context ctx = (Context) context; - BigDecimal val = (BigDecimal) ctx.candidateValuesChunk.get(pos); + int pos = ctx.windowIndices.front(); + BigDecimal val = ctx.objectInfluencerValuesChunk.get(pos); // reduce the running sum if (val != null) { - ctx.currentVal = ctx.currentVal.subtract(val, mathContext); + ctx.curVal = ctx.curVal.subtract(val, mathContext); } else { ctx.nullCount--; } } - - @Override - public void reset(UpdateContext context) { - final Context ctx = (Context) context; - ctx.currentVal = null; - } - - @Override - public void doProcessChunk(@NotNull final BaseWindowedObjectUpdateByOperator.Context context, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @NotNull final Chunk workingChunk) { - final Context ctx = (Context) context; - - if (timestampColumnName == null) { - computeTicks(ctx, posChunk, inputKeys.intSize()); - } else { - computeTime(ctx, inputKeys); - } - - // noinspection unchecked - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); - } - - private void computeTicks(@NotNull final Context ctx, - @Nullable final LongChunk posChunk, - final int runLength) { - - final WritableObjectChunk localOutputValues = ctx.outputValues.get(); - for (int ii = 0; ii < runLength; ii++) { - // the output value is computed by push/pop operations triggered by fillWindow - ctx.fillWindowTicks(ctx, posChunk.get(ii)); - localOutputValues.set(ii, ctx.currentVal); - } - } - - private void computeTime(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys) { - - final WritableObjectChunk localOutputValues = ctx.outputValues.get(); - // get the timestamp values for this chunk - try (final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(inputKeys.intSize())) { - LongChunk timestampChunk = - timestampColumnSource.getChunk(context, inputKeys).asLongChunk(); - - for (int ii = 0; ii < inputKeys.intSize(); ii++) { - // the output value is computed by push/pop operations triggered by fillWindow - ctx.fillWindowTime(ctx, timestampChunk.get(ii)); - localOutputValues.set(ii, ctx.currentVal); - } - } - } - - @NotNull - @Override - public Map> getOutputColumns() { - return Collections.singletonMap(pair.leftColumn, outputSource); - } - - @Override - public void startTrackingPrev() { - outputSource.startTrackingPrevValues(); - if (redirContext.isRedirected()) { - maybeInnerSource.startTrackingPrevValues(); - } - } - - @Override - public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { - ((ObjectSparseArraySource) outputSource).shift(subIndexToShift, delta); - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java index 189060bbb1e..18cb21a72c5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java @@ -1,9 +1,7 @@ package io.deephaven.engine.table.impl.updateby.rollingsum; import io.deephaven.api.updateby.OperationControl; -import io.deephaven.chunk.Chunk; -import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.WritableObjectChunk; +import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.sized.SizedObjectChunk; import io.deephaven.engine.rowset.RowSequence; @@ -26,69 +24,36 @@ import java.util.Map; public final class BigIntegerRollingSumOperator extends BaseWindowedObjectUpdateByOperator { - private final WritableColumnSource outputSource; - private final WritableColumnSource maybeInnerSource; - protected class Context extends BaseWindowedObjectUpdateByOperator.Context { - public final SizedSafeCloseable fillContext; - public final SizedObjectChunk outputValues; - - public BigInteger currentVal = null; + public ObjectChunk objectInfluencerValuesChunk; - protected Context(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { - this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); - this.fillContext.ensureCapacity(chunkSize); - this.outputValues = new SizedObjectChunk<>(chunkSize); - this.timestampSsa = timestampSsa; + protected Context(final int chunkSize) { + super(chunkSize); } @Override - public void close() { - super.close(); - outputValues.close(); - fillContext.close(); + public void storeInfluencerValuesChunk(@NotNull final Chunk influencerValuesChunk) { + objectInfluencerValuesChunk = influencerValuesChunk.asObjectChunk(); } } @NotNull @Override - public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { - return new Context(chunkSize, timestampSsa); - } - - @Override - public void setChunkSize(@NotNull UpdateContext context, int chunkSize) { - final Context ctx = (Context) context; - ctx.outputValues.ensureCapacity(chunkSize); - ctx.fillContext.ensureCapacity(chunkSize); + public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize) { + return new Context(chunkSize); } - public BigIntegerRollingSumOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final String timestampColumnName, - @Nullable final ColumnSource timestampColumnSource, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, - @NotNull final ColumnSource valueSource + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, timestampColumnSource, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource, BigInteger.class); - if(redirContext.isRedirected()) { - // region create-dense - this.maybeInnerSource = new ObjectArraySource(BigInteger.class); - // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(redirContext.getRowRedirection(), maybeInnerSource, 0); - } else { - this.maybeInnerSource = null; - // region create-sparse - this.outputSource = new ObjectSparseArraySource(BigInteger.class); - // endregion create-sparse - } - + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, BigInteger.class); // region constructor // endregion constructor } @@ -96,14 +61,14 @@ public BigIntegerRollingSumOperator(@NotNull final MatchPair pair, @Override public void push(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; - BigInteger val = ctx.candidateValuesChunk.get(pos); + BigInteger val = ctx.objectInfluencerValuesChunk.get(pos); // increase the running sum if (val != null) { - if (ctx.currentVal == null) { - ctx.currentVal = val; + if (ctx.curVal == null) { + ctx.curVal = val; } else { - ctx.currentVal = ctx.currentVal.add(val); + ctx.curVal = ctx.curVal.add(val); } } else { ctx.nullCount++; @@ -111,13 +76,14 @@ public void push(UpdateContext context, long key, int pos) { } @Override - public void pop(UpdateContext context, long key, int pos) { - final BigIntegerRollingSumOperator.Context ctx = (BigIntegerRollingSumOperator.Context) context; - BigInteger val = ctx.candidateValuesChunk.get(pos); + public void pop(UpdateContext context) { + final Context ctx = (Context) context; + int pos = ctx.windowIndices.front(); + BigInteger val = ctx.objectInfluencerValuesChunk.get(pos); // reduce the running sum if (val != null) { - ctx.currentVal = ctx.currentVal.subtract(val); + ctx.curVal = ctx.curVal.subtract(val); } else { ctx.nullCount--; } @@ -125,73 +91,7 @@ public void pop(UpdateContext context, long key, int pos) { @Override public void reset(UpdateContext context) { - final BigIntegerRollingSumOperator.Context ctx = (BigIntegerRollingSumOperator.Context) context; - ctx.currentVal = null; - } - - @Override - public void doProcessChunk(@NotNull final BaseWindowedObjectUpdateByOperator.Context context, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @NotNull final Chunk workingChunk) { - final BigIntegerRollingSumOperator.Context ctx = (BigIntegerRollingSumOperator.Context) context; - - if (timestampColumnName == null) { - computeTicks(ctx, posChunk, inputKeys.intSize()); - } else { - computeTime(ctx, inputKeys); - } - - //noinspection unchecked - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); - } - - private void computeTicks(@NotNull final BigIntegerRollingSumOperator.Context ctx, - @Nullable final LongChunk posChunk, - final int runLength) { - - final WritableObjectChunk localOutputValues = ctx.outputValues.get(); - for (int ii = 0; ii < runLength; ii++) { - // the output value is computed by push/pop operations triggered by fillWindow - ctx.fillWindowTicks(ctx, posChunk.get(ii)); - localOutputValues.set(ii, ctx.currentVal); - } - } - - private void computeTime(@NotNull final BigIntegerRollingSumOperator.Context ctx, - @NotNull final RowSequence inputKeys) { - - final WritableObjectChunk localOutputValues = ctx.outputValues.get(); - // get the timestamp values for this chunk - try (final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(inputKeys.intSize())) { - LongChunk timestampChunk = timestampColumnSource.getChunk(context, inputKeys).asLongChunk(); - - for (int ii = 0; ii < inputKeys.intSize(); ii++) { - // the output value is computed by push/pop operations triggered by fillWindow - ctx.fillWindowTime(ctx, timestampChunk.get(ii)); - localOutputValues.set(ii, ctx.currentVal); - } - } - } - - @NotNull - @Override - public Map> getOutputColumns() { - return Collections.singletonMap(pair.leftColumn, outputSource); - } - - @Override - public void startTrackingPrev() { - outputSource.startTrackingPrevValues(); - if(redirContext.isRedirected()) { - maybeInnerSource.startTrackingPrevValues(); - } - } - - @Override - public void applyOutputShift(@NotNull final RowSet subIndexToShift, - final long delta) { - ((ObjectSparseArraySource)outputSource).shift(subIndexToShift, delta); + final Context ctx = (Context) context; + ctx.curVal = null; } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java index b1787862a07..ddc430d1a1f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java @@ -6,19 +6,13 @@ package io.deephaven.engine.table.impl.updateby.rollingsum; import io.deephaven.api.updateby.OperationControl; -import io.deephaven.chunk.*; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.ByteChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.sized.SizedLongChunk; -import io.deephaven.engine.rowset.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; -import io.deephaven.engine.table.*; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.UpdateByOperator; -import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; -import io.deephaven.engine.table.impl.updateby.internal.*; -import io.deephaven.engine.table.impl.util.SizedSafeCloseable; +import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedLongUpdateByOperator; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -28,76 +22,42 @@ import static io.deephaven.util.QueryConstants.NULL_LONG; import static io.deephaven.util.QueryConstants.NULL_BYTE; -public class ByteRollingSumOperator extends BaseWindowedByteUpdateByOperator { - - // RollingSum will output Long values for integral types - private final WritableColumnSource outputSource; - private final WritableColumnSource maybeInnerSource; - +public class ByteRollingSumOperator extends BaseWindowedLongUpdateByOperator { // region extra-fields final byte nullValue; // endregion extra-fields - protected class Context extends BaseWindowedByteUpdateByOperator.Context { - public final SizedSafeCloseable fillContext; - public final SizedLongChunk outputValues; - - public long currentVal = NULL_LONG; + protected class Context extends BaseWindowedLongUpdateByOperator.Context { + public ByteChunk byteInfluencerValuesChunk; - protected Context(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { - this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); - this.fillContext.ensureCapacity(chunkSize); - this.outputValues = new SizedLongChunk<>(chunkSize); - this.timestampSsa = timestampSsa; + protected Context(int chunkSize) { + super(chunkSize); } @Override - public void close() { - super.close(); - outputValues.close(); - fillContext.close(); + public void storeInfluencerValuesChunk(@NotNull final Chunk influencerValuesChunk) { + byteInfluencerValuesChunk = influencerValuesChunk.asByteChunk(); } } @NotNull @Override - public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { - return new Context(chunkSize, timestampSsa); - } - - @Override - public void setChunkSize(@NotNull UpdateContext context, int chunkSize) { - final Context ctx = (Context) context; - ctx.outputValues.ensureCapacity(chunkSize); - ctx.fillContext.ensureCapacity(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize) { + return new Context(chunkSize); } public ByteRollingSumOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final String timestampColumnName, - @Nullable final ColumnSource timestampColumnSource, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, - @NotNull final ColumnSource valueSource + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args ,final byte nullValue // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, timestampColumnSource, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource); - if(redirContext.isRedirected()) { - // region create-dense - this.maybeInnerSource = new LongArraySource(); - // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(redirContext.getRowRedirection(), maybeInnerSource, 0); - } else { - this.maybeInnerSource = null; - // region create-sparse - this.outputSource = new LongSparseArraySource(); - // endregion create-sparse - } - + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); // region constructor this.nullValue = nullValue; // endregion constructor @@ -106,14 +66,14 @@ public ByteRollingSumOperator(@NotNull final MatchPair pair, @Override public void push(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; - byte val = ctx.candidateValuesChunk.get(pos); + byte val = ctx.byteInfluencerValuesChunk.get(pos); // increase the running sum if (val != NULL_BYTE) { - if (ctx.currentVal == NULL_LONG) { - ctx.currentVal = val; + if (ctx.curVal == NULL_LONG) { + ctx.curVal = val; } else { - ctx.currentVal += val; + ctx.curVal += val; } } else { ctx.nullCount++; @@ -121,86 +81,16 @@ public void push(UpdateContext context, long key, int pos) { } @Override - public void pop(UpdateContext context, long key, int pos) { + public void pop(UpdateContext context) { final Context ctx = (Context) context; - byte val = ctx.candidateValuesChunk.get(pos); + int pos = ctx.windowIndices.front(); + byte val = ctx.byteInfluencerValuesChunk.get(pos); // reduce the running sum if (val != NULL_BYTE) { - ctx.currentVal -= val; + ctx.curVal -= val; } else { ctx.nullCount--; } } - - @Override - public void reset(UpdateContext context) { - final Context ctx = (Context) context; - ctx.currentVal = NULL_LONG; - } - - @Override - public void doProcessChunk(@NotNull final BaseWindowedByteUpdateByOperator.Context context, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @NotNull final Chunk workingChunk) { - final Context ctx = (Context) context; - - if (timestampColumnName == null) { - computeTicks(ctx, posChunk, inputKeys.intSize()); - } else { - computeTime(ctx, inputKeys); - } - - //noinspection unchecked - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); - } - - private void computeTicks(@NotNull final Context ctx, - @Nullable final LongChunk posChunk, - final int runLength) { - - final WritableLongChunk localOutputValues = ctx.outputValues.get(); - for (int ii = 0; ii < runLength; ii++) { - // the output value is computed by push/pop operations triggered by fillWindow - ctx.fillWindowTicks(ctx, posChunk.get(ii)); - localOutputValues.set(ii, ctx.currentVal); - } - } - - private void computeTime(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys) { - - final WritableLongChunk localOutputValues = ctx.outputValues.get(); - // get the timestamp values for this chunk - try (final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(inputKeys.intSize())) { - LongChunk timestampChunk = timestampColumnSource.getChunk(context, inputKeys).asLongChunk(); - - for (int ii = 0; ii < inputKeys.intSize(); ii++) { - // the output value is computed by push/pop operations triggered by fillWindow - ctx.fillWindowTime(ctx, timestampChunk.get(ii)); - localOutputValues.set(ii, ctx.currentVal); - } - } - } - - @NotNull - @Override - public Map> getOutputColumns() { - return Collections.singletonMap(pair.leftColumn, outputSource); - } - - @Override - public void startTrackingPrev() { - outputSource.startTrackingPrevValues(); - if(redirContext.isRedirected()) { - maybeInnerSource.startTrackingPrevValues(); - } - } - - @Override - public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { - ((LongSparseArraySource)outputSource).shift(subIndexToShift, delta); - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java index 640cf45b833..3e9b40836e8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java @@ -6,9 +6,7 @@ package io.deephaven.engine.table.impl.updateby.rollingsum; import io.deephaven.api.updateby.OperationControl; -import io.deephaven.chunk.Chunk; -import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.WritableDoubleChunk; +import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.sized.SizedDoubleChunk; import io.deephaven.chunk.sized.SizedLongChunk; @@ -32,88 +30,43 @@ import static io.deephaven.util.QueryConstants.NULL_DOUBLE; public class DoubleRollingSumOperator extends BaseWindowedDoubleUpdateByOperator { - - // RollingSum will output Long values for integral types - private final WritableColumnSource outputSource; - private final WritableColumnSource maybeInnerSource; - // region extra-fields // endregion extra-fields protected class Context extends BaseWindowedDoubleUpdateByOperator.Context { - public final SizedSafeCloseable fillContext; - public final SizedDoubleChunk outputValues; - - // position data for the chunk being currently processed - public SizedLongChunk valuePositionChunk; + public DoubleChunk doubleInfluencerValuesChunk; public PairwiseDoubleRingBuffer pairwiseSum; - protected Context(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { - this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); - this.fillContext.ensureCapacity(chunkSize); - this.outputValues = new SizedDoubleChunk<>(chunkSize); - this.valuePositionChunk = new SizedLongChunk<>(chunkSize); - this.timestampSsa = timestampSsa; - final int initialSize; - if (timestampColumnName == null) { - // we know exactly the size and will never grow when using ticks - initialSize = (int)(reverseTimeScaleUnits + forwardTimeScaleUnits + 1); - } else { - initialSize = 64; // too big and the log(m) operation costs but growth also costs - } - this.pairwiseSum = new PairwiseDoubleRingBuffer(initialSize, 0.0f, (a, b) -> a + b); + protected Context(final int chunkSize) { + super(chunkSize); + this.pairwiseSum = new PairwiseDoubleRingBuffer(64, 0.0f, Double::sum); } @Override public void close() { super.close(); - outputValues.close(); - fillContext.close(); - this.valuePositionChunk.close(); this.pairwiseSum.close(); } } @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { - return new Context(chunkSize, timestampSsa); - } - - @Override - public void setChunkSize(@NotNull UpdateContext context, int chunkSize) { - final Context ctx = (Context) context; - ctx.outputValues.ensureCapacity(chunkSize); - ctx.fillContext.ensureCapacity(chunkSize); - ctx.valuePositionChunk.ensureCapacity(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize) { + return new Context(chunkSize); } public DoubleRollingSumOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final String timestampColumnName, - @Nullable final ColumnSource timestampColumnSource, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, - @NotNull final ColumnSource valueSource + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, timestampColumnSource, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource); - if(redirContext.isRedirected()) { - // region create-dense - this.maybeInnerSource = new DoubleArraySource(); - // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(redirContext.getRowRedirection(), maybeInnerSource, 0); - } else { - this.maybeInnerSource = null; - // region create-sparse - this.outputSource = new DoubleSparseArraySource(); - // endregion create-sparse - } - + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); // region constructor // endregion constructor } @@ -121,7 +74,7 @@ public DoubleRollingSumOperator(@NotNull final MatchPair pair, @Override public void push(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; - double val = ctx.candidateValuesChunk.get(pos); + double val = ctx.doubleInfluencerValuesChunk.get(pos); if (val != NULL_DOUBLE) { ctx.pairwiseSum.push(val); @@ -132,9 +85,11 @@ public void push(UpdateContext context, long key, int pos) { } @Override - public void pop(UpdateContext context, long key, int pos) { + public void pop(UpdateContext context) { final Context ctx = (Context) context; - double val = ctx.candidateValuesChunk.get(pos); + int pos = ctx.windowIndices.front(); + double val = ctx.doubleInfluencerValuesChunk.get(pos); + if (val == NULL_DOUBLE) { ctx.nullCount--; } @@ -144,63 +99,18 @@ public void pop(UpdateContext context, long key, int pos) { @Override public void reset(UpdateContext context) { final Context ctx = (Context) context; + // take this opportunity to clear the pairwise structure + ctx.pairwiseSum.clear(); + ctx.nullCount = 0; } @Override - public void doProcessChunk(@NotNull final BaseWindowedDoubleUpdateByOperator.Context context, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @NotNull final Chunk workingChunk) { + public double result(UpdateContext context) { final Context ctx = (Context) context; - - if (timestampColumnName == null) { - computeTicks(ctx, posChunk, inputKeys.intSize()); - } else { - computeTime(ctx, inputKeys); - } - - //noinspection unchecked - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); - } - - private void computeTicks(@NotNull final Context ctx, - @Nullable final LongChunk posChunk, - final int runLength) { - - final WritableDoubleChunk localOutputValues = ctx.outputValues.get(); - for (int ii = 0; ii < runLength; ii++) { - ctx.fillWindowTicks(ctx, posChunk.get(ii)); - - if (ctx.pairwiseSum.isEmpty() || ctx.pairwiseSum.size() == ctx.nullCount) { - localOutputValues.set(ii, NULL_DOUBLE); - } else { - double val = ctx.pairwiseSum.evaluate(); - localOutputValues.set(ii, val); - } - } - } - - private void computeTime(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys) { - - final WritableDoubleChunk localOutputValues = ctx.outputValues.get(); - // get the timestamp values for this chunk - try (final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(inputKeys.intSize())) { - LongChunk timestampChunk = timestampColumnSource.getChunk(context, inputKeys).asLongChunk(); - - for (int ii = 0; ii < inputKeys.intSize(); ii++) { - // the output value is computed by push/pop operations triggered by fillWindow - ctx.fillWindowTime(ctx, timestampChunk.get(ii)); - - if (ctx.pairwiseSum.isEmpty() || ctx.pairwiseSum.size() == ctx.nullCount) { - localOutputValues.set(ii, NULL_DOUBLE); - } else { - double val = ctx.pairwiseSum.evaluate(); - localOutputValues.set(ii, val); - } - } + if (ctx.pairwiseSum.size() == ctx.nullCount) { + return NULL_DOUBLE; } + return ctx.pairwiseSum.evaluate(); } @NotNull @@ -208,17 +118,4 @@ private void computeTime(@NotNull final Context ctx, public Map> getOutputColumns() { return Collections.singletonMap(pair.leftColumn, outputSource); } - - @Override - public void startTrackingPrev() { - outputSource.startTrackingPrevValues(); - if(redirContext.isRedirected()) { - maybeInnerSource.startTrackingPrevValues(); - } - } - - @Override - public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { - ((DoubleSparseArraySource)outputSource).shift(subIndexToShift, delta); - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java index 5d20daffd35..bd3684e3a9f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java @@ -1,9 +1,7 @@ package io.deephaven.engine.table.impl.updateby.rollingsum; import io.deephaven.api.updateby.OperationControl; -import io.deephaven.chunk.Chunk; -import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.WritableFloatChunk; +import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.sized.SizedFloatChunk; import io.deephaven.chunk.sized.SizedLongChunk; @@ -27,88 +25,43 @@ import static io.deephaven.util.QueryConstants.NULL_FLOAT; public class FloatRollingSumOperator extends BaseWindowedFloatUpdateByOperator { - - // RollingSum will output Long values for integral types - private final WritableColumnSource outputSource; - private final WritableColumnSource maybeInnerSource; - // region extra-fields // endregion extra-fields protected class Context extends BaseWindowedFloatUpdateByOperator.Context { - public final SizedSafeCloseable fillContext; - public final SizedFloatChunk outputValues; - - // position data for the chunk being currently processed - public SizedLongChunk valuePositionChunk; + public FloatChunk floatInfluencerValuesChunk; public PairwiseFloatRingBuffer pairwiseSum; - protected Context(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { - this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); - this.fillContext.ensureCapacity(chunkSize); - this.outputValues = new SizedFloatChunk<>(chunkSize); - this.valuePositionChunk = new SizedLongChunk<>(chunkSize); - this.timestampSsa = timestampSsa; - final int initialSize; - if (timestampColumnName == null) { - // we know exactly the size and will never grow when using ticks - initialSize = (int)(reverseTimeScaleUnits + forwardTimeScaleUnits + 1); - } else { - initialSize = 64; // too big and the log(m) operation costs but growth also costs - } - this.pairwiseSum = new PairwiseFloatRingBuffer(initialSize, 0.0f, (a, b) -> a + b); + protected Context(final int chunkSize) { + super(chunkSize); + this.pairwiseSum = new PairwiseFloatRingBuffer(64, 0.0f, Float::sum); } @Override public void close() { super.close(); - outputValues.close(); - fillContext.close(); - this.valuePositionChunk.close(); this.pairwiseSum.close(); } } @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { - return new Context(chunkSize, timestampSsa); - } - - @Override - public void setChunkSize(@NotNull UpdateContext context, int chunkSize) { - final Context ctx = (Context) context; - ctx.outputValues.ensureCapacity(chunkSize); - ctx.fillContext.ensureCapacity(chunkSize); - ctx.valuePositionChunk.ensureCapacity(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize) { + return new Context(chunkSize); } public FloatRollingSumOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final String timestampColumnName, - @Nullable final ColumnSource timestampColumnSource, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, - @NotNull final ColumnSource valueSource + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, timestampColumnSource, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource); - if(redirContext.isRedirected()) { - // region create-dense - this.maybeInnerSource = new FloatArraySource(); - // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(redirContext.getRowRedirection(), maybeInnerSource, 0); - } else { - this.maybeInnerSource = null; - // region create-sparse - this.outputSource = new FloatSparseArraySource(); - // endregion create-sparse - } - + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); // region constructor // endregion constructor } @@ -116,7 +69,7 @@ public FloatRollingSumOperator(@NotNull final MatchPair pair, @Override public void push(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; - float val = ctx.candidateValuesChunk.get(pos); + float val = ctx.floatInfluencerValuesChunk.get(pos); if (val != NULL_FLOAT) { ctx.pairwiseSum.push(val); @@ -127,9 +80,11 @@ public void push(UpdateContext context, long key, int pos) { } @Override - public void pop(UpdateContext context, long key, int pos) { + public void pop(UpdateContext context) { final Context ctx = (Context) context; - float val = ctx.candidateValuesChunk.get(pos); + int pos = ctx.windowIndices.front(); + float val = ctx.floatInfluencerValuesChunk.get(pos); + if (val == NULL_FLOAT) { ctx.nullCount--; } @@ -139,63 +94,18 @@ public void pop(UpdateContext context, long key, int pos) { @Override public void reset(UpdateContext context) { final Context ctx = (Context) context; + // take this opportunity to clear the pairwise structure + ctx.pairwiseSum.clear(); + ctx.nullCount = 0; } @Override - public void doProcessChunk(@NotNull final BaseWindowedFloatUpdateByOperator.Context context, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @NotNull final Chunk workingChunk) { + public float result(UpdateContext context) { final Context ctx = (Context) context; - - if (timestampColumnName == null) { - computeTicks(ctx, posChunk, inputKeys.intSize()); - } else { - computeTime(ctx, inputKeys); - } - - //noinspection unchecked - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); - } - - private void computeTicks(@NotNull final Context ctx, - @Nullable final LongChunk posChunk, - final int runLength) { - - final WritableFloatChunk localOutputValues = ctx.outputValues.get(); - for (int ii = 0; ii < runLength; ii++) { - ctx.fillWindowTicks(ctx, posChunk.get(ii)); - - if (ctx.pairwiseSum.isEmpty() || ctx.pairwiseSum.size() == ctx.nullCount) { - localOutputValues.set(ii, NULL_FLOAT); - } else { - float val = ctx.pairwiseSum.evaluate(); - localOutputValues.set(ii, val); - } - } - } - - private void computeTime(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys) { - - final WritableFloatChunk localOutputValues = ctx.outputValues.get(); - // get the timestamp values for this chunk - try (final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(inputKeys.intSize())) { - LongChunk timestampChunk = timestampColumnSource.getChunk(context, inputKeys).asLongChunk(); - - for (int ii = 0; ii < inputKeys.intSize(); ii++) { - // the output value is computed by push/pop operations triggered by fillWindow - ctx.fillWindowTime(ctx, timestampChunk.get(ii)); - - if (ctx.pairwiseSum.isEmpty() || ctx.pairwiseSum.size() == ctx.nullCount) { - localOutputValues.set(ii, NULL_FLOAT); - } else { - float val = ctx.pairwiseSum.evaluate(); - localOutputValues.set(ii, val); - } - } + if (ctx.pairwiseSum.size() == ctx.nullCount) { + return NULL_FLOAT; } + return ctx.pairwiseSum.evaluate(); } @NotNull @@ -203,17 +113,4 @@ private void computeTime(@NotNull final Context ctx, public Map> getOutputColumns() { return Collections.singletonMap(pair.leftColumn, outputSource); } - - @Override - public void startTrackingPrev() { - outputSource.startTrackingPrevValues(); - if(redirContext.isRedirected()) { - maybeInnerSource.startTrackingPrevValues(); - } - } - - @Override - public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { - ((FloatSparseArraySource)outputSource).shift(subIndexToShift, delta); - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java index 35125f3faa6..3604688e0df 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java @@ -6,19 +6,13 @@ package io.deephaven.engine.table.impl.updateby.rollingsum; import io.deephaven.api.updateby.OperationControl; -import io.deephaven.chunk.*; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.sized.SizedLongChunk; -import io.deephaven.engine.rowset.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; -import io.deephaven.engine.table.*; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.UpdateByOperator; -import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; -import io.deephaven.engine.table.impl.updateby.internal.*; -import io.deephaven.engine.table.impl.util.SizedSafeCloseable; +import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedLongUpdateByOperator; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -28,74 +22,40 @@ import static io.deephaven.util.QueryConstants.NULL_LONG; import static io.deephaven.util.QueryConstants.NULL_INT; -public class IntRollingSumOperator extends BaseWindowedIntUpdateByOperator { - - // RollingSum will output Long values for integral types - private final WritableColumnSource outputSource; - private final WritableColumnSource maybeInnerSource; - +public class IntRollingSumOperator extends BaseWindowedLongUpdateByOperator { // region extra-fields // endregion extra-fields - protected class Context extends BaseWindowedIntUpdateByOperator.Context { - public final SizedSafeCloseable fillContext; - public final SizedLongChunk outputValues; - - public long currentVal = NULL_LONG; + protected class Context extends BaseWindowedLongUpdateByOperator.Context { + public IntChunk intInfluencerValuesChunk; - protected Context(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { - this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); - this.fillContext.ensureCapacity(chunkSize); - this.outputValues = new SizedLongChunk<>(chunkSize); - this.timestampSsa = timestampSsa; + protected Context(int chunkSize) { + super(chunkSize); } @Override - public void close() { - super.close(); - outputValues.close(); - fillContext.close(); + public void storeInfluencerValuesChunk(@NotNull final Chunk influencerValuesChunk) { + intInfluencerValuesChunk = influencerValuesChunk.asIntChunk(); } } @NotNull @Override - public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { - return new Context(chunkSize, timestampSsa); - } - - @Override - public void setChunkSize(@NotNull UpdateContext context, int chunkSize) { - final Context ctx = (Context) context; - ctx.outputValues.ensureCapacity(chunkSize); - ctx.fillContext.ensureCapacity(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize) { + return new Context(chunkSize); } public IntRollingSumOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final String timestampColumnName, - @Nullable final ColumnSource timestampColumnSource, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, - @NotNull final ColumnSource valueSource + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, timestampColumnSource, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource); - if(redirContext.isRedirected()) { - // region create-dense - this.maybeInnerSource = new LongArraySource(); - // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(redirContext.getRowRedirection(), maybeInnerSource, 0); - } else { - this.maybeInnerSource = null; - // region create-sparse - this.outputSource = new LongSparseArraySource(); - // endregion create-sparse - } - + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); // region constructor // endregion constructor } @@ -103,14 +63,14 @@ public IntRollingSumOperator(@NotNull final MatchPair pair, @Override public void push(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; - int val = ctx.candidateValuesChunk.get(pos); + int val = ctx.intInfluencerValuesChunk.get(pos); // increase the running sum if (val != NULL_INT) { - if (ctx.currentVal == NULL_LONG) { - ctx.currentVal = val; + if (ctx.curVal == NULL_LONG) { + ctx.curVal = val; } else { - ctx.currentVal += val; + ctx.curVal += val; } } else { ctx.nullCount++; @@ -118,86 +78,16 @@ public void push(UpdateContext context, long key, int pos) { } @Override - public void pop(UpdateContext context, long key, int pos) { + public void pop(UpdateContext context) { final Context ctx = (Context) context; - int val = ctx.candidateValuesChunk.get(pos); + int pos = ctx.windowIndices.front(); + int val = ctx.intInfluencerValuesChunk.get(pos); // reduce the running sum if (val != NULL_INT) { - ctx.currentVal -= val; + ctx.curVal -= val; } else { ctx.nullCount--; } } - - @Override - public void reset(UpdateContext context) { - final Context ctx = (Context) context; - ctx.currentVal = NULL_LONG; - } - - @Override - public void doProcessChunk(@NotNull final BaseWindowedIntUpdateByOperator.Context context, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @NotNull final Chunk workingChunk) { - final Context ctx = (Context) context; - - if (timestampColumnName == null) { - computeTicks(ctx, posChunk, inputKeys.intSize()); - } else { - computeTime(ctx, inputKeys); - } - - //noinspection unchecked - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); - } - - private void computeTicks(@NotNull final Context ctx, - @Nullable final LongChunk posChunk, - final int runLength) { - - final WritableLongChunk localOutputValues = ctx.outputValues.get(); - for (int ii = 0; ii < runLength; ii++) { - // the output value is computed by push/pop operations triggered by fillWindow - ctx.fillWindowTicks(ctx, posChunk.get(ii)); - localOutputValues.set(ii, ctx.currentVal); - } - } - - private void computeTime(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys) { - - final WritableLongChunk localOutputValues = ctx.outputValues.get(); - // get the timestamp values for this chunk - try (final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(inputKeys.intSize())) { - LongChunk timestampChunk = timestampColumnSource.getChunk(context, inputKeys).asLongChunk(); - - for (int ii = 0; ii < inputKeys.intSize(); ii++) { - // the output value is computed by push/pop operations triggered by fillWindow - ctx.fillWindowTime(ctx, timestampChunk.get(ii)); - localOutputValues.set(ii, ctx.currentVal); - } - } - } - - @NotNull - @Override - public Map> getOutputColumns() { - return Collections.singletonMap(pair.leftColumn, outputSource); - } - - @Override - public void startTrackingPrev() { - outputSource.startTrackingPrevValues(); - if(redirContext.isRedirected()) { - maybeInnerSource.startTrackingPrevValues(); - } - } - - @Override - public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { - ((LongSparseArraySource)outputSource).shift(subIndexToShift, delta); - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java index a5ed7d7bbe1..0b199f86802 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java @@ -6,19 +6,13 @@ package io.deephaven.engine.table.impl.updateby.rollingsum; import io.deephaven.api.updateby.OperationControl; -import io.deephaven.chunk.*; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.sized.SizedLongChunk; -import io.deephaven.engine.rowset.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; -import io.deephaven.engine.table.*; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.UpdateByOperator; -import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; -import io.deephaven.engine.table.impl.updateby.internal.*; -import io.deephaven.engine.table.impl.util.SizedSafeCloseable; +import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedLongUpdateByOperator; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -29,73 +23,39 @@ import static io.deephaven.util.QueryConstants.NULL_LONG; public class LongRollingSumOperator extends BaseWindowedLongUpdateByOperator { - - // RollingSum will output Long values for integral types - private final WritableColumnSource outputSource; - private final WritableColumnSource maybeInnerSource; - // region extra-fields // endregion extra-fields protected class Context extends BaseWindowedLongUpdateByOperator.Context { - public final SizedSafeCloseable fillContext; - public final SizedLongChunk outputValues; - - public long currentVal = NULL_LONG; + public LongChunk longInfluencerValuesChunk; - protected Context(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { - this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); - this.fillContext.ensureCapacity(chunkSize); - this.outputValues = new SizedLongChunk<>(chunkSize); - this.timestampSsa = timestampSsa; + protected Context(int chunkSize) { + super(chunkSize); } @Override - public void close() { - super.close(); - outputValues.close(); - fillContext.close(); + public void storeInfluencerValuesChunk(@NotNull final Chunk influencerValuesChunk) { + longInfluencerValuesChunk = influencerValuesChunk.asLongChunk(); } } @NotNull @Override - public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { - return new Context(chunkSize, timestampSsa); - } - - @Override - public void setChunkSize(@NotNull UpdateContext context, int chunkSize) { - final Context ctx = (Context) context; - ctx.outputValues.ensureCapacity(chunkSize); - ctx.fillContext.ensureCapacity(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize) { + return new Context(chunkSize); } public LongRollingSumOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final String timestampColumnName, - @Nullable final ColumnSource timestampColumnSource, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, - @NotNull final ColumnSource valueSource + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, timestampColumnSource, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource); - if(redirContext.isRedirected()) { - // region create-dense - this.maybeInnerSource = new LongArraySource(); - // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(redirContext.getRowRedirection(), maybeInnerSource, 0); - } else { - this.maybeInnerSource = null; - // region create-sparse - this.outputSource = new LongSparseArraySource(); - // endregion create-sparse - } - + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); // region constructor // endregion constructor } @@ -103,14 +63,14 @@ public LongRollingSumOperator(@NotNull final MatchPair pair, @Override public void push(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; - long val = ctx.candidateValuesChunk.get(pos); + long val = ctx.longInfluencerValuesChunk.get(pos); // increase the running sum if (val != NULL_LONG) { - if (ctx.currentVal == NULL_LONG) { - ctx.currentVal = val; + if (ctx.curVal == NULL_LONG) { + ctx.curVal = val; } else { - ctx.currentVal += val; + ctx.curVal += val; } } else { ctx.nullCount++; @@ -118,86 +78,16 @@ public void push(UpdateContext context, long key, int pos) { } @Override - public void pop(UpdateContext context, long key, int pos) { + public void pop(UpdateContext context) { final Context ctx = (Context) context; - long val = ctx.candidateValuesChunk.get(pos); + int pos = ctx.windowIndices.front(); + long val = ctx.longInfluencerValuesChunk.get(pos); // reduce the running sum if (val != NULL_LONG) { - ctx.currentVal -= val; + ctx.curVal -= val; } else { ctx.nullCount--; } } - - @Override - public void reset(UpdateContext context) { - final Context ctx = (Context) context; - ctx.currentVal = NULL_LONG; - } - - @Override - public void doProcessChunk(@NotNull final BaseWindowedLongUpdateByOperator.Context context, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @NotNull final Chunk workingChunk) { - final Context ctx = (Context) context; - - if (timestampColumnName == null) { - computeTicks(ctx, posChunk, inputKeys.intSize()); - } else { - computeTime(ctx, inputKeys); - } - - //noinspection unchecked - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); - } - - private void computeTicks(@NotNull final Context ctx, - @Nullable final LongChunk posChunk, - final int runLength) { - - final WritableLongChunk localOutputValues = ctx.outputValues.get(); - for (int ii = 0; ii < runLength; ii++) { - // the output value is computed by push/pop operations triggered by fillWindow - ctx.fillWindowTicks(ctx, posChunk.get(ii)); - localOutputValues.set(ii, ctx.currentVal); - } - } - - private void computeTime(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys) { - - final WritableLongChunk localOutputValues = ctx.outputValues.get(); - // get the timestamp values for this chunk - try (final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(inputKeys.intSize())) { - LongChunk timestampChunk = timestampColumnSource.getChunk(context, inputKeys).asLongChunk(); - - for (int ii = 0; ii < inputKeys.intSize(); ii++) { - // the output value is computed by push/pop operations triggered by fillWindow - ctx.fillWindowTime(ctx, timestampChunk.get(ii)); - localOutputValues.set(ii, ctx.currentVal); - } - } - } - - @NotNull - @Override - public Map> getOutputColumns() { - return Collections.singletonMap(pair.leftColumn, outputSource); - } - - @Override - public void startTrackingPrev() { - outputSource.startTrackingPrevValues(); - if(redirContext.isRedirected()) { - maybeInnerSource.startTrackingPrevValues(); - } - } - - @Override - public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { - ((LongSparseArraySource)outputSource).shift(subIndexToShift, delta); - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java index fb2b8a60d70..6c4ee6e4247 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java @@ -1,19 +1,13 @@ package io.deephaven.engine.table.impl.updateby.rollingsum; import io.deephaven.api.updateby.OperationControl; -import io.deephaven.chunk.*; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.ShortChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.sized.SizedLongChunk; -import io.deephaven.engine.rowset.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; -import io.deephaven.engine.table.*; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.UpdateByOperator; -import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; -import io.deephaven.engine.table.impl.updateby.internal.*; -import io.deephaven.engine.table.impl.util.SizedSafeCloseable; +import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedLongUpdateByOperator; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -23,74 +17,40 @@ import static io.deephaven.util.QueryConstants.NULL_LONG; import static io.deephaven.util.QueryConstants.NULL_SHORT; -public class ShortRollingSumOperator extends BaseWindowedShortUpdateByOperator { - - // RollingSum will output Long values for integral types - private final WritableColumnSource outputSource; - private final WritableColumnSource maybeInnerSource; - +public class ShortRollingSumOperator extends BaseWindowedLongUpdateByOperator { // region extra-fields // endregion extra-fields - protected class Context extends BaseWindowedShortUpdateByOperator.Context { - public final SizedSafeCloseable fillContext; - public final SizedLongChunk outputValues; - - public long currentVal = NULL_LONG; + protected class Context extends BaseWindowedLongUpdateByOperator.Context { + public ShortChunk shortInfluencerValuesChunk; - protected Context(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { - this.fillContext = new SizedSafeCloseable<>(outputSource::makeFillFromContext); - this.fillContext.ensureCapacity(chunkSize); - this.outputValues = new SizedLongChunk<>(chunkSize); - this.timestampSsa = timestampSsa; + protected Context(int chunkSize) { + super(chunkSize); } @Override - public void close() { - super.close(); - outputValues.close(); - fillContext.close(); + public void storeInfluencerValuesChunk(@NotNull final Chunk influencerValuesChunk) { + shortInfluencerValuesChunk = influencerValuesChunk.asShortChunk(); } } @NotNull @Override - public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize, final LongSegmentedSortedArray timestampSsa) { - return new Context(chunkSize, timestampSsa); - } - - @Override - public void setChunkSize(@NotNull UpdateContext context, int chunkSize) { - final Context ctx = (Context) context; - ctx.outputValues.ensureCapacity(chunkSize); - ctx.fillContext.ensureCapacity(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize) { + return new Context(chunkSize); } public ShortRollingSumOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final String timestampColumnName, - @Nullable final ColumnSource timestampColumnSource, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, - @NotNull final ColumnSource valueSource + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, timestampColumnSource, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, valueSource); - if(redirContext.isRedirected()) { - // region create-dense - this.maybeInnerSource = new LongArraySource(); - // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(redirContext.getRowRedirection(), maybeInnerSource, 0); - } else { - this.maybeInnerSource = null; - // region create-sparse - this.outputSource = new LongSparseArraySource(); - // endregion create-sparse - } - + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); // region constructor // endregion constructor } @@ -98,14 +58,14 @@ public ShortRollingSumOperator(@NotNull final MatchPair pair, @Override public void push(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; - short val = ctx.candidateValuesChunk.get(pos); + short val = ctx.shortInfluencerValuesChunk.get(pos); // increase the running sum if (val != NULL_SHORT) { - if (ctx.currentVal == NULL_LONG) { - ctx.currentVal = val; + if (ctx.curVal == NULL_LONG) { + ctx.curVal = val; } else { - ctx.currentVal += val; + ctx.curVal += val; } } else { ctx.nullCount++; @@ -113,86 +73,16 @@ public void push(UpdateContext context, long key, int pos) { } @Override - public void pop(UpdateContext context, long key, int pos) { + public void pop(UpdateContext context) { final Context ctx = (Context) context; - short val = ctx.candidateValuesChunk.get(pos); + int pos = ctx.windowIndices.front(); + short val = ctx.shortInfluencerValuesChunk.get(pos); // reduce the running sum if (val != NULL_SHORT) { - ctx.currentVal -= val; + ctx.curVal -= val; } else { ctx.nullCount--; } } - - @Override - public void reset(UpdateContext context) { - final Context ctx = (Context) context; - ctx.currentVal = NULL_LONG; - } - - @Override - public void doProcessChunk(@NotNull final BaseWindowedShortUpdateByOperator.Context context, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @NotNull final Chunk workingChunk) { - final Context ctx = (Context) context; - - if (timestampColumnName == null) { - computeTicks(ctx, posChunk, inputKeys.intSize()); - } else { - computeTime(ctx, inputKeys); - } - - //noinspection unchecked - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); - } - - private void computeTicks(@NotNull final Context ctx, - @Nullable final LongChunk posChunk, - final int runLength) { - - final WritableLongChunk localOutputValues = ctx.outputValues.get(); - for (int ii = 0; ii < runLength; ii++) { - // the output value is computed by push/pop operations triggered by fillWindow - ctx.fillWindowTicks(ctx, posChunk.get(ii)); - localOutputValues.set(ii, ctx.currentVal); - } - } - - private void computeTime(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys) { - - final WritableLongChunk localOutputValues = ctx.outputValues.get(); - // get the timestamp values for this chunk - try (final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(inputKeys.intSize())) { - LongChunk timestampChunk = timestampColumnSource.getChunk(context, inputKeys).asLongChunk(); - - for (int ii = 0; ii < inputKeys.intSize(); ii++) { - // the output value is computed by push/pop operations triggered by fillWindow - ctx.fillWindowTime(ctx, timestampChunk.get(ii)); - localOutputValues.set(ii, ctx.currentVal); - } - } - } - - @NotNull - @Override - public Map> getOutputColumns() { - return Collections.singletonMap(pair.leftColumn, outputSource); - } - - @Override - public void startTrackingPrev() { - outputSource.startTrackingPrevValues(); - if(redirContext.isRedirected()) { - maybeInnerSource.startTrackingPrevValues(); - } - } - - @Override - public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { - ((LongSparseArraySource)outputSource).shift(subIndexToShift, delta); - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java index 19e9951b72b..4005b825f48 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java @@ -1,28 +1,60 @@ package io.deephaven.engine.table.impl.updateby.sum; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.ObjectChunk; +import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.updateby.internal.BaseObjectBinaryOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; +import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import java.math.BigDecimal; import java.math.MathContext; -public final class BigDecimalCumSumOperator extends BaseObjectBinaryOperator { +public final class BigDecimalCumSumOperator extends BaseObjectUpdateByOperator { @NotNull private final MathContext mathContext; + protected class Context extends BaseObjectUpdateByOperator.Context { + public ObjectChunk objectValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + objectValueChunk = valuesChunk.asObjectChunk(); + } + } + public BigDecimalCumSumOperator(@NotNull final MatchPair inputPair, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, @NotNull final MathContext mathContext) { - super(BigDecimal.class, inputPair, new String[] {inputPair.rightColumn}, redirContext); + super(inputPair, new String[] {inputPair.rightColumn}, redirContext, BigDecimal.class); this.mathContext = mathContext; } + @NotNull + @Override + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); + } + @Override - protected BigDecimal doOperation(BigDecimal bucketCurVal, BigDecimal chunkCurVal) { - return bucketCurVal.add(chunkCurVal, mathContext); + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + // read the value from the values chunk + final BigDecimal currentVal = ctx.objectValueChunk.get(pos); + + final boolean isCurrentNull = currentVal == null; + if (ctx.curVal == null) { + ctx.curVal = isCurrentNull ? null : currentVal; + } else { + if (!isCurrentNull) { + ctx.curVal = ctx.curVal.add(ctx.objectValueChunk.get(pos), mathContext); + } + } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java index 860873417fa..fb93f583ae0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java @@ -1,23 +1,54 @@ package io.deephaven.engine.table.impl.updateby.sum; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.ObjectChunk; +import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.updateby.internal.BaseObjectBinaryOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; +import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import java.math.BigInteger; -public final class BigIntegerCumSumOperator extends BaseObjectBinaryOperator { +public final class BigIntegerCumSumOperator extends BaseObjectUpdateByOperator { + protected class Context extends BaseObjectUpdateByOperator.Context { + public ObjectChunk objectValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + objectValueChunk = valuesChunk.asObjectChunk(); + } + } public BigIntegerCumSumOperator(@NotNull final MatchPair inputPair, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { - super(BigInteger.class, inputPair, new String[] { inputPair.rightColumn }, redirContext); + super(inputPair, new String[] {inputPair.rightColumn}, redirContext, BigInteger.class); + } + + @NotNull + @Override + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); } @Override - protected BigInteger doOperation(BigInteger bucketCurVal, BigInteger chunkCurVal) { - return bucketCurVal.add(chunkCurVal); + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + // read the value from the values chunk + final BigInteger currentVal = ctx.objectValueChunk.get(pos); + + final boolean isCurrentNull = currentVal == null; + if(ctx.curVal == null) { + ctx.curVal = isCurrentNull ? null : currentVal; + } else { + if(!isCurrentNull) { + ctx.curVal = ctx.curVal.add(ctx.objectValueChunk.get(pos)); + } + } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java index 96ad3af7571..26f6b8b4702 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java @@ -6,26 +6,32 @@ package io.deephaven.engine.table.impl.updateby.sum; 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.rowset.RowSequence; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.NULL_LONG; -import static io.deephaven.util.QueryConstants.NULL_BYTE; +import static io.deephaven.util.QueryConstants.*; public class ByteCumSumOperator extends BaseLongUpdateByOperator { // region extra-fields final byte nullValue; // endregion extra-fields + protected class Context extends BaseLongUpdateByOperator.Context { + public ByteChunk byteValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + byteValueChunk = valuesChunk.asByteChunk(); + } + } + public ByteCumSumOperator(@NotNull final MatchPair pair, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args @@ -38,33 +44,23 @@ public ByteCumSumOperator(@NotNull final MatchPair pair, // endregion constructor } + @NotNull @Override - protected void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { -// ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_LONG; - accumulate(workingChunk.asByteChunk(), ctx, 0, workingChunk.size()); -// singletonGroup = groupPosition; -// singletonVal = ctx.curVal; - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); } - private void accumulate(@NotNull final ByteChunk asBytes, - @NotNull final Context ctx, - final int runStart, - final int runLength) { - final WritableLongChunk localOutputValues = ctx.outputValues.get(); - for (int ii = runStart; ii < runStart + runLength; ii++) { - final byte currentVal = asBytes.get(ii); - final boolean isCurrentNull = currentVal == NULL_BYTE; - if(ctx.curVal == NULL_LONG) { - ctx.curVal = isCurrentNull ? NULL_LONG : currentVal; - } else { - if(!isCurrentNull) { - ctx.curVal += currentVal; - } - } - localOutputValues.set(ii, ctx.curVal); + @Override + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + // read the value from the values chunk + final byte currentVal = ctx.byteValueChunk.get(pos); + + if(ctx.curVal == NULL_LONG) { + ctx.curVal = currentVal == NULL_BYTE ? NULL_LONG : currentVal; + } else if (currentVal != NULL_BYTE) { + ctx.curVal += currentVal; } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java index 23a5a104981..cfec36050e4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java @@ -5,22 +5,32 @@ */ package io.deephaven.engine.table.impl.updateby.sum; -import io.deephaven.chunk.*; -import io.deephaven.chunk.attributes.ChunkLengths; -import io.deephaven.chunk.attributes.ChunkPositions; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.DoubleChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseDoubleUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.NULL_DOUBLE; public class DoubleCumSumOperator extends BaseDoubleUpdateByOperator { + // region extra-fields + // endregion extra-fields + + protected class Context extends BaseDoubleUpdateByOperator.Context { + public DoubleChunk doubleValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + doubleValueChunk = valuesChunk.asDoubleChunk(); + } + } public DoubleCumSumOperator(@NotNull final MatchPair pair, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext @@ -32,34 +42,23 @@ public DoubleCumSumOperator(@NotNull final MatchPair pair, // endregion constructor } - protected void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { - if(Double.isNaN(ctx.curVal)) { - if(!ctx.filledWithPermanentValue) { - ctx.filledWithPermanentValue = true; - ctx.outputValues.get().fillWithValue(0, ctx.outputValues.get().capacity(), Double.NaN); - } - } else { - accumulate(workingChunk.asDoubleChunk(), ctx, 0, workingChunk.size()); - } - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); + @NotNull + @Override + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); } - private void accumulate(@NotNull final DoubleChunk asDoubles, - @NotNull final Context ctx, - final int runStart, - final int runLength) { + @Override + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + // read the value from the values chunk + final double currentVal = ctx.doubleValueChunk.get(pos); - final WritableDoubleChunk localOutputChunk = ctx.outputValues.get(); - for (int ii = runStart; ii < runStart + runLength; ii++) { - final double currentVal = asDoubles.get(ii); - if (ctx.curVal == NULL_DOUBLE) { - ctx.curVal = currentVal; - } else if (currentVal != NULL_DOUBLE) { - ctx.curVal += currentVal; - } - localOutputChunk.set(ii, ctx.curVal); + if (ctx.curVal == NULL_DOUBLE) { + ctx.curVal = currentVal; + } else if (currentVal != NULL_DOUBLE) { + ctx.curVal += currentVal; } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java index 321ab2c1805..9620aa9b876 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java @@ -1,21 +1,31 @@ package io.deephaven.engine.table.impl.updateby.sum; -import io.deephaven.chunk.*; -import io.deephaven.chunk.attributes.ChunkLengths; -import io.deephaven.chunk.attributes.ChunkPositions; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.FloatChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseFloatUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.NULL_FLOAT; public class FloatCumSumOperator extends BaseFloatUpdateByOperator { + // region extra-fields + // endregion extra-fields + + protected class Context extends BaseFloatUpdateByOperator.Context { + public FloatChunk floatValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + floatValueChunk = valuesChunk.asFloatChunk(); + } + } public FloatCumSumOperator(@NotNull final MatchPair pair, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext @@ -27,34 +37,23 @@ public FloatCumSumOperator(@NotNull final MatchPair pair, // endregion constructor } - protected void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { - if(Float.isNaN(ctx.curVal)) { - if(!ctx.filledWithPermanentValue) { - ctx.filledWithPermanentValue = true; - ctx.outputValues.get().fillWithValue(0, ctx.outputValues.get().capacity(), Float.NaN); - } - } else { - accumulate(workingChunk.asFloatChunk(), ctx, 0, workingChunk.size()); - } - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); + @NotNull + @Override + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); } - private void accumulate(@NotNull final FloatChunk asFloats, - @NotNull final Context ctx, - final int runStart, - final int runLength) { - - final WritableFloatChunk localOutputChunk = ctx.outputValues.get(); - for (int ii = runStart; ii < runStart + runLength; ii++) { - final float currentVal = asFloats.get(ii); - if (ctx.curVal == NULL_FLOAT) { - ctx.curVal = currentVal; - } else if (currentVal != NULL_FLOAT) { - ctx.curVal += currentVal; - } - localOutputChunk.set(ii, ctx.curVal); + @Override + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + // read the value from the values chunk + final float currentVal = ctx.floatValueChunk.get(pos); + + if (ctx.curVal == NULL_FLOAT) { + ctx.curVal = currentVal; + } else if (currentVal != NULL_FLOAT) { + ctx.curVal += currentVal; } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java index f7512763054..7fdd4616148 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java @@ -6,25 +6,31 @@ package io.deephaven.engine.table.impl.updateby.sum; 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.rowset.RowSequence; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.NULL_LONG; -import static io.deephaven.util.QueryConstants.NULL_INT; +import static io.deephaven.util.QueryConstants.*; public class IntCumSumOperator extends BaseLongUpdateByOperator { // region extra-fields // endregion extra-fields + protected class Context extends BaseLongUpdateByOperator.Context { + public IntChunk intValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + intValueChunk = valuesChunk.asIntChunk(); + } + } + public IntCumSumOperator(@NotNull final MatchPair pair, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args @@ -35,33 +41,23 @@ public IntCumSumOperator(@NotNull final MatchPair pair, // endregion constructor } + @NotNull @Override - protected void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { -// ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_LONG; - accumulate(workingChunk.asIntChunk(), ctx, 0, workingChunk.size()); -// singletonGroup = groupPosition; -// singletonVal = ctx.curVal; - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); } - private void accumulate(@NotNull final IntChunk asIntegers, - @NotNull final Context ctx, - final int runStart, - final int runLength) { - final WritableLongChunk localOutputValues = ctx.outputValues.get(); - for (int ii = runStart; ii < runStart + runLength; ii++) { - final int currentVal = asIntegers.get(ii); - final boolean isCurrentNull = currentVal == NULL_INT; - if(ctx.curVal == NULL_LONG) { - ctx.curVal = isCurrentNull ? NULL_LONG : currentVal; - } else { - if(!isCurrentNull) { - ctx.curVal += currentVal; - } - } - localOutputValues.set(ii, ctx.curVal); + @Override + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + // read the value from the values chunk + final int currentVal = ctx.intValueChunk.get(pos); + + if(ctx.curVal == NULL_LONG) { + ctx.curVal = currentVal == NULL_INT ? NULL_LONG : currentVal; + } else if (currentVal != NULL_INT) { + ctx.curVal += currentVal; } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java index a279b4edea2..931a2228d8b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java @@ -6,25 +6,31 @@ package io.deephaven.engine.table.impl.updateby.sum; 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.rowset.RowSequence; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.NULL_LONG; -import static io.deephaven.util.QueryConstants.NULL_LONG; +import static io.deephaven.util.QueryConstants.*; public class LongCumSumOperator extends BaseLongUpdateByOperator { // region extra-fields // endregion extra-fields + protected class Context extends BaseLongUpdateByOperator.Context { + public LongChunk longValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + longValueChunk = valuesChunk.asLongChunk(); + } + } + public LongCumSumOperator(@NotNull final MatchPair pair, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args @@ -35,33 +41,23 @@ public LongCumSumOperator(@NotNull final MatchPair pair, // endregion constructor } + @NotNull @Override - protected void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { -// ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_LONG; - accumulate(workingChunk.asLongChunk(), ctx, 0, workingChunk.size()); -// singletonGroup = groupPosition; -// singletonVal = ctx.curVal; - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); } - private void accumulate(@NotNull final LongChunk asLongs, - @NotNull final Context ctx, - final int runStart, - final int runLength) { - final WritableLongChunk localOutputValues = ctx.outputValues.get(); - for (int ii = runStart; ii < runStart + runLength; ii++) { - final long currentVal = asLongs.get(ii); - final boolean isCurrentNull = currentVal == NULL_LONG; - if(ctx.curVal == NULL_LONG) { - ctx.curVal = isCurrentNull ? NULL_LONG : currentVal; - } else { - if(!isCurrentNull) { - ctx.curVal += currentVal; - } - } - localOutputValues.set(ii, ctx.curVal); + @Override + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + // read the value from the values chunk + final long currentVal = ctx.longValueChunk.get(pos); + + if(ctx.curVal == NULL_LONG) { + ctx.curVal = currentVal == NULL_LONG ? NULL_LONG : currentVal; + } else if (currentVal != NULL_LONG) { + ctx.curVal += currentVal; } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java index 542bc776f1d..0eed0f88d08 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java @@ -1,25 +1,31 @@ package io.deephaven.engine.table.impl.updateby.sum; 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.rowset.RowSequence; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.NULL_LONG; -import static io.deephaven.util.QueryConstants.NULL_SHORT; +import static io.deephaven.util.QueryConstants.*; public class ShortCumSumOperator extends BaseLongUpdateByOperator { // region extra-fields // endregion extra-fields + protected class Context extends BaseLongUpdateByOperator.Context { + public ShortChunk shortValueChunk; + + protected Context(int chunkSize) { + super(chunkSize); + } + + @Override + public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + shortValueChunk = valuesChunk.asShortChunk(); + } + } + public ShortCumSumOperator(@NotNull final MatchPair pair, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args @@ -30,33 +36,23 @@ public ShortCumSumOperator(@NotNull final MatchPair pair, // endregion constructor } + @NotNull @Override - protected void doProcessChunk(@NotNull final Context ctx, - @NotNull final RowSequence inputKeys, - @NotNull final Chunk workingChunk) { -// ctx.curVal = groupPosition == singletonGroup ? singletonVal : NULL_LONG; - accumulate(workingChunk.asShortChunk(), ctx, 0, workingChunk.size()); -// singletonGroup = groupPosition; -// singletonVal = ctx.curVal; - outputSource.fillFromChunk(ctx.fillContext.get(), ctx.outputValues.get(), inputKeys); + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); } - private void accumulate(@NotNull final ShortChunk asShorts, - @NotNull final Context ctx, - final int runStart, - final int runLength) { - final WritableLongChunk localOutputValues = ctx.outputValues.get(); - for (int ii = runStart; ii < runStart + runLength; ii++) { - final short currentVal = asShorts.get(ii); - final boolean isCurrentNull = currentVal == NULL_SHORT; - if(ctx.curVal == NULL_LONG) { - ctx.curVal = isCurrentNull ? NULL_LONG : currentVal; - } else { - if(!isCurrentNull) { - ctx.curVal += currentVal; - } - } - localOutputValues.set(ii, ctx.curVal); + @Override + public void push(UpdateContext context, long key, int pos) { + final Context ctx = (Context) context; + + // read the value from the values chunk + final short currentVal = ctx.shortValueChunk.get(pos); + + if(ctx.curVal == NULL_LONG) { + ctx.curVal = currentVal == NULL_SHORT ? NULL_LONG : currentVal; + } else if (currentVal != NULL_SHORT) { + ctx.curVal += currentVal; } } } diff --git a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java index 45da27436d1..0f95aa217a6 100644 --- a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java +++ b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java @@ -151,7 +151,8 @@ private static void fixupByteBase(String byteResult) throws IOException { lines = replaceRegion(lines, "Shifts", Collections.singletonList( " @Override\n" + - " public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) {\n" + + " public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) {\n" + + " if (outputSource instanceof BooleanSparseArraySource.ReinterpretedAsByte) {\n" + " ((BooleanSparseArraySource.ReinterpretedAsByte)outputSource).shift(subIndexToShift, delta);\n" @@ -179,8 +180,7 @@ private static void fixupByteBase(String byteResult) throws IOException { " this.maybeInnerSource = makeDenseSource();")); lines = replaceRegion(lines, "create-sparse", Collections.singletonList( " this.outputSource = makeSparseSource();")); - lines = replaceRegion(lines, "create-bucket", Collections.singletonList( - " this.bucketLastVal = makeDenseSource();")); + FileUtils.writeLines(objectFile, lines); } From 4df71beae5d0740199d9073ec829d6e003de5f0b Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 27 Sep 2022 14:05:56 -0700 Subject: [PATCH 026/123] huge refactor near complete, all cumulative tests passing --- .../base/ringbuffer/ByteRingBufferTest.java | 12 +- .../base/ringbuffer/CharRingBufferTest.java | 12 +- .../base/ringbuffer/DoubleRingBufferTest.java | 12 +- .../base/ringbuffer/FloatRingBufferTest.java | 12 +- .../base/ringbuffer/IntRingBufferTest.java | 12 +- .../base/ringbuffer/LongRingBufferTest.java | 12 +- .../base/ringbuffer/ObjectRingBufferTest.java | 12 +- .../base/ringbuffer/ShortRingBufferTest.java | 12 +- .../engine/rowset/impl/OrderedLongSet.java | 10 +- .../impl/BucketedPartitionedUpdateBy.java | 12 - .../deephaven/engine/table/impl/UpdateBy.java | 43 +-- .../impl/UpdateByCumulativeOperator.java | 11 +- .../engine/table/impl/UpdateByOperator.java | 6 +- .../table/impl/UpdateByOperatorFactory.java | 24 +- .../engine/table/impl/ZeroKeyUpdateBy.java | 57 ++-- .../impl/by/PartitionByChunkedOperator.java | 162 +++++----- .../table/impl/updateby/UpdateByWindow.java | 287 +++++++++++++----- .../ema/BasePrimitiveEMAOperator.java | 7 +- .../updateby/ema/BigDecimalEMAOperator.java | 5 +- .../updateby/ema/BigIntegerEMAOperator.java | 5 +- .../updateby/ema/BigNumberEMAOperator.java | 31 +- .../impl/updateby/ema/ByteEMAOperator.java | 23 +- .../impl/updateby/ema/DoubleEMAOperator.java | 69 +++-- .../impl/updateby/ema/FloatEMAOperator.java | 69 +++-- .../impl/updateby/ema/IntEMAOperator.java | 23 +- .../impl/updateby/ema/LongEMAOperator.java | 23 +- .../impl/updateby/ema/ShortEMAOperator.java | 23 +- .../updateby/fill/BooleanFillByOperator.java | 8 +- .../updateby/fill/ByteFillByOperator.java | 8 +- .../updateby/fill/CharFillByOperator.java | 8 +- .../updateby/fill/DoubleFillByOperator.java | 8 +- .../updateby/fill/FloatFillByOperator.java | 8 +- .../impl/updateby/fill/IntFillByOperator.java | 8 +- .../updateby/fill/LongFillByOperator.java | 8 +- .../updateby/fill/ObjectFillByOperator.java | 8 +- .../updateby/fill/ShortFillByOperator.java | 8 +- .../internal/BaseByteUpdateByOperator.java | 3 +- .../internal/BaseCharUpdateByOperator.java | 3 +- .../internal/BaseDoubleUpdateByOperator.java | 4 +- .../internal/BaseFloatUpdateByOperator.java | 4 +- .../internal/BaseIntUpdateByOperator.java | 3 +- .../internal/BaseLongUpdateByOperator.java | 3 +- .../internal/BaseObjectBinaryOperator.java | 5 +- .../internal/BaseObjectUpdateByOperator.java | 3 +- .../internal/BaseShortUpdateByOperator.java | 3 +- .../BaseWindowedByteUpdateByOperator.java | 4 +- .../BaseWindowedCharUpdateByOperator.java | 4 +- .../BaseWindowedDoubleUpdateByOperator.java | 4 +- .../BaseWindowedFloatUpdateByOperator.java | 4 +- .../BaseWindowedIntUpdateByOperator.java | 4 +- .../BaseWindowedLongUpdateByOperator.java | 4 +- .../BaseWindowedObjectUpdateByOperator.java | 4 +- .../BaseWindowedShortUpdateByOperator.java | 4 +- .../minmax/ByteCumMinMaxOperator.java | 4 +- .../minmax/DoubleCumMinMaxOperator.java | 4 +- .../minmax/FloatCumMinMaxOperator.java | 4 +- .../updateby/minmax/IntCumMinMaxOperator.java | 4 +- .../minmax/LongCumMinMaxOperator.java | 4 +- .../minmax/ShortCumMinMaxOperator.java | 4 +- .../prod/BigDecimalCumProdOperator.java | 3 +- .../prod/BigIntegerCumProdOperator.java | 3 +- .../updateby/prod/ByteCumProdOperator.java | 3 +- .../updateby/prod/DoubleCumProdOperator.java | 4 +- .../updateby/prod/FloatCumProdOperator.java | 4 +- .../updateby/prod/IntCumProdOperator.java | 3 +- .../updateby/prod/LongCumProdOperator.java | 3 +- .../updateby/prod/ShortCumProdOperator.java | 3 +- .../BigDecimalRollingSumOperator.java | 13 +- .../BigIntegerRollingSumOperator.java | 13 +- .../rollingsum/ByteRollingSumOperator.java | 6 +- .../rollingsum/DoubleRollingSumOperator.java | 11 +- .../rollingsum/FloatRollingSumOperator.java | 11 +- .../rollingsum/IntRollingSumOperator.java | 6 +- .../rollingsum/LongRollingSumOperator.java | 6 +- .../rollingsum/ShortRollingSumOperator.java | 6 +- .../sum/BigDecimalCumSumOperator.java | 3 +- .../sum/BigIntegerCumSumOperator.java | 3 +- .../impl/updateby/sum/ByteCumSumOperator.java | 3 +- .../updateby/sum/DoubleCumSumOperator.java | 3 +- .../updateby/sum/FloatCumSumOperator.java | 3 +- .../impl/updateby/sum/IntCumSumOperator.java | 3 +- .../impl/updateby/sum/LongCumSumOperator.java | 3 +- .../updateby/sum/ShortCumSumOperator.java | 3 +- .../replicators/ReplicateRingBuffers.java | 3 +- 84 files changed, 689 insertions(+), 581 deletions(-) diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/ByteRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/ByteRingBufferTest.java index 6f7ca5452db..87a89a104d1 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/ByteRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/ByteRingBufferTest.java @@ -355,18 +355,18 @@ public void testLargeAmounts() { ByteRingBuffer rb = new ByteRingBuffer(3); for (int i = 0; i < 100; i++) - rb.add((byte)i); + rb.add((byte) i); for (int i = 100; i < 200; i++) { - rb.add((byte)i); - assertEquals((byte)(i - 100 + 1), rb.front(1)); - assertEquals((byte)(i - 100), rb.poll(SENTINEL)); + rb.add((byte) i); + assertEquals((byte) (i - 100 + 1), rb.front(1)); + assertEquals((byte) (i - 100), rb.poll(SENTINEL)); } for (int i = 200; i < 300; i++) { if (i < 299) - assertEquals((byte)(i - 100 + 1), rb.front(1)); - assertEquals((byte)(i - 100), rb.poll(SENTINEL)); + assertEquals((byte) (i - 100 + 1), rb.front(1)); + assertEquals((byte) (i - 100), rb.poll(SENTINEL)); } } } diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java index 2b21ff2e92c..80a98c13546 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java @@ -350,18 +350,18 @@ public void testLargeAmounts() { CharRingBuffer rb = new CharRingBuffer(3); for (int i = 0; i < 100; i++) - rb.add((char)i); + rb.add((char) i); for (int i = 100; i < 200; i++) { - rb.add((char)i); - assertEquals((char)(i - 100 + 1), rb.front(1)); - assertEquals((char)(i - 100), rb.poll(SENTINEL)); + rb.add((char) i); + assertEquals((char) (i - 100 + 1), rb.front(1)); + assertEquals((char) (i - 100), rb.poll(SENTINEL)); } for (int i = 200; i < 300; i++) { if (i < 299) - assertEquals((char)(i - 100 + 1), rb.front(1)); - assertEquals((char)(i - 100), rb.poll(SENTINEL)); + assertEquals((char) (i - 100 + 1), rb.front(1)); + assertEquals((char) (i - 100), rb.poll(SENTINEL)); } } } diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/DoubleRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/DoubleRingBufferTest.java index d7a2e87ce35..214d9c1a284 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/DoubleRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/DoubleRingBufferTest.java @@ -355,18 +355,18 @@ public void testLargeAmounts() { DoubleRingBuffer rb = new DoubleRingBuffer(3); for (int i = 0; i < 100; i++) - rb.add((double)i); + rb.add((double) i); for (int i = 100; i < 200; i++) { - rb.add((double)i); - assertEquals((double)(i - 100 + 1), rb.front(1)); - assertEquals((double)(i - 100), rb.poll(SENTINEL)); + rb.add((double) i); + assertEquals((double) (i - 100 + 1), rb.front(1)); + assertEquals((double) (i - 100), rb.poll(SENTINEL)); } for (int i = 200; i < 300; i++) { if (i < 299) - assertEquals((double)(i - 100 + 1), rb.front(1)); - assertEquals((double)(i - 100), rb.poll(SENTINEL)); + assertEquals((double) (i - 100 + 1), rb.front(1)); + assertEquals((double) (i - 100), rb.poll(SENTINEL)); } } } diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/FloatRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/FloatRingBufferTest.java index a77c1036f49..0eed8d1965c 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/FloatRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/FloatRingBufferTest.java @@ -355,18 +355,18 @@ public void testLargeAmounts() { FloatRingBuffer rb = new FloatRingBuffer(3); for (int i = 0; i < 100; i++) - rb.add((float)i); + rb.add((float) i); for (int i = 100; i < 200; i++) { - rb.add((float)i); - assertEquals((float)(i - 100 + 1), rb.front(1)); - assertEquals((float)(i - 100), rb.poll(SENTINEL)); + rb.add((float) i); + assertEquals((float) (i - 100 + 1), rb.front(1)); + assertEquals((float) (i - 100), rb.poll(SENTINEL)); } for (int i = 200; i < 300; i++) { if (i < 299) - assertEquals((float)(i - 100 + 1), rb.front(1)); - assertEquals((float)(i - 100), rb.poll(SENTINEL)); + assertEquals((float) (i - 100 + 1), rb.front(1)); + assertEquals((float) (i - 100), rb.poll(SENTINEL)); } } } diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/IntRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/IntRingBufferTest.java index 1c0815a1eda..3c798cfb97f 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/IntRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/IntRingBufferTest.java @@ -355,18 +355,18 @@ public void testLargeAmounts() { IntRingBuffer rb = new IntRingBuffer(3); for (int i = 0; i < 100; i++) - rb.add((int)i); + rb.add((int) i); for (int i = 100; i < 200; i++) { - rb.add((int)i); - assertEquals((int)(i - 100 + 1), rb.front(1)); - assertEquals((int)(i - 100), rb.poll(SENTINEL)); + rb.add((int) i); + assertEquals((int) (i - 100 + 1), rb.front(1)); + assertEquals((int) (i - 100), rb.poll(SENTINEL)); } for (int i = 200; i < 300; i++) { if (i < 299) - assertEquals((int)(i - 100 + 1), rb.front(1)); - assertEquals((int)(i - 100), rb.poll(SENTINEL)); + assertEquals((int) (i - 100 + 1), rb.front(1)); + assertEquals((int) (i - 100), rb.poll(SENTINEL)); } } } diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/LongRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/LongRingBufferTest.java index f66de64e4d3..8f94e96d03c 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/LongRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/LongRingBufferTest.java @@ -355,18 +355,18 @@ public void testLargeAmounts() { LongRingBuffer rb = new LongRingBuffer(3); for (int i = 0; i < 100; i++) - rb.add((long)i); + rb.add((long) i); for (int i = 100; i < 200; i++) { - rb.add((long)i); - assertEquals((long)(i - 100 + 1), rb.front(1)); - assertEquals((long)(i - 100), rb.poll(SENTINEL)); + rb.add((long) i); + assertEquals((long) (i - 100 + 1), rb.front(1)); + assertEquals((long) (i - 100), rb.poll(SENTINEL)); } for (int i = 200; i < 300; i++) { if (i < 299) - assertEquals((long)(i - 100 + 1), rb.front(1)); - assertEquals((long)(i - 100), rb.poll(SENTINEL)); + assertEquals((long) (i - 100 + 1), rb.front(1)); + assertEquals((long) (i - 100), rb.poll(SENTINEL)); } } } diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/ObjectRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/ObjectRingBufferTest.java index 526152444a6..266f5ba2b49 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/ObjectRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/ObjectRingBufferTest.java @@ -355,18 +355,18 @@ public void testLargeAmounts() { ObjectRingBuffer rb = new ObjectRingBuffer(3); for (int i = 0; i < 100; i++) - rb.add((Object)i); + rb.add((Object) i); for (int i = 100; i < 200; i++) { - rb.add((Object)i); - assertEquals((Object)(i - 100 + 1), rb.front(1)); - assertEquals((Object)(i - 100), rb.poll(SENTINEL)); + rb.add((Object) i); + assertEquals((Object) (i - 100 + 1), rb.front(1)); + assertEquals((Object) (i - 100), rb.poll(SENTINEL)); } for (int i = 200; i < 300; i++) { if (i < 299) - assertEquals((Object)(i - 100 + 1), rb.front(1)); - assertEquals((Object)(i - 100), rb.poll(SENTINEL)); + assertEquals((Object) (i - 100 + 1), rb.front(1)); + assertEquals((Object) (i - 100), rb.poll(SENTINEL)); } } } diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/ShortRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/ShortRingBufferTest.java index 30b980e5348..e5843b22e5d 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/ShortRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/ShortRingBufferTest.java @@ -355,18 +355,18 @@ public void testLargeAmounts() { ShortRingBuffer rb = new ShortRingBuffer(3); for (int i = 0; i < 100; i++) - rb.add((short)i); + rb.add((short) i); for (int i = 100; i < 200; i++) { - rb.add((short)i); - assertEquals((short)(i - 100 + 1), rb.front(1)); - assertEquals((short)(i - 100), rb.poll(SENTINEL)); + rb.add((short) i); + assertEquals((short) (i - 100 + 1), rb.front(1)); + assertEquals((short) (i - 100), rb.poll(SENTINEL)); } for (int i = 200; i < 300; i++) { if (i < 299) - assertEquals((short)(i - 100 + 1), rb.front(1)); - assertEquals((short)(i - 100), rb.poll(SENTINEL)); + assertEquals((short) (i - 100 + 1), rb.front(1)); + assertEquals((short) (i - 100), rb.poll(SENTINEL)); } } } diff --git a/engine/rowset/src/main/java/io/deephaven/engine/rowset/impl/OrderedLongSet.java b/engine/rowset/src/main/java/io/deephaven/engine/rowset/impl/OrderedLongSet.java index e4abef7b76e..48d9d9ea24e 100644 --- a/engine/rowset/src/main/java/io/deephaven/engine/rowset/impl/OrderedLongSet.java +++ b/engine/rowset/src/main/java/io/deephaven/engine/rowset/impl/OrderedLongSet.java @@ -40,7 +40,7 @@ public interface OrderedLongSet { @FinalDefault default OrderedLongSet ixInsert(final LongChunk keys, final int offset, - final int length) { + final int length) { if (length <= 1) { if (length == 0) { return this; @@ -70,7 +70,7 @@ default OrderedLongSet ixInsert(final LongChunk keys, final int @FinalDefault default OrderedLongSet ixRemove(final LongChunk keys, final int offset, - final int length) { + final int length) { if (ixIsEmpty()) { return this; } @@ -191,7 +191,7 @@ default void ixValidate() { * @return A new {@link OrderedLongSet} containing the specified slice of {@code keys} */ static OrderedLongSet fromChunk(final LongChunk keys, final int offset, final int length, - final boolean disposable) { + final boolean disposable) { if (length == 0) { return EMPTY; } @@ -238,13 +238,13 @@ public OrderedLongSet ixInsertRange(final long startKey, final long endKey) { @Override public OrderedLongSet ixInsertSecondHalf(final LongChunk keys, final int offset, - final int length) { + final int length) { return fromChunk(keys, offset, length, false); } @Override public OrderedLongSet ixRemoveSecondHalf(final LongChunk keys, final int offset, - final int length) { + final int length) { throw new IllegalStateException(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java index ebac3a92e85..0fa085fb84a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java @@ -165,20 +165,8 @@ private final class BucketedPartitionedUpdateByListenerRecorder extends Listener * The Listener for apply to the constituent table updates */ class BucketedPartitionedUpdateByListener extends MergedListener { - private final ModifiedColumnSet[] inputModifiedColumnSets; - private final ModifiedColumnSet[] outputModifiedColumnSets; - public BucketedPartitionedUpdateByListener(@Nullable String description) { super(recorders, List.of(), description, resultTable); - - this.inputModifiedColumnSets = new ModifiedColumnSet[operators.length]; - this.outputModifiedColumnSets = new ModifiedColumnSet[operators.length]; - - for (int ii = 0; ii < operators.length; ii++) { - final String[] outputColumnNames = operators[ii].getOutputColumnNames(); - inputModifiedColumnSets[ii] = source.newModifiedColumnSet(operators[ii].getAffectingColumnNames()); - outputModifiedColumnSets[ii] = resultTable.newModifiedColumnSet(outputColumnNames); - } } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java index a26967fb417..9b9d5ca80e0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java @@ -1,14 +1,11 @@ package io.deephaven.engine.table.impl; import gnu.trove.list.array.TIntArrayList; -import gnu.trove.map.TIntIntMap; import gnu.trove.map.hash.TIntObjectHashMap; import gnu.trove.map.hash.TObjectIntHashMap; import io.deephaven.api.ColumnName; import io.deephaven.api.updateby.UpdateByOperation; import io.deephaven.api.updateby.UpdateByControl; -import io.deephaven.chunk.Chunk; -import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.exceptions.UncheckedTableException; import io.deephaven.engine.rowset.*; @@ -20,11 +17,13 @@ import io.deephaven.engine.table.impl.util.InverseRowRedirectionImpl; import io.deephaven.engine.table.impl.util.LongColumnSourceWritableRowRedirection; import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import org.apache.commons.lang3.mutable.MutableInt; import org.apache.commons.lang3.mutable.MutableLong; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import java.util.*; +import java.util.stream.Stream; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; @@ -32,7 +31,7 @@ * The core of the {@link Table#updateBy(UpdateByControl, Collection, Collection)} operation. */ public abstract class UpdateBy { - protected final ChunkSource[] inputSources; + protected final ColumnSource[] inputSources; protected final int[] inputSourceSlots; protected final UpdateByWindow[] windows; protected final UpdateByOperator[] operators; @@ -145,7 +144,7 @@ protected UpdateBy(@NotNull final UpdateByOperator[] operators, // the next bit is complicated but the goal is simple. We don't want to have duplicate input column sources, so // we will store each one only once in inputSources and setup some mapping from the opIdx to the input column. // noinspection unchecked - inputSources = new ChunkSource[operators.length]; + inputSources = new ColumnSource[operators.length]; inputSourceSlots = new int[operators.length]; final TObjectIntHashMap> sourceToSlotMap = new TObjectIntHashMap<>(); for (int opIdx = 0; opIdx < operators.length; opIdx++) { @@ -161,34 +160,36 @@ protected UpdateBy(@NotNull final UpdateByOperator[] operators, } // now we want to divide the operators into similar windows for efficient processing - TIntObjectHashMap windowHashToObjectMap = new TIntObjectHashMap<>(); TIntObjectHashMap windowHashToOperatorIndicesMap = new TIntObjectHashMap<>(); for (int opIdx = 0; opIdx < operators.length; opIdx++) { - UpdateByWindow newWindow = UpdateByWindow.createFromOperator(operators[opIdx]); - final int hash = newWindow.hashCode(); + // get the hash + final int hash = UpdateByWindow.hashCodeFromOperator(operators[opIdx]); // add this if not found - if (!windowHashToObjectMap.containsKey(hash)) { - windowHashToObjectMap.put(hash, newWindow); + if (!windowHashToOperatorIndicesMap.containsKey(hash)) { windowHashToOperatorIndicesMap.put(hash, new TIntArrayList()); } windowHashToOperatorIndicesMap.get(hash).add(opIdx); } // store the operator information into the windows - windowHashToObjectMap.valueCollection().forEach(window -> { - final int hash = window.hashCode(); - final int[] opIndices = windowHashToOperatorIndicesMap.get(hash).toArray(); - - final UpdateByOperator[] ops = - Arrays.stream(opIndices).mapToObj(idx -> operators[idx]).toArray(UpdateByOperator[]::new); - final int[] opInputSourceSlots = Arrays.stream(opIndices).map(idx -> inputSourceSlots[idx]).toArray(); - - window.setOperators(ops, opInputSourceSlots); + this.windows = new UpdateByWindow[windowHashToOperatorIndicesMap.size()]; + final MutableInt winIdx = new MutableInt(0); + + windowHashToOperatorIndicesMap.forEachEntry((final int hash, final TIntArrayList opIndices) -> { + final UpdateByOperator[] windowOperators = + Arrays.stream(opIndices.toArray()) + .mapToObj(idx -> operators[idx]) + .toArray(UpdateByOperator[]::new); + final int[] windowOperatorSourceSlots = + Arrays.stream(opIndices.toArray()) + .map(idx -> inputSourceSlots[idx]) + .toArray(); + this.windows[winIdx.getAndIncrement()] = + UpdateByWindow.createFromOperatorArray(windowOperators, windowOperatorSourceSlots); + return true; }); - - this.windows = windowHashToObjectMap.valueCollection().toArray(UpdateByOperator.ZERO_LENGTH_WINDOW_ARRAY); } // region UpdateBy implementation diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java index 6072bf1d610..64a65bebe8c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java @@ -7,6 +7,7 @@ import io.deephaven.engine.table.ChunkSink; import io.deephaven.engine.table.MatchPair; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.NULL_LONG; @@ -21,13 +22,17 @@ public abstract class UpdateByCumulativeOperator implements UpdateByOperator { protected long timeScaleUnits; protected String timestampColumnName; - protected class Context implements UpdateContext { + public class Context implements UpdateContext { public long curTimestamp; protected Context(final int chunkSize) { curTimestamp = NULL_LONG; } + public boolean isValueValid(long atKey) { + throw new UnsupportedOperationException( + "isValueValid() must be overriden by time-aware cumulative operators"); + } @Override public void close() {} @@ -47,10 +52,6 @@ public UpdateByCumulativeOperator(@NotNull final MatchPair pair, abstract public void initializeUpdate(@NotNull final UpdateContext context, final long firstUnmodifiedKey, long firstUnmodifiedTimestamp); - public boolean isValueValid(long atKey) { - throw new UnsupportedOperationException("isValueValid() must be overriden by time-aware cumulative operators"); - } - @Override public void finishUpdate(@NotNull final UpdateContext context) {} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java index 3c1e4d2e434..f5cd1ec3914 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java @@ -8,6 +8,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.impl.updateby.UpdateByWindow; @@ -139,10 +140,11 @@ interface UpdateContext extends SafeCloseable { * Make an {@link UpdateContext} suitable for use with non-bucketed updates. * * @param chunkSize The expected size of chunks that will be provided during the update, + * @param inputSource * @return a new context */ @NotNull - UpdateContext makeUpdateContext(final int chunkSize); + UpdateContext makeUpdateContext(final int chunkSize, ColumnSource inputSource); /** * Perform and bookkeeping required at the end of a single part of the update. This is always preceded with a call @@ -173,5 +175,5 @@ void processChunk(@NotNull final UpdateContext context, @Nullable final LongChunk keyChunk, @Nullable final LongChunk posChunk, @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk); + @Nullable final LongChunk timestampValuesChunk); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java index ef340be5298..b0ea8ddee63 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java @@ -226,36 +226,28 @@ private UpdateByOperator makeEmaOperator(@NotNull final MatchPair pair, if (csType == byte.class || csType == Byte.class) { return new ByteEMAOperator(pair, affectingColumns, ema.controlOrDefault(), - ema.timeScale().timestampCol(), timeScaleUnits, - columnSource, redirContext); + ema.timeScale().timestampCol(), timeScaleUnits, redirContext); } else if (csType == short.class || csType == Short.class) { return new ShortEMAOperator(pair, affectingColumns, ema.controlOrDefault(), - ema.timeScale().timestampCol(), timeScaleUnits, - columnSource, redirContext); + ema.timeScale().timestampCol(), timeScaleUnits, redirContext); } else if (csType == int.class || csType == Integer.class) { return new IntEMAOperator(pair, affectingColumns, ema.controlOrDefault(), - ema.timeScale().timestampCol(), timeScaleUnits, - columnSource, redirContext); + ema.timeScale().timestampCol(), timeScaleUnits, redirContext); } else if (csType == long.class || csType == Long.class) { return new LongEMAOperator(pair, affectingColumns, ema.controlOrDefault(), - ema.timeScale().timestampCol(), timeScaleUnits, - columnSource, redirContext); + ema.timeScale().timestampCol(), timeScaleUnits, redirContext); } else if (csType == float.class || csType == Float.class) { return new FloatEMAOperator(pair, affectingColumns, ema.controlOrDefault(), - ema.timeScale().timestampCol(), timeScaleUnits, - columnSource, redirContext); + ema.timeScale().timestampCol(), timeScaleUnits, redirContext); } else if (csType == double.class || csType == Double.class) { return new DoubleEMAOperator(pair, affectingColumns, ema.controlOrDefault(), - ema.timeScale().timestampCol(), - timeScaleUnits, columnSource, redirContext); + ema.timeScale().timestampCol(), timeScaleUnits, redirContext); } else if (csType == BigDecimal.class) { return new BigDecimalEMAOperator(pair, affectingColumns, ema.controlOrDefault(), - ema.timeScale().timestampCol(), - timeScaleUnits, columnSource, redirContext); + ema.timeScale().timestampCol(), timeScaleUnits, redirContext); } else if (csType == BigInteger.class) { return new BigIntegerEMAOperator(pair, affectingColumns, ema.controlOrDefault(), - ema.timeScale().timestampCol(), - timeScaleUnits, columnSource, redirContext); + ema.timeScale().timestampCol(), timeScaleUnits, redirContext); } throw new IllegalArgumentException("Can not perform EMA on type " + csType); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java index e56c1eb52a6..bfcb2bae668 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java @@ -1,6 +1,8 @@ package io.deephaven.engine.table.impl; +import gnu.trove.set.hash.TLongHashSet; import io.deephaven.api.updateby.UpdateByControl; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.*; @@ -116,6 +118,8 @@ private void processUpdateForSsa(TableUpdate upstream) { final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(size); final WritableLongChunk ssaValues = WritableLongChunk.makeWritableChunk(size); final WritableLongChunk ssaKeys = WritableLongChunk.makeWritableChunk(size)) { + TLongHashSet ssaKeySet = new TLongHashSet(); + timestampSsa.forAllKeys(ssaKeySet::add); MutableLong lastTimestamp = new MutableLong(NULL_LONG); while (it.hasMore()) { @@ -128,6 +132,15 @@ private void processUpdateForSsa(TableUpdate upstream) { // push only non-null values/keys into the Ssa fillChunkWithNonNull(keysChunk, valuesChunk, ssaKeys, ssaValues, lastTimestamp); + + // verify that all values to remove are actually in this SSA + for (int ii = 0; ii < valuesChunk.size(); ii++) { + final long ts = valuesChunk.get(ii); + final long key = keysChunk.get(ii); + if (!ssaKeySet.contains(key)) { + System.out.println(ts + " : " + key); + } + } timestampSsa.remove(ssaValues, ssaKeys); } } @@ -138,14 +151,14 @@ private void processUpdateForSsa(TableUpdate upstream) { final int size = Math.max( upstream.modified().intSize() + Math.max(upstream.added().intSize(), upstream.removed().intSize()), (int) upstream.shifted().getEffectiveSize()); - try (final RowSet prevRowSet = source.getRowSet().copyPrev(); - final RowSet withoutMods = prevRowSet.minus(upstream.getModifiedPreShift()); + try (final RowSet fullPrevRowSet = source.getRowSet().copyPrev(); + final RowSet previousToShift = fullPrevRowSet.minus(restampRemovals); final ColumnSource.GetContext getContext = timestampColumnSource.makeGetContext(size)) { final RowSetShiftData.Iterator sit = upstream.shifted().applyIterator(); while (sit.hasNext()) { sit.next(); - try (final RowSet subRowSet = withoutMods.subSetByKeyRange(sit.beginRange(), sit.endRange()); + try (final RowSet subRowSet = previousToShift.subSetByKeyRange(sit.beginRange(), sit.endRange()); final RowSet rowSetToShift = subRowSet.minus(upstream.removed())) { if (rowSetToShift.isEmpty()) { continue; @@ -154,7 +167,7 @@ private void processUpdateForSsa(TableUpdate upstream) { final LongChunk shiftValues = timestampColumnSource.getPrevChunk(getContext, rowSetToShift).asLongChunk(); - timestampSsa.applyShift(shiftValues, rowSetToShift.asRowKeyChunk(), sit.shiftDelta()); + timestampSsa.applyShiftReverse(shiftValues, rowSetToShift.asRowKeyChunk(), sit.shiftDelta()); } } } @@ -167,8 +180,10 @@ private void processUpdateForSsa(TableUpdate upstream) { final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(size); final WritableLongChunk ssaValues = WritableLongChunk.makeWritableChunk(size); final WritableLongChunk ssaKeys = WritableLongChunk.makeWritableChunk(size)) { - MutableLong lastTimestamp = new MutableLong(NULL_LONG); + TLongHashSet ssaKeySet = new TLongHashSet(); + timestampSsa.forAllKeys(ssaKeySet::add); + MutableLong lastTimestamp = new MutableLong(NULL_LONG); while (it.hasMore()) { RowSequence chunkRs = it.getNextRowSequenceWithLength(4096); @@ -179,6 +194,15 @@ private void processUpdateForSsa(TableUpdate upstream) { // push only non-null values/keys into the Ssa fillChunkWithNonNull(keysChunk, valuesChunk, ssaKeys, ssaValues, lastTimestamp); + + // verify that the items are not already in the SSA + for (int ii = 0; ii < valuesChunk.size(); ii++) { + final long ts = valuesChunk.get(ii); + final long key = keysChunk.get(ii); + if (ssaKeySet.contains(key)) { + System.out.println(ts + " : " + key); + } + } timestampSsa.insert(ssaValues, ssaKeys); } } @@ -227,7 +251,7 @@ void doInitialAdditions() { } // do the processing for this fake update - try (final UpdateContext ctx = new UpdateContext(fakeUpdate, null, true)) { + try (final UpdateContext ctx = new UpdateContext(fakeUpdate, true)) { ctx.processRows(); } } @@ -254,7 +278,6 @@ private class UpdateContext implements SafeCloseable { @SuppressWarnings("resource") UpdateContext(@NotNull final TableUpdate upstream, - @Nullable final ModifiedColumnSet[] inputModifiedColumnSets, final boolean isInitializeStep) { final int updateSize = UpdateSizeCalculator.chunkSize(upstream, control.chunkCapacityOrDefault()); @@ -275,8 +298,7 @@ private class UpdateContext implements SafeCloseable { isInitializeStep); // compute the affected/influenced operators and rowset within this window - windowAffected[winIdx] = windowContexts[winIdx].computeAffectedRowsAndOperators(upstream, - inputModifiedColumnSets[winIdx]); + windowAffected[winIdx] = windowContexts[winIdx].computeAffectedRowsAndOperators(upstream); } } @@ -324,8 +346,6 @@ private void processRows() { */ class ZeroKeyUpdateByListener extends InstrumentedTableUpdateListenerAdapter { private final QueryTable result; - private final ModifiedColumnSet[] inputModifiedColumnSets; - private final ModifiedColumnSet[] outputModifiedColumnSets; private final ModifiedColumnSet.Transformer transformer; public ZeroKeyUpdateByListener(@Nullable String description, @@ -333,12 +353,9 @@ public ZeroKeyUpdateByListener(@Nullable String description, @NotNull final QueryTable result) { super(description, source, false); this.result = result; - this.inputModifiedColumnSets = new ModifiedColumnSet[windows.length]; - this.outputModifiedColumnSets = new ModifiedColumnSet[windows.length]; for (int ii = 0; ii < windows.length; ii++) { - inputModifiedColumnSets[ii] = source.newModifiedColumnSet(windows[ii].getAffectingColumnNames()); - outputModifiedColumnSets[ii] = result.newModifiedColumnSet(windows[ii].getOutputColumnNames()); + windows[ii].startTrackingModifications(source, result); } this.transformer = @@ -352,14 +369,13 @@ public void onUpdate(TableUpdate upstream) { processUpdateForSsa(upstream); } - try (final UpdateContext ctx = new UpdateContext(upstream, inputModifiedColumnSets, false)) { + try (final UpdateContext ctx = new UpdateContext(upstream, false)) { if (applyShifts) { if (redirContext.isRedirected()) { redirContext.processUpdateForRedirection(upstream, source.getRowSet()); } else { // We will not mess with shifts if we are using a redirection because we'll have applied the - // shift - // to the redirection index already by now. + // shift to the redirection index already by now. if (upstream.shifted().nonempty()) { try (final RowSet prevIdx = source.getRowSet().copyPrev()) { upstream.shifted().apply((begin, end, delta) -> { @@ -401,7 +417,7 @@ public void onUpdate(TableUpdate upstream) { for (int winIdx = 0; winIdx < windows.length; winIdx++) { if (ctx.windowAffected[winIdx]) { if (ctx.windowContexts[winIdx].anyModified()) { - modifiedRowSet.insert(ctx.windowContexts[winIdx].getAdditionalModifications()); + modifiedRowSet.insert(ctx.windowContexts[winIdx].getModifiedRows()); } } } @@ -416,8 +432,7 @@ public void onUpdate(TableUpdate upstream) { // set the modified columns if any operators made changes (add/rem/modify) for (int winIdx = 0; winIdx < windows.length; winIdx++) { if (ctx.windowAffected[winIdx]) { - // TODO: need to add only the affected column sets from the window, not all - downstream.modifiedColumnSet.setAll(outputModifiedColumnSets[winIdx]); + ctx.windowContexts[winIdx].updateOutputModifiedColumnSet(downstream.modifiedColumnSet); } } 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 fc33144ea15..68cd4c7aaa2 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 @@ -140,10 +140,10 @@ public interface AttributeCopier { * @param keyColumnNames The key columns */ PartitionByChunkedOperator(@NotNull final QueryTable unadjustedParentTable, - @NotNull final QueryTable parentTable, - @NotNull final String resultName, - @NotNull final AttributeCopier attributeCopier, - @NotNull final String... keyColumnNames) { + @NotNull final QueryTable parentTable, + @NotNull final String resultName, + @NotNull final AttributeCopier attributeCopier, + @NotNull final String... keyColumnNames) { this.parentTable = parentTable; this.resultName = resultName; this.initialized = false; @@ -195,9 +195,9 @@ public interface AttributeCopier { @Override public void addChunk(final BucketedContext bucketedContext, final Chunk values, - @NotNull final LongChunk inputRowKeys, - @NotNull final IntChunk destinations, @NotNull final IntChunk startPositions, - @NotNull final IntChunk length, @NotNull final WritableBooleanChunk stateModified) { + @NotNull final LongChunk inputRowKeys, + @NotNull final IntChunk destinations, @NotNull final IntChunk startPositions, + @NotNull final IntChunk length, @NotNull final WritableBooleanChunk stateModified) { Assert.eqNull(values, "values"); final AdaptiveOrderedLongSetBuilderRandom chunkDestinationsBuilder = stepUpdatedDestinations == null ? null : new AdaptiveOrderedLongSetBuilderRandom(); @@ -221,7 +221,7 @@ public void addChunk(final BucketedContext bucketedContext, final Chunk values, - @NotNull final LongChunk inputRowKeys, - @NotNull final IntChunk destinations, @NotNull final IntChunk startPositions, - @NotNull final IntChunk length, @NotNull final WritableBooleanChunk stateModified) { + @NotNull final LongChunk inputRowKeys, + @NotNull final IntChunk destinations, @NotNull final IntChunk startPositions, + @NotNull final IntChunk length, @NotNull final WritableBooleanChunk stateModified) { Assert.eqNull(values, "values"); Assert.neqNull(stepUpdatedDestinations, "stepUpdatedDestinations"); final AdaptiveOrderedLongSetBuilderRandom chunkDestinationsBuilder = new AdaptiveOrderedLongSetBuilderRandom(); @@ -247,28 +247,28 @@ public void removeChunk(final BucketedContext bucketedContext, final Chunk previousValues, - final Chunk newValues, - @NotNull final LongChunk postShiftRowKeys, - @NotNull final IntChunk destinations, @NotNull final IntChunk startPositions, - @NotNull final IntChunk length, @NotNull final WritableBooleanChunk stateModified) { + final Chunk newValues, + @NotNull final LongChunk postShiftRowKeys, + @NotNull final IntChunk destinations, @NotNull final IntChunk startPositions, + @NotNull final IntChunk length, @NotNull final WritableBooleanChunk stateModified) { // We have no inputs, so we should never get here. throw new IllegalStateException(); } @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) { + 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) { Assert.eqNull(previousValues, "previousValues"); Assert.eqNull(newValues, "newValues"); Assert.neqNull(stepUpdatedDestinations, "stepUpdatedDestinations"); @@ -284,16 +284,16 @@ public void shiftChunk(final BucketedContext bucketedContext, final Chunk inputRowKeys, - @NotNull final IntChunk destinations, @NotNull final IntChunk startPositions, - @NotNull final IntChunk length, @NotNull final WritableBooleanChunk stateModified) { + @NotNull final LongChunk inputRowKeys, + @NotNull final IntChunk destinations, @NotNull final IntChunk startPositions, + @NotNull final IntChunk length, @NotNull final WritableBooleanChunk stateModified) { if (!stepValuesModified) { return; } @@ -311,15 +311,15 @@ public void modifyRowKeys(final BucketedContext context, chunkDestinationsBuilder.addKey(destination); } try (final RowSet chunkUpdatedDestinations = - new WritableRowSetImpl(chunkDestinationsBuilder.getOrderedLongSet())) { + new WritableRowSetImpl(chunkDestinationsBuilder.getOrderedLongSet())) { stepUpdatedDestinations.insert(chunkUpdatedDestinations); } } @Override public boolean addChunk(final SingletonContext singletonContext, final int chunkSize, - final Chunk values, - @NotNull final LongChunk inputRowKeys, final long destination) { + final Chunk values, + @NotNull final LongChunk inputRowKeys, final long destination) { Assert.eqNull(values, "values"); // noinspection unchecked final LongChunk inputRowKeysAsOrdered = (LongChunk) inputRowKeys; @@ -350,8 +350,8 @@ public boolean addRowSet(SingletonContext context, RowSet rowSet, long destinati @Override public boolean removeChunk(final SingletonContext singletonContext, final int chunkSize, - final Chunk values, - @NotNull final LongChunk inputRowKeys, final long destination) { + final Chunk values, + @NotNull final LongChunk inputRowKeys, final long destination) { Assert.eqNull(values, "values"); Assert.neqNull(stepUpdatedDestinations, "stepUpdatedDestinations"); // noinspection unchecked @@ -363,19 +363,19 @@ public boolean removeChunk(final SingletonContext singletonContext, final int ch @Override public boolean modifyChunk(final SingletonContext singletonContext, final int chunkSize, - final Chunk previousValues, final Chunk newValues, - @NotNull final LongChunk postShiftRowKeys, - final long destination) { + final Chunk previousValues, final Chunk newValues, + @NotNull final LongChunk postShiftRowKeys, + final long destination) { // We have no inputs, so we should never get here. throw new IllegalStateException(); } @Override public boolean shiftChunk(final SingletonContext singletonContext, final Chunk previousValues, - final Chunk newValues, - @NotNull final LongChunk preShiftRowKeys, - @NotNull final LongChunk postShiftRowKeys, - final long destination) { + final Chunk newValues, + @NotNull final LongChunk preShiftRowKeys, + @NotNull final LongChunk postShiftRowKeys, + final long destination) { Assert.eqNull(previousValues, "previousValues"); Assert.eqNull(newValues, "newValues"); Assert.neqNull(stepUpdatedDestinations, "stepUpdatedDestinations"); @@ -387,7 +387,7 @@ public boolean shiftChunk(final SingletonContext singletonContext, final Chunk rowKeys, - final long destination) { + final long destination) { if (!stepValuesModified) { return false; } @@ -439,8 +439,8 @@ private static void accumulateToBuilderSequential( private static void accumulateToBuilderRandom(@NotNull final ObjectArraySource rowSetColumn, - @NotNull final LongChunk rowKeysToAdd, - final int start, final int length, final long destination) { + @NotNull final LongChunk rowKeysToAdd, + final int start, final int length, final long destination) { final RowSetBuilderRandom builder = rowSetColumn.getUnsafe(destination); if (builder == NONEXISTENT_TABLE_ROW_SET_BUILDER_RANDOM) { return; @@ -459,7 +459,7 @@ private static void accumulateToBuilderRandom(@NotNull final ObjectArraySource rowSetColumn, - @NotNull final RowSet rowSetToAdd, final long destination) { + @NotNull final RowSet rowSetToAdd, final long destination) { final RowSetBuilderRandom builder = rowSetColumn.getUnsafe(destination); if (builder == NONEXISTENT_TABLE_ROW_SET_BUILDER_RANDOM) { return; @@ -476,8 +476,8 @@ private static void accumulateToBuilderRandom(@NotNull final ObjectArraySource preShiftRowKeys, - @NotNull final LongChunk postShiftRowKeys, - final int startPosition, final int runLength, final long destination) { + @NotNull final LongChunk postShiftRowKeys, + final int startPosition, final int runLength, final long destination) { RowSetShiftData.SmartCoalescingBuilder builder = shiftDataBuilders.getUnsafe(destination); if (builder == NONEXISTENT_TABLE_SHIFT_BUILDER) { return false; @@ -553,11 +553,11 @@ public void propagateInitialState(@NotNull final QueryTable resultTable) { // initialization scope, and don't need to do anything special to ensure liveness. final boolean setCallSite = QueryPerformanceRecorder.setCallsite(callSite); try (final ResettableWritableObjectChunk tablesResettableChunk = - ResettableWritableObjectChunk.makeResettableChunk(); - final ResettableWritableObjectChunk addedRowSetsResettableChunk = - !initialized ? ResettableWritableObjectChunk.makeResettableChunk() : null; - final RowSequence.Iterator initialDestinationsIterator = - initialDestinations.getRowSequenceIterator()) { + ResettableWritableObjectChunk.makeResettableChunk(); + final ResettableWritableObjectChunk addedRowSetsResettableChunk = + !initialized ? ResettableWritableObjectChunk.makeResettableChunk() : null; + final RowSequence.Iterator initialDestinationsIterator = + initialDestinations.getRowSequenceIterator()) { // noinspection unchecked final WritableObjectChunk tablesBackingChunk = @@ -667,11 +667,11 @@ private void propagateResurrectedDestinations(@NotNull final RowSequence resurre return; } try (final ResettableWritableObjectChunk tablesResettableChunk = - ResettableWritableObjectChunk.makeResettableChunk(); - final ResettableWritableObjectChunk addedRowSetsResettableChunk = - ResettableWritableObjectChunk.makeResettableChunk(); - final RowSequence.Iterator resurrectedDestinationsIterator = - resurrectedDestinations.getRowSequenceIterator()) { + ResettableWritableObjectChunk.makeResettableChunk(); + final ResettableWritableObjectChunk addedRowSetsResettableChunk = + ResettableWritableObjectChunk.makeResettableChunk(); + final RowSequence.Iterator resurrectedDestinationsIterator = + resurrectedDestinations.getRowSequenceIterator()) { // Destinations that were added can't have any removals, modifications, or shifts. // noinspection unchecked @@ -733,16 +733,16 @@ private void propagateNewDestinations(@NotNull final RowSequence newDestinations final boolean allowCreation = retainedResultTable && retainedAggregationUpdateListener; final boolean setCallSite = QueryPerformanceRecorder.setCallsite(callSite); try (final ResettableWritableObjectChunk tablesResettableChunk = - ResettableWritableObjectChunk.makeResettableChunk(); - final ResettableWritableObjectChunk addedRowSetsResettableChunk = - ResettableWritableObjectChunk.makeResettableChunk(); - final ResettableWritableObjectChunk removedRowSetsResettableChunk = - allowCreation ? null : ResettableWritableObjectChunk.makeResettableChunk(); - final ResettableWritableObjectChunk modifiedRowSetsResettableChunk = - allowCreation ? null : ResettableWritableObjectChunk.makeResettableChunk(); - final ResettableWritableObjectChunk shiftDataBuildersResettableChunk = - allowCreation ? null : ResettableWritableObjectChunk.makeResettableChunk(); - final RowSequence.Iterator newDestinationsIterator = newDestinations.getRowSequenceIterator()) { + ResettableWritableObjectChunk.makeResettableChunk(); + final ResettableWritableObjectChunk addedRowSetsResettableChunk = + ResettableWritableObjectChunk.makeResettableChunk(); + final ResettableWritableObjectChunk removedRowSetsResettableChunk = + allowCreation ? null : ResettableWritableObjectChunk.makeResettableChunk(); + final ResettableWritableObjectChunk modifiedRowSetsResettableChunk = + allowCreation ? null : ResettableWritableObjectChunk.makeResettableChunk(); + final ResettableWritableObjectChunk shiftDataBuildersResettableChunk = + allowCreation ? null : ResettableWritableObjectChunk.makeResettableChunk(); + final RowSequence.Iterator newDestinationsIterator = newDestinations.getRowSequenceIterator()) { // noinspection unchecked final WritableObjectChunk tablesBackingChunk = @@ -822,10 +822,10 @@ private void propagateUpdatesToRemovedDestinations(@NotNull final RowSequence re return; } try (final ResettableWritableObjectChunk tablesResettableChunk = - ResettableWritableObjectChunk.makeResettableChunk(); - final ResettableWritableObjectChunk removedRowSetsResettableChunk = - ResettableWritableObjectChunk.makeResettableChunk(); - final RowSequence.Iterator removedDestinationsIterator = removedDestinations.getRowSequenceIterator()) { + ResettableWritableObjectChunk.makeResettableChunk(); + final ResettableWritableObjectChunk removedRowSetsResettableChunk = + ResettableWritableObjectChunk.makeResettableChunk(); + final RowSequence.Iterator removedDestinationsIterator = removedDestinations.getRowSequenceIterator()) { // Destinations that were completely removed can't have any additions, modifications, or shifts. // noinspection unchecked @@ -878,20 +878,20 @@ private void propagateUpdatesToModifiedDestinations(@NotNull final RowSequence m return; } try (final ResettableWritableObjectChunk tablesResettableChunk = - ResettableWritableObjectChunk.makeResettableChunk(); - final ResettableWritableObjectChunk addedRowSetsResettableChunk = - initialized ? ResettableWritableObjectChunk.makeResettableChunk() : null; - final ResettableWritableObjectChunk addedRowSetsSequentialResettableChunk = - !initialized ? ResettableWritableObjectChunk.makeResettableChunk() : null; - - final ResettableWritableObjectChunk removedRowSetsResettableChunk = - ResettableWritableObjectChunk.makeResettableChunk(); - final ResettableWritableObjectChunk modifiedRowSetsResettableChunk = - ResettableWritableObjectChunk.makeResettableChunk(); - final ResettableWritableObjectChunk shiftDataBuildersResettableChunk = - ResettableWritableObjectChunk.makeResettableChunk(); - final RowSequence.Iterator modifiedDestinationsIterator = - modifiedDestinations.getRowSequenceIterator()) { + ResettableWritableObjectChunk.makeResettableChunk(); + final ResettableWritableObjectChunk addedRowSetsResettableChunk = + initialized ? ResettableWritableObjectChunk.makeResettableChunk() : null; + final ResettableWritableObjectChunk addedRowSetsSequentialResettableChunk = + !initialized ? ResettableWritableObjectChunk.makeResettableChunk() : null; + + final ResettableWritableObjectChunk removedRowSetsResettableChunk = + ResettableWritableObjectChunk.makeResettableChunk(); + final ResettableWritableObjectChunk modifiedRowSetsResettableChunk = + ResettableWritableObjectChunk.makeResettableChunk(); + final ResettableWritableObjectChunk shiftDataBuildersResettableChunk = + ResettableWritableObjectChunk.makeResettableChunk(); + final RowSequence.Iterator modifiedDestinationsIterator = + modifiedDestinations.getRowSequenceIterator()) { // noinspection unchecked final ObjectChunk tablesBackingChunk = tablesResettableChunk.asObjectChunk(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java index 2e9399c9c28..0007fa54c62 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java @@ -9,6 +9,7 @@ import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.ModifiedColumnSet; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.QueryTable; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; @@ -30,9 +31,14 @@ public class UpdateByWindow { protected final long fwdUnits; // store the operators for this window - protected UpdateByOperator[] operators; + protected final UpdateByOperator[] operators; // store the index in the {@link UpdateBy.inputSources} - protected int[] operatorSourceSlots; + protected final int[] operatorSourceSlots; + // individual input/output modifiedColumnSets for the operators + protected final ModifiedColumnSet[] operatorInputModifiedColumnSets; + protected final ModifiedColumnSet[] operatorOutputModifiedColumnSets; + + protected boolean trackModifications; public class UpdateByWindowContext implements SafeCloseable { /** store a reference to the source rowset */ @@ -53,7 +59,7 @@ public class UpdateByWindowContext implements SafeCloseable { final UpdateByOperator.UpdateContext[] opContext; /** An array of ColumnSources for each underlying operator */ - final ChunkSource[] inputSource; + final ColumnSource[] inputSources; /** An array of {@link ChunkSource.FillContext}s for each input column */ final ChunkSource.FillContext[] inputSourceFillContexts; @@ -80,11 +86,11 @@ public class UpdateByWindowContext implements SafeCloseable { final int chunkSize; final boolean initialStep; - public UpdateByWindowContext(final TrackingRowSet sourceRowSet, final ChunkSource[] opInputSource, + public UpdateByWindowContext(final TrackingRowSet sourceRowSet, final ColumnSource[] inputSources, @Nullable final ColumnSource timestampColumnSource, @Nullable final LongSegmentedSortedArray timestampSsa, final int chunkSize, final boolean initialStep) { this.sourceRowSet = sourceRowSet; - this.inputSource = opInputSource; + this.inputSources = inputSources; this.timestampColumnSource = timestampColumnSource; this.timestampSsa = timestampSsa; @@ -99,8 +105,7 @@ public UpdateByWindowContext(final TrackingRowSet sourceRowSet, final ChunkSourc this.initialStep = initialStep; } - public boolean computeAffectedRowsAndOperators(@NotNull final TableUpdate upstream, - @Nullable final ModifiedColumnSet inputModifiedColumnSets) { + public boolean computeAffectedRowsAndOperators(@NotNull final TableUpdate upstream) { // all rows are affected on the initial step if (initialStep) { affectedRows = sourceRowSet.copy(); @@ -125,15 +130,14 @@ public boolean computeAffectedRowsAndOperators(@NotNull final TableUpdate upstre for (int opIdx = 0; opIdx < operators.length; opIdx++) { opAffected[opIdx] = allAffected - || (upstream.modifiedColumnSet().nonempty() && (inputModifiedColumnSets == null - || upstream.modifiedColumnSet().containsAny(inputModifiedColumnSets))); + || (upstream.modifiedColumnSet().nonempty() && (operatorInputModifiedColumnSets[opIdx] == null + || upstream.modifiedColumnSet().containsAny(operatorInputModifiedColumnSets[opIdx]))); if (opAffected[opIdx]) { anyAffected = true; } } - if (sourceRowSet.isEmpty() || !anyAffected) { - // no work to do for this window this cycle + if (!anyAffected) { return false; } @@ -222,16 +226,16 @@ private void makeOperatorContexts() { // create contexts for the affected operators for (int opIdx = 0; opIdx < operators.length; opIdx++) { if (opAffected[opIdx]) { - opContext[opIdx] = operators[opIdx].makeUpdateContext(chunkSize); - - // create the fill contexts and + // create the fill contexts for the input sources int sourceSlot = operatorSourceSlots[opIdx]; if (!inputSourceChunkPopulated[sourceSlot]) { inputSourceChunks[sourceSlot] = - inputSource[sourceSlot].getChunkType().makeWritableChunk(chunkSize); - inputSourceFillContexts[sourceSlot] = inputSource[sourceSlot].makeFillContext(chunkSize); + inputSources[sourceSlot].getChunkType().makeWritableChunk(chunkSize); + inputSourceFillContexts[sourceSlot] = inputSources[sourceSlot].makeFillContext(chunkSize); inputSourceChunkPopulated[sourceSlot] = true; } + opContext[opIdx] = operators[opIdx].makeUpdateContext(chunkSize, inputSources[sourceSlot]); + } } @@ -241,10 +245,18 @@ public boolean anyModified() { return newModified != null && newModified.isNonempty(); } - public RowSet getAdditionalModifications() { + public RowSet getModifiedRows() { return newModified; } + public void updateOutputModifiedColumnSet(ModifiedColumnSet outputModifiedColumnSet) { + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (opAffected[opIdx]) { + outputModifiedColumnSet.setAll(operatorOutputModifiedColumnSets[opIdx]); + } + } + } + public RowSet getAffectedRows() { return affectedRows; } @@ -254,25 +266,26 @@ public RowSet getInfluencerRows() { } public void processRows() { - modifiedBuilder = RowSetFactory.builderSequential(); + if (trackModifications) { + modifiedBuilder = RowSetFactory.builderSequential(); + } - // these could be nested and/or simplified but this is most readable - if (!windowed && timestampColumnName == null) { + if (!windowed) { processRowsCumulative(); - } else if (!windowed && timestampColumnName != null) { - processRowsCumulativeTimestamp(); - } else if (windowed && timestampColumnName == null) { + } else if (timestampColumnName == null) { processRowsWindowedTicks(); } else { processRowsWindowedTime(); } - newModified = modifiedBuilder.build(); + if (trackModifications) { + newModified = modifiedBuilder.build(); + } } private void prepareValuesChunkForSource(final int srcIdx, final RowSequence rs) { if (!inputSourceChunkPopulated[srcIdx]) { - inputSource[srcIdx].fillChunk( + inputSources[srcIdx].fillChunk( inputSourceFillContexts[srcIdx], inputSourceChunks[srcIdx], rs); @@ -290,18 +303,47 @@ private void processRowsCumulative() { for (int opIdx = 0; opIdx < operators.length; opIdx++) { if (opAffected[opIdx]) { - // call the specialized version of `intializeUpdate()` for these operators - // TODO: make sure the time-based cumulative oerators are starting from a valid value and timestamp - ((UpdateByCumulativeOperator) operators[opIdx]).initializeUpdate(opContext[opIdx], keyBefore, - NULL_LONG); + UpdateByCumulativeOperator cumOp = (UpdateByCumulativeOperator) operators[opIdx]; + if (cumOp.getTimestampColumnName() == null || keyBefore == -1) { + cumOp.initializeUpdate(opContext[opIdx], keyBefore, NULL_LONG); + } else { + UpdateByCumulativeOperator.Context cumOpContext = + (UpdateByCumulativeOperator.Context) opContext[opIdx]; + // make sure the time-based cumulative operators are starting from a valid value and timestamp + long potentialResetTimestamp = timestampColumnSource.getLong(keyBefore); + + if (potentialResetTimestamp == NULL_LONG || + !cumOpContext.isValueValid(keyBefore)) { + try (final RowSet.SearchIterator rIt = sourceRowSet.reverseIterator()) { + if (rIt.advance(keyBefore)) { + while (rIt.hasNext()) { + final long nextKey = rIt.nextLong(); + potentialResetTimestamp = timestampColumnSource.getLong(nextKey); + if (potentialResetTimestamp != NULL_LONG && + cumOpContext.isValueValid(nextKey)) { + break; + } + } + } + } + } + // call the specialized version of `intializeUpdate()` for these operators + cumOp.initializeUpdate(opContext[opIdx], keyBefore, potentialResetTimestamp); + } } } - try (final RowSequence.Iterator it = affectedRows.getRowSequenceIterator()) { + try (final RowSequence.Iterator it = affectedRows.getRowSequenceIterator(); + ChunkSource.GetContext tsGetCtx = + timestampColumnSource == null ? null : timestampColumnSource.makeGetContext(chunkSize)) { while (it.hasMore()) { final RowSequence rs = it.getNextRowSequenceWithLength(chunkSize); Arrays.fill(inputSourceChunkPopulated, false); + // create the timestamp chunk if needed + LongChunk tsChunk = timestampColumnSource == null ? null + : timestampColumnSource.getChunk(tsGetCtx, rs).asLongChunk(); + for (int opIdx = 0; opIdx < operators.length; opIdx++) { if (opAffected[opIdx]) { final int srcIdx = operatorSourceSlots[opIdx]; @@ -316,11 +358,13 @@ private void processRowsCumulative() { null, null, inputSourceChunks[srcIdx], - null); + tsChunk); } } // all these rows were modified - modifiedBuilder.appendRowSequence(rs); + if (modifiedBuilder != null) { + modifiedBuilder.appendRowSequence(rs); + } } } @@ -330,16 +374,90 @@ private void processRowsCumulative() { operators[opIdx].finishUpdate(opContext[opIdx]); } } - - } - - private void processRowsCumulativeTimestamp() { - // find the key before the first affected row (that has a valid timestamp) and preload - // that data for these operators } private void processRowsWindowedTicks() { // start loading the window for these operators using position data + // find the key before the first affected row and preload that data for these operators + final long keyBefore; + try (final RowSet.SearchIterator sit = sourceRowSet.searchIterator()) { + keyBefore = sit.binarySearchValue( + (compareTo, ignored) -> Long.compare(affectedRows.firstRowKey() - 1, compareTo), 1); + } + + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (opAffected[opIdx]) { + UpdateByCumulativeOperator cumOp = (UpdateByCumulativeOperator) operators[opIdx]; + if (cumOp.getTimestampColumnName() == null || keyBefore == -1) { + cumOp.initializeUpdate(opContext[opIdx], keyBefore, NULL_LONG); + } else { + UpdateByCumulativeOperator.Context cumOpContext = + (UpdateByCumulativeOperator.Context) opContext[opIdx]; + // make sure the time-based cumulative operators are starting from a valid value and timestamp + long potentialResetTimestamp = timestampColumnSource.getLong(keyBefore); + + if (potentialResetTimestamp == NULL_LONG || + !cumOpContext.isValueValid(keyBefore)) { + try (final RowSet.SearchIterator rIt = sourceRowSet.reverseIterator()) { + if (rIt.advance(keyBefore)) { + while (rIt.hasNext()) { + final long nextKey = rIt.nextLong(); + potentialResetTimestamp = timestampColumnSource.getLong(nextKey); + if (potentialResetTimestamp != NULL_LONG && + cumOpContext.isValueValid(nextKey)) { + break; + } + } + } + } + } + // call the specialized version of `intializeUpdate()` for these operators + cumOp.initializeUpdate(opContext[opIdx], keyBefore, potentialResetTimestamp); + } + } + } + + try (final RowSequence.Iterator it = affectedRows.getRowSequenceIterator(); + ChunkSource.GetContext tsGetCtx = + timestampColumnSource == null ? null : timestampColumnSource.makeGetContext(chunkSize)) { + while (it.hasMore()) { + final RowSequence rs = it.getNextRowSequenceWithLength(chunkSize); + Arrays.fill(inputSourceChunkPopulated, false); + + // create the timestamp chunk if needed + LongChunk tsChunk = timestampColumnSource == null ? null + : timestampColumnSource.getChunk(tsGetCtx, rs).asLongChunk(); + + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (opAffected[opIdx]) { + final int srcIdx = operatorSourceSlots[opIdx]; + + // get the values this operator needs + prepareValuesChunkForSource(srcIdx, rs); + + // process the chunk + operators[opIdx].processChunk( + opContext[opIdx], + rs, + null, + null, + inputSourceChunks[srcIdx], + tsChunk); + } + } + // all these rows were modified + if (modifiedBuilder != null) { + modifiedBuilder.appendRowSequence(rs); + } + } + } + + // call the generic `finishUpdate()` function for each operator + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (opAffected[opIdx]) { + operators[opIdx].finishUpdate(opContext[opIdx]); + } + } } private void processRowsWindowedTime() { @@ -376,7 +494,7 @@ public void close() { } public UpdateByWindowContext makeWindowContext(final TrackingRowSet sourceRowSet, - final ChunkSource[] inputSources, + final ColumnSource[] inputSources, final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa, final int chunkSize, @@ -385,29 +503,14 @@ public UpdateByWindowContext makeWindowContext(final TrackingRowSet sourceRowSet isInitializeStep); } - public void setOperators(final UpdateByOperator[] operators, final int[] operatorSourceSlots) { - this.operators = operators; - this.operatorSourceSlots = operatorSourceSlots; - } - - - @NotNull - public String[] getAffectingColumnNames() { - Set columns = new TreeSet<>(); - for (UpdateByOperator operator : operators) { - columns.addAll(Arrays.asList(operator.getAffectingColumnNames())); - } - return columns.toArray(new String[0]); - } - - @NotNull - public String[] getOutputColumnNames() { - // we can use a list since we have previously checked for duplicates - List columns = new ArrayList<>(); - for (UpdateByOperator operator : operators) { - columns.addAll(Arrays.asList(operator.getOutputColumnNames())); + public void startTrackingModifications(@NotNull final QueryTable source, @NotNull final QueryTable result) { + trackModifications = true; + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + operatorInputModifiedColumnSets[opIdx] = + source.newModifiedColumnSet(operators[opIdx].getAffectingColumnNames()); + operatorOutputModifiedColumnSets[opIdx] = + result.newModifiedColumnSet(operators[opIdx].getOutputColumnNames()); } - return columns.toArray(new String[0]); } /** @@ -594,18 +697,37 @@ private static long locatePreviousTimestamp(final RowSet sourceSet, final Column return NULL_LONG; } - private UpdateByWindow(boolean windowed, @Nullable String timestampColumnName, long prevUnits, long fwdUnits) { + private UpdateByWindow(UpdateByOperator[] operators, int[] operatorSourceSlots, boolean windowed, + @Nullable String timestampColumnName, long prevUnits, long fwdUnits) { + this.operators = operators; + this.operatorSourceSlots = operatorSourceSlots; this.windowed = windowed; this.timestampColumnName = timestampColumnName; this.prevUnits = prevUnits; this.fwdUnits = fwdUnits; + + operatorInputModifiedColumnSets = new ModifiedColumnSet[operators.length]; + operatorOutputModifiedColumnSets = new ModifiedColumnSet[operators.length]; + trackModifications = false; } - public static UpdateByWindow createFromOperator(final UpdateByOperator op) { - return new UpdateByWindow(op instanceof UpdateByWindowedOperator, - op.getTimestampColumnName(), - op.getPrevWindowUnits(), - op.getPrevWindowUnits()); + public static UpdateByWindow createFromOperatorArray(final UpdateByOperator[] operators, + final int[] operatorSourceSlots) { + // review operators to extract timestamp column (if one exists) + String timestampColumnName = null; + for (UpdateByOperator operator : operators) { + if (operator.getTimestampColumnName() != null) { + timestampColumnName = operator.getTimestampColumnName(); + break; + } + } + + return new UpdateByWindow(operators, + operatorSourceSlots, + operators[0] instanceof UpdateByWindowedOperator, + timestampColumnName, + operators[0].getPrevWindowUnits(), + operators[0].getPrevWindowUnits()); } @Nullable @@ -613,17 +735,13 @@ public String getTimestampColumnName() { return timestampColumnName; } - @NotNull - final public RowSet getAdditionalModifications(@NotNull final UpdateByWindowContext context) { - return context.newModified; - } - - final public boolean anyModified(@NotNull final UpdateByWindowContext context) { - return context.newModified != null && context.newModified.isNonempty(); - } + private static int hashCode(boolean windowed, @Nullable String timestampColumnName, long prevUnits, long fwdUnits) { + // treat all cumulative as identical, even if they rely on timestamps + if (!windowed) { + return Boolean.hashCode(windowed); + } - @Override - public int hashCode() { + // windowed are unique per timestamp column and window-size int hash = Boolean.hashCode(windowed); if (timestampColumnName != null) { hash = 31 * hash + timestampColumnName.hashCode(); @@ -632,4 +750,19 @@ public int hashCode() { hash = 31 * hash + Long.hashCode(fwdUnits); return hash; } + + public static int hashCodeFromOperator(final UpdateByOperator op) { + return hashCode(op instanceof UpdateByWindowedOperator, + op.getTimestampColumnName(), + op.getPrevWindowUnits(), + op.getPrevWindowUnits()); + } + + @Override + public int hashCode() { + return hashCode(windowed, + timestampColumnName, + prevUnits, + fwdUnits); + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java index fb0994b3540..71e6c0768f9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java @@ -21,13 +21,12 @@ public abstract class BasePrimitiveEMAOperator extends BaseDoubleUpdateByOperator { protected final OperationControl control; - protected final String timestampColumnName; protected final double timeScaleUnits; protected final double alpha; protected double oneMinusAlpha; class Context extends BaseDoubleUpdateByOperator.Context { - public LongChunk timestampValueChunk; + public LongChunk timestampValueChunk; long lastStamp = NULL_LONG; @@ -86,8 +85,8 @@ public void processChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk keyChunk, @Nullable final LongChunk posChunk, @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { - Assert.notEquals(valuesChunk, "valuesChunk must not be null for a cumulative operator", null); + @Nullable final LongChunk timestampValuesChunk) { + Assert.neqNull(valuesChunk, "valuesChunk must not be null for a cumulative operator"); final Context ctx = (Context) updateContext; ctx.storeValuesChunk(valuesChunk); ctx.timestampValueChunk = timestampValuesChunk; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java index 7596ae6d1d2..aff3a6ac4c9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java @@ -1,7 +1,6 @@ package io.deephaven.engine.table.impl.updateby.ema; import io.deephaven.api.updateby.OperationControl; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import org.jetbrains.annotations.NotNull; @@ -23,7 +22,6 @@ public class BigDecimalEMAOperator extends BigNumberEMAOperator { * integer ticks. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timeRecorder} is provided, this is measured * in ticks, otherwise it is measured in nanoseconds - * @param valueSource the input column source. Used when determining reset positions for reprocessing * @param redirContext the row redirection context to use for the EMA */ public BigDecimalEMAOperator(@NotNull final MatchPair pair, @@ -31,9 +29,8 @@ public BigDecimalEMAOperator(@NotNull final MatchPair pair, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull final ColumnSource valueSource, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { - super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, valueSource, redirContext); + super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, redirContext); } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java index be019d8dc25..8f867c40900 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java @@ -1,7 +1,6 @@ package io.deephaven.engine.table.impl.updateby.ema; import io.deephaven.api.updateby.OperationControl; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import org.jetbrains.annotations.NotNull; @@ -23,7 +22,6 @@ public class BigIntegerEMAOperator extends BigNumberEMAOperator { * integer ticks. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timeRecorder} is provided, this is measured * in ticks, otherwise it is measured in nanoseconds - * @param valueSource the input column source. Used when determining reset positions for reprocessing * @param redirContext the row redirection context to use for the EMA */ public BigIntegerEMAOperator(@NotNull final MatchPair pair, @@ -31,10 +29,9 @@ public BigIntegerEMAOperator(@NotNull final MatchPair pair, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull final ColumnSource valueSource, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext ) { - super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, valueSource, redirContext); + super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, redirContext); } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java index 33fbe250758..0766ed45bdd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java @@ -9,6 +9,7 @@ import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; @@ -23,29 +24,33 @@ import static io.deephaven.util.QueryConstants.*; public abstract class BigNumberEMAOperator extends BaseObjectUpdateByOperator { - protected final ColumnSource valueSource; - protected final OperationControl control; - protected final String timestampColumnName; protected final double timeScaleUnits; protected final BigDecimal alpha; protected final BigDecimal oneMinusAlpha; class Context extends BaseObjectUpdateByOperator.Context { - public LongChunk timestampValueChunk; + protected final ColumnSource valueSource; + public LongChunk timestampValueChunk; public ObjectChunk objectValueChunk; long lastStamp = NULL_LONG; - Context(final int chunkSize) { + protected Context(int chunkSize, ColumnSource inputSource) { super(chunkSize); + this.valueSource = inputSource; } @Override public void storeValuesChunk(@NotNull final Chunk valuesChunk) { objectValueChunk = valuesChunk.asObjectChunk(); } + + @Override + public boolean isValueValid(long atKey) { + return valueSource.get(atKey) != null; + } } /** @@ -56,21 +61,18 @@ public void storeValuesChunk(@NotNull final Chunk valuesChunk) { * @param control defines how to handle {@code null} input values. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timeRecorder} is provided, this is measured * in ticks, otherwise it is measured in nanoseconds - * @param valueSource the input column source. Used when determining reset positions for reprocessing */ public BigNumberEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull ColumnSource valueSource, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { super(pair, affectingColumns, redirContext, BigDecimal.class); this.control = control; this.timestampColumnName = timestampColumnName; this.timeScaleUnits = (double) timeScaleUnits; - this.valueSource = valueSource; alpha = BigDecimal.valueOf(Math.exp(-1.0 / (double) timeScaleUnits)); oneMinusAlpha = @@ -79,8 +81,8 @@ public BigNumberEMAOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, ColumnSource inputSource) { + return new Context(chunkSize, inputSource); } @Override @@ -105,8 +107,8 @@ public void processChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk keyChunk, @Nullable final LongChunk posChunk, @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { - Assert.notEquals(valuesChunk, "valuesChunk must not be null for a cumulative operator", null); + @Nullable final LongChunk timestampValuesChunk) { + Assert.neqNull(valuesChunk, "valuesChunk must not be null for a cumulative operator"); final Context ctx = (Context) updateContext; ctx.storeValuesChunk(valuesChunk); ctx.timestampValueChunk = timestampValuesChunk; @@ -160,9 +162,4 @@ void handleBadTime(@NotNull final Context ctx, final long dt) { ctx.lastStamp = NULL_LONG; } } - - @Override - public boolean isValueValid(long atKey) { - return valueSource.get(atKey) != null; - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java index f5730d8af25..d0e05580c0a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java @@ -18,19 +18,26 @@ import static io.deephaven.util.QueryConstants.*; public class ByteEMAOperator extends BasePrimitiveEMAOperator { - private final ColumnSource valueSource; protected class Context extends BasePrimitiveEMAOperator.Context { + public final ColumnSource valueSource; + public ByteChunk byteValueChunk; - protected Context(int chunkSize) { + protected Context(int chunkSize, ColumnSource inputSource) { super(chunkSize); + this.valueSource = inputSource; } @Override public void storeValuesChunk(@NotNull final Chunk valuesChunk) { byteValueChunk = valuesChunk.asByteChunk(); } + + @Override + public boolean isValueValid(long atKey) { + return valueSource.getByte(atKey) != NULL_BYTE; + } } /** @@ -41,28 +48,25 @@ public void storeValuesChunk(@NotNull final Chunk valuesChunk) { * @param control defines how to handle {@code null} input values. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timeRecorder} is provided, this is measured * in ticks, otherwise it is measured in nanoseconds - * @param valueSource the input column source. Used when determining reset positions for reprocessing */ public ByteEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull final ColumnSource valueSource, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits,redirContext); - this.valueSource = valueSource; // region constructor // endregion constructor } @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + return new Context(chunkSize, inputSource); } @Override @@ -107,9 +111,4 @@ public void push(UpdateContext context, long key, int pos) { } } } - - @Override - public boolean isValueValid(long atKey) { - return valueSource.getByte(atKey) != NULL_BYTE; - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java index f47fb81d7c9..181b025e18b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java @@ -13,26 +13,38 @@ import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.updateby.prod.ShortCumProdOperator; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.*; public class DoubleEMAOperator extends BasePrimitiveEMAOperator { - private final ColumnSource valueSource; - protected class Context extends BasePrimitiveEMAOperator.Context { + private final ColumnSource valueSource; + public DoubleChunk doubleValueChunk; - protected Context(int chunkSize) { + protected Context(int chunkSize, ColumnSource inputSource) { super(chunkSize); + this.valueSource = inputSource; } @Override public void storeValuesChunk(@NotNull final Chunk valuesChunk) { doubleValueChunk = valuesChunk.asDoubleChunk(); } + + @Override + public boolean isValueValid(long atKey) { + final double value = valueSource.getDouble(atKey); + if(value == NULL_DOUBLE) { + return false; + } + + // Note that we don't care about Reset because in that case the current EMA at this key would be null + // and the superclass will do the right thing. + return !Double.isNaN(value) || control.onNanValueOrDefault() != BadDataBehavior.SKIP; + } } /** @@ -43,28 +55,25 @@ public void storeValuesChunk(@NotNull final Chunk valuesChunk) { * @param control defines how to handle {@code null} input values. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timeRecorder} is provided, this is measured * in ticks, otherwise it is measured in nanoseconds - * @param valueSource the input column source. Used when determining reset positions for reprocessing */ public DoubleEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull final ColumnSource valueSource, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, redirContext); - this.valueSource = valueSource; // region constructor // endregion constructor } @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + return new Context(chunkSize, inputSource); } @Override @@ -72,10 +81,13 @@ public void push(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; final double input = ctx.doubleValueChunk.get(pos); + final boolean isNull = input == NULL_DOUBLE; + final boolean isNan = Double.isNaN(input); + if (timestampColumnName == null) { // compute with ticks - if(input == NULL_DOUBLE) { - handleBadData(ctx, true, false, false); + if(isNull || isNan) { + handleBadData(ctx, isNull, isNan, false); } else { if(ctx.curVal == NULL_DOUBLE) { ctx.curVal = input; @@ -86,20 +98,27 @@ public void push(UpdateContext context, long key, int pos) { } else { // compute with time final long timestamp = ctx.timestampValueChunk.get(pos); - //noinspection ConstantConditions - final boolean isNull = input == NULL_DOUBLE; final boolean isNullTime = timestamp == NULL_LONG; - if(isNull || isNullTime) { - handleBadData(ctx, isNull, false, isNullTime); + + + // Handle bad data first + if (isNull || isNan || isNullTime) { + handleBadData(ctx, isNull, isNan, isNullTime); + } else if (ctx.curVal == NULL_DOUBLE) { + // If the data looks good, and we have a null ema, just accept the current value + ctx.curVal = input; + ctx.lastStamp = timestamp; } else { - if(ctx.curVal == NULL_DOUBLE) { - ctx.curVal = input; + final boolean currentPoisoned = Double.isNaN(ctx.curVal); + if (currentPoisoned && ctx.lastStamp == NULL_LONG) { + // If the current EMA was a NaN, we should accept the first good timestamp so that + // we can handle reset behavior properly in the following else ctx.lastStamp = timestamp; } else { final long dt = timestamp - ctx.lastStamp; - if(dt <= 0) { + if (dt <= 0) { handleBadTime(ctx, dt); - } else { + } else if (!currentPoisoned) { final double alpha = Math.exp(-dt / timeScaleUnits); ctx.curVal = alpha * ctx.curVal + ((1 - alpha) * input); ctx.lastStamp = timestamp; @@ -108,16 +127,4 @@ public void push(UpdateContext context, long key, int pos) { } } } - - @Override - public boolean isValueValid(long atKey) { - final double value = valueSource.getDouble(atKey); - if(value == NULL_DOUBLE) { - return false; - } - - // Note that we don't care about Reset because in that case the current EMA at this key would be null - // and the superclass will do the right thing. - return !Double.isNaN(value) || control.onNanValueOrDefault() != BadDataBehavior.SKIP; - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java index 2962bd8d4cd..f46a3cd8b62 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java @@ -8,26 +8,38 @@ import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.updateby.prod.ShortCumProdOperator; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.*; public class FloatEMAOperator extends BasePrimitiveEMAOperator { - private final ColumnSource valueSource; - protected class Context extends BasePrimitiveEMAOperator.Context { + private final ColumnSource valueSource; + public FloatChunk floatValueChunk; - protected Context(int chunkSize) { + protected Context(int chunkSize, ColumnSource inputSource) { super(chunkSize); + this.valueSource = inputSource; } @Override public void storeValuesChunk(@NotNull final Chunk valuesChunk) { floatValueChunk = valuesChunk.asFloatChunk(); } + + @Override + public boolean isValueValid(long atKey) { + final float value = valueSource.getFloat(atKey); + if(value == NULL_FLOAT) { + return false; + } + + // Note that we don't care about Reset because in that case the current EMA at this key would be null + // and the superclass will do the right thing. + return !Float.isNaN(value) || control.onNanValueOrDefault() != BadDataBehavior.SKIP; + } } /** @@ -38,28 +50,25 @@ public void storeValuesChunk(@NotNull final Chunk valuesChunk) { * @param control defines how to handle {@code null} input values. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timeRecorder} is provided, this is measured * in ticks, otherwise it is measured in nanoseconds - * @param valueSource the input column source. Used when determining reset positions for reprocessing */ public FloatEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull final ColumnSource valueSource, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, redirContext); - this.valueSource = valueSource; // region constructor // endregion constructor } @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + return new Context(chunkSize, inputSource); } @Override @@ -67,10 +76,13 @@ public void push(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; final float input = ctx.floatValueChunk.get(pos); + final boolean isNull = input == NULL_FLOAT; + final boolean isNan = Float.isNaN(input); + if (timestampColumnName == null) { // compute with ticks - if(input == NULL_FLOAT) { - handleBadData(ctx, true, false, false); + if(isNull || isNan) { + handleBadData(ctx, isNull, isNan, false); } else { if(ctx.curVal == NULL_DOUBLE) { ctx.curVal = input; @@ -81,20 +93,27 @@ public void push(UpdateContext context, long key, int pos) { } else { // compute with time final long timestamp = ctx.timestampValueChunk.get(pos); - //noinspection ConstantConditions - final boolean isNull = input == NULL_FLOAT; final boolean isNullTime = timestamp == NULL_LONG; - if(isNull || isNullTime) { - handleBadData(ctx, isNull, false, isNullTime); + + + // Handle bad data first + if (isNull || isNan || isNullTime) { + handleBadData(ctx, isNull, isNan, isNullTime); + } else if (ctx.curVal == NULL_DOUBLE) { + // If the data looks good, and we have a null ema, just accept the current value + ctx.curVal = input; + ctx.lastStamp = timestamp; } else { - if(ctx.curVal == NULL_DOUBLE) { - ctx.curVal = input; + final boolean currentPoisoned = Double.isNaN(ctx.curVal); + if (currentPoisoned && ctx.lastStamp == NULL_LONG) { + // If the current EMA was a NaN, we should accept the first good timestamp so that + // we can handle reset behavior properly in the following else ctx.lastStamp = timestamp; } else { final long dt = timestamp - ctx.lastStamp; - if(dt <= 0) { + if (dt <= 0) { handleBadTime(ctx, dt); - } else { + } else if (!currentPoisoned) { final double alpha = Math.exp(-dt / timeScaleUnits); ctx.curVal = alpha * ctx.curVal + ((1 - alpha) * input); ctx.lastStamp = timestamp; @@ -103,16 +122,4 @@ public void push(UpdateContext context, long key, int pos) { } } } - - @Override - public boolean isValueValid(long atKey) { - final float value = valueSource.getFloat(atKey); - if(value == NULL_FLOAT) { - return false; - } - - // Note that we don't care about Reset because in that case the current EMA at this key would be null - // and the superclass will do the right thing. - return !Float.isNaN(value) || control.onNanValueOrDefault() != BadDataBehavior.SKIP; - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java index 805df4d30a5..ba5c5bc145d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java @@ -18,19 +18,26 @@ import static io.deephaven.util.QueryConstants.*; public class IntEMAOperator extends BasePrimitiveEMAOperator { - private final ColumnSource valueSource; protected class Context extends BasePrimitiveEMAOperator.Context { + public final ColumnSource valueSource; + public IntChunk intValueChunk; - protected Context(int chunkSize) { + protected Context(int chunkSize, ColumnSource inputSource) { super(chunkSize); + this.valueSource = inputSource; } @Override public void storeValuesChunk(@NotNull final Chunk valuesChunk) { intValueChunk = valuesChunk.asIntChunk(); } + + @Override + public boolean isValueValid(long atKey) { + return valueSource.getInt(atKey) != NULL_INT; + } } /** @@ -41,28 +48,25 @@ public void storeValuesChunk(@NotNull final Chunk valuesChunk) { * @param control defines how to handle {@code null} input values. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timeRecorder} is provided, this is measured * in ticks, otherwise it is measured in nanoseconds - * @param valueSource the input column source. Used when determining reset positions for reprocessing */ public IntEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull final ColumnSource valueSource, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits,redirContext); - this.valueSource = valueSource; // region constructor // endregion constructor } @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + return new Context(chunkSize, inputSource); } @Override @@ -107,9 +111,4 @@ public void push(UpdateContext context, long key, int pos) { } } } - - @Override - public boolean isValueValid(long atKey) { - return valueSource.getInt(atKey) != NULL_INT; - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java index 3593e0c26d0..58df703a843 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java @@ -18,19 +18,26 @@ import static io.deephaven.util.QueryConstants.*; public class LongEMAOperator extends BasePrimitiveEMAOperator { - private final ColumnSource valueSource; protected class Context extends BasePrimitiveEMAOperator.Context { + public final ColumnSource valueSource; + public LongChunk longValueChunk; - protected Context(int chunkSize) { + protected Context(int chunkSize, ColumnSource inputSource) { super(chunkSize); + this.valueSource = inputSource; } @Override public void storeValuesChunk(@NotNull final Chunk valuesChunk) { longValueChunk = valuesChunk.asLongChunk(); } + + @Override + public boolean isValueValid(long atKey) { + return valueSource.getLong(atKey) != NULL_LONG; + } } /** @@ -41,28 +48,25 @@ public void storeValuesChunk(@NotNull final Chunk valuesChunk) { * @param control defines how to handle {@code null} input values. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timeRecorder} is provided, this is measured * in ticks, otherwise it is measured in nanoseconds - * @param valueSource the input column source. Used when determining reset positions for reprocessing */ public LongEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull final ColumnSource valueSource, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits,redirContext); - this.valueSource = valueSource; // region constructor // endregion constructor } @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + return new Context(chunkSize, inputSource); } @Override @@ -107,9 +111,4 @@ public void push(UpdateContext context, long key, int pos) { } } } - - @Override - public boolean isValueValid(long atKey) { - return valueSource.getLong(atKey) != NULL_LONG; - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java index f1259e16618..30491c1e7f0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java @@ -13,19 +13,26 @@ import static io.deephaven.util.QueryConstants.*; public class ShortEMAOperator extends BasePrimitiveEMAOperator { - private final ColumnSource valueSource; protected class Context extends BasePrimitiveEMAOperator.Context { + public final ColumnSource valueSource; + public ShortChunk shortValueChunk; - protected Context(int chunkSize) { + protected Context(int chunkSize, ColumnSource inputSource) { super(chunkSize); + this.valueSource = inputSource; } @Override public void storeValuesChunk(@NotNull final Chunk valuesChunk) { shortValueChunk = valuesChunk.asShortChunk(); } + + @Override + public boolean isValueValid(long atKey) { + return valueSource.getShort(atKey) != NULL_SHORT; + } } /** @@ -36,28 +43,25 @@ public void storeValuesChunk(@NotNull final Chunk valuesChunk) { * @param control defines how to handle {@code null} input values. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timeRecorder} is provided, this is measured * in ticks, otherwise it is measured in nanoseconds - * @param valueSource the input column source. Used when determining reset positions for reprocessing */ public ShortEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull final ColumnSource valueSource, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext // region extra-constructor-args // endregion extra-constructor-args ) { super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits,redirContext); - this.valueSource = valueSource; // region constructor // endregion constructor } @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + return new Context(chunkSize, inputSource); } @Override @@ -102,9 +106,4 @@ public void push(UpdateContext context, long key, int pos) { } } } - - @Override - public boolean isValueValid(long atKey) { - return valueSource.getShort(atKey) != NULL_SHORT; - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java index e8bf4a703b5..d16226d0acc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java @@ -15,6 +15,7 @@ import io.deephaven.chunk.ByteChunk; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseByteUpdateByOperator; @@ -51,7 +52,7 @@ public BooleanFillByOperator(@NotNull final MatchPair fillPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } @@ -59,8 +60,9 @@ public UpdateContext makeUpdateContext(int chunkSize) { public void push(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; - if(ctx.curVal == NULL_BOOLEAN_AS_BYTE) { - ctx.curVal = ctx.booleanValueChunk.get(pos); + byte currentVal = ctx.booleanValueChunk.get(pos); + if(currentVal != NULL_BOOLEAN_AS_BYTE) { + ctx.curVal = currentVal; } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java index 671f8cd4268..23c10ceabf9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java @@ -8,6 +8,7 @@ import io.deephaven.chunk.ByteChunk; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseByteUpdateByOperator; @@ -44,7 +45,7 @@ public ByteFillByOperator(@NotNull final MatchPair fillPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } @@ -52,8 +53,9 @@ public UpdateContext makeUpdateContext(int chunkSize) { public void push(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; - if(ctx.curVal == NULL_BYTE) { - ctx.curVal = ctx.byteValueChunk.get(pos); + byte currentVal = ctx.byteValueChunk.get(pos); + if(currentVal != NULL_BYTE) { + ctx.curVal = currentVal; } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java index 14a8a438e2b..dba1c56b99f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java @@ -3,6 +3,7 @@ import io.deephaven.chunk.CharChunk; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseCharUpdateByOperator; @@ -39,7 +40,7 @@ public CharFillByOperator(@NotNull final MatchPair fillPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } @@ -47,8 +48,9 @@ public UpdateContext makeUpdateContext(int chunkSize) { public void push(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; - if(ctx.curVal == NULL_CHAR) { - ctx.curVal = ctx.charValueChunk.get(pos); + char currentVal = ctx.charValueChunk.get(pos); + if(currentVal != NULL_CHAR) { + ctx.curVal = currentVal; } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java index 9e9fb8ef8d8..9408fd6a670 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java @@ -8,6 +8,7 @@ import io.deephaven.chunk.DoubleChunk; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseDoubleUpdateByOperator; @@ -44,7 +45,7 @@ public DoubleFillByOperator(@NotNull final MatchPair fillPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } @@ -52,8 +53,9 @@ public UpdateContext makeUpdateContext(int chunkSize) { public void push(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; - if(ctx.curVal == NULL_DOUBLE) { - ctx.curVal = ctx.doubleValueChunk.get(pos); + double currentVal = ctx.doubleValueChunk.get(pos); + if(currentVal != NULL_DOUBLE) { + ctx.curVal = currentVal; } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java index cfd2a329a14..415f1114824 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java @@ -8,6 +8,7 @@ import io.deephaven.chunk.FloatChunk; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseFloatUpdateByOperator; @@ -44,7 +45,7 @@ public FloatFillByOperator(@NotNull final MatchPair fillPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } @@ -52,8 +53,9 @@ public UpdateContext makeUpdateContext(int chunkSize) { public void push(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; - if(ctx.curVal == NULL_FLOAT) { - ctx.curVal = ctx.floatValueChunk.get(pos); + float currentVal = ctx.floatValueChunk.get(pos); + if(currentVal != NULL_FLOAT) { + ctx.curVal = currentVal; } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java index c0775271095..2e67362f337 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java @@ -8,6 +8,7 @@ import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseIntUpdateByOperator; @@ -44,7 +45,7 @@ public IntFillByOperator(@NotNull final MatchPair fillPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } @@ -52,8 +53,9 @@ public UpdateContext makeUpdateContext(int chunkSize) { public void push(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; - if(ctx.curVal == NULL_INT) { - ctx.curVal = ctx.intValueChunk.get(pos); + int currentVal = ctx.intValueChunk.get(pos); + if(currentVal != NULL_INT) { + ctx.curVal = currentVal; } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java index 981e68c0b2d..c71af219cad 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java @@ -15,6 +15,7 @@ import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; @@ -54,7 +55,7 @@ public LongFillByOperator(@NotNull final MatchPair fillPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } @@ -62,8 +63,9 @@ public UpdateContext makeUpdateContext(int chunkSize) { public void push(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; - if(ctx.curVal == NULL_LONG) { - ctx.curVal = ctx.longValueChunk.get(pos); + long currentVal = ctx.longValueChunk.get(pos); + if(currentVal != NULL_LONG) { + ctx.curVal = currentVal; } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java index 46a3790a237..8f4561fc24e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java @@ -8,6 +8,7 @@ import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; @@ -44,7 +45,7 @@ public ObjectFillByOperator(@NotNull final MatchPair fillPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } @@ -52,8 +53,9 @@ public UpdateContext makeUpdateContext(int chunkSize) { public void push(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; - if(ctx.curVal == null) { - ctx.curVal = ctx.ObjectValueChunk.get(pos); + T currentVal = ctx.ObjectValueChunk.get(pos); + if(currentVal != null) { + ctx.curVal = currentVal; } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java index 2ef8be83796..c76c06773eb 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java @@ -8,6 +8,7 @@ import io.deephaven.chunk.ShortChunk; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseShortUpdateByOperator; @@ -44,7 +45,7 @@ public ShortFillByOperator(@NotNull final MatchPair fillPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } @@ -52,8 +53,9 @@ public UpdateContext makeUpdateContext(int chunkSize) { public void push(UpdateContext context, long key, int pos) { final Context ctx = (Context) context; - if(ctx.curVal == NULL_SHORT) { - ctx.curVal = ctx.shortValueChunk.get(pos); + short currentVal = ctx.shortValueChunk.get(pos); + if(currentVal != NULL_SHORT) { + ctx.curVal = currentVal; } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java index 745d414c7c2..1e6f4ab9144 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java @@ -15,7 +15,6 @@ import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableByteChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.sized.SizedByteChunk; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.*; @@ -158,7 +157,7 @@ public void processChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk keyChunk, @Nullable final LongChunk posChunk, @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { + @Nullable final LongChunk timestampValuesChunk) { Assert.neqNull(valuesChunk, "valuesChunk must not be null for a cumulative operator"); final Context ctx = (Context) updateContext; ctx.storeValuesChunk(valuesChunk); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java index cb2af8012c3..b8841471f82 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java @@ -5,7 +5,6 @@ import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableCharChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.sized.SizedCharChunk; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.*; @@ -130,7 +129,7 @@ public void processChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk keyChunk, @Nullable final LongChunk posChunk, @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { + @Nullable final LongChunk timestampValuesChunk) { Assert.neqNull(valuesChunk, "valuesChunk must not be null for a cumulative operator"); final Context ctx = (Context) updateContext; ctx.storeValuesChunk(valuesChunk); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java index 04944ed4469..72e10ff34ff 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java @@ -8,10 +8,8 @@ import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.WritableCharChunk; import io.deephaven.chunk.WritableDoubleChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.sized.SizedDoubleChunk; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.*; @@ -137,7 +135,7 @@ public void processChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk keyChunk, @Nullable final LongChunk posChunk, @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { + @Nullable final LongChunk timestampValuesChunk) { Assert.neqNull(valuesChunk, "valuesChunk must not be null for a cumulative operator"); final Context ctx = (Context) updateContext; ctx.storeValuesChunk(valuesChunk); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java index 7db73c690dd..00aa692f2a2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java @@ -3,10 +3,8 @@ import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.WritableCharChunk; import io.deephaven.chunk.WritableFloatChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.sized.SizedFloatChunk; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.*; @@ -132,7 +130,7 @@ public void processChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk keyChunk, @Nullable final LongChunk posChunk, @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { + @Nullable final LongChunk timestampValuesChunk) { Assert.neqNull(valuesChunk, "valuesChunk must not be null for a cumulative operator"); final Context ctx = (Context) updateContext; ctx.storeValuesChunk(valuesChunk); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java index 14f47729480..1a45190a05f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java @@ -10,7 +10,6 @@ import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.sized.SizedIntChunk; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.*; @@ -135,7 +134,7 @@ public void processChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk keyChunk, @Nullable final LongChunk posChunk, @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { + @Nullable final LongChunk timestampValuesChunk) { Assert.neqNull(valuesChunk, "valuesChunk must not be null for a cumulative operator"); final Context ctx = (Context) updateContext; ctx.storeValuesChunk(valuesChunk); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java index fc20a35e843..314d625d00a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java @@ -10,7 +10,6 @@ import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableLongChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.sized.SizedLongChunk; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.*; @@ -135,7 +134,7 @@ public void processChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk keyChunk, @Nullable final LongChunk posChunk, @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { + @Nullable final LongChunk timestampValuesChunk) { Assert.neqNull(valuesChunk, "valuesChunk must not be null for a cumulative operator"); final Context ctx = (Context) updateContext; ctx.storeValuesChunk(valuesChunk); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java index d26108c5f68..0550220fff3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java @@ -7,6 +7,7 @@ import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import org.jetbrains.annotations.NotNull; @@ -39,7 +40,7 @@ public BaseObjectBinaryOperator(@NotNull final Class type, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } @@ -50,7 +51,7 @@ public void processChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk keyChunk, @Nullable final LongChunk posChunk, @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { + @Nullable final LongChunk timestampValuesChunk) { Assert.neqNull(valuesChunk, "valuesChunk must not be null for a cumulative operator"); final Context ctx = (Context) updateContext; ctx.storeValuesChunk(valuesChunk); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java index 47c8affde87..2395d356a50 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java @@ -10,7 +10,6 @@ import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.sized.SizedObjectChunk; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.*; @@ -138,7 +137,7 @@ public void processChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk keyChunk, @Nullable final LongChunk posChunk, @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { + @Nullable final LongChunk timestampValuesChunk) { Assert.neqNull(valuesChunk, "valuesChunk must not be null for a cumulative operator"); final Context ctx = (Context) updateContext; ctx.storeValuesChunk(valuesChunk); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java index 17157fa0760..b04bbf7f127 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java @@ -10,7 +10,6 @@ import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableShortChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.sized.SizedShortChunk; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.*; @@ -135,7 +134,7 @@ public void processChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk keyChunk, @Nullable final LongChunk posChunk, @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { + @Nullable final LongChunk timestampValuesChunk) { Assert.neqNull(valuesChunk, "valuesChunk must not be null for a cumulative operator"); final Context ctx = (Context) updateContext; ctx.storeValuesChunk(valuesChunk); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java index 914dda218d6..543777f0371 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java @@ -109,7 +109,7 @@ protected WritableColumnSource makeDenseSource() { @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } @@ -139,7 +139,7 @@ public void processChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk keyChunk, @Nullable final LongChunk posChunk, @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { + @Nullable final LongChunk timestampValuesChunk) { final Context ctx = (Context) updateContext; for (int ii = 0; ii < valuesChunk.size(); ii++) { push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java index 2563e20c316..0e5ca3a9f63 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java @@ -85,7 +85,7 @@ public void reset(UpdateContext context) { @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } @@ -111,7 +111,7 @@ public void processChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk keyChunk, @Nullable final LongChunk posChunk, @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { + @Nullable final LongChunk timestampValuesChunk) { final Context ctx = (Context) updateContext; for (int ii = 0; ii < valuesChunk.size(); ii++) { push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java index 825ebffdaee..395a57f386a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java @@ -82,7 +82,7 @@ public BaseWindowedDoubleUpdateByOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } @@ -111,7 +111,7 @@ public void processChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk keyChunk, @Nullable final LongChunk posChunk, @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { + @Nullable final LongChunk timestampValuesChunk) { final Context ctx = (Context) updateContext; ctx.storeWorkingChunk(valuesChunk); for (int ii = 0; ii < valuesChunk.size(); ii++) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java index 4547ca38e14..8fb34a7685a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java @@ -77,7 +77,7 @@ public BaseWindowedFloatUpdateByOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } @@ -106,7 +106,7 @@ public void processChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk keyChunk, @Nullable final LongChunk posChunk, @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { + @Nullable final LongChunk timestampValuesChunk) { final Context ctx = (Context) updateContext; ctx.storeWorkingChunk(valuesChunk); for (int ii = 0; ii < valuesChunk.size(); ii++) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java index b8df5b2c86f..e495c11e73b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java @@ -90,7 +90,7 @@ public void reset(UpdateContext context) { @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } @@ -116,7 +116,7 @@ public void processChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk keyChunk, @Nullable final LongChunk posChunk, @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { + @Nullable final LongChunk timestampValuesChunk) { final Context ctx = (Context) updateContext; for (int ii = 0; ii < valuesChunk.size(); ii++) { push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java index 824f09ac90d..7448cfd9367 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java @@ -90,7 +90,7 @@ public void reset(UpdateContext context) { @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } @@ -116,7 +116,7 @@ public void processChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk keyChunk, @Nullable final LongChunk posChunk, @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { + @Nullable final LongChunk timestampValuesChunk) { final Context ctx = (Context) updateContext; for (int ii = 0; ii < valuesChunk.size(); ii++) { push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java index 84398f8a537..86ed3e4b897 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java @@ -93,7 +93,7 @@ public void reset(UpdateContext context) { @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } @@ -119,7 +119,7 @@ public void processChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk keyChunk, @Nullable final LongChunk posChunk, @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { + @Nullable final LongChunk timestampValuesChunk) { final Context ctx = (Context) updateContext; for (int ii = 0; ii < valuesChunk.size(); ii++) { push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java index 98e4a42e599..ef09d222af5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java @@ -90,7 +90,7 @@ public void reset(UpdateContext context) { @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } @@ -116,7 +116,7 @@ public void processChunk(@NotNull final UpdateContext updateContext, @Nullable final LongChunk keyChunk, @Nullable final LongChunk posChunk, @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { + @Nullable final LongChunk timestampValuesChunk) { final Context ctx = (Context) updateContext; for (int ii = 0; ii < valuesChunk.size(); ii++) { push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java index 070dfacb145..0b49e5b73d5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java @@ -8,9 +8,9 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ByteChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseByteUpdateByOperator; import org.jetbrains.annotations.NotNull; @@ -49,7 +49,7 @@ public ByteCumMinMaxOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java index edcab22e88f..7f449d25770 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java @@ -8,13 +8,13 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.DoubleChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseDoubleUpdateByOperator; import org.jetbrains.annotations.NotNull; import static io.deephaven.util.QueryConstants.NULL_DOUBLE; -import static io.deephaven.util.QueryConstants.NULL_LONG; public class DoubleCumMinMaxOperator extends BaseDoubleUpdateByOperator { private final boolean isMax; @@ -49,7 +49,7 @@ public DoubleCumMinMaxOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java index 2b4aa2fb4fe..caf9c09db11 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java @@ -3,13 +3,13 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.FloatChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseFloatUpdateByOperator; import org.jetbrains.annotations.NotNull; import static io.deephaven.util.QueryConstants.NULL_FLOAT; -import static io.deephaven.util.QueryConstants.NULL_LONG; public class FloatCumMinMaxOperator extends BaseFloatUpdateByOperator { private final boolean isMax; @@ -44,7 +44,7 @@ public FloatCumMinMaxOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java index 442910db0ef..b86a9cc5bca 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java @@ -8,9 +8,9 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseIntUpdateByOperator; import org.jetbrains.annotations.NotNull; @@ -49,7 +49,7 @@ public IntCumMinMaxOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java index 660d0f20113..5582cbe3c26 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java @@ -15,10 +15,10 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; -import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; import org.jetbrains.annotations.NotNull; import static io.deephaven.util.QueryConstants.*; @@ -59,7 +59,7 @@ public LongCumMinMaxOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java index 17613b46cc9..a17c8a00c53 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java @@ -3,9 +3,9 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ShortChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseShortUpdateByOperator; import org.jetbrains.annotations.NotNull; @@ -44,7 +44,7 @@ public ShortCumMinMaxOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java index a13e2dc5cdf..6518fde111f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java @@ -3,6 +3,7 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; @@ -37,7 +38,7 @@ public BigDecimalCumProdOperator(@NotNull final MatchPair inputPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java index 105f018a49f..164a1478bd7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java @@ -3,6 +3,7 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; @@ -31,7 +32,7 @@ public BigIntegerCumProdOperator(@NotNull final MatchPair inputPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java index 122a99dc946..74e5abbfe29 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java @@ -8,6 +8,7 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ByteChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; @@ -45,7 +46,7 @@ public ByteCumProdOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java index 81dea1f128b..f08058a7281 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java @@ -8,13 +8,13 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.DoubleChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseDoubleUpdateByOperator; import org.jetbrains.annotations.NotNull; import static io.deephaven.util.QueryConstants.NULL_DOUBLE; -import static io.deephaven.util.QueryConstants.NULL_LONG; public class DoubleCumProdOperator extends BaseDoubleUpdateByOperator { // region extra-fields @@ -45,7 +45,7 @@ public DoubleCumProdOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java index 72d288e0892..049d1181b25 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java @@ -3,13 +3,13 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.FloatChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseFloatUpdateByOperator; import org.jetbrains.annotations.NotNull; import static io.deephaven.util.QueryConstants.NULL_FLOAT; -import static io.deephaven.util.QueryConstants.NULL_LONG; public class FloatCumProdOperator extends BaseFloatUpdateByOperator { // region extra-fields @@ -40,7 +40,7 @@ public FloatCumProdOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java index d13ec4d7b77..2834a8e14f2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java @@ -8,6 +8,7 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; @@ -45,7 +46,7 @@ public IntCumProdOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java index a9550f56e7a..2c16628292a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java @@ -8,6 +8,7 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; @@ -45,7 +46,7 @@ public LongCumProdOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java index 303c8a70d32..7858b8f1f5b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java @@ -3,6 +3,7 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ShortChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; @@ -40,7 +41,7 @@ public ShortCumProdOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java index 2ef1c33879a..aa8b3a12321 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java @@ -3,26 +3,15 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.sized.SizedObjectChunk; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByOperator; -import io.deephaven.engine.table.impl.sources.ObjectArraySource; -import io.deephaven.engine.table.impl.sources.ObjectSparseArraySource; -import io.deephaven.engine.table.impl.sources.WritableRedirectedColumnSource; -import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedObjectUpdateByOperator; -import io.deephaven.engine.table.impl.util.SizedSafeCloseable; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import java.math.BigDecimal; import java.math.MathContext; -import java.util.Collections; -import java.util.Map; public final class BigDecimalRollingSumOperator extends BaseWindowedObjectUpdateByOperator { @NotNull @@ -43,7 +32,7 @@ public void storeInfluencerValuesChunk(@NotNull final Chunk influencerVa @NotNull @Override - public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize) { + public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java index 18cb21a72c5..8f9d69ecc4d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java @@ -3,25 +3,14 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.sized.SizedObjectChunk; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByOperator; -import io.deephaven.engine.table.impl.sources.ObjectArraySource; -import io.deephaven.engine.table.impl.sources.ObjectSparseArraySource; -import io.deephaven.engine.table.impl.sources.WritableRedirectedColumnSource; -import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedObjectUpdateByOperator; -import io.deephaven.engine.table.impl.util.SizedSafeCloseable; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import java.math.BigInteger; -import java.util.Collections; -import java.util.Map; public final class BigIntegerRollingSumOperator extends BaseWindowedObjectUpdateByOperator { protected class Context extends BaseWindowedObjectUpdateByOperator.Context { @@ -39,7 +28,7 @@ public void storeInfluencerValuesChunk(@NotNull final Chunk influencerVa @NotNull @Override - public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize) { + public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java index ddc430d1a1f..0fb3ac0a71d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java @@ -9,6 +9,7 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ByteChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; @@ -16,9 +17,6 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.util.Collections; -import java.util.Map; - import static io.deephaven.util.QueryConstants.NULL_LONG; import static io.deephaven.util.QueryConstants.NULL_BYTE; @@ -42,7 +40,7 @@ public void storeInfluencerValuesChunk(@NotNull final Chunk influencerVa @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize) { + public UpdateContext makeUpdateContext(final int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java index 3e9b40836e8..c9587ac2e8a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java @@ -8,19 +8,10 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.sized.SizedDoubleChunk; -import io.deephaven.chunk.sized.SizedLongChunk; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedDoubleUpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.PairwiseDoubleRingBuffer; -import io.deephaven.engine.table.impl.util.SizedSafeCloseable; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -52,7 +43,7 @@ public void close() { @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize) { + public UpdateContext makeUpdateContext(final int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java index bd3684e3a9f..a870fd76784 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java @@ -3,19 +3,10 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.sized.SizedFloatChunk; -import io.deephaven.chunk.sized.SizedLongChunk; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedFloatUpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.PairwiseFloatRingBuffer; -import io.deephaven.engine.table.impl.util.SizedSafeCloseable; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -47,7 +38,7 @@ public void close() { @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize) { + public UpdateContext makeUpdateContext(final int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java index 3604688e0df..08c8a0b51b3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java @@ -9,6 +9,7 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; @@ -16,9 +17,6 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.util.Collections; -import java.util.Map; - import static io.deephaven.util.QueryConstants.NULL_LONG; import static io.deephaven.util.QueryConstants.NULL_INT; @@ -41,7 +39,7 @@ public void storeInfluencerValuesChunk(@NotNull final Chunk influencerVa @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize) { + public UpdateContext makeUpdateContext(final int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java index 0b199f86802..d5bef5a9a33 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java @@ -9,6 +9,7 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; @@ -16,9 +17,6 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.util.Collections; -import java.util.Map; - import static io.deephaven.util.QueryConstants.NULL_LONG; import static io.deephaven.util.QueryConstants.NULL_LONG; @@ -41,7 +39,7 @@ public void storeInfluencerValuesChunk(@NotNull final Chunk influencerVa @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize) { + public UpdateContext makeUpdateContext(final int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java index 6c4ee6e4247..fbb80611ea6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java @@ -4,6 +4,7 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ShortChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; @@ -11,9 +12,6 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.util.Collections; -import java.util.Map; - import static io.deephaven.util.QueryConstants.NULL_LONG; import static io.deephaven.util.QueryConstants.NULL_SHORT; @@ -36,7 +34,7 @@ public void storeInfluencerValuesChunk(@NotNull final Chunk influencerVa @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize) { + public UpdateContext makeUpdateContext(final int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java index 4005b825f48..baa42c3d826 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java @@ -3,6 +3,7 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; @@ -37,7 +38,7 @@ public BigDecimalCumSumOperator(@NotNull final MatchPair inputPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java index fb93f583ae0..cb795e0fe9a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java @@ -3,6 +3,7 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; @@ -31,7 +32,7 @@ public BigIntegerCumSumOperator(@NotNull final MatchPair inputPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java index 26f6b8b4702..8820dc500f3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java @@ -7,6 +7,7 @@ import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; @@ -46,7 +47,7 @@ public ByteCumSumOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java index cfec36050e4..9bc75559a19 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java @@ -8,6 +8,7 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.DoubleChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseDoubleUpdateByOperator; @@ -44,7 +45,7 @@ public DoubleCumSumOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java index 9620aa9b876..80ed891ea77 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java @@ -3,6 +3,7 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.FloatChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseFloatUpdateByOperator; @@ -39,7 +40,7 @@ public FloatCumSumOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java index 7fdd4616148..8ac45493840 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java @@ -7,6 +7,7 @@ import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; @@ -43,7 +44,7 @@ public IntCumSumOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java index 931a2228d8b..8faf7124fcd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java @@ -7,6 +7,7 @@ import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; @@ -43,7 +44,7 @@ public LongCumSumOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java index 0eed0f88d08..fa8b9a24715 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java @@ -2,6 +2,7 @@ import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; @@ -38,7 +39,7 @@ public ShortCumSumOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } diff --git a/replication/static/src/main/java/io/deephaven/replicators/ReplicateRingBuffers.java b/replication/static/src/main/java/io/deephaven/replicators/ReplicateRingBuffers.java index bc81497cbf8..6d2ff4ac0d5 100644 --- a/replication/static/src/main/java/io/deephaven/replicators/ReplicateRingBuffers.java +++ b/replication/static/src/main/java/io/deephaven/replicators/ReplicateRingBuffers.java @@ -35,4 +35,5 @@ private static void replaceLines(String fileResult, String... replacements) thro List lines = FileUtils.readLines(objectFile, Charset.defaultCharset()); lines = ReplicationUtils.globalReplacements(lines, replacements); FileUtils.writeLines(objectFile, lines); - }} + } +} From 02d6b27410c40d34e60171fc5be94996c7f45dec Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 28 Sep 2022 13:24:09 -0700 Subject: [PATCH 027/123] Massive refactor complete, tests passing, performance TBD --- .../impl/UpdateByCumulativeOperator.java | 15 +- .../engine/table/impl/UpdateByOperator.java | 88 ++- .../table/impl/UpdateByWindowedOperator.java | 149 +---- .../engine/table/impl/ZeroKeyUpdateBy.java | 23 - .../table/impl/updateby/UpdateByWindow.java | 594 ++---------------- .../updateby/UpdateByWindowCumulative.java | 292 +++++++++ .../impl/updateby/UpdateByWindowTicks.java | 370 +++++++++++ .../impl/updateby/UpdateByWindowTime.java | 361 +++++++++++ .../ema/BasePrimitiveEMAOperator.java | 37 +- .../updateby/ema/BigDecimalEMAOperator.java | 105 ++-- .../updateby/ema/BigIntegerEMAOperator.java | 104 +-- .../updateby/ema/BigNumberEMAOperator.java | 51 +- .../impl/updateby/ema/ByteEMAOperator.java | 86 ++- .../impl/updateby/ema/DoubleEMAOperator.java | 105 ++-- .../impl/updateby/ema/FloatEMAOperator.java | 105 ++-- .../impl/updateby/ema/IntEMAOperator.java | 86 ++- .../impl/updateby/ema/LongEMAOperator.java | 86 ++- .../impl/updateby/ema/ShortEMAOperator.java | 86 ++- .../updateby/fill/BooleanFillByOperator.java | 25 +- .../updateby/fill/ByteFillByOperator.java | 25 +- .../updateby/fill/CharFillByOperator.java | 25 +- .../updateby/fill/DoubleFillByOperator.java | 25 +- .../updateby/fill/FloatFillByOperator.java | 25 +- .../impl/updateby/fill/IntFillByOperator.java | 25 +- .../updateby/fill/LongFillByOperator.java | 25 +- .../updateby/fill/ObjectFillByOperator.java | 25 +- .../updateby/fill/ShortFillByOperator.java | 25 +- .../internal/BaseByteUpdateByOperator.java | 63 +- .../internal/BaseCharUpdateByOperator.java | 63 +- .../internal/BaseDoubleUpdateByOperator.java | 55 +- .../internal/BaseFloatUpdateByOperator.java | 55 +- .../internal/BaseIntUpdateByOperator.java | 63 +- .../internal/BaseLongUpdateByOperator.java | 63 +- .../internal/BaseObjectBinaryOperator.java | 50 +- .../internal/BaseObjectUpdateByOperator.java | 63 +- .../internal/BaseShortUpdateByOperator.java | 63 +- .../BaseWindowedByteUpdateByOperator.java | 67 +- .../BaseWindowedCharUpdateByOperator.java | 67 +- .../BaseWindowedDoubleUpdateByOperator.java | 71 +-- .../BaseWindowedFloatUpdateByOperator.java | 71 +-- .../BaseWindowedIntUpdateByOperator.java | 67 +- .../BaseWindowedLongUpdateByOperator.java | 67 +- .../BaseWindowedObjectUpdateByOperator.java | 67 +- .../BaseWindowedShortUpdateByOperator.java | 67 +- .../internal/PairwiseDoubleRingBuffer.java | 2 +- .../internal/PairwiseFloatRingBuffer.java | 2 +- .../minmax/ByteCumMinMaxOperator.java | 41 +- .../minmax/DoubleCumMinMaxOperator.java | 41 +- .../minmax/FloatCumMinMaxOperator.java | 41 +- .../updateby/minmax/IntCumMinMaxOperator.java | 41 +- .../minmax/LongCumMinMaxOperator.java | 52 +- .../minmax/ShortCumMinMaxOperator.java | 41 +- .../prod/BigDecimalCumProdOperator.java | 41 +- .../prod/BigIntegerCumProdOperator.java | 41 +- .../updateby/prod/ByteCumProdOperator.java | 36 +- .../updateby/prod/DoubleCumProdOperator.java | 32 +- .../updateby/prod/FloatCumProdOperator.java | 32 +- .../updateby/prod/IntCumProdOperator.java | 36 +- .../updateby/prod/LongCumProdOperator.java | 36 +- .../updateby/prod/ShortCumProdOperator.java | 36 +- .../BigDecimalRollingSumOperator.java | 85 ++- .../BigIntegerRollingSumOperator.java | 91 +-- .../rollingsum/ByteRollingSumOperator.java | 88 ++- .../rollingsum/DoubleRollingSumOperator.java | 92 +-- .../rollingsum/FloatRollingSumOperator.java | 92 +-- .../rollingsum/IntRollingSumOperator.java | 88 ++- .../rollingsum/LongRollingSumOperator.java | 88 ++- .../rollingsum/ShortRollingSumOperator.java | 88 ++- .../sum/BigDecimalCumSumOperator.java | 42 +- .../sum/BigIntegerCumSumOperator.java | 41 +- .../impl/updateby/sum/ByteCumSumOperator.java | 33 +- .../updateby/sum/DoubleCumSumOperator.java | 33 +- .../updateby/sum/FloatCumSumOperator.java | 33 +- .../impl/updateby/sum/IntCumSumOperator.java | 33 +- .../impl/updateby/sum/LongCumSumOperator.java | 33 +- .../updateby/sum/ShortCumSumOperator.java | 33 +- .../replicators/ReplicateUpdateBy.java | 1 + 77 files changed, 3002 insertions(+), 2593 deletions(-) create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java index 64a65bebe8c..55078719cce 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java @@ -6,6 +6,7 @@ import io.deephaven.chunk.sized.SizedLongChunk; import io.deephaven.engine.table.ChunkSink; import io.deephaven.engine.table.MatchPair; +import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -22,7 +23,7 @@ public abstract class UpdateByCumulativeOperator implements UpdateByOperator { protected long timeScaleUnits; protected String timestampColumnName; - public class Context implements UpdateContext { + public abstract class Context implements UpdateContext { public long curTimestamp; protected Context(final int chunkSize) { @@ -31,11 +32,16 @@ protected Context(final int chunkSize) { public boolean isValueValid(long atKey) { throw new UnsupportedOperationException( - "isValueValid() must be overriden by time-aware cumulative operators"); + "isValueValid() must be overridden by time-aware cumulative operators"); } @Override public void close() {} + + @FinalDefault + public void pop() { + throw new UnsupportedOperationException("Cumulative operators should never call pop()"); + } } public UpdateByCumulativeOperator(@NotNull final MatchPair pair, @@ -88,9 +94,4 @@ public String[] getAffectingColumnNames() { public String[] getOutputColumnNames() { return new String[] {pair.leftColumn}; } - - @Override - public void pop(UpdateContext context) { - throw new UnsupportedOperationException("Cumulative operators should never call pop()"); - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java index f5cd1ec3914..96ed27f22ed 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java @@ -1,6 +1,5 @@ package io.deephaven.engine.table.impl; -import io.deephaven.api.updateby.UpdateByControl; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.LongChunk; @@ -8,7 +7,6 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.impl.updateby.UpdateByWindow; @@ -16,7 +14,6 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.util.Collection; import java.util.Map; /** @@ -44,38 +41,46 @@ public interface UpdateByOperator { UpdateByOperator[] ZERO_LENGTH_OP_ARRAY = new UpdateByOperator[0]; /** - * A context item for use with {@link Table#updateBy(UpdateByControl, Collection, String...)} for non-bucketed - * updates. + * A context item for use with updateBy operators */ interface UpdateContext extends SafeCloseable { - } - - /** - * Add a value to the operators current data set - * - * @param context the operator context for this action - * @param key the row key associated with the value - * @param pos the index in the associated chunk where this value can be found. Depending on the usage, might be a - * values chunk (for cumulative operators) or an influencer values chunk (for windowed). It is the task of - * the operator to pull the data from the chunk and use it properly - */ - void push(UpdateContext context, long key, int pos); - - /** - * Remove a value from the operators current data set. This is only valid for windowed operators since cumulative - * operators only append values - * - * @param context the operator context for this action - */ - void pop(UpdateContext context); - /** - * Reset the operator data values to a known state. This may occur during initialization or when a windowed operator - * has an empty window - * - * @param context the operator context for this action - */ - void reset(UpdateContext context); + void setValuesChunk(@NotNull final Chunk valuesChunk); + + void setTimestampChunk(@NotNull final LongChunk valuesChunk); + + /** + * Add a value to the operators current data set + * + * @param key the row key associated with the value + * @param pos the index in the associated chunk where this value can be found. Depending on the usage, might be + * a values chunk (for cumulative operators) or an influencer values chunk (for windowed). It is the task + * of the operator to pull the data from the chunk and use it properly + */ + void push(long key, int pos); + + /** + * Remove a value from the operators current data set. This is only valid for windowed operators since + * cumulative operators only append values + */ + void pop(); + + /** + * TODO: update doc + */ + void writeToOutputChunk(int outIdx); + + /** + * Reset the operator data values to a known state. This may occur during initialization or when a windowed + * operator has an empty window + */ + void reset(); + + /** + * TODO: update doc + */ + void writeToOutputColumn(@NotNull final RowSequence inputKeys); + } /** * Get the name of the input column this operator depends on. @@ -140,7 +145,7 @@ interface UpdateContext extends SafeCloseable { * Make an {@link UpdateContext} suitable for use with non-bucketed updates. * * @param chunkSize The expected size of chunks that will be provided during the update, - * @param inputSource + * @param inputSource The column source that contains the input values needed by the operator * @return a new context */ @NotNull @@ -159,21 +164,4 @@ interface UpdateContext extends SafeCloseable { * */ void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta); - - /** - * Process a chunk of data for an updateBy table. - * - * @param context the context object - * @param inputKeys the keys contained in the chunk - * @param keyChunk a {@link LongChunk} containing the keys if requested - * @param posChunk a {@link LongChunk} containing the positions if requested - * @param valuesChunk the current chunk of working values. - * @param timestampValuesChunk a {@link LongChunk} containing the working timestamps if requested - */ - void processChunk(@NotNull final UpdateContext context, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java index 99d476900a2..591787bbddd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java @@ -1,28 +1,10 @@ package io.deephaven.engine.table.impl; import io.deephaven.api.updateby.OperationControl; -import io.deephaven.base.ringbuffer.IntRingBuffer; -import io.deephaven.base.ringbuffer.LongRingBuffer; -import io.deephaven.base.verify.Assert; -import io.deephaven.chunk.Chunk; -import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.WritableChunk; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.sized.SizedLongChunk; -import io.deephaven.engine.rowset.*; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; -import io.deephaven.engine.table.ChunkSource; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.TableUpdate; -import io.deephaven.engine.table.impl.sources.ReinterpretUtils; -import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; -import org.apache.commons.lang3.mutable.MutableLong; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.NULL_LONG; - public abstract class UpdateByWindowedOperator implements UpdateByOperator { protected final MatchPair pair; protected final String[] affectingColumns; @@ -35,15 +17,12 @@ public abstract class UpdateByWindowedOperator implements UpdateByOperator { protected final long reverseTimeScaleUnits; protected final long forwardTimeScaleUnits; - public abstract class UpdateWindowedContext implements UpdateContext { - public void storeInfluencerValuesChunk(@NotNull final Chunk influencerValuesChunk) {} - + public abstract class Context implements UpdateContext { public int nullCount = 0; // // protected long currentInfluencerKey; // // candidate data for the window - public final int WINDOW_CHUNK_SIZE = 4096; // // // allocate some chunks for holding the key, position and timestamp data // protected SizedLongChunk influencerKeyChunk; @@ -57,105 +36,9 @@ public void storeInfluencerValuesChunk(@NotNull final Chunk influencerVa // protected long currentInfluencerPosOrTimestamp; // protected int currentInfluencerIndex; - protected LongRingBuffer windowKeys = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); - protected LongRingBuffer windowPosOrTimestamp = new LongRingBuffer(WINDOW_CHUNK_SIZE, true); - public IntRingBuffer windowIndices = new IntRingBuffer(WINDOW_CHUNK_SIZE, true); - // public abstract void loadInfluencerValueChunk(); - // public void fillWindowTicks(UpdateWindowedContext context, long currentPos) { - // // compute the head and tail positions (inclusive) - // final long head = Math.max(0, currentPos - reverseTimeScaleUnits + 1); - // final long tail = Math.min(sourceRowSet.size() - 1, currentPos + forwardTimeScaleUnits); - // - // // pop out all values from the current window that are not in the new window - // while (!windowPosOrTimestamp.isEmpty() && windowPosOrTimestamp.front() < head) { - // pop(context, windowKeys.remove(), (int) windowIndices.remove()); - // windowPosOrTimestamp.remove(); - // } - // - // // if the window is empty or completly filled with null, call reset() - // if (windowPosOrTimestamp.isEmpty() || context.nullCount == windowPosOrTimestamp.size()) { - // reset(context); - // } - // - // // skip values until they match the window - // while (currentInfluencerPosOrTimestamp < head) { - // currentInfluencerIndex++; - // - // if (currentInfluencerIndex < influencerPosChunk.get().size()) { - // currentInfluencerPosOrTimestamp = influencerPosChunk.get().get(currentInfluencerIndex); - // currentInfluencerKey = influencerKeyChunk.get().get(currentInfluencerIndex); - // } else { - // currentInfluencerPosOrTimestamp = Long.MAX_VALUE; - // currentInfluencerKey = Long.MAX_VALUE; - // } - // } - // - // // push matching values - // while (currentInfluencerPosOrTimestamp <= tail) { - // push(context, currentInfluencerKey, currentInfluencerIndex); - // windowKeys.add(currentInfluencerKey); - // windowPosOrTimestamp.add(currentInfluencerPosOrTimestamp); - // windowIndices.add(currentInfluencerIndex); - // currentInfluencerIndex++; - // - // if (currentInfluencerIndex < influencerPosChunk.get().size()) { - // currentInfluencerPosOrTimestamp = influencerPosChunk.get().get(currentInfluencerIndex); - // currentInfluencerKey = influencerKeyChunk.get().get(currentInfluencerIndex); - // } else { - // currentInfluencerPosOrTimestamp = Long.MAX_VALUE; - // currentInfluencerKey = Long.MAX_VALUE; - // } - // } - // } - // - // public void fillWindowTime(UpdateWindowedContext context, long currentTimestamp) { - // // compute the head and tail positions (inclusive) - // final long head = currentTimestamp - reverseTimeScaleUnits; - // final long tail = currentTimestamp + forwardTimeScaleUnits; - // - // // pop out all values from the current window that are not in the new window - // while (!windowPosOrTimestamp.isEmpty() && windowPosOrTimestamp.front() < head) { - // pop(context, windowKeys.remove(), (int) windowIndices.remove()); - // windowPosOrTimestamp.remove(); - // } - // - // // if the window is empty or completly filled with null, call reset() - // if (windowPosOrTimestamp.isEmpty() || context.nullCount == windowPosOrTimestamp.size()) { - // reset(context); - // } - // - // // skip values until they match the window - // while (currentInfluencerPosOrTimestamp < head) { - // currentInfluencerIndex++; - // - // if (currentInfluencerIndex < influencerTimestampChunk.get().size()) { - // currentInfluencerPosOrTimestamp = influencerTimestampChunk.get().get(currentInfluencerIndex); - // currentInfluencerKey = influencerKeyChunk.get().get(currentInfluencerIndex); - // } else { - // currentInfluencerPosOrTimestamp = Long.MAX_VALUE; - // currentInfluencerKey = Long.MAX_VALUE; - // } - // } - // - // // push matching values - // while (currentInfluencerPosOrTimestamp <= tail) { - // push(context, currentInfluencerKey, currentInfluencerIndex); - // windowKeys.add(currentInfluencerKey); - // windowPosOrTimestamp.add(currentInfluencerPosOrTimestamp); - // windowIndices.add(currentInfluencerIndex); - // currentInfluencerIndex++; - // - // if (currentInfluencerIndex < influencerTimestampChunk.get().size()) { - // currentInfluencerPosOrTimestamp = influencerTimestampChunk.get().get(currentInfluencerIndex); - // currentInfluencerKey = influencerKeyChunk.get().get(currentInfluencerIndex); - // } else { - // currentInfluencerPosOrTimestamp = Long.MAX_VALUE; - // currentInfluencerKey = Long.MAX_VALUE; - // } - // } - // } + @Override public void close() { @@ -194,33 +77,7 @@ public UpdateByWindowedOperator(@NotNull final MatchPair pair, this.redirContext = redirContext; } - public void initializeUpdate(@NotNull final UpdateContext context) { - final UpdateWindowedContext ctx = (UpdateWindowedContext) context; - // // load all the influencer values this update will need - // ctx.loadInfluencerValueChunk(); - // - // // load all the influencer keys - // ctx.influencerKeyChunk = new SizedLongChunk<>(ctx.influencerRows.intSize()); - // ctx.influencerRows.fillRowKeyChunk(ctx.influencerKeyChunk.get()); - // ctx.currentInfluencerKey = ctx.influencerRows.firstRowKey(); - // - // if (timestampColumnName == null) { - // // load all the influencer positions - // ctx.influencerPosChunk = new SizedLongChunk<>(ctx.influencerRows.intSize()); - // ctx.influencerPositions.fillRowKeyChunk(ctx.influencerPosChunk.get()); - // ctx.currentInfluencerPosOrTimestamp = ctx.influencerPositions.firstRowKey(); - // } else { - // // load all the influencer timestamp data - // ctx.influencerTimestampChunk = new SizedLongChunk<>(ctx.influencerRows.intSize()); - // try (final ChunkSource.FillContext fillContext = - // timestampColumnSource.makeFillContext(ctx.influencerRows.intSize())) { - // timestampColumnSource.fillChunk(fillContext, - // (WritableChunk) ctx.influencerTimestampChunk.get(), ctx.influencerRows); - // } - // ctx.currentInfluencerPosOrTimestamp = ctx.influencerTimestampChunk.get().get(0); - // } - // ctx.currentInfluencerIndex = 0; - } + public abstract void initializeUpdate(@NotNull final UpdateContext context); @Override public void finishUpdate(@NotNull final UpdateContext context) {} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java index bfcb2bae668..6fb17421b55 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java @@ -118,8 +118,6 @@ private void processUpdateForSsa(TableUpdate upstream) { final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(size); final WritableLongChunk ssaValues = WritableLongChunk.makeWritableChunk(size); final WritableLongChunk ssaKeys = WritableLongChunk.makeWritableChunk(size)) { - TLongHashSet ssaKeySet = new TLongHashSet(); - timestampSsa.forAllKeys(ssaKeySet::add); MutableLong lastTimestamp = new MutableLong(NULL_LONG); while (it.hasMore()) { @@ -132,15 +130,6 @@ private void processUpdateForSsa(TableUpdate upstream) { // push only non-null values/keys into the Ssa fillChunkWithNonNull(keysChunk, valuesChunk, ssaKeys, ssaValues, lastTimestamp); - - // verify that all values to remove are actually in this SSA - for (int ii = 0; ii < valuesChunk.size(); ii++) { - final long ts = valuesChunk.get(ii); - final long key = keysChunk.get(ii); - if (!ssaKeySet.contains(key)) { - System.out.println(ts + " : " + key); - } - } timestampSsa.remove(ssaValues, ssaKeys); } } @@ -180,9 +169,6 @@ private void processUpdateForSsa(TableUpdate upstream) { final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(size); final WritableLongChunk ssaValues = WritableLongChunk.makeWritableChunk(size); final WritableLongChunk ssaKeys = WritableLongChunk.makeWritableChunk(size)) { - TLongHashSet ssaKeySet = new TLongHashSet(); - timestampSsa.forAllKeys(ssaKeySet::add); - MutableLong lastTimestamp = new MutableLong(NULL_LONG); while (it.hasMore()) { RowSequence chunkRs = it.getNextRowSequenceWithLength(4096); @@ -194,15 +180,6 @@ private void processUpdateForSsa(TableUpdate upstream) { // push only non-null values/keys into the Ssa fillChunkWithNonNull(keysChunk, valuesChunk, ssaKeys, ssaValues, lastTimestamp); - - // verify that the items are not already in the SSA - for (int ii = 0; ii < valuesChunk.size(); ii++) { - final long ts = valuesChunk.get(ii); - final long key = keysChunk.get(ii); - if (ssaKeySet.contains(key)) { - System.out.println(ts + " : " + key); - } - } timestampSsa.insert(ssaValues, ssaKeys); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java index 0007fa54c62..6dc29babca3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java @@ -1,10 +1,12 @@ package io.deephaven.engine.table.impl.updateby; +import io.deephaven.base.ringbuffer.LongRingBuffer; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.*; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.ModifiedColumnSet; @@ -21,14 +23,12 @@ import java.util.*; +import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; import static io.deephaven.util.QueryConstants.NULL_LONG; -public class UpdateByWindow { - protected final boolean windowed; +public abstract class UpdateByWindow { @Nullable protected final String timestampColumnName; - protected final long prevUnits; - protected final long fwdUnits; // store the operators for this window protected final UpdateByOperator[] operators; @@ -40,51 +40,47 @@ public class UpdateByWindow { protected boolean trackModifications; - public class UpdateByWindowContext implements SafeCloseable { + public abstract class UpdateByWindowContext implements SafeCloseable { /** store a reference to the source rowset */ - final TrackingRowSet sourceRowSet; + protected final TrackingRowSet sourceRowSet; /** the column source providing the timestamp data for this window */ @Nullable - final ColumnSource timestampColumnSource; + protected final ColumnSource timestampColumnSource; /** the timestamp SSA providing fast lookup for time windows */ @Nullable - final LongSegmentedSortedArray timestampSsa; + protected final LongSegmentedSortedArray timestampSsa; /** An array of boolean denoting which operators are affected by the current update. */ - final boolean[] opAffected; + protected final boolean[] opAffected; /** An array of context objects for each underlying operator */ - final UpdateByOperator.UpdateContext[] opContext; + protected final UpdateByOperator.UpdateContext[] opContext; /** An array of ColumnSources for each underlying operator */ - final ColumnSource[] inputSources; + protected final ColumnSource[] inputSources; /** An array of {@link ChunkSource.FillContext}s for each input column */ - final ChunkSource.FillContext[] inputSourceFillContexts; + protected final ChunkSource.FillContext[] inputSourceFillContexts; /** A set of chunks used to store working values */ - final WritableChunk[] inputSourceChunks; + protected final WritableChunk[] inputSourceChunks; /** An indicator of if each slot has been populated with data or not for this phase. */ - final boolean[] inputSourceChunkPopulated; + protected final boolean[] inputSourceChunkPopulated; /** the rows affected by this update */ - RowSet affectedRows; + protected RowSet affectedRows; /** the rows that contain values used to compute affected row values */ - RowSet influencerRows; - - /** for use with a ticking window */ - RowSet affectedRowPositions; - RowSet influencerPositions; + protected RowSet influencerRows; /** keep track of what rows were modified (we'll use a single set for all operators sharing a window) */ - RowSetBuilderSequential modifiedBuilder; - RowSet newModified; + protected RowSetBuilderSequential modifiedBuilder; + protected RowSet newModified; - final int chunkSize; - final boolean initialStep; + protected final int chunkSize; + protected final boolean initialStep; public UpdateByWindowContext(final TrackingRowSet sourceRowSet, final ColumnSource[] inputSources, @Nullable final ColumnSource timestampColumnSource, @@ -105,121 +101,11 @@ public UpdateByWindowContext(final TrackingRowSet sourceRowSet, final ColumnSour this.initialStep = initialStep; } - public boolean computeAffectedRowsAndOperators(@NotNull final TableUpdate upstream) { - // all rows are affected on the initial step - if (initialStep) { - affectedRows = sourceRowSet.copy(); - influencerRows = affectedRows; - - // no need to invert, just create a flat rowset - if (windowed && timestampColumnName == null) { - affectedRowPositions = RowSetFactory.flat(sourceRowSet.size()); - influencerPositions = RowSetFactory.flat(sourceRowSet.size()); - } - // mark all operators as affected by this update - Arrays.fill(opAffected, true); - makeOperatorContexts(); - return true; - } - - // determine which operators are affected by this update - boolean anyAffected = false; - boolean allAffected = upstream.added().isNonempty() || - upstream.removed().isNonempty() || - upstream.shifted().nonempty(); - - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - opAffected[opIdx] = allAffected - || (upstream.modifiedColumnSet().nonempty() && (operatorInputModifiedColumnSets[opIdx] == null - || upstream.modifiedColumnSet().containsAny(operatorInputModifiedColumnSets[opIdx]))); - if (opAffected[opIdx]) { - anyAffected = true; - } - } - - if (!anyAffected) { - return false; - } - - if (!windowed) { - computeCumulativeRowsAffected(upstream); - } else { - computeWindowedRowsAffected(upstream); - } - - makeOperatorContexts(); - return true; - } - - // cumulative windows is simple, just find the smallest key and return the range from smallest to end - private void computeCumulativeRowsAffected(@NotNull TableUpdate upstream) { - long smallestModifiedKey = smallestAffectedKey(upstream.added(), upstream.modified(), upstream.removed(), - upstream.shifted(), sourceRowSet); - - affectedRows = smallestModifiedKey == Long.MAX_VALUE - ? RowSetFactory.empty() - : sourceRowSet.subSetByKeyRange(smallestModifiedKey, sourceRowSet.lastRowKey()); - influencerRows = affectedRows; - } - - // windowed by time/ticks is more complex to compute: find all the changed rows and the rows that would - // be affected by the changes (includes newly added rows) and need to be recomputed. Then include all - // the rows that are affected by deletions (if any). After the affected rows have been identified, - // determine which rows will be needed to compute new values for the affected rows (influencer rows) - private void computeWindowedRowsAffected(@NotNull TableUpdate upstream) { - // changed rows are all mods+adds - WritableRowSet changed = upstream.added().copy(); - changed.insert(upstream.modified()); - - // need a writable rowset - WritableRowSet tmpAffected; - - // compute the rows affected from these changes - if (timestampColumnName == null) { - try (final WritableRowSet changedInverted = sourceRowSet.invert(changed)) { - tmpAffected = computeAffectedRowsTicks(sourceRowSet, changed, changedInverted, prevUnits, fwdUnits); - } - } else { - tmpAffected = computeAffectedRowsTime(sourceRowSet, changed, prevUnits, fwdUnits, timestampColumnSource, - timestampSsa); - } - - // other rows can be affected by removes - if (upstream.removed().isNonempty()) { - try (final RowSet prev = sourceRowSet.copyPrev(); - final RowSet removedPositions = timestampColumnName == null - ? null - : prev.invert(upstream.removed()); - final WritableRowSet affectedByRemoves = timestampColumnName == null - ? computeAffectedRowsTicks(prev, upstream.removed(), removedPositions, prevUnits, - fwdUnits) - : computeAffectedRowsTime(prev, upstream.removed(), prevUnits, fwdUnits, - timestampColumnSource, timestampSsa)) { - // apply shifts to get back to pos-shift space - upstream.shifted().apply(affectedByRemoves); - // retain only the rows that still exist in the sourceRowSet - affectedByRemoves.retain(sourceRowSet); - tmpAffected.insert(affectedByRemoves); - } - } - - affectedRows = tmpAffected; + public abstract boolean computeAffectedRowsAndOperators(@NotNull final TableUpdate upstream); - // now get influencer rows for the affected rows - if (timestampColumnName == null) { - // generate position data rowsets for efficiently computed position offsets - affectedRowPositions = sourceRowSet.invert(affectedRows); - - influencerRows = computeInfluencerRowsTicks(sourceRowSet, affectedRows, affectedRowPositions, prevUnits, - fwdUnits); - influencerPositions = sourceRowSet.invert(influencerRows); - } else { - influencerRows = computeInfluencerRowsTime(sourceRowSet, affectedRows, prevUnits, fwdUnits, - timestampColumnSource, timestampSsa); - } - } + public abstract void processRows(); - private void makeOperatorContexts() { + protected void makeOperatorContexts() { // use this to make which input sources are initialized Arrays.fill(inputSourceChunkPopulated, false); @@ -235,10 +121,8 @@ private void makeOperatorContexts() { inputSourceChunkPopulated[sourceSlot] = true; } opContext[opIdx] = operators[opIdx].makeUpdateContext(chunkSize, inputSources[sourceSlot]); - } } - } public boolean anyModified() { @@ -265,25 +149,7 @@ public RowSet getInfluencerRows() { return influencerRows; } - public void processRows() { - if (trackModifications) { - modifiedBuilder = RowSetFactory.builderSequential(); - } - - if (!windowed) { - processRowsCumulative(); - } else if (timestampColumnName == null) { - processRowsWindowedTicks(); - } else { - processRowsWindowedTime(); - } - - if (trackModifications) { - newModified = modifiedBuilder.build(); - } - } - - private void prepareValuesChunkForSource(final int srcIdx, final RowSequence rs) { + protected void prepareValuesChunkForSource(final int srcIdx, final RowSequence rs) { if (!inputSourceChunkPopulated[srcIdx]) { inputSources[srcIdx].fillChunk( inputSourceFillContexts[srcIdx], @@ -293,188 +159,15 @@ private void prepareValuesChunkForSource(final int srcIdx, final RowSequence rs) } } - private void processRowsCumulative() { - // find the key before the first affected row and preload that data for these operators - final long keyBefore; - try (final RowSet.SearchIterator sit = sourceRowSet.searchIterator()) { - keyBefore = sit.binarySearchValue( - (compareTo, ignored) -> Long.compare(affectedRows.firstRowKey() - 1, compareTo), 1); - } - - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - UpdateByCumulativeOperator cumOp = (UpdateByCumulativeOperator) operators[opIdx]; - if (cumOp.getTimestampColumnName() == null || keyBefore == -1) { - cumOp.initializeUpdate(opContext[opIdx], keyBefore, NULL_LONG); - } else { - UpdateByCumulativeOperator.Context cumOpContext = - (UpdateByCumulativeOperator.Context) opContext[opIdx]; - // make sure the time-based cumulative operators are starting from a valid value and timestamp - long potentialResetTimestamp = timestampColumnSource.getLong(keyBefore); - - if (potentialResetTimestamp == NULL_LONG || - !cumOpContext.isValueValid(keyBefore)) { - try (final RowSet.SearchIterator rIt = sourceRowSet.reverseIterator()) { - if (rIt.advance(keyBefore)) { - while (rIt.hasNext()) { - final long nextKey = rIt.nextLong(); - potentialResetTimestamp = timestampColumnSource.getLong(nextKey); - if (potentialResetTimestamp != NULL_LONG && - cumOpContext.isValueValid(nextKey)) { - break; - } - } - } - } - } - // call the specialized version of `intializeUpdate()` for these operators - cumOp.initializeUpdate(opContext[opIdx], keyBefore, potentialResetTimestamp); - } - } - } - - try (final RowSequence.Iterator it = affectedRows.getRowSequenceIterator(); - ChunkSource.GetContext tsGetCtx = - timestampColumnSource == null ? null : timestampColumnSource.makeGetContext(chunkSize)) { - while (it.hasMore()) { - final RowSequence rs = it.getNextRowSequenceWithLength(chunkSize); - Arrays.fill(inputSourceChunkPopulated, false); - - // create the timestamp chunk if needed - LongChunk tsChunk = timestampColumnSource == null ? null - : timestampColumnSource.getChunk(tsGetCtx, rs).asLongChunk(); - - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - final int srcIdx = operatorSourceSlots[opIdx]; - - // get the values this operator needs - prepareValuesChunkForSource(srcIdx, rs); - - // process the chunk - operators[opIdx].processChunk( - opContext[opIdx], - rs, - null, - null, - inputSourceChunks[srcIdx], - tsChunk); - } - } - // all these rows were modified - if (modifiedBuilder != null) { - modifiedBuilder.appendRowSequence(rs); - } - } - } - - // call the generic `finishUpdate()` function for each operator - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - operators[opIdx].finishUpdate(opContext[opIdx]); - } - } - } - - private void processRowsWindowedTicks() { - // start loading the window for these operators using position data - // find the key before the first affected row and preload that data for these operators - final long keyBefore; - try (final RowSet.SearchIterator sit = sourceRowSet.searchIterator()) { - keyBefore = sit.binarySearchValue( - (compareTo, ignored) -> Long.compare(affectedRows.firstRowKey() - 1, compareTo), 1); - } - - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - UpdateByCumulativeOperator cumOp = (UpdateByCumulativeOperator) operators[opIdx]; - if (cumOp.getTimestampColumnName() == null || keyBefore == -1) { - cumOp.initializeUpdate(opContext[opIdx], keyBefore, NULL_LONG); - } else { - UpdateByCumulativeOperator.Context cumOpContext = - (UpdateByCumulativeOperator.Context) opContext[opIdx]; - // make sure the time-based cumulative operators are starting from a valid value and timestamp - long potentialResetTimestamp = timestampColumnSource.getLong(keyBefore); - - if (potentialResetTimestamp == NULL_LONG || - !cumOpContext.isValueValid(keyBefore)) { - try (final RowSet.SearchIterator rIt = sourceRowSet.reverseIterator()) { - if (rIt.advance(keyBefore)) { - while (rIt.hasNext()) { - final long nextKey = rIt.nextLong(); - potentialResetTimestamp = timestampColumnSource.getLong(nextKey); - if (potentialResetTimestamp != NULL_LONG && - cumOpContext.isValueValid(nextKey)) { - break; - } - } - } - } - } - // call the specialized version of `intializeUpdate()` for these operators - cumOp.initializeUpdate(opContext[opIdx], keyBefore, potentialResetTimestamp); - } - } - } - - try (final RowSequence.Iterator it = affectedRows.getRowSequenceIterator(); - ChunkSource.GetContext tsGetCtx = - timestampColumnSource == null ? null : timestampColumnSource.makeGetContext(chunkSize)) { - while (it.hasMore()) { - final RowSequence rs = it.getNextRowSequenceWithLength(chunkSize); - Arrays.fill(inputSourceChunkPopulated, false); - - // create the timestamp chunk if needed - LongChunk tsChunk = timestampColumnSource == null ? null - : timestampColumnSource.getChunk(tsGetCtx, rs).asLongChunk(); - - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - final int srcIdx = operatorSourceSlots[opIdx]; - - // get the values this operator needs - prepareValuesChunkForSource(srcIdx, rs); - - // process the chunk - operators[opIdx].processChunk( - opContext[opIdx], - rs, - null, - null, - inputSourceChunks[srcIdx], - tsChunk); - } - } - // all these rows were modified - if (modifiedBuilder != null) { - modifiedBuilder.appendRowSequence(rs); - } - } - } - - // call the generic `finishUpdate()` function for each operator - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - operators[opIdx].finishUpdate(opContext[opIdx]); - } - } - } - - private void processRowsWindowedTime() { - // start loading the window for these operators using timestamp data - } - @Override public void close() { + // these might be the same object, don't close both! if (influencerRows != null && influencerRows != affectedRows) { influencerRows.close(); - } - if (influencerPositions != null && influencerPositions != affectedRowPositions) { - influencerPositions.close(); + influencerRows = null; } try (final RowSet ignoredRs1 = affectedRows; - final RowSet ignoredRs2 = affectedRowPositions; - final RowSet ignoredRs3 = newModified) { + final RowSet ignoredRs2 = newModified) { } for (int opIdx = 0; opIdx < operators.length; opIdx++) { if (opAffected[opIdx]) { @@ -493,15 +186,12 @@ public void close() { } } - public UpdateByWindowContext makeWindowContext(final TrackingRowSet sourceRowSet, + public abstract UpdateByWindowContext makeWindowContext(final TrackingRowSet sourceRowSet, final ColumnSource[] inputSources, final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa, final int chunkSize, - final boolean isInitializeStep) { - return new UpdateByWindowContext(sourceRowSet, inputSources, timestampColumnSource, timestampSsa, chunkSize, - isInitializeStep); - } + final boolean isInitializeStep); public void startTrackingModifications(@NotNull final QueryTable source, @NotNull final QueryTable result) { trackModifications = true; @@ -513,198 +203,11 @@ public void startTrackingModifications(@NotNull final QueryTable source, @NotNul } } - /** - * Find the smallest valued key that participated in the upstream {@link TableUpdate}. - * - * @param added the added rows - * @param modified the modified rows - * @param removed the removed rows - * @param shifted the shifted rows - * - * @return the smallest key that participated in any part of the update. - */ - private static long smallestAffectedKey(@NotNull final RowSet added, - @NotNull final RowSet modified, - @NotNull final RowSet removed, - @NotNull final RowSetShiftData shifted, - @NotNull final RowSet affectedIndex) { - - long smallestModifiedKey = Long.MAX_VALUE; - if (removed.isNonempty()) { - smallestModifiedKey = removed.firstRowKey(); - } - - if (added.isNonempty()) { - smallestModifiedKey = Math.min(smallestModifiedKey, added.firstRowKey()); - } - - if (modified.isNonempty()) { - smallestModifiedKey = Math.min(smallestModifiedKey, modified.firstRowKey()); - } - - if (shifted.nonempty()) { - final long firstModKey = modified.isEmpty() ? Long.MAX_VALUE : modified.firstRowKey(); - boolean modShiftFound = !modified.isEmpty(); - boolean affectedFound = false; - try (final RowSequence.Iterator it = affectedIndex.getRowSequenceIterator()) { - for (int shiftIdx = 0; shiftIdx < shifted.size() - && (!modShiftFound || !affectedFound); shiftIdx++) { - final long shiftStart = shifted.getBeginRange(shiftIdx); - final long shiftEnd = shifted.getEndRange(shiftIdx); - final long shiftDelta = shifted.getShiftDelta(shiftIdx); - - if (!affectedFound) { - if (it.advance(shiftStart + shiftDelta)) { - final long maybeAffectedKey = it.peekNextKey(); - if (maybeAffectedKey <= shiftEnd + shiftDelta) { - affectedFound = true; - final long keyToCompare = - shiftDelta > 0 ? maybeAffectedKey - shiftDelta : maybeAffectedKey; - smallestModifiedKey = Math.min(smallestModifiedKey, keyToCompare); - } - } else { - affectedFound = true; - } - } - - if (!modShiftFound) { - if (firstModKey <= (shiftEnd + shiftDelta)) { - modShiftFound = true; - // If the first modified key is in the range we should include it - if (firstModKey >= (shiftStart + shiftDelta)) { - smallestModifiedKey = Math.min(smallestModifiedKey, firstModKey - shiftDelta); - } else { - // Otherwise it's not included in any shifts, and since shifts can't reorder rows - // it is the smallest possible value and we've already accounted for it above. - break; - } - } - } - } - } - } - - return smallestModifiedKey; - } - - private static WritableRowSet computeAffectedRowsTime(final RowSet sourceSet, final RowSet subset, long revNanos, - long fwdNanos, final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa) { - // swap fwd/rev to get the affected windows - return computeInfluencerRowsTime(sourceSet, subset, fwdNanos, revNanos, timestampColumnSource, timestampSsa); - } - - private static WritableRowSet computeInfluencerRowsTime(final RowSet sourceSet, final RowSet subset, long revNanos, - long fwdNanos, final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa) { - if (sourceSet.size() == subset.size()) { - return sourceSet.copy(); - } - - int chunkSize = (int) Math.min(subset.size(), 4096); - try (final RowSequence.Iterator it = subset.getRowSequenceIterator(); - final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(chunkSize)) { - final RowSetBuilderSequential builder = RowSetFactory.builderSequential(); - LongSegmentedSortedArray.Iterator ssaIt = timestampSsa.iterator(false, false); - while (it.hasMore() && ssaIt.hasNext()) { - final RowSequence rs = it.getNextRowSequenceWithLength(chunkSize); - LongChunk timestamps = timestampColumnSource.getChunk(context, rs).asLongChunk(); - - for (int ii = 0; ii < rs.intSize(); ii++) { - // if the timestamp of the row is null, it won't belong to any set and we can ignore it - // completely - final long ts = timestamps.get(ii); - if (ts != NULL_LONG) { - // look at every row timestamp, compute the head and tail in nanos - final long head = ts - revNanos; - final long tail = ts + fwdNanos; - - // advance the iterator to the beginning of the window - if (ssaIt.nextValue() < head) { - ssaIt.advanceToBeforeFirst(head); - if (!ssaIt.hasNext()) { - // SSA is exhausted - break; - } - } - - Assert.assertion(ssaIt.hasNext() && ssaIt.nextValue() >= head, - "SSA Iterator outside of window"); - - // step through the SSA and collect keys until outside of the window - while (ssaIt.hasNext() && ssaIt.nextValue() <= tail) { - builder.appendKey(ssaIt.nextKey()); - ssaIt.next(); - } - - if (!ssaIt.hasNext()) { - // SSA is exhausted - break; - } - } - } - } - return builder.build(); - } - } - - private static WritableRowSet computeAffectedRowsTicks(final RowSet sourceSet, final RowSet subset, - final RowSet invertedSubSet, long revTicks, long fwdTicks) { - // swap fwd/rev to get the influencer windows - return computeInfluencerRowsTicks(sourceSet, subset, invertedSubSet, fwdTicks, revTicks); - } - - private static WritableRowSet computeInfluencerRowsTicks(final RowSet sourceSet, final RowSet subset, - final RowSet invertedSubSet, long revTicks, long fwdTicks) { - if (sourceSet.size() == subset.size()) { - return sourceSet.copy(); - } - - long maxPos = sourceSet.size() - 1; - - final RowSetBuilderSequential builder = RowSetFactory.builderSequential(); - final MutableLong minPos = new MutableLong(0L); - - invertedSubSet.forAllRowKeyRanges((s, e) -> { - long sPos = Math.max(s - revTicks, minPos.longValue()); - long ePos = Math.min(e + fwdTicks, maxPos); - builder.appendRange(sPos, ePos); - minPos.setValue(ePos + 1); - }); - - try (final RowSet positions = builder.build()) { - return sourceSet.subSetForPositions(positions); - } - } - - private static long locatePreviousTimestamp(final RowSet sourceSet, final ColumnSource timestampColumnSource, - final long firstUnmodifiedKey) { - long potentialResetTimestamp = timestampColumnSource.getLong(firstUnmodifiedKey); - if (potentialResetTimestamp != NULL_LONG) { - return potentialResetTimestamp; - } - - try (final RowSet.SearchIterator rIt = sourceSet.reverseIterator()) { - if (rIt.advance(firstUnmodifiedKey)) { - while (rIt.hasNext()) { - final long nextKey = rIt.nextLong(); - potentialResetTimestamp = timestampColumnSource.getLong(nextKey); - if (potentialResetTimestamp != NULL_LONG) { - return potentialResetTimestamp; - } - } - } - } - - return NULL_LONG; - } - - private UpdateByWindow(UpdateByOperator[] operators, int[] operatorSourceSlots, boolean windowed, - @Nullable String timestampColumnName, long prevUnits, long fwdUnits) { + protected UpdateByWindow(UpdateByOperator[] operators, int[] operatorSourceSlots, + @Nullable String timestampColumnName) { this.operators = operators; this.operatorSourceSlots = operatorSourceSlots; - this.windowed = windowed; this.timestampColumnName = timestampColumnName; - this.prevUnits = prevUnits; - this.fwdUnits = fwdUnits; operatorInputModifiedColumnSets = new ModifiedColumnSet[operators.length]; operatorOutputModifiedColumnSets = new ModifiedColumnSet[operators.length]; @@ -722,12 +225,24 @@ public static UpdateByWindow createFromOperatorArray(final UpdateByOperator[] op } } - return new UpdateByWindow(operators, - operatorSourceSlots, - operators[0] instanceof UpdateByWindowedOperator, - timestampColumnName, - operators[0].getPrevWindowUnits(), - operators[0].getPrevWindowUnits()); + // return the correct type of UpdateByWindow + final boolean windowed = operators[0] instanceof UpdateByWindowedOperator; + if (!windowed) { + return new UpdateByWindowCumulative(operators, + operatorSourceSlots, + timestampColumnName); + } else if (timestampColumnName == null) { + return new UpdateByWindowTicks(operators, + operatorSourceSlots, + operators[0].getPrevWindowUnits(), + operators[0].getFwdWindowUnits()); + } else { + return new UpdateByWindowTime(operators, + operatorSourceSlots, + timestampColumnName, + operators[0].getPrevWindowUnits(), + operators[0].getFwdWindowUnits()); + } } @Nullable @@ -735,7 +250,8 @@ public String getTimestampColumnName() { return timestampColumnName; } - private static int hashCode(boolean windowed, @Nullable String timestampColumnName, long prevUnits, long fwdUnits) { + protected static int hashCode(boolean windowed, @Nullable String timestampColumnName, long prevUnits, + long fwdUnits) { // treat all cumulative as identical, even if they rely on timestamps if (!windowed) { return Boolean.hashCode(windowed); @@ -757,12 +273,4 @@ public static int hashCodeFromOperator(final UpdateByOperator op) { op.getPrevWindowUnits(), op.getPrevWindowUnits()); } - - @Override - public int hashCode() { - return hashCode(windowed, - timestampColumnName, - prevUnits, - fwdUnits); - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java new file mode 100644 index 00000000000..9b922e2e71d --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java @@ -0,0 +1,292 @@ +package io.deephaven.engine.table.impl.updateby; + +import io.deephaven.base.ringbuffer.LongRingBuffer; +import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.WritableChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.*; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.table.ChunkSource; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.ModifiedColumnSet; +import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.QueryTable; +import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; +import io.deephaven.engine.table.impl.UpdateByOperator; +import io.deephaven.engine.table.impl.UpdateByWindowedOperator; +import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; +import io.deephaven.util.SafeCloseable; +import org.apache.commons.lang3.mutable.MutableLong; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.util.Arrays; + +import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; +import static io.deephaven.util.QueryConstants.NULL_LONG; + +// this class is currently too big, should specialize into CumWindow, TickWindow, TimeWindow to simplify implementation +public class UpdateByWindowCumulative extends UpdateByWindow { + public class UpdateByWindowCumulativeContext extends UpdateByWindowContext { + public UpdateByWindowCumulativeContext(final TrackingRowSet sourceRowSet, final ColumnSource[] inputSources, + @Nullable final ColumnSource timestampColumnSource, + @Nullable final LongSegmentedSortedArray timestampSsa, final int chunkSize, final boolean initialStep) { + super(sourceRowSet, inputSources, timestampColumnSource, timestampSsa, chunkSize, initialStep); + } + + @Override + public void close() { + super.close(); + } + + @Override + public boolean computeAffectedRowsAndOperators(@NotNull final TableUpdate upstream) { + // all rows are affected on the initial step + if (initialStep) { + affectedRows = sourceRowSet.copy(); + influencerRows = affectedRows; + + // mark all operators as affected by this update + Arrays.fill(opAffected, true); + + makeOperatorContexts(); + return true; + } + + // determine which operators are affected by this update + boolean anyAffected = false; + boolean allAffected = upstream.added().isNonempty() || + upstream.removed().isNonempty() || + upstream.shifted().nonempty(); + + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + opAffected[opIdx] = allAffected + || (upstream.modifiedColumnSet().nonempty() && (operatorInputModifiedColumnSets[opIdx] == null + || upstream.modifiedColumnSet().containsAny(operatorInputModifiedColumnSets[opIdx]))); + if (opAffected[opIdx]) { + anyAffected = true; + } + } + + if (!anyAffected) { + return false; + } + + long smallestModifiedKey = smallestAffectedKey(upstream.added(), upstream.modified(), upstream.removed(), + upstream.shifted(), sourceRowSet); + + affectedRows = smallestModifiedKey == Long.MAX_VALUE + ? RowSetFactory.empty() + : sourceRowSet.subSetByKeyRange(smallestModifiedKey, sourceRowSet.lastRowKey()); + influencerRows = affectedRows; + + makeOperatorContexts(); + return true; + } + + @Override + public void processRows() { + if (trackModifications) { + modifiedBuilder = RowSetFactory.builderSequential(); + } + + // find the key before the first affected row + final long keyBefore; + try (final RowSet.SearchIterator sit = sourceRowSet.searchIterator()) { + keyBefore = sit.binarySearchValue( + (compareTo, ignored) -> Long.compare(affectedRows.firstRowKey() - 1, compareTo), 1); + } + + // and preload that data for these operators + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (opAffected[opIdx]) { + UpdateByCumulativeOperator cumOp = (UpdateByCumulativeOperator) operators[opIdx]; + if (cumOp.getTimestampColumnName() == null || keyBefore == -1) { + cumOp.initializeUpdate(opContext[opIdx], keyBefore, NULL_LONG); + } else { + UpdateByCumulativeOperator.Context cumOpContext = + (UpdateByCumulativeOperator.Context) opContext[opIdx]; + // make sure the time-based cumulative operators are starting from a valid value and timestamp + long potentialResetTimestamp = timestampColumnSource.getLong(keyBefore); + + if (potentialResetTimestamp == NULL_LONG || + !cumOpContext.isValueValid(keyBefore)) { + try (final RowSet.SearchIterator rIt = sourceRowSet.reverseIterator()) { + if (rIt.advance(keyBefore)) { + while (rIt.hasNext()) { + final long nextKey = rIt.nextLong(); + potentialResetTimestamp = timestampColumnSource.getLong(nextKey); + if (potentialResetTimestamp != NULL_LONG && + cumOpContext.isValueValid(nextKey)) { + break; + } + } + } + } + } + // call the specialized version of `intializeUpdate()` for these operators + cumOp.initializeUpdate(opContext[opIdx], keyBefore, potentialResetTimestamp); + } + } + } + + try (final RowSequence.Iterator it = affectedRows.getRowSequenceIterator(); + ChunkSource.GetContext tsGetCtx = + timestampColumnSource == null ? null : timestampColumnSource.makeGetContext(chunkSize)) { + while (it.hasMore()) { + final RowSequence rs = it.getNextRowSequenceWithLength(chunkSize); + Arrays.fill(inputSourceChunkPopulated, false); + + // create the timestamp chunk if needed + LongChunk tsChunk = timestampColumnSource == null ? null + : timestampColumnSource.getChunk(tsGetCtx, rs).asLongChunk(); + + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (opAffected[opIdx]) { + final int srcIdx = operatorSourceSlots[opIdx]; + + // chunk prep + prepareValuesChunkForSource(srcIdx, rs); + opContext[opIdx].setValuesChunk(inputSourceChunks[srcIdx]); + opContext[opIdx].setTimestampChunk(tsChunk); + + // chunk processing + for (int ii = 0; ii < rs.size(); ii++) { + opContext[opIdx].push(NULL_ROW_KEY, ii); + opContext[opIdx].writeToOutputChunk(ii); + } + + // chunk output to column + opContext[opIdx].writeToOutputColumn(rs); + } + } + + // all these rows were modified + if (modifiedBuilder != null) { + modifiedBuilder.appendRowSequence(rs); + } + } + } + + // call `finishUpdate()` function for each operator + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (opAffected[opIdx]) { + operators[opIdx].finishUpdate(opContext[opIdx]); + } + } + + if (trackModifications) { + newModified = modifiedBuilder.build(); + } + } + } + + public UpdateByWindowContext makeWindowContext(final TrackingRowSet sourceRowSet, + final ColumnSource[] inputSources, + final ColumnSource timestampColumnSource, + final LongSegmentedSortedArray timestampSsa, + final int chunkSize, + final boolean isInitializeStep) { + return new UpdateByWindowCumulativeContext(sourceRowSet, inputSources, timestampColumnSource, timestampSsa, + chunkSize, + isInitializeStep); + } + + public void startTrackingModifications(@NotNull final QueryTable source, @NotNull final QueryTable result) { + trackModifications = true; + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + operatorInputModifiedColumnSets[opIdx] = + source.newModifiedColumnSet(operators[opIdx].getAffectingColumnNames()); + operatorOutputModifiedColumnSets[opIdx] = + result.newModifiedColumnSet(operators[opIdx].getOutputColumnNames()); + } + } + + /** + * Find the smallest valued key that participated in the upstream {@link TableUpdate}. + * + * @param added the added rows + * @param modified the modified rows + * @param removed the removed rows + * @param shifted the shifted rows + * + * @return the smallest key that participated in any part of the update. + */ + private static long smallestAffectedKey(@NotNull final RowSet added, + @NotNull final RowSet modified, + @NotNull final RowSet removed, + @NotNull final RowSetShiftData shifted, + @NotNull final RowSet affectedIndex) { + + long smallestModifiedKey = Long.MAX_VALUE; + if (removed.isNonempty()) { + smallestModifiedKey = removed.firstRowKey(); + } + + if (added.isNonempty()) { + smallestModifiedKey = Math.min(smallestModifiedKey, added.firstRowKey()); + } + + if (modified.isNonempty()) { + smallestModifiedKey = Math.min(smallestModifiedKey, modified.firstRowKey()); + } + + if (shifted.nonempty()) { + final long firstModKey = modified.isEmpty() ? Long.MAX_VALUE : modified.firstRowKey(); + boolean modShiftFound = !modified.isEmpty(); + boolean affectedFound = false; + try (final RowSequence.Iterator it = affectedIndex.getRowSequenceIterator()) { + for (int shiftIdx = 0; shiftIdx < shifted.size() + && (!modShiftFound || !affectedFound); shiftIdx++) { + final long shiftStart = shifted.getBeginRange(shiftIdx); + final long shiftEnd = shifted.getEndRange(shiftIdx); + final long shiftDelta = shifted.getShiftDelta(shiftIdx); + + if (!affectedFound) { + if (it.advance(shiftStart + shiftDelta)) { + final long maybeAffectedKey = it.peekNextKey(); + if (maybeAffectedKey <= shiftEnd + shiftDelta) { + affectedFound = true; + final long keyToCompare = + shiftDelta > 0 ? maybeAffectedKey - shiftDelta : maybeAffectedKey; + smallestModifiedKey = Math.min(smallestModifiedKey, keyToCompare); + } + } else { + affectedFound = true; + } + } + + if (!modShiftFound) { + if (firstModKey <= (shiftEnd + shiftDelta)) { + modShiftFound = true; + // If the first modified key is in the range we should include it + if (firstModKey >= (shiftStart + shiftDelta)) { + smallestModifiedKey = Math.min(smallestModifiedKey, firstModKey - shiftDelta); + } else { + // Otherwise it's not included in any shifts, and since shifts can't reorder rows + // it is the smallest possible value and we've already accounted for it above. + break; + } + } + } + } + } + } + + return smallestModifiedKey; + } + + public UpdateByWindowCumulative(UpdateByOperator[] operators, int[] operatorSourceSlots, + @Nullable String timestampColumnName) { + super(operators, operatorSourceSlots, timestampColumnName); + } + + @Override + public int hashCode() { + return hashCode(false, + timestampColumnName, + 0L, + 0L); + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java new file mode 100644 index 00000000000..c38c7655857 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java @@ -0,0 +1,370 @@ +package io.deephaven.engine.table.impl.updateby; + +import io.deephaven.base.ringbuffer.IntRingBuffer; +import io.deephaven.chunk.LongChunk; +import io.deephaven.engine.rowset.*; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.QueryTable; +import io.deephaven.engine.table.impl.UpdateByOperator; +import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; +import io.deephaven.util.datastructures.LongSizedDataStructure; +import org.apache.commons.lang3.mutable.MutableLong; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.util.Arrays; + +// this class is currently too big, should specialize into CumWindow, TickWindow, TimeWindow to simplify implementation +public class UpdateByWindowTicks extends UpdateByWindow { + protected final long prevUnits; + protected final long fwdUnits; + + public class UpdateByWindowTicksContext extends UpdateByWindow.UpdateByWindowContext { + protected final IntRingBuffer currentWindowPositions; + + protected RowSet affectedRowPositions; + protected RowSet influencerPositions; + + protected int nextInfluencerIndex; + protected int nextInfluencerPos; + protected long nextInfluencerKey; + + protected RowSequence.Iterator influencerIt; + protected RowSequence.Iterator influencerPosIt; + protected LongChunk influencerPosChunk; + protected LongChunk influencerKeyChunk; + + public UpdateByWindowTicksContext(final TrackingRowSet sourceRowSet, final ColumnSource[] inputSources, + @Nullable final ColumnSource timestampColumnSource, + @Nullable final LongSegmentedSortedArray timestampSsa, final int chunkSize, final boolean initialStep) { + super(sourceRowSet, inputSources, null, null, chunkSize, initialStep); + + currentWindowPositions = new IntRingBuffer(512, true); + } + + + @Override + public void close() { + super.close(); + // these might be identical, don't close both! + if (influencerPositions != null && influencerPositions != affectedRowPositions) { + influencerPositions.close(); + } + try (final RowSet ignoredRs1 = affectedRowPositions; + final RowSequence.Iterator ignoreIt1 = influencerIt; + final RowSequence.Iterator ignoreIt2 = influencerPosIt) { + // leveraging try with resources to auto-close + } + } + + // windowed by time/ticks is more complex to compute: find all the changed rows and the rows that would + // be affected by the changes (includes newly added rows) and need to be recomputed. Then include all + // the rows that are affected by deletions (if any). After the affected rows have been identified, + // determine which rows will be needed to compute new values for the affected rows (influencer rows) + @Override + public boolean computeAffectedRowsAndOperators(@NotNull final TableUpdate upstream) { + // all rows are affected on the initial step + if (initialStep) { + affectedRows = sourceRowSet.copy(); + influencerRows = affectedRows; + + // no need to invert, just create a flat rowset + affectedRowPositions = RowSetFactory.flat(sourceRowSet.size()); + influencerPositions = RowSetFactory.flat(sourceRowSet.size()); + + // mark all operators as affected by this update + Arrays.fill(opAffected, true); + + makeOperatorContexts(); + return true; + } + + // determine which operators are affected by this update + boolean anyAffected = false; + boolean allAffected = upstream.added().isNonempty() || + upstream.removed().isNonempty() || + upstream.shifted().nonempty(); + + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + opAffected[opIdx] = allAffected + || (upstream.modifiedColumnSet().nonempty() && (operatorInputModifiedColumnSets[opIdx] == null + || upstream.modifiedColumnSet().containsAny(operatorInputModifiedColumnSets[opIdx]))); + if (opAffected[opIdx]) { + anyAffected = true; + } + } + + if (!anyAffected) { + return false; + } + + // changed rows are all mods+adds + WritableRowSet changed = upstream.added().union(upstream.modified()); + + // need a writable rowset + WritableRowSet tmpAffected; + + // compute the rows affected from these changes + try (final WritableRowSet changedInverted = sourceRowSet.invert(changed)) { + tmpAffected = computeAffectedRowsTicks(sourceRowSet, changed, changedInverted, prevUnits, fwdUnits); + } + + // other rows can be affected by removes + if (upstream.removed().isNonempty()) { + try (final RowSet prev = sourceRowSet.copyPrev(); + final RowSet removedPositions = prev.invert(upstream.removed()); + final WritableRowSet affectedByRemoves = + computeAffectedRowsTicks(prev, upstream.removed(), removedPositions, prevUnits, + fwdUnits)) { + // apply shifts to get back to pos-shift space + upstream.shifted().apply(affectedByRemoves); + // retain only the rows that still exist in the sourceRowSet + affectedByRemoves.retain(sourceRowSet); + tmpAffected.insert(affectedByRemoves); + } + } + + affectedRows = tmpAffected; + + // now get influencer rows for the affected rows + // generate position data rowsets for efficiently computed position offsets + affectedRowPositions = sourceRowSet.invert(affectedRows); + + influencerRows = computeInfluencerRowsTicks(sourceRowSet, affectedRows, affectedRowPositions, prevUnits, + fwdUnits); + influencerPositions = sourceRowSet.invert(influencerRows); + + makeOperatorContexts(); + return true; + } + + private void loadNextInfluencerValueChunks() { + if (!influencerIt.hasMore()) { + nextInfluencerPos = Integer.MAX_VALUE; + nextInfluencerKey = Long.MAX_VALUE; + return; + } + + final RowSequence influencerRs = influencerIt.getNextRowSequenceWithLength(chunkSize); + influencerKeyChunk = influencerRs.asRowKeyChunk(); + + final RowSequence influencePosRs = influencerPosIt.getNextRowSequenceWithLength(chunkSize); + influencerPosChunk = influencePosRs.asRowKeyChunk(); + + Arrays.fill(inputSourceChunkPopulated, false); + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (opAffected[opIdx]) { + final int srcIdx = operatorSourceSlots[opIdx]; + prepareValuesChunkForSource(srcIdx, influencerRs); + + opContext[opIdx].setValuesChunk(inputSourceChunks[srcIdx]); + } + } + + nextInfluencerIndex = 0; + nextInfluencerPos = LongSizedDataStructure.intSize( + "updateBy window positions exceeded maximum size", + influencerPosChunk.get(nextInfluencerIndex)); + nextInfluencerKey = influencerKeyChunk.get(nextInfluencerIndex); + } + + private void fillWindowTicks(long currentPos) { + // compute the head and tail positions (inclusive) + final long head = Math.max(0, currentPos - prevUnits + 1); + final long tail = Math.min(sourceRowSet.size() - 1, currentPos + fwdUnits); + + // pop out all values from the current window that are not in the new window + while (!currentWindowPositions.isEmpty() && currentWindowPositions.front() < head) { + // operator pop + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (opAffected[opIdx]) { + opContext[opIdx].pop(); + } + } + currentWindowPositions.remove(); + } + + // if the window is empty, reset + if (currentWindowPositions.isEmpty()) { + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (opAffected[opIdx]) { + opContext[opIdx].reset(); + } + } + } + + // skip values until they match the window (this can only happen on the initial addition of rows to the + // table, because we short-circuit the precise building of the influencer rows for efficiency) + while (nextInfluencerPos < head) { + nextInfluencerIndex++; + + if (nextInfluencerIndex < influencerPosChunk.size()) { + nextInfluencerPos = LongSizedDataStructure.intSize( + "updateBy window positions exceeded maximum size", + influencerPosChunk.get(nextInfluencerIndex)); + nextInfluencerKey = influencerKeyChunk.get(nextInfluencerIndex); + } else { + // try to bring in new data + loadNextInfluencerValueChunks(); + } + } + + // push matching values + while (nextInfluencerPos <= tail) { + // operator push + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (opAffected[opIdx]) { + opContext[opIdx].push(nextInfluencerKey, nextInfluencerIndex); + } + } + currentWindowPositions.add(nextInfluencerPos); + nextInfluencerIndex++; + + if (nextInfluencerIndex < influencerPosChunk.size()) { + nextInfluencerPos = LongSizedDataStructure.intSize( + "updateBy window positions exceeded maximum size", + influencerPosChunk.get(nextInfluencerIndex)); + nextInfluencerKey = influencerKeyChunk.get(nextInfluencerIndex); + } else { + // try to bring in new data + loadNextInfluencerValueChunks(); + } + } + } + + // this function process the affected rows chunkwise, but will call fillWindowTicks() for each + // new row. fillWindowTicks() will advance the moving window (which is the same for all operators in this + // collection) and will call push/pop for each operator as it advances the window. + // + // We track the minimum amount of data needed, only the window position data. The downstream operators + // should manage local storage in a RingBuffer or other efficient structure + @Override + public void processRows() { + if (trackModifications) { + modifiedBuilder = RowSetFactory.builderSequential(); + } + + influencerIt = influencerRows.getRowSequenceIterator(); + influencerPosIt = influencerPositions.getRowSequenceIterator(); + + try (final RowSequence.Iterator it = affectedRows.getRowSequenceIterator(); + final RowSequence.Iterator posIt = affectedRowPositions.getRowSequenceIterator()) { + + // load the first chunk of influencer values (fillWindowTicks() will call in future) + loadNextInfluencerValueChunks(); + + while (it.hasMore()) { + final RowSequence rs = it.getNextRowSequenceWithLength(chunkSize); + final RowSequence posRs = posIt.getNextRowSequenceWithLength(chunkSize); + + final LongChunk posChunk = posRs.asRowKeyChunk(); + + // chunk processing + for (int ii = 0; ii < rs.size(); ii++) { + // read the current position + final long currentPos = posChunk.get(ii); + + // fill the operator windows (calls push/pop/reset as appropriate) + fillWindowTicks(currentPos); + + // now the operators have seen the correct window data, write to the output chunk + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (opAffected[opIdx]) { + opContext[opIdx].writeToOutputChunk(ii); + } + } + } + + // chunk output to column + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (opAffected[opIdx]) { + opContext[opIdx].writeToOutputColumn(rs); + } + } + + // all these rows were modified + if (modifiedBuilder != null) { + modifiedBuilder.appendRowSequence(rs); + } + } + } + + // call `finishUpdate()` function for each operator + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (opAffected[opIdx]) { + operators[opIdx].finishUpdate(opContext[opIdx]); + } + } + + if (trackModifications) { + newModified = modifiedBuilder.build(); + } + } + } + + public UpdateByWindowContext makeWindowContext(final TrackingRowSet sourceRowSet, + final ColumnSource[] inputSources, + final ColumnSource timestampColumnSource, + final LongSegmentedSortedArray timestampSsa, + final int chunkSize, + final boolean isInitializeStep) { + return new UpdateByWindowTicksContext(sourceRowSet, inputSources, timestampColumnSource, timestampSsa, + chunkSize, + isInitializeStep); + } + + public void startTrackingModifications(@NotNull final QueryTable source, @NotNull final QueryTable result) { + trackModifications = true; + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + operatorInputModifiedColumnSets[opIdx] = + source.newModifiedColumnSet(operators[opIdx].getAffectingColumnNames()); + operatorOutputModifiedColumnSets[opIdx] = + result.newModifiedColumnSet(operators[opIdx].getOutputColumnNames()); + } + } + + private static WritableRowSet computeAffectedRowsTicks(final RowSet sourceSet, final RowSet subset, + final RowSet invertedSubSet, long revTicks, long fwdTicks) { + // swap fwd/rev to get the influencer windows + return computeInfluencerRowsTicks(sourceSet, subset, invertedSubSet, fwdTicks, revTicks); + } + + private static WritableRowSet computeInfluencerRowsTicks(final RowSet sourceSet, final RowSet subset, + final RowSet invertedSubSet, long revTicks, long fwdTicks) { + if (sourceSet.size() == subset.size()) { + return sourceSet.copy(); + } + + long maxPos = sourceSet.size() - 1; + + final RowSetBuilderSequential builder = RowSetFactory.builderSequential(); + final MutableLong minPos = new MutableLong(0L); + + invertedSubSet.forAllRowKeyRanges((s, e) -> { + long sPos = Math.max(s - revTicks, minPos.longValue()); + long ePos = Math.min(e + fwdTicks, maxPos); + builder.appendRange(sPos, ePos); + minPos.setValue(ePos + 1); + }); + + try (final RowSet positions = builder.build()) { + return sourceSet.subSetForPositions(positions); + } + } + + UpdateByWindowTicks(UpdateByOperator[] operators, int[] operatorSourceSlots, long prevUnits, long fwdUnits) { + super(operators, operatorSourceSlots, null); + this.prevUnits = prevUnits; + this.fwdUnits = fwdUnits; + } + + @Override + public int hashCode() { + return hashCode(true, + null, + prevUnits, + fwdUnits); + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java new file mode 100644 index 00000000000..ee092c28a3a --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java @@ -0,0 +1,361 @@ +package io.deephaven.engine.table.impl.updateby; + +import io.deephaven.base.ringbuffer.LongRingBuffer; +import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.*; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.table.ChunkSource; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.UpdateByOperator; +import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.util.Arrays; + +import static io.deephaven.util.QueryConstants.NULL_LONG; + +// this class is currently too big, should specialize into CumWindow, TickWindow, TimeWindow to simplify implementation +public class UpdateByWindowTime extends UpdateByWindow { + protected final long prevUnits; + protected final long fwdUnits; + + public class UpdateByWindowTimeContext extends UpdateByWindowContext { + protected final ChunkSource.GetContext influencerTimestampContext; + protected final LongRingBuffer currentWindowTimestamps; + + protected int nextInfluencerIndex; + protected long nextInfluencerTimestamp; + protected long nextInfluencerKey; + + protected RowSequence.Iterator influencerIt; + protected LongChunk influencerKeyChunk; + protected LongChunk influencerTimestampChunk; + + public UpdateByWindowTimeContext(final TrackingRowSet sourceRowSet, final ColumnSource[] inputSources, + @NotNull final ColumnSource timestampColumnSource, + @Nullable final LongSegmentedSortedArray timestampSsa, final int chunkSize, final boolean initialStep) { + super(sourceRowSet, inputSources, timestampColumnSource, timestampSsa, chunkSize, initialStep); + + influencerTimestampContext = timestampColumnSource.makeGetContext(chunkSize); + currentWindowTimestamps = new LongRingBuffer(512, true); + } + + @Override + public void close() { + super.close(); + try (final RowSequence.Iterator ignoreIt1 = influencerIt; + final ChunkSource.GetContext ignoreCtx1 = influencerTimestampContext) { + // leveraging try with resources to auto-close + } + } + + @Override + public boolean computeAffectedRowsAndOperators(@NotNull final TableUpdate upstream) { + // all rows are affected on the initial step + if (initialStep) { + affectedRows = sourceRowSet.copy(); + influencerRows = affectedRows; + + // mark all operators as affected by this update + Arrays.fill(opAffected, true); + + makeOperatorContexts(); + return true; + } + + // determine which operators are affected by this update + boolean anyAffected = false; + boolean allAffected = upstream.added().isNonempty() || + upstream.removed().isNonempty() || + upstream.shifted().nonempty(); + + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + opAffected[opIdx] = allAffected + || (upstream.modifiedColumnSet().nonempty() && (operatorInputModifiedColumnSets[opIdx] == null + || upstream.modifiedColumnSet().containsAny(operatorInputModifiedColumnSets[opIdx]))); + if (opAffected[opIdx]) { + anyAffected = true; + } + } + + if (!anyAffected) { + return false; + } + + // changed rows are all mods+adds + WritableRowSet changed = upstream.added().union(upstream.modified()); + + // need a writable rowset + WritableRowSet tmpAffected = computeAffectedRowsTime(sourceRowSet, changed, prevUnits, fwdUnits, + timestampColumnSource, timestampSsa); + + // other rows can be affected by removes + if (upstream.removed().isNonempty()) { + try (final RowSet prev = sourceRowSet.copyPrev(); + final WritableRowSet affectedByRemoves = + computeAffectedRowsTime(prev, upstream.removed(), prevUnits, fwdUnits, + timestampColumnSource, timestampSsa)) { + // apply shifts to get back to pos-shift space + upstream.shifted().apply(affectedByRemoves); + // retain only the rows that still exist in the sourceRowSet + affectedByRemoves.retain(sourceRowSet); + tmpAffected.insert(affectedByRemoves); + } + } + + affectedRows = tmpAffected; + + // now get influencer rows for the affected rows + influencerRows = computeInfluencerRowsTime(sourceRowSet, affectedRows, prevUnits, fwdUnits, + timestampColumnSource, timestampSsa); + + makeOperatorContexts(); + return true; + } + + private void loadNextInfluencerValueChunks() { + if (!influencerIt.hasMore()) { + nextInfluencerTimestamp = Long.MAX_VALUE; + nextInfluencerKey = Long.MAX_VALUE; + return; + } + + final RowSequence influencerRs = influencerIt.getNextRowSequenceWithLength(chunkSize); + influencerKeyChunk = influencerRs.asRowKeyChunk(); + influencerTimestampChunk = + timestampColumnSource.getChunk(influencerTimestampContext, influencerRs).asLongChunk(); + + Arrays.fill(inputSourceChunkPopulated, false); + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (opAffected[opIdx]) { + final int srcIdx = operatorSourceSlots[opIdx]; + prepareValuesChunkForSource(srcIdx, influencerRs); + + opContext[opIdx].setValuesChunk(inputSourceChunks[srcIdx]); + } + } + + nextInfluencerIndex = 0; + nextInfluencerTimestamp = influencerTimestampChunk.get(nextInfluencerIndex); + nextInfluencerKey = influencerKeyChunk.get(nextInfluencerIndex); + } + + public void fillWindowTime(long currentTimestamp) { + // compute the head and tail positions (inclusive) + final long head = currentTimestamp - prevUnits; + final long tail = currentTimestamp + fwdUnits; + + // pop out all values from the current window that are not in the new window + while (!currentWindowTimestamps.isEmpty() && currentWindowTimestamps.front() < head) { + // operator pop + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (opAffected[opIdx]) { + opContext[opIdx].pop(); + } + } + currentWindowTimestamps.remove(); + } + + // if the window is empty, reset + if (currentWindowTimestamps.isEmpty()) { + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (opAffected[opIdx]) { + opContext[opIdx].reset(); + } + } + } + + // skip values until they match the window (this can only happen on the initial addition of rows to the + // table, because we short-circuit the precise building of the influencer rows for efficiency) + while (nextInfluencerTimestamp < head) { + nextInfluencerIndex++; + + if (nextInfluencerIndex < influencerTimestampChunk.size()) { + nextInfluencerTimestamp = influencerTimestampChunk.get(nextInfluencerIndex); + nextInfluencerKey = influencerKeyChunk.get(nextInfluencerIndex); + } else { + // try to bring in new data + loadNextInfluencerValueChunks(); + } + } + + // push matching values + while (nextInfluencerTimestamp <= tail) { + // operator push + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (opAffected[opIdx]) { + opContext[opIdx].push(nextInfluencerKey, nextInfluencerIndex); + } + } + currentWindowTimestamps.add(nextInfluencerTimestamp); + nextInfluencerIndex++; + + if (nextInfluencerIndex < influencerTimestampChunk.size()) { + nextInfluencerTimestamp = influencerTimestampChunk.get(nextInfluencerIndex); + nextInfluencerKey = influencerKeyChunk.get(nextInfluencerIndex); + } else { + // try to bring in new data + loadNextInfluencerValueChunks(); + } + } + } + + // overview: this function process the affected rows chunkwise, but will call fillWindowTime() for each + // new row. fillWindowTime() will advance the moving window (which is the same for all operators in this + // collection) and will call push/pop for each operator as it advances the window. + // + // We track the minimum amount of data needed, only the window timestamp data. The downstream operators + // should manage local storage in a RingBuffer or other efficient structure + @Override + public void processRows() { + if (trackModifications) { + modifiedBuilder = RowSetFactory.builderSequential(); + } + + influencerIt = influencerRows.getRowSequenceIterator(); + try (final RowSequence.Iterator it = affectedRows.getRowSequenceIterator(); + final ChunkSource.GetContext localTimestampContext = + timestampColumnSource.makeGetContext(chunkSize)) { + + // load the first chunk of influencer values (fillWindowTime() will call in future) + loadNextInfluencerValueChunks(); + + while (it.hasMore()) { + final RowSequence rs = it.getNextRowSequenceWithLength(chunkSize); + + // just a note, we did not put null values into our SSA and our influencer rowset is built using the + // SSA. there should be no null timestamps considered in the rolling windows + final LongChunk timestampChunk = + timestampColumnSource.getChunk(localTimestampContext, rs).asLongChunk(); + + // chunk processing + for (int ii = 0; ii < rs.size(); ii++) { + // read the current position + final long ts = timestampChunk.get(ii); + + // fill the operator windows (calls push/pop/reset as appropriate) + fillWindowTime(ts); + + // now the operators have seen the correct window data, write to the output chunk + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (opAffected[opIdx]) { + opContext[opIdx].writeToOutputChunk(ii); + } + } + } + + // chunk output to column + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (opAffected[opIdx]) { + opContext[opIdx].writeToOutputColumn(rs); + } + } + + // all these rows were modified + if (modifiedBuilder != null) { + modifiedBuilder.appendRowSequence(rs); + } + } + } + + // call `finishUpdate()` function for each operator + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (opAffected[opIdx]) { + operators[opIdx].finishUpdate(opContext[opIdx]); + } + } + + if (trackModifications) { + newModified = modifiedBuilder.build(); + } + } + } + + public UpdateByWindowContext makeWindowContext(final TrackingRowSet sourceRowSet, + final ColumnSource[] inputSources, + final ColumnSource timestampColumnSource, + final LongSegmentedSortedArray timestampSsa, + final int chunkSize, + final boolean isInitializeStep) { + return new UpdateByWindowTimeContext(sourceRowSet, inputSources, timestampColumnSource, timestampSsa, chunkSize, + isInitializeStep); + } + + private static WritableRowSet computeAffectedRowsTime(final RowSet sourceSet, final RowSet subset, long revNanos, + long fwdNanos, final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa) { + // swap fwd/rev to get the affected windows + return computeInfluencerRowsTime(sourceSet, subset, fwdNanos, revNanos, timestampColumnSource, timestampSsa); + } + + private static WritableRowSet computeInfluencerRowsTime(final RowSet sourceSet, final RowSet subset, long revNanos, + long fwdNanos, final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa) { + if (sourceSet.size() == subset.size()) { + return sourceSet.copy(); + } + + int chunkSize = (int) Math.min(subset.size(), 4096); + try (final RowSequence.Iterator it = subset.getRowSequenceIterator(); + final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(chunkSize)) { + final RowSetBuilderSequential builder = RowSetFactory.builderSequential(); + LongSegmentedSortedArray.Iterator ssaIt = timestampSsa.iterator(false, false); + while (it.hasMore() && ssaIt.hasNext()) { + final RowSequence rs = it.getNextRowSequenceWithLength(chunkSize); + LongChunk timestamps = timestampColumnSource.getChunk(context, rs).asLongChunk(); + + for (int ii = 0; ii < rs.intSize(); ii++) { + // if the timestamp of the row is null, it won't belong to any set and we can ignore it + // completely + final long ts = timestamps.get(ii); + if (ts != NULL_LONG) { + // look at every row timestamp, compute the head and tail in nanos + final long head = ts - revNanos; + final long tail = ts + fwdNanos; + + // advance the iterator to the beginning of the window + if (ssaIt.nextValue() < head) { + ssaIt.advanceToBeforeFirst(head); + if (!ssaIt.hasNext()) { + // SSA is exhausted + break; + } + } + + Assert.eqTrue(ssaIt.hasNext() && ssaIt.nextValue() >= head, + "SSA Iterator outside of window"); + + // step through the SSA and collect keys until outside of the window + while (ssaIt.hasNext() && ssaIt.nextValue() <= tail) { + builder.appendKey(ssaIt.nextKey()); + ssaIt.next(); + } + + if (!ssaIt.hasNext()) { + // SSA is exhausted + break; + } + } + } + } + return builder.build(); + } + } + + UpdateByWindowTime(UpdateByOperator[] operators, int[] operatorSourceSlots, @Nullable String timestampColumnName, + long prevUnits, long fwdUnits) { + super(operators, operatorSourceSlots, timestampColumnName); + this.prevUnits = prevUnits; + this.fwdUnits = fwdUnits; + } + + @Override + public int hashCode() { + return hashCode(true, + timestampColumnName, + prevUnits, + fwdUnits); + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java index 71e6c0768f9..00e79efae63 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java @@ -25,14 +25,25 @@ public abstract class BasePrimitiveEMAOperator extends BaseDoubleUpdateByOperato protected final double alpha; protected double oneMinusAlpha; - class Context extends BaseDoubleUpdateByOperator.Context { + public abstract class Context extends BaseDoubleUpdateByOperator.Context { public LongChunk timestampValueChunk; long lastStamp = NULL_LONG; + @Override + public void setTimestampChunk(@NotNull final LongChunk valuesChunk) { + timestampValueChunk = valuesChunk; + } + Context(final int chunkSize) { super(chunkSize); } + + @Override + public void reset() { + curVal = NULL_DOUBLE; + lastStamp = NULL_LONG; + } } /** @@ -79,24 +90,6 @@ public void initializeUpdate(@NotNull final UpdateContext updateContext, } } - @Override - public void processChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { - Assert.neqNull(valuesChunk, "valuesChunk must not be null for a cumulative operator"); - final Context ctx = (Context) updateContext; - ctx.storeValuesChunk(valuesChunk); - ctx.timestampValueChunk = timestampValuesChunk; - for (int ii = 0; ii < valuesChunk.size(); ii++) { - push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); - ctx.outputValues.set(ii, ctx.curVal); - } - outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues, inputKeys); - } - void handleBadData(@NotNull final Context ctx, final boolean isNull, final boolean isNan, @@ -125,8 +118,7 @@ void handleBadData(@NotNull final Context ctx, } if (doReset) { - ctx.curVal = NULL_DOUBLE; - ctx.lastStamp = NULL_LONG; + ctx.reset(); } } @@ -145,8 +137,7 @@ void handleBadTime(@NotNull final Context ctx, final long dt) { } if (doReset) { - ctx.curVal = NULL_DOUBLE; - ctx.lastStamp = NULL_LONG; + ctx.reset(); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java index aff3a6ac4c9..b9b5d920e8b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java @@ -1,8 +1,14 @@ package io.deephaven.engine.table.impl.updateby.ema; import io.deephaven.api.updateby.OperationControl; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.ObjectChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -11,6 +17,58 @@ import static io.deephaven.util.QueryConstants.NULL_LONG; public class BigDecimalEMAOperator extends BigNumberEMAOperator { + public class Context extends BigNumberEMAOperator.Context { + protected Context(int chunkSize, ColumnSource inputSource) { + super(chunkSize, inputSource); + } + + @Override + public void push(long key, int pos) { + final BigDecimal input = objectValueChunk.get(pos); + if (timestampColumnName == null) { + // compute with ticks + if (input == null) { + handleBadData(this, true, false); + } else { + if (curVal == null) { + curVal = input; + } else { + curVal = curVal.multiply(alpha, control.bigValueContextOrDefault()) + .add(input.multiply(oneMinusAlpha, control.bigValueContextOrDefault()), + control.bigValueContextOrDefault()); + } + } + } else { + // compute with time + final long timestamp = timestampValueChunk.get(pos); + final boolean isNull = input == null; + final boolean isNullTime = timestamp == NULL_LONG; + if (isNull || isNullTime) { + handleBadData(this, isNull, isNullTime); + } else { + if (curVal == null) { + curVal = input; + lastStamp = timestamp; + } else { + final long dt = timestamp - lastStamp; + if (dt <= 0) { + handleBadTime(this, dt); + } else { + // alpha is dynamic, based on time + BigDecimal alpha = BigDecimal.valueOf(Math.exp(-dt / timeScaleUnits)); + BigDecimal oneMinusAlpha = + BigDecimal.ONE.subtract(alpha, control.bigValueContextOrDefault()); + + curVal = curVal.multiply(alpha, control.bigValueContextOrDefault()) + .add(input.multiply(oneMinusAlpha, control.bigValueContextOrDefault()), + control.bigValueContextOrDefault()); + lastStamp = timestamp; + } + } + } + } + } + } /** * An operator that computes an EMA from a int column using an exponential decay function. @@ -33,50 +91,9 @@ public BigDecimalEMAOperator(@NotNull final MatchPair pair, super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, redirContext); } + @NotNull @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - final BigDecimal input = ctx.objectValueChunk.get(pos); - if (timestampColumnName == null) { - // compute with ticks - if (input == null) { - handleBadData(ctx, true, false); - } else { - if (ctx.curVal == null) { - ctx.curVal = input; - } else { - ctx.curVal = ctx.curVal.multiply(alpha, control.bigValueContextOrDefault()) - .add(input.multiply(oneMinusAlpha, control.bigValueContextOrDefault()), - control.bigValueContextOrDefault()); - } - } - } else { - // compute with time - final long timestamp = ctx.timestampValueChunk.get(pos); - final boolean isNull = input == null; - final boolean isNullTime = timestamp == NULL_LONG; - if (isNull || isNullTime) { - handleBadData(ctx, isNull, isNullTime); - } else { - if (ctx.curVal == null) { - ctx.curVal = input; - ctx.lastStamp = timestamp; - } else { - final long dt = timestamp - ctx.lastStamp; - if (dt <= 0) { - handleBadTime(ctx, dt); - } else { - // alpha is dynamic, based on time - BigDecimal alpha = BigDecimal.valueOf(Math.exp(-dt / timeScaleUnits)); - BigDecimal oneMinusAlpha = BigDecimal.ONE.subtract(alpha, control.bigValueContextOrDefault()); - - ctx.curVal = ctx.curVal.multiply(alpha, control.bigValueContextOrDefault()) - .add(input.multiply(oneMinusAlpha, control.bigValueContextOrDefault()), - control.bigValueContextOrDefault()); - ctx.lastStamp = timestamp; - } - } - } - } + public UpdateContext makeUpdateContext(final int chunkSize, ColumnSource inputSource) { + return new Context(chunkSize, inputSource); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java index 8f867c40900..965f7632d29 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java @@ -1,6 +1,7 @@ package io.deephaven.engine.table.impl.updateby.ema; import io.deephaven.api.updateby.OperationControl; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import org.jetbrains.annotations.NotNull; @@ -12,6 +13,60 @@ import static io.deephaven.util.QueryConstants.NULL_LONG; public class BigIntegerEMAOperator extends BigNumberEMAOperator { + public class Context extends BigNumberEMAOperator.Context { + protected Context(int chunkSize, ColumnSource inputSource) { + super(chunkSize, inputSource); + } + + @Override + public void push(long key, int pos) { + final BigInteger input = objectValueChunk.get(pos); + if (timestampColumnName == null) { + // compute with ticks + if (input == null) { + handleBadData(this, true, false); + } else { + final BigDecimal decimalInput = new BigDecimal(input, control.bigValueContextOrDefault()); + if(curVal == null) { + curVal = decimalInput; + } else { + curVal = curVal.multiply(alpha, control.bigValueContextOrDefault()) + .add(decimalInput.multiply(oneMinusAlpha, control.bigValueContextOrDefault()), + control.bigValueContextOrDefault()); + } + } + } else { + // compute with time + final long timestamp = timestampValueChunk.get(pos); + final boolean isNull = input == null; + final boolean isNullTime = timestamp == NULL_LONG; + if (isNull || isNullTime) { + handleBadData(this, isNull, isNullTime); + } else { + final BigDecimal decimalInput = new BigDecimal(input, control.bigValueContextOrDefault()); + if(curVal == null) { + curVal = decimalInput; + lastStamp = timestamp; + } else { + final long dt = timestamp - lastStamp; + if(dt <= 0) { + handleBadTime(this, dt); + } else { + // alpha is dynamic, based on time + BigDecimal alpha = BigDecimal.valueOf(Math.exp(-dt / timeScaleUnits)); + BigDecimal oneMinusAlpha = BigDecimal.ONE.subtract(alpha, control.bigValueContextOrDefault()); + + curVal = curVal.multiply(alpha, control.bigValueContextOrDefault()) + .add(decimalInput.multiply(oneMinusAlpha, control.bigValueContextOrDefault()), + control.bigValueContextOrDefault()); + lastStamp = timestamp; + } + } + } + } + } + } + /** * An operator that computes an EMA from a int column using an exponential decay function. * @@ -34,52 +89,9 @@ public BigIntegerEMAOperator(@NotNull final MatchPair pair, super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, redirContext); } + @NotNull @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - final BigInteger input = ctx.objectValueChunk.get(pos); - if (timestampColumnName == null) { - // compute with ticks - if (input == null) { - handleBadData(ctx, true, false); - } else { - final BigDecimal decimalInput = new BigDecimal(input, control.bigValueContextOrDefault()); - if(ctx.curVal == null) { - ctx.curVal = decimalInput; - } else { - ctx.curVal = ctx.curVal.multiply(alpha, control.bigValueContextOrDefault()) - .add(decimalInput.multiply(oneMinusAlpha, control.bigValueContextOrDefault()), - control.bigValueContextOrDefault()); - } - } - } else { - // compute with time - final long timestamp = ctx.timestampValueChunk.get(pos); - final boolean isNull = input == null; - final boolean isNullTime = timestamp == NULL_LONG; - if (isNull || isNullTime) { - handleBadData(ctx, isNull, isNullTime); - } else { - final BigDecimal decimalInput = new BigDecimal(input, control.bigValueContextOrDefault()); - if(ctx.curVal == null) { - ctx.curVal = decimalInput; - ctx.lastStamp = timestamp; - } else { - final long dt = timestamp - ctx.lastStamp; - if(dt <= 0) { - handleBadTime(ctx, dt); - } else { - // alpha is dynamic, based on time - BigDecimal alpha = BigDecimal.valueOf(Math.exp(-dt / timeScaleUnits)); - BigDecimal oneMinusAlpha = BigDecimal.ONE.subtract(alpha, control.bigValueContextOrDefault()); - - ctx.curVal = ctx.curVal.multiply(alpha, control.bigValueContextOrDefault()) - .add(decimalInput.multiply(oneMinusAlpha, control.bigValueContextOrDefault()), - control.bigValueContextOrDefault()); - ctx.lastStamp = timestamp; - } - } - } - } + public UpdateContext makeUpdateContext(final int chunkSize, ColumnSource inputSource) { + return new Context(chunkSize, inputSource); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java index 0766ed45bdd..893bf1b0215 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java @@ -2,14 +2,10 @@ import io.deephaven.api.updateby.BadDataBehavior; import io.deephaven.api.updateby.OperationControl; -import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; @@ -21,7 +17,7 @@ import java.math.BigDecimal; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; -import static io.deephaven.util.QueryConstants.*; +import static io.deephaven.util.QueryConstants.NULL_LONG; public abstract class BigNumberEMAOperator extends BaseObjectUpdateByOperator { protected final OperationControl control; @@ -30,7 +26,7 @@ public abstract class BigNumberEMAOperator extends BaseObjectUpdateByOperator protected final BigDecimal oneMinusAlpha; - class Context extends BaseObjectUpdateByOperator.Context { + public abstract class Context extends BaseObjectUpdateByOperator.Context { protected final ColumnSource valueSource; public LongChunk timestampValueChunk; public ObjectChunk objectValueChunk; @@ -43,14 +39,25 @@ protected Context(int chunkSize, ColumnSource inputSource) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { objectValueChunk = valuesChunk.asObjectChunk(); } + @Override + public void setTimestampChunk(@NotNull final LongChunk valuesChunk) { + timestampValueChunk = valuesChunk; + } + @Override public boolean isValueValid(long atKey) { return valueSource.get(atKey) != null; } + + @Override + public void reset() { + curVal = null; + lastStamp = NULL_LONG; + } } /** @@ -79,12 +86,6 @@ public BigNumberEMAOperator(@NotNull final MatchPair pair, timestampColumnName == null ? BigDecimal.ONE.subtract(alpha, control.bigValueContextOrDefault()) : null; } - @NotNull - @Override - public UpdateContext makeUpdateContext(final int chunkSize, ColumnSource inputSource) { - return new Context(chunkSize, inputSource); - } - @Override public void initializeUpdate(@NotNull final UpdateContext updateContext, @NotNull final long firstUnmodifiedKey, long firstUnmodifiedTimestamp) { @@ -101,24 +102,6 @@ public void initializeUpdate(@NotNull final UpdateContext updateContext, } } - @Override - public void processChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { - Assert.neqNull(valuesChunk, "valuesChunk must not be null for a cumulative operator"); - final Context ctx = (Context) updateContext; - ctx.storeValuesChunk(valuesChunk); - ctx.timestampValueChunk = timestampValuesChunk; - for (int ii = 0; ii < valuesChunk.size(); ii++) { - push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); - ctx.outputValues.set(ii, ctx.curVal); - } - outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues, inputKeys); - } - void handleBadData(@NotNull final Context ctx, final boolean isNull, final boolean isNullTime) { @@ -138,8 +121,7 @@ void handleBadData(@NotNull final Context ctx, } if (doReset) { - ctx.curVal = null; - ctx.lastStamp = NULL_LONG; + ctx.reset(); } } @@ -158,8 +140,7 @@ void handleBadTime(@NotNull final Context ctx, final long dt) { } if (doReset) { - ctx.curVal = null; - ctx.lastStamp = NULL_LONG; + ctx.reset(); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java index d0e05580c0a..bceb3a109e6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java @@ -30,7 +30,7 @@ protected Context(int chunkSize, ColumnSource inputSource) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { byteValueChunk = valuesChunk.asByteChunk(); } @@ -38,6 +38,47 @@ public void storeValuesChunk(@NotNull final Chunk valuesChunk) { public boolean isValueValid(long atKey) { return valueSource.getByte(atKey) != NULL_BYTE; } + + @Override + public void push(long key, int pos) { + final byte input = byteValueChunk.get(pos); + if (timestampColumnName == null) { + // compute with ticks + if(input == NULL_BYTE) { + handleBadData(this, true, false, false); + } else { + if(curVal == NULL_DOUBLE) { + curVal = input; + } else { + curVal = alpha * curVal + (oneMinusAlpha * input); + } + } + } else { + // compute with time + final long timestamp = timestampValueChunk.get(pos); + //noinspection ConstantConditions + final boolean isNull = input == NULL_BYTE; + final boolean isNullTime = timestamp == NULL_LONG; + if(isNull || isNullTime) { + handleBadData(this, isNull, false, isNullTime); + } else { + if(curVal == NULL_DOUBLE) { + curVal = input; + lastStamp = timestamp; + } else { + final long dt = timestamp - lastStamp; + if(dt <= 0) { + handleBadTime(this, dt); + } else { + // alpha is dynamic, based on time + final double alpha = Math.exp(-dt / timeScaleUnits); + curVal = alpha * curVal + ((1 - alpha) * input); + lastStamp = timestamp; + } + } + } + } + } } /** @@ -68,47 +109,4 @@ public ByteEMAOperator(@NotNull final MatchPair pair, public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize, inputSource); } - - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - final byte input = ctx.byteValueChunk.get(pos); - if (timestampColumnName == null) { - // compute with ticks - if(input == NULL_BYTE) { - handleBadData(ctx, true, false, false); - } else { - if(ctx.curVal == NULL_DOUBLE) { - ctx.curVal = input; - } else { - ctx.curVal = alpha * ctx.curVal + (oneMinusAlpha * input); - } - } - } else { - // compute with time - final long timestamp = ctx.timestampValueChunk.get(pos); - //noinspection ConstantConditions - final boolean isNull = input == NULL_BYTE; - final boolean isNullTime = timestamp == NULL_LONG; - if(isNull || isNullTime) { - handleBadData(ctx, isNull, false, isNullTime); - } else { - if(ctx.curVal == NULL_DOUBLE) { - ctx.curVal = input; - ctx.lastStamp = timestamp; - } else { - final long dt = timestamp - ctx.lastStamp; - if(dt <= 0) { - handleBadTime(ctx, dt); - } else { - // alpha is dynamic, based on time - final double alpha = Math.exp(-dt / timeScaleUnits); - ctx.curVal = alpha * ctx.curVal + ((1 - alpha) * input); - ctx.lastStamp = timestamp; - } - } - } - } - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java index 181b025e18b..2a06a1fe187 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java @@ -30,7 +30,7 @@ protected Context(int chunkSize, ColumnSource inputSource) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { doubleValueChunk = valuesChunk.asDoubleChunk(); } @@ -45,6 +45,57 @@ public boolean isValueValid(long atKey) { // and the superclass will do the right thing. return !Double.isNaN(value) || control.onNanValueOrDefault() != BadDataBehavior.SKIP; } + + + @Override + public void push(long key, int pos) { + final double input = doubleValueChunk.get(pos); + final boolean isNull = input == NULL_DOUBLE; + final boolean isNan = Double.isNaN(input); + + if (timestampColumnName == null) { + // compute with ticks + if(isNull || isNan) { + handleBadData(this, isNull, isNan, false); + } else { + if(curVal == NULL_DOUBLE) { + curVal = input; + } else { + curVal = alpha * curVal + (oneMinusAlpha * input); + } + } + } else { + // compute with time + final long timestamp = timestampValueChunk.get(pos); + final boolean isNullTime = timestamp == NULL_LONG; + + + // Handle bad data first + if (isNull || isNan || isNullTime) { + handleBadData(this, isNull, isNan, isNullTime); + } else if (curVal == NULL_DOUBLE) { + // If the data looks good, and we have a null ema, just accept the current value + curVal = input; + lastStamp = timestamp; + } else { + final boolean currentPoisoned = Double.isNaN(curVal); + if (currentPoisoned && lastStamp == NULL_LONG) { + // If the current EMA was a NaN, we should accept the first good timestamp so that + // we can handle reset behavior properly in the following else + lastStamp = timestamp; + } else { + final long dt = timestamp - lastStamp; + if (dt <= 0) { + handleBadTime(this, dt); + } else if (!currentPoisoned) { + final double alpha = Math.exp(-dt / timeScaleUnits); + curVal = alpha * curVal + ((1 - alpha) * input); + lastStamp = timestamp; + } + } + } + } + } } /** @@ -75,56 +126,4 @@ public DoubleEMAOperator(@NotNull final MatchPair pair, public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize, inputSource); } - - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - final double input = ctx.doubleValueChunk.get(pos); - final boolean isNull = input == NULL_DOUBLE; - final boolean isNan = Double.isNaN(input); - - if (timestampColumnName == null) { - // compute with ticks - if(isNull || isNan) { - handleBadData(ctx, isNull, isNan, false); - } else { - if(ctx.curVal == NULL_DOUBLE) { - ctx.curVal = input; - } else { - ctx.curVal = alpha * ctx.curVal + (oneMinusAlpha * input); - } - } - } else { - // compute with time - final long timestamp = ctx.timestampValueChunk.get(pos); - final boolean isNullTime = timestamp == NULL_LONG; - - - // Handle bad data first - if (isNull || isNan || isNullTime) { - handleBadData(ctx, isNull, isNan, isNullTime); - } else if (ctx.curVal == NULL_DOUBLE) { - // If the data looks good, and we have a null ema, just accept the current value - ctx.curVal = input; - ctx.lastStamp = timestamp; - } else { - final boolean currentPoisoned = Double.isNaN(ctx.curVal); - if (currentPoisoned && ctx.lastStamp == NULL_LONG) { - // If the current EMA was a NaN, we should accept the first good timestamp so that - // we can handle reset behavior properly in the following else - ctx.lastStamp = timestamp; - } else { - final long dt = timestamp - ctx.lastStamp; - if (dt <= 0) { - handleBadTime(ctx, dt); - } else if (!currentPoisoned) { - final double alpha = Math.exp(-dt / timeScaleUnits); - ctx.curVal = alpha * ctx.curVal + ((1 - alpha) * input); - ctx.lastStamp = timestamp; - } - } - } - } - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java index f46a3cd8b62..dd25971cb46 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java @@ -25,7 +25,7 @@ protected Context(int chunkSize, ColumnSource inputSource) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { floatValueChunk = valuesChunk.asFloatChunk(); } @@ -40,6 +40,57 @@ public boolean isValueValid(long atKey) { // and the superclass will do the right thing. return !Float.isNaN(value) || control.onNanValueOrDefault() != BadDataBehavior.SKIP; } + + + @Override + public void push(long key, int pos) { + final float input = floatValueChunk.get(pos); + final boolean isNull = input == NULL_FLOAT; + final boolean isNan = Float.isNaN(input); + + if (timestampColumnName == null) { + // compute with ticks + if(isNull || isNan) { + handleBadData(this, isNull, isNan, false); + } else { + if(curVal == NULL_DOUBLE) { + curVal = input; + } else { + curVal = alpha * curVal + (oneMinusAlpha * input); + } + } + } else { + // compute with time + final long timestamp = timestampValueChunk.get(pos); + final boolean isNullTime = timestamp == NULL_LONG; + + + // Handle bad data first + if (isNull || isNan || isNullTime) { + handleBadData(this, isNull, isNan, isNullTime); + } else if (curVal == NULL_DOUBLE) { + // If the data looks good, and we have a null ema, just accept the current value + curVal = input; + lastStamp = timestamp; + } else { + final boolean currentPoisoned = Double.isNaN(curVal); + if (currentPoisoned && lastStamp == NULL_LONG) { + // If the current EMA was a NaN, we should accept the first good timestamp so that + // we can handle reset behavior properly in the following else + lastStamp = timestamp; + } else { + final long dt = timestamp - lastStamp; + if (dt <= 0) { + handleBadTime(this, dt); + } else if (!currentPoisoned) { + final double alpha = Math.exp(-dt / timeScaleUnits); + curVal = alpha * curVal + ((1 - alpha) * input); + lastStamp = timestamp; + } + } + } + } + } } /** @@ -70,56 +121,4 @@ public FloatEMAOperator(@NotNull final MatchPair pair, public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize, inputSource); } - - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - final float input = ctx.floatValueChunk.get(pos); - final boolean isNull = input == NULL_FLOAT; - final boolean isNan = Float.isNaN(input); - - if (timestampColumnName == null) { - // compute with ticks - if(isNull || isNan) { - handleBadData(ctx, isNull, isNan, false); - } else { - if(ctx.curVal == NULL_DOUBLE) { - ctx.curVal = input; - } else { - ctx.curVal = alpha * ctx.curVal + (oneMinusAlpha * input); - } - } - } else { - // compute with time - final long timestamp = ctx.timestampValueChunk.get(pos); - final boolean isNullTime = timestamp == NULL_LONG; - - - // Handle bad data first - if (isNull || isNan || isNullTime) { - handleBadData(ctx, isNull, isNan, isNullTime); - } else if (ctx.curVal == NULL_DOUBLE) { - // If the data looks good, and we have a null ema, just accept the current value - ctx.curVal = input; - ctx.lastStamp = timestamp; - } else { - final boolean currentPoisoned = Double.isNaN(ctx.curVal); - if (currentPoisoned && ctx.lastStamp == NULL_LONG) { - // If the current EMA was a NaN, we should accept the first good timestamp so that - // we can handle reset behavior properly in the following else - ctx.lastStamp = timestamp; - } else { - final long dt = timestamp - ctx.lastStamp; - if (dt <= 0) { - handleBadTime(ctx, dt); - } else if (!currentPoisoned) { - final double alpha = Math.exp(-dt / timeScaleUnits); - ctx.curVal = alpha * ctx.curVal + ((1 - alpha) * input); - ctx.lastStamp = timestamp; - } - } - } - } - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java index ba5c5bc145d..df030e6a657 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java @@ -30,7 +30,7 @@ protected Context(int chunkSize, ColumnSource inputSource) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { intValueChunk = valuesChunk.asIntChunk(); } @@ -38,6 +38,47 @@ public void storeValuesChunk(@NotNull final Chunk valuesChunk) { public boolean isValueValid(long atKey) { return valueSource.getInt(atKey) != NULL_INT; } + + @Override + public void push(long key, int pos) { + final int input = intValueChunk.get(pos); + if (timestampColumnName == null) { + // compute with ticks + if(input == NULL_INT) { + handleBadData(this, true, false, false); + } else { + if(curVal == NULL_DOUBLE) { + curVal = input; + } else { + curVal = alpha * curVal + (oneMinusAlpha * input); + } + } + } else { + // compute with time + final long timestamp = timestampValueChunk.get(pos); + //noinspection ConstantConditions + final boolean isNull = input == NULL_INT; + final boolean isNullTime = timestamp == NULL_LONG; + if(isNull || isNullTime) { + handleBadData(this, isNull, false, isNullTime); + } else { + if(curVal == NULL_DOUBLE) { + curVal = input; + lastStamp = timestamp; + } else { + final long dt = timestamp - lastStamp; + if(dt <= 0) { + handleBadTime(this, dt); + } else { + // alpha is dynamic, based on time + final double alpha = Math.exp(-dt / timeScaleUnits); + curVal = alpha * curVal + ((1 - alpha) * input); + lastStamp = timestamp; + } + } + } + } + } } /** @@ -68,47 +109,4 @@ public IntEMAOperator(@NotNull final MatchPair pair, public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize, inputSource); } - - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - final int input = ctx.intValueChunk.get(pos); - if (timestampColumnName == null) { - // compute with ticks - if(input == NULL_INT) { - handleBadData(ctx, true, false, false); - } else { - if(ctx.curVal == NULL_DOUBLE) { - ctx.curVal = input; - } else { - ctx.curVal = alpha * ctx.curVal + (oneMinusAlpha * input); - } - } - } else { - // compute with time - final long timestamp = ctx.timestampValueChunk.get(pos); - //noinspection ConstantConditions - final boolean isNull = input == NULL_INT; - final boolean isNullTime = timestamp == NULL_LONG; - if(isNull || isNullTime) { - handleBadData(ctx, isNull, false, isNullTime); - } else { - if(ctx.curVal == NULL_DOUBLE) { - ctx.curVal = input; - ctx.lastStamp = timestamp; - } else { - final long dt = timestamp - ctx.lastStamp; - if(dt <= 0) { - handleBadTime(ctx, dt); - } else { - // alpha is dynamic, based on time - final double alpha = Math.exp(-dt / timeScaleUnits); - ctx.curVal = alpha * ctx.curVal + ((1 - alpha) * input); - ctx.lastStamp = timestamp; - } - } - } - } - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java index 58df703a843..71bcc44b81b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java @@ -30,7 +30,7 @@ protected Context(int chunkSize, ColumnSource inputSource) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { longValueChunk = valuesChunk.asLongChunk(); } @@ -38,6 +38,47 @@ public void storeValuesChunk(@NotNull final Chunk valuesChunk) { public boolean isValueValid(long atKey) { return valueSource.getLong(atKey) != NULL_LONG; } + + @Override + public void push(long key, int pos) { + final long input = longValueChunk.get(pos); + if (timestampColumnName == null) { + // compute with ticks + if(input == NULL_LONG) { + handleBadData(this, true, false, false); + } else { + if(curVal == NULL_DOUBLE) { + curVal = input; + } else { + curVal = alpha * curVal + (oneMinusAlpha * input); + } + } + } else { + // compute with time + final long timestamp = timestampValueChunk.get(pos); + //noinspection ConstantConditions + final boolean isNull = input == NULL_LONG; + final boolean isNullTime = timestamp == NULL_LONG; + if(isNull || isNullTime) { + handleBadData(this, isNull, false, isNullTime); + } else { + if(curVal == NULL_DOUBLE) { + curVal = input; + lastStamp = timestamp; + } else { + final long dt = timestamp - lastStamp; + if(dt <= 0) { + handleBadTime(this, dt); + } else { + // alpha is dynamic, based on time + final double alpha = Math.exp(-dt / timeScaleUnits); + curVal = alpha * curVal + ((1 - alpha) * input); + lastStamp = timestamp; + } + } + } + } + } } /** @@ -68,47 +109,4 @@ public LongEMAOperator(@NotNull final MatchPair pair, public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize, inputSource); } - - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - final long input = ctx.longValueChunk.get(pos); - if (timestampColumnName == null) { - // compute with ticks - if(input == NULL_LONG) { - handleBadData(ctx, true, false, false); - } else { - if(ctx.curVal == NULL_DOUBLE) { - ctx.curVal = input; - } else { - ctx.curVal = alpha * ctx.curVal + (oneMinusAlpha * input); - } - } - } else { - // compute with time - final long timestamp = ctx.timestampValueChunk.get(pos); - //noinspection ConstantConditions - final boolean isNull = input == NULL_LONG; - final boolean isNullTime = timestamp == NULL_LONG; - if(isNull || isNullTime) { - handleBadData(ctx, isNull, false, isNullTime); - } else { - if(ctx.curVal == NULL_DOUBLE) { - ctx.curVal = input; - ctx.lastStamp = timestamp; - } else { - final long dt = timestamp - ctx.lastStamp; - if(dt <= 0) { - handleBadTime(ctx, dt); - } else { - // alpha is dynamic, based on time - final double alpha = Math.exp(-dt / timeScaleUnits); - ctx.curVal = alpha * ctx.curVal + ((1 - alpha) * input); - ctx.lastStamp = timestamp; - } - } - } - } - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java index 30491c1e7f0..c255a21977f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java @@ -25,7 +25,7 @@ protected Context(int chunkSize, ColumnSource inputSource) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { shortValueChunk = valuesChunk.asShortChunk(); } @@ -33,6 +33,47 @@ public void storeValuesChunk(@NotNull final Chunk valuesChunk) { public boolean isValueValid(long atKey) { return valueSource.getShort(atKey) != NULL_SHORT; } + + @Override + public void push(long key, int pos) { + final short input = shortValueChunk.get(pos); + if (timestampColumnName == null) { + // compute with ticks + if(input == NULL_SHORT) { + handleBadData(this, true, false, false); + } else { + if(curVal == NULL_DOUBLE) { + curVal = input; + } else { + curVal = alpha * curVal + (oneMinusAlpha * input); + } + } + } else { + // compute with time + final long timestamp = timestampValueChunk.get(pos); + //noinspection ConstantConditions + final boolean isNull = input == NULL_SHORT; + final boolean isNullTime = timestamp == NULL_LONG; + if(isNull || isNullTime) { + handleBadData(this, isNull, false, isNullTime); + } else { + if(curVal == NULL_DOUBLE) { + curVal = input; + lastStamp = timestamp; + } else { + final long dt = timestamp - lastStamp; + if(dt <= 0) { + handleBadTime(this, dt); + } else { + // alpha is dynamic, based on time + final double alpha = Math.exp(-dt / timeScaleUnits); + curVal = alpha * curVal + ((1 - alpha) * input); + lastStamp = timestamp; + } + } + } + } + } } /** @@ -63,47 +104,4 @@ public ShortEMAOperator(@NotNull final MatchPair pair, public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize, inputSource); } - - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - final short input = ctx.shortValueChunk.get(pos); - if (timestampColumnName == null) { - // compute with ticks - if(input == NULL_SHORT) { - handleBadData(ctx, true, false, false); - } else { - if(ctx.curVal == NULL_DOUBLE) { - ctx.curVal = input; - } else { - ctx.curVal = alpha * ctx.curVal + (oneMinusAlpha * input); - } - } - } else { - // compute with time - final long timestamp = ctx.timestampValueChunk.get(pos); - //noinspection ConstantConditions - final boolean isNull = input == NULL_SHORT; - final boolean isNullTime = timestamp == NULL_LONG; - if(isNull || isNullTime) { - handleBadData(ctx, isNull, false, isNullTime); - } else { - if(ctx.curVal == NULL_DOUBLE) { - ctx.curVal = input; - ctx.lastStamp = timestamp; - } else { - final long dt = timestamp - ctx.lastStamp; - if(dt <= 0) { - handleBadTime(ctx, dt); - } else { - // alpha is dynamic, based on time - final double alpha = Math.exp(-dt / timeScaleUnits); - ctx.curVal = alpha * ctx.curVal + ((1 - alpha) * input); - ctx.lastStamp = timestamp; - } - } - } - } - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java index d16226d0acc..b28a5f74c71 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java @@ -35,9 +35,22 @@ protected Context(int chunkSize) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { booleanValueChunk = valuesChunk.asByteChunk(); } + + @Override + public void push(long key, int pos) { + byte currentVal = booleanValueChunk.get(pos); + if(currentVal != NULL_BOOLEAN_AS_BYTE) { + curVal = currentVal; + } + } + + @Override + public void reset() { + curVal = NULL_BOOLEAN_AS_BYTE; + } } public BooleanFillByOperator(@NotNull final MatchPair fillPair, @@ -56,16 +69,6 @@ public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSourc return new Context(chunkSize); } - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - byte currentVal = ctx.booleanValueChunk.get(pos); - if(currentVal != NULL_BOOLEAN_AS_BYTE) { - ctx.curVal = currentVal; - } - } - // region extra-methods @Override protected byte getNullValue() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java index 23c10ceabf9..8cdd33dd894 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java @@ -28,9 +28,22 @@ protected Context(int chunkSize) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { byteValueChunk = valuesChunk.asByteChunk(); } + + @Override + public void push(long key, int pos) { + byte currentVal = byteValueChunk.get(pos); + if(currentVal != NULL_BYTE) { + curVal = currentVal; + } + } + + @Override + public void reset() { + curVal = NULL_BYTE; + } } public ByteFillByOperator(@NotNull final MatchPair fillPair, @@ -49,16 +62,6 @@ public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSourc return new Context(chunkSize); } - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - byte currentVal = ctx.byteValueChunk.get(pos); - if(currentVal != NULL_BYTE) { - ctx.curVal = currentVal; - } - } - // region extra-methods // endregion extra-methods } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java index dba1c56b99f..0d84441d629 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java @@ -23,9 +23,22 @@ protected Context(int chunkSize) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { charValueChunk = valuesChunk.asCharChunk(); } + + @Override + public void push(long key, int pos) { + char currentVal = charValueChunk.get(pos); + if(currentVal != NULL_CHAR) { + curVal = currentVal; + } + } + + @Override + public void reset() { + curVal = NULL_CHAR; + } } public CharFillByOperator(@NotNull final MatchPair fillPair, @@ -44,16 +57,6 @@ public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSourc return new Context(chunkSize); } - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - char currentVal = ctx.charValueChunk.get(pos); - if(currentVal != NULL_CHAR) { - ctx.curVal = currentVal; - } - } - // region extra-methods // endregion extra-methods } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java index 9408fd6a670..78ddd49ccf9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java @@ -28,9 +28,22 @@ protected Context(int chunkSize) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { doubleValueChunk = valuesChunk.asDoubleChunk(); } + + @Override + public void push(long key, int pos) { + double currentVal = doubleValueChunk.get(pos); + if(currentVal != NULL_DOUBLE) { + curVal = currentVal; + } + } + + @Override + public void reset() { + curVal = NULL_DOUBLE; + } } public DoubleFillByOperator(@NotNull final MatchPair fillPair, @@ -49,16 +62,6 @@ public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSourc return new Context(chunkSize); } - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - double currentVal = ctx.doubleValueChunk.get(pos); - if(currentVal != NULL_DOUBLE) { - ctx.curVal = currentVal; - } - } - // region extra-methods // endregion extra-methods } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java index 415f1114824..9de8bc4e639 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java @@ -28,9 +28,22 @@ protected Context(int chunkSize) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { floatValueChunk = valuesChunk.asFloatChunk(); } + + @Override + public void push(long key, int pos) { + float currentVal = floatValueChunk.get(pos); + if(currentVal != NULL_FLOAT) { + curVal = currentVal; + } + } + + @Override + public void reset() { + curVal = NULL_FLOAT; + } } public FloatFillByOperator(@NotNull final MatchPair fillPair, @@ -49,16 +62,6 @@ public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSourc return new Context(chunkSize); } - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - float currentVal = ctx.floatValueChunk.get(pos); - if(currentVal != NULL_FLOAT) { - ctx.curVal = currentVal; - } - } - // region extra-methods // endregion extra-methods } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java index 2e67362f337..c770d3776e3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java @@ -28,9 +28,22 @@ protected Context(int chunkSize) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { intValueChunk = valuesChunk.asIntChunk(); } + + @Override + public void push(long key, int pos) { + int currentVal = intValueChunk.get(pos); + if(currentVal != NULL_INT) { + curVal = currentVal; + } + } + + @Override + public void reset() { + curVal = NULL_INT; + } } public IntFillByOperator(@NotNull final MatchPair fillPair, @@ -49,16 +62,6 @@ public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSourc return new Context(chunkSize); } - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - int currentVal = ctx.intValueChunk.get(pos); - if(currentVal != NULL_INT) { - ctx.curVal = currentVal; - } - } - // region extra-methods // endregion extra-methods } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java index c71af219cad..36e6a50e10a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java @@ -36,9 +36,22 @@ protected Context(int chunkSize) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { longValueChunk = valuesChunk.asLongChunk(); } + + @Override + public void push(long key, int pos) { + long currentVal = longValueChunk.get(pos); + if(currentVal != NULL_LONG) { + curVal = currentVal; + } + } + + @Override + public void reset() { + curVal = NULL_LONG; + } } public LongFillByOperator(@NotNull final MatchPair fillPair, @@ -59,16 +72,6 @@ public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSourc return new Context(chunkSize); } - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - long currentVal = ctx.longValueChunk.get(pos); - if(currentVal != NULL_LONG) { - ctx.curVal = currentVal; - } - } - // region extra-methods @NotNull @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java index 8f4561fc24e..d18af2a20ef 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java @@ -27,9 +27,22 @@ protected Context(int chunkSize) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { ObjectValueChunk = valuesChunk.asObjectChunk(); } + + @Override + public void push(long key, int pos) { + T currentVal = ObjectValueChunk.get(pos); + if(currentVal != null) { + curVal = currentVal; + } + } + + @Override + public void reset() { + curVal = null; + } } public ObjectFillByOperator(@NotNull final MatchPair fillPair, @@ -49,16 +62,6 @@ public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSourc return new Context(chunkSize); } - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - T currentVal = ctx.ObjectValueChunk.get(pos); - if(currentVal != null) { - ctx.curVal = currentVal; - } - } - // region extra-methods // endregion extra-methods } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java index c76c06773eb..696cad5280a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java @@ -28,9 +28,22 @@ protected Context(int chunkSize) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { shortValueChunk = valuesChunk.asShortChunk(); } + + @Override + public void push(long key, int pos) { + short currentVal = shortValueChunk.get(pos); + if(currentVal != NULL_SHORT) { + curVal = currentVal; + } + } + + @Override + public void reset() { + curVal = NULL_SHORT; + } } public ShortFillByOperator(@NotNull final MatchPair fillPair, @@ -49,16 +62,6 @@ public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSourc return new Context(chunkSize); } - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - short currentVal = ctx.shortValueChunk.get(pos); - if(currentVal != NULL_SHORT) { - ctx.curVal = currentVal; - } - } - // region extra-methods // endregion extra-methods } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java index 1e6f4ab9144..0c5a87d17d7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java @@ -10,19 +10,20 @@ import io.deephaven.engine.table.impl.sources.ByteSparseArraySource; import io.deephaven.engine.table.WritableColumnSource; -import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableByteChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.table.*; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.table.ChunkSink; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import java.util.Collections; import java.util.Map; @@ -38,25 +39,39 @@ public abstract class BaseByteUpdateByOperator extends UpdateByCumulativeOperato final byte nullValue; // endregion extra-fields - protected class Context extends UpdateByCumulativeOperator.Context { - public final ChunkSink.FillFromContext fillContext; + protected abstract class Context extends UpdateByCumulativeOperator.Context { + public final ChunkSink.FillFromContext outputFillContext; public final WritableByteChunk outputValues; public byte curVal = NULL_BYTE; protected Context(final int chunkSize) { super(chunkSize); - this.fillContext = outputSource.makeFillFromContext(chunkSize); + this.outputFillContext = outputSource.makeFillFromContext(chunkSize); this.outputValues = WritableByteChunk.makeWritableChunk(chunkSize); } - public void storeValuesChunk(@NotNull final Chunk valuesChunk) {} - @Override public void close() { super.close(); outputValues.close(); - fillContext.close(); + outputFillContext.close(); + } + + @Override + public void setValuesChunk(@NotNull final Chunk valuesChunk) {} + + @Override + public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} + + @Override + public void writeToOutputChunk(int outIdx) { + outputValues.set(outIdx, curVal); + } + + @Override + public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { + outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); } } @@ -92,11 +107,6 @@ public BaseByteUpdateByOperator(@NotNull final MatchPair pair, // endregion constructor } - @Override - public void reset(UpdateContext context) { - final Context ctx = (Context)context; - ctx.curVal = NULL_BYTE; - } // region extra-methods protected byte getNullValue() { @@ -119,7 +129,7 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie if (firstUnmodifiedKey != NULL_ROW_KEY) { ctx.curVal = outputSource.getByte(firstUnmodifiedKey); } else { - reset(ctx); + ctx.reset(); } // If we're redirected we have to make sure we tell the output source it's actual size, or we're going @@ -150,25 +160,6 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d } // endregion Shifts - // region Processing - @Override - public void processChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { - Assert.neqNull(valuesChunk, "valuesChunk must not be null for a cumulative operator"); - final Context ctx = (Context) updateContext; - ctx.storeValuesChunk(valuesChunk); - for (int ii = 0; ii < valuesChunk.size(); ii++) { - push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); - ctx.outputValues.set(ii, ctx.curVal); - } - outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues, inputKeys); - } - // endregion - @NotNull @Override public Map> getOutputColumns() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java index b8841471f82..0d251168aca 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java @@ -1,18 +1,19 @@ package io.deephaven.engine.table.impl.updateby.internal; -import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableCharChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.table.*; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.table.ChunkSink; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import java.util.Collections; import java.util.Map; @@ -27,25 +28,39 @@ public abstract class BaseCharUpdateByOperator extends UpdateByCumulativeOperato // region extra-fields // endregion extra-fields - protected class Context extends UpdateByCumulativeOperator.Context { - public final ChunkSink.FillFromContext fillContext; + protected abstract class Context extends UpdateByCumulativeOperator.Context { + public final ChunkSink.FillFromContext outputFillContext; public final WritableCharChunk outputValues; public char curVal = NULL_CHAR; protected Context(final int chunkSize) { super(chunkSize); - this.fillContext = outputSource.makeFillFromContext(chunkSize); + this.outputFillContext = outputSource.makeFillFromContext(chunkSize); this.outputValues = WritableCharChunk.makeWritableChunk(chunkSize); } - public void storeValuesChunk(@NotNull final Chunk valuesChunk) {} - @Override public void close() { super.close(); outputValues.close(); - fillContext.close(); + outputFillContext.close(); + } + + @Override + public void setValuesChunk(@NotNull final Chunk valuesChunk) {} + + @Override + public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} + + @Override + public void writeToOutputChunk(int outIdx) { + outputValues.set(outIdx, curVal); + } + + @Override + public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { + outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); } } @@ -80,11 +95,6 @@ public BaseCharUpdateByOperator(@NotNull final MatchPair pair, // endregion constructor } - @Override - public void reset(UpdateContext context) { - final Context ctx = (Context)context; - ctx.curVal = NULL_CHAR; - } // region extra-methods // endregion extra-methods @@ -95,7 +105,7 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie if (firstUnmodifiedKey != NULL_ROW_KEY) { ctx.curVal = outputSource.getChar(firstUnmodifiedKey); } else { - reset(ctx); + ctx.reset(); } // If we're redirected we have to make sure we tell the output source it's actual size, or we're going @@ -122,25 +132,6 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d } // endregion Shifts - // region Processing - @Override - public void processChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { - Assert.neqNull(valuesChunk, "valuesChunk must not be null for a cumulative operator"); - final Context ctx = (Context) updateContext; - ctx.storeValuesChunk(valuesChunk); - for (int ii = 0; ii < valuesChunk.size(); ii++) { - push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); - ctx.outputValues.set(ii, ctx.curVal); - } - outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues, inputKeys); - } - // endregion - @NotNull @Override public Map> getOutputColumns() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java index 72e10ff34ff..cd2c9153412 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java @@ -33,25 +33,39 @@ public abstract class BaseDoubleUpdateByOperator extends UpdateByCumulativeOpera // region extra-fields // endregion extra-fields - protected class Context extends UpdateByCumulativeOperator.Context { - public final ChunkSink.FillFromContext fillContext; + protected abstract class Context extends UpdateByCumulativeOperator.Context { + public final ChunkSink.FillFromContext outputFillContext; public final WritableDoubleChunk outputValues; - public double curVal = NULL_CHAR; + public double curVal = NULL_DOUBLE; protected Context(final int chunkSize) { super(chunkSize); - this.fillContext = outputSource.makeFillFromContext(chunkSize); + this.outputFillContext = outputSource.makeFillFromContext(chunkSize); this.outputValues = WritableDoubleChunk.makeWritableChunk(chunkSize); } - public void storeValuesChunk(@NotNull final Chunk valuesChunk) {} - @Override public void close() { super.close(); outputValues.close(); - fillContext.close(); + outputFillContext.close(); + } + + @Override + public void setValuesChunk(@NotNull final Chunk valuesChunk) {} + + @Override + public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} + + @Override + public void writeToOutputChunk(int outIdx) { + outputValues.set(outIdx, curVal); + } + + @Override + public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { + outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); } } @@ -86,12 +100,6 @@ public BaseDoubleUpdateByOperator(@NotNull final MatchPair pair, // endregion constructor } - @Override - public void reset(UpdateContext context) { - final Context ctx = (Context)context; - ctx.curVal = NULL_DOUBLE; - } - // region extra-methods // endregion extra-methods @@ -101,7 +109,7 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie if (firstUnmodifiedKey != NULL_ROW_KEY) { ctx.curVal = outputSource.getDouble(firstUnmodifiedKey); } else { - reset(ctx); + ctx.reset(); } // If we're redirected we have to make sure we tell the output source it's actual size, or we're going @@ -128,25 +136,6 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long } // endregion - // region Processing - @Override - public void processChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { - Assert.neqNull(valuesChunk, "valuesChunk must not be null for a cumulative operator"); - final Context ctx = (Context) updateContext; - ctx.storeValuesChunk(valuesChunk); - for (int ii = 0; ii < valuesChunk.size(); ii++) { - push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); - ctx.outputValues.set(ii, ctx.curVal); - } - outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues, inputKeys); - } - // endregion - @NotNull @Override public Map> getOutputColumns() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java index 00aa692f2a2..cdfdf12b0b1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java @@ -28,25 +28,39 @@ public abstract class BaseFloatUpdateByOperator extends UpdateByCumulativeOperat // region extra-fields // endregion extra-fields - protected class Context extends UpdateByCumulativeOperator.Context { - public final ChunkSink.FillFromContext fillContext; + protected abstract class Context extends UpdateByCumulativeOperator.Context { + public final ChunkSink.FillFromContext outputFillContext; public final WritableFloatChunk outputValues; - public float curVal = NULL_CHAR; + public float curVal = NULL_FLOAT; protected Context(final int chunkSize) { super(chunkSize); - this.fillContext = outputSource.makeFillFromContext(chunkSize); + this.outputFillContext = outputSource.makeFillFromContext(chunkSize); this.outputValues = WritableFloatChunk.makeWritableChunk(chunkSize); } - public void storeValuesChunk(@NotNull final Chunk valuesChunk) {} - @Override public void close() { super.close(); outputValues.close(); - fillContext.close(); + outputFillContext.close(); + } + + @Override + public void setValuesChunk(@NotNull final Chunk valuesChunk) {} + + @Override + public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} + + @Override + public void writeToOutputChunk(int outIdx) { + outputValues.set(outIdx, curVal); + } + + @Override + public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { + outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); } } @@ -81,12 +95,6 @@ public BaseFloatUpdateByOperator(@NotNull final MatchPair pair, // endregion constructor } - @Override - public void reset(UpdateContext context) { - final Context ctx = (Context)context; - ctx.curVal = NULL_FLOAT; - } - // region extra-methods // endregion extra-methods @@ -96,7 +104,7 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie if (firstUnmodifiedKey != NULL_ROW_KEY) { ctx.curVal = outputSource.getFloat(firstUnmodifiedKey); } else { - reset(ctx); + ctx.reset(); } // If we're redirected we have to make sure we tell the output source it's actual size, or we're going @@ -123,25 +131,6 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long } // endregion - // region Processing - @Override - public void processChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { - Assert.neqNull(valuesChunk, "valuesChunk must not be null for a cumulative operator"); - final Context ctx = (Context) updateContext; - ctx.storeValuesChunk(valuesChunk); - for (int ii = 0; ii < valuesChunk.size(); ii++) { - push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); - ctx.outputValues.set(ii, ctx.curVal); - } - outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues, inputKeys); - } - // endregion - @NotNull @Override public Map> getOutputColumns() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java index 1a45190a05f..f46caaadf2b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java @@ -5,19 +5,20 @@ */ package io.deephaven.engine.table.impl.updateby.internal; -import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.table.*; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.table.ChunkSink; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import java.util.Collections; import java.util.Map; @@ -32,25 +33,39 @@ public abstract class BaseIntUpdateByOperator extends UpdateByCumulativeOperator // region extra-fields // endregion extra-fields - protected class Context extends UpdateByCumulativeOperator.Context { - public final ChunkSink.FillFromContext fillContext; + protected abstract class Context extends UpdateByCumulativeOperator.Context { + public final ChunkSink.FillFromContext outputFillContext; public final WritableIntChunk outputValues; public int curVal = NULL_INT; protected Context(final int chunkSize) { super(chunkSize); - this.fillContext = outputSource.makeFillFromContext(chunkSize); + this.outputFillContext = outputSource.makeFillFromContext(chunkSize); this.outputValues = WritableIntChunk.makeWritableChunk(chunkSize); } - public void storeValuesChunk(@NotNull final Chunk valuesChunk) {} - @Override public void close() { super.close(); outputValues.close(); - fillContext.close(); + outputFillContext.close(); + } + + @Override + public void setValuesChunk(@NotNull final Chunk valuesChunk) {} + + @Override + public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} + + @Override + public void writeToOutputChunk(int outIdx) { + outputValues.set(outIdx, curVal); + } + + @Override + public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { + outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); } } @@ -85,11 +100,6 @@ public BaseIntUpdateByOperator(@NotNull final MatchPair pair, // endregion constructor } - @Override - public void reset(UpdateContext context) { - final Context ctx = (Context)context; - ctx.curVal = NULL_INT; - } // region extra-methods // endregion extra-methods @@ -100,7 +110,7 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie if (firstUnmodifiedKey != NULL_ROW_KEY) { ctx.curVal = outputSource.getInt(firstUnmodifiedKey); } else { - reset(ctx); + ctx.reset(); } // If we're redirected we have to make sure we tell the output source it's actual size, or we're going @@ -127,25 +137,6 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d } // endregion Shifts - // region Processing - @Override - public void processChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { - Assert.neqNull(valuesChunk, "valuesChunk must not be null for a cumulative operator"); - final Context ctx = (Context) updateContext; - ctx.storeValuesChunk(valuesChunk); - for (int ii = 0; ii < valuesChunk.size(); ii++) { - push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); - ctx.outputValues.set(ii, ctx.curVal); - } - outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues, inputKeys); - } - // endregion - @NotNull @Override public Map> getOutputColumns() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java index 314d625d00a..c921c5e227c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java @@ -5,19 +5,20 @@ */ package io.deephaven.engine.table.impl.updateby.internal; -import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableLongChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.table.*; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.table.ChunkSink; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import java.util.Collections; import java.util.Map; @@ -32,25 +33,39 @@ public abstract class BaseLongUpdateByOperator extends UpdateByCumulativeOperato // region extra-fields // endregion extra-fields - protected class Context extends UpdateByCumulativeOperator.Context { - public final ChunkSink.FillFromContext fillContext; + protected abstract class Context extends UpdateByCumulativeOperator.Context { + public final ChunkSink.FillFromContext outputFillContext; public final WritableLongChunk outputValues; public long curVal = NULL_LONG; protected Context(final int chunkSize) { super(chunkSize); - this.fillContext = outputSource.makeFillFromContext(chunkSize); + this.outputFillContext = outputSource.makeFillFromContext(chunkSize); this.outputValues = WritableLongChunk.makeWritableChunk(chunkSize); } - public void storeValuesChunk(@NotNull final Chunk valuesChunk) {} - @Override public void close() { super.close(); outputValues.close(); - fillContext.close(); + outputFillContext.close(); + } + + @Override + public void setValuesChunk(@NotNull final Chunk valuesChunk) {} + + @Override + public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} + + @Override + public void writeToOutputChunk(int outIdx) { + outputValues.set(outIdx, curVal); + } + + @Override + public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { + outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); } } @@ -85,11 +100,6 @@ public BaseLongUpdateByOperator(@NotNull final MatchPair pair, // endregion constructor } - @Override - public void reset(UpdateContext context) { - final Context ctx = (Context)context; - ctx.curVal = NULL_LONG; - } // region extra-methods // endregion extra-methods @@ -100,7 +110,7 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie if (firstUnmodifiedKey != NULL_ROW_KEY) { ctx.curVal = outputSource.getLong(firstUnmodifiedKey); } else { - reset(ctx); + ctx.reset(); } // If we're redirected we have to make sure we tell the output source it's actual size, or we're going @@ -127,25 +137,6 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d } // endregion Shifts - // region Processing - @Override - public void processChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { - Assert.neqNull(valuesChunk, "valuesChunk must not be null for a cumulative operator"); - final Context ctx = (Context) updateContext; - ctx.storeValuesChunk(valuesChunk); - for (int ii = 0; ii < valuesChunk.size(); ii++) { - push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); - ctx.outputValues.set(ii, ctx.curVal); - } - outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues, inputKeys); - } - // endregion - @NotNull @Override public Map> getOutputColumns() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java index 0550220fff3..10881dd0bbe 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java @@ -24,9 +24,25 @@ protected Context(int chunkSize) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { objectValueChunk = valuesChunk.asObjectChunk(); } + + @Override + public void push(long key, int pos) { + // read the value from the values chunk + final T currentVal = objectValueChunk.get(pos); + if(curVal == null) { + curVal = currentVal; + } else if(currentVal != null) { + curVal = doOperation(curVal, currentVal); + } + } + + @Override + public void reset() { + curVal = null; + } } public BaseObjectBinaryOperator(@NotNull final Class type, @@ -43,36 +59,4 @@ public BaseObjectBinaryOperator(@NotNull final Class type, public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } - - // region Processing - @Override - public void processChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { - Assert.neqNull(valuesChunk, "valuesChunk must not be null for a cumulative operator"); - final Context ctx = (Context) updateContext; - ctx.storeValuesChunk(valuesChunk); - for (int ii = 0; ii < valuesChunk.size(); ii++) { - push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); - ctx.outputValues.set(ii, ctx.curVal); - } - outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues, inputKeys); - } - // endregion - - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - // read the value from the values chunk - final T currentVal = ctx.objectValueChunk.get(pos); - if(ctx.curVal == null) { - ctx.curVal = currentVal; - } else if(currentVal != null) { - ctx.curVal = doOperation(ctx.curVal, currentVal); - } - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java index 2395d356a50..5661a536753 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java @@ -5,19 +5,20 @@ */ package io.deephaven.engine.table.impl.updateby.internal; -import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.table.*; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.table.ChunkSink; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import java.util.Collections; import java.util.Map; @@ -33,25 +34,39 @@ public abstract class BaseObjectUpdateByOperator extends UpdateByCumulativeOp private final Class colType; // endregion extra-fields - protected class Context extends UpdateByCumulativeOperator.Context { - public final ChunkSink.FillFromContext fillContext; + protected abstract class Context extends UpdateByCumulativeOperator.Context { + public final ChunkSink.FillFromContext outputFillContext; public final WritableObjectChunk outputValues; public T curVal = null; protected Context(final int chunkSize) { super(chunkSize); - this.fillContext = outputSource.makeFillFromContext(chunkSize); + this.outputFillContext = outputSource.makeFillFromContext(chunkSize); this.outputValues = WritableObjectChunk.makeWritableChunk(chunkSize); } - public void storeValuesChunk(@NotNull final Chunk valuesChunk) {} - @Override public void close() { super.close(); outputValues.close(); - fillContext.close(); + outputFillContext.close(); + } + + @Override + public void setValuesChunk(@NotNull final Chunk valuesChunk) {} + + @Override + public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} + + @Override + public void writeToOutputChunk(int outIdx) { + outputValues.set(outIdx, curVal); + } + + @Override + public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { + outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); } } @@ -88,11 +103,6 @@ public BaseObjectUpdateByOperator(@NotNull final MatchPair pair, // endregion constructor } - @Override - public void reset(UpdateContext context) { - final Context ctx = (Context)context; - ctx.curVal = null; - } // region extra-methods // endregion extra-methods @@ -103,7 +113,7 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie if (firstUnmodifiedKey != NULL_ROW_KEY) { ctx.curVal = outputSource.get(firstUnmodifiedKey); } else { - reset(ctx); + ctx.reset(); } // If we're redirected we have to make sure we tell the output source it's actual size, or we're going @@ -130,25 +140,6 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d } // endregion Shifts - // region Processing - @Override - public void processChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { - Assert.neqNull(valuesChunk, "valuesChunk must not be null for a cumulative operator"); - final Context ctx = (Context) updateContext; - ctx.storeValuesChunk(valuesChunk); - for (int ii = 0; ii < valuesChunk.size(); ii++) { - push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); - ctx.outputValues.set(ii, ctx.curVal); - } - outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues, inputKeys); - } - // endregion - @NotNull @Override public Map> getOutputColumns() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java index b04bbf7f127..c82b48fa107 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java @@ -5,19 +5,20 @@ */ package io.deephaven.engine.table.impl.updateby.internal; -import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableShortChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.table.*; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.table.ChunkSink; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import java.util.Collections; import java.util.Map; @@ -32,25 +33,39 @@ public abstract class BaseShortUpdateByOperator extends UpdateByCumulativeOperat // region extra-fields // endregion extra-fields - protected class Context extends UpdateByCumulativeOperator.Context { - public final ChunkSink.FillFromContext fillContext; + protected abstract class Context extends UpdateByCumulativeOperator.Context { + public final ChunkSink.FillFromContext outputFillContext; public final WritableShortChunk outputValues; public short curVal = NULL_SHORT; protected Context(final int chunkSize) { super(chunkSize); - this.fillContext = outputSource.makeFillFromContext(chunkSize); + this.outputFillContext = outputSource.makeFillFromContext(chunkSize); this.outputValues = WritableShortChunk.makeWritableChunk(chunkSize); } - public void storeValuesChunk(@NotNull final Chunk valuesChunk) {} - @Override public void close() { super.close(); outputValues.close(); - fillContext.close(); + outputFillContext.close(); + } + + @Override + public void setValuesChunk(@NotNull final Chunk valuesChunk) {} + + @Override + public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} + + @Override + public void writeToOutputChunk(int outIdx) { + outputValues.set(outIdx, curVal); + } + + @Override + public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { + outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); } } @@ -85,11 +100,6 @@ public BaseShortUpdateByOperator(@NotNull final MatchPair pair, // endregion constructor } - @Override - public void reset(UpdateContext context) { - final Context ctx = (Context)context; - ctx.curVal = NULL_SHORT; - } // region extra-methods // endregion extra-methods @@ -100,7 +110,7 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie if (firstUnmodifiedKey != NULL_ROW_KEY) { ctx.curVal = outputSource.getShort(firstUnmodifiedKey); } else { - reset(ctx); + ctx.reset(); } // If we're redirected we have to make sure we tell the output source it's actual size, or we're going @@ -127,25 +137,6 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d } // endregion Shifts - // region Processing - @Override - public void processChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { - Assert.neqNull(valuesChunk, "valuesChunk must not be null for a cumulative operator"); - final Context ctx = (Context) updateContext; - ctx.storeValuesChunk(valuesChunk); - for (int ii = 0; ii < valuesChunk.size(); ii++) { - push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); - ctx.outputValues.set(ii, ctx.curVal); - } - outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues, inputKeys); - } - // endregion - @NotNull @Override public Map> getOutputColumns() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java index 543777f0371..e5839f47134 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java @@ -13,6 +13,7 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.WritableByteChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.sized.SizedByteChunk; import io.deephaven.engine.rowset.*; @@ -38,21 +39,44 @@ public abstract class BaseWindowedByteUpdateByOperator extends UpdateByWindowedO final byte nullValue; // endregion extra-fields - protected class Context extends UpdateWindowedContext { - public final ChunkSink.FillFromContext fillContext; - public final SizedByteChunk outputValues; + protected abstract class Context extends UpdateByWindowedOperator.Context { + public final ChunkSink.FillFromContext outputFillContext; + public final WritableByteChunk outputValues; public byte curVal = NULL_BYTE; protected Context(final int chunkSize) { - this.fillContext = outputSource.makeFillFromContext(chunkSize); - this.outputValues = new SizedByteChunk<>(chunkSize); + this.outputFillContext = outputSource.makeFillFromContext(chunkSize); + this.outputValues = WritableByteChunk.makeWritableChunk(chunkSize); + } + + @Override + public void setValuesChunk(@NotNull final Chunk valuesChunk) {} + + @Override + public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} + + @Override + public void writeToOutputChunk(int outIdx) { + outputValues.set(outIdx, curVal); + } + + @Override + public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { + outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); } @Override public void close() { + super.close(); outputValues.close(); - fillContext.close(); + outputFillContext.close(); + } + + @Override + public void reset() { + curVal = NULL_BYTE; + nullCount = 0; } } @@ -84,13 +108,6 @@ public BaseWindowedByteUpdateByOperator(@NotNull final MatchPair pair, // endregion constructor } - @Override - public void reset(UpdateContext context) { - final Context ctx = (Context)context; - ctx.curVal = NULL_BYTE; - ctx.nullCount = 0; - } - // region extra-methods protected byte getNullValue() { return QueryConstants.NULL_BYTE; @@ -106,13 +123,6 @@ protected WritableColumnSource makeDenseSource() { } // endregion extra-methods - - @NotNull - @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { - return new Context(chunkSize); - } - @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); @@ -132,23 +142,6 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d } // endregion Shifts - // region Processing - @Override - public void processChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { - final Context ctx = (Context) updateContext; - for (int ii = 0; ii < valuesChunk.size(); ii++) { - push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); - ctx.outputValues.get().set(ii, ctx.curVal); - } - outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues.get(), inputKeys); - } - // endregion - @NotNull @Override public Map> getOutputColumns() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java index 0e5ca3a9f63..9cb765e941b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java @@ -3,6 +3,7 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.WritableCharChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.sized.SizedCharChunk; import io.deephaven.engine.rowset.*; @@ -27,21 +28,44 @@ public abstract class BaseWindowedCharUpdateByOperator extends UpdateByWindowedO // region extra-fields // endregion extra-fields - protected class Context extends UpdateWindowedContext { - public final ChunkSink.FillFromContext fillContext; - public final SizedCharChunk outputValues; + protected abstract class Context extends UpdateByWindowedOperator.Context { + public final ChunkSink.FillFromContext outputFillContext; + public final WritableCharChunk outputValues; public char curVal = NULL_CHAR; protected Context(final int chunkSize) { - this.fillContext = outputSource.makeFillFromContext(chunkSize); - this.outputValues = new SizedCharChunk<>(chunkSize); + this.outputFillContext = outputSource.makeFillFromContext(chunkSize); + this.outputValues = WritableCharChunk.makeWritableChunk(chunkSize); + } + + @Override + public void setValuesChunk(@NotNull final Chunk valuesChunk) {} + + @Override + public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} + + @Override + public void writeToOutputChunk(int outIdx) { + outputValues.set(outIdx, curVal); + } + + @Override + public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { + outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); } @Override public void close() { + super.close(); outputValues.close(); - fillContext.close(); + outputFillContext.close(); + } + + @Override + public void reset() { + curVal = NULL_CHAR; + nullCount = 0; } } @@ -72,23 +96,9 @@ public BaseWindowedCharUpdateByOperator(@NotNull final MatchPair pair, // endregion constructor } - @Override - public void reset(UpdateContext context) { - final Context ctx = (Context)context; - ctx.curVal = NULL_CHAR; - ctx.nullCount = 0; - } - // region extra-methods // endregion extra-methods - - @NotNull - @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { - return new Context(chunkSize); - } - @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); @@ -104,23 +114,6 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d } // endregion Shifts - // region Processing - @Override - public void processChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { - final Context ctx = (Context) updateContext; - for (int ii = 0; ii < valuesChunk.size(); ii++) { - push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); - ctx.outputValues.get().set(ii, ctx.curVal); - } - outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues.get(), inputKeys); - } - // endregion - @NotNull @Override public Map> getOutputColumns() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java index 395a57f386a..468d4bef0e6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java @@ -8,7 +8,6 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.sized.SizedDoubleChunk; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.*; @@ -21,29 +20,56 @@ import org.jetbrains.annotations.Nullable; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; +import static io.deephaven.util.QueryConstants.NULL_CHAR; +import static io.deephaven.util.QueryConstants.NULL_DOUBLE; public abstract class BaseWindowedDoubleUpdateByOperator extends UpdateByWindowedOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; + public double curVal = NULL_DOUBLE; + // region extra-fields // endregion extra-fields - protected class Context extends UpdateWindowedContext { - public final ChunkSink.FillFromContext fillContext; - public final SizedDoubleChunk outputValues; + protected abstract class Context extends UpdateByWindowedOperator.Context { + public final ChunkSink.FillFromContext outputFillContext; + public final WritableDoubleChunk outputValues; protected Context(final int chunkSize) { - this.fillContext = outputSource.makeFillFromContext(chunkSize); - this.outputValues = new SizedDoubleChunk<>(chunkSize); + this.outputFillContext = outputSource.makeFillFromContext(chunkSize); + this.outputValues = WritableDoubleChunk.makeWritableChunk(chunkSize); } public void storeWorkingChunk(@NotNull final Chunk valuesChunk) {} + @Override + public void setValuesChunk(@NotNull final Chunk valuesChunk) {} + + @Override + public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} + + @Override + public void writeToOutputChunk(int outIdx) { + outputValues.set(outIdx, curVal); + } + + @Override + public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { + outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); + } + @Override public void close() { + super.close(); outputValues.close(); - fillContext.close(); + outputFillContext.close(); + } + + @Override + public void reset() { + curVal = NULL_DOUBLE; + nullCount = 0; } } @@ -74,18 +100,9 @@ public BaseWindowedDoubleUpdateByOperator(@NotNull final MatchPair pair, // endregion constructor } - //*** for doubleing point operators, we want a computed result */ - public abstract double result(UpdateContext context); - // region extra-methods // endregion extra-methods - @NotNull - @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { - return new Context(chunkSize); - } - @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); @@ -95,31 +112,9 @@ public void startTrackingPrev() { } // region Shifts - @Override public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { ((DoubleSparseArraySource)outputSource).shift(subIndexToShift, delta); } - // endregion Shifts - - // region Processing - - @Override - public void processChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { - final Context ctx = (Context) updateContext; - ctx.storeWorkingChunk(valuesChunk); - for (int ii = 0; ii < valuesChunk.size(); ii++) { - push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); - ctx.outputValues.get().set(ii, result(ctx)); - } - outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues.get(), inputKeys); - } - - // endregion } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java index 8fb34a7685a..ea794d34671 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java @@ -3,7 +3,6 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.sized.SizedFloatChunk; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.*; @@ -16,29 +15,56 @@ import org.jetbrains.annotations.Nullable; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; +import static io.deephaven.util.QueryConstants.NULL_CHAR; +import static io.deephaven.util.QueryConstants.NULL_FLOAT; public abstract class BaseWindowedFloatUpdateByOperator extends UpdateByWindowedOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; + public float curVal = NULL_FLOAT; + // region extra-fields // endregion extra-fields - protected class Context extends UpdateWindowedContext { - public final ChunkSink.FillFromContext fillContext; - public final SizedFloatChunk outputValues; + protected abstract class Context extends UpdateByWindowedOperator.Context { + public final ChunkSink.FillFromContext outputFillContext; + public final WritableFloatChunk outputValues; protected Context(final int chunkSize) { - this.fillContext = outputSource.makeFillFromContext(chunkSize); - this.outputValues = new SizedFloatChunk<>(chunkSize); + this.outputFillContext = outputSource.makeFillFromContext(chunkSize); + this.outputValues = WritableFloatChunk.makeWritableChunk(chunkSize); } public void storeWorkingChunk(@NotNull final Chunk valuesChunk) {} + @Override + public void setValuesChunk(@NotNull final Chunk valuesChunk) {} + + @Override + public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} + + @Override + public void writeToOutputChunk(int outIdx) { + outputValues.set(outIdx, curVal); + } + + @Override + public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { + outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); + } + @Override public void close() { + super.close(); outputValues.close(); - fillContext.close(); + outputFillContext.close(); + } + + @Override + public void reset() { + curVal = NULL_FLOAT; + nullCount = 0; } } @@ -69,18 +95,9 @@ public BaseWindowedFloatUpdateByOperator(@NotNull final MatchPair pair, // endregion constructor } - //*** for floating point operators, we want a computed result */ - public abstract float result(UpdateContext context); - // region extra-methods // endregion extra-methods - @NotNull - @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { - return new Context(chunkSize); - } - @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); @@ -90,31 +107,9 @@ public void startTrackingPrev() { } // region Shifts - @Override public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { ((FloatSparseArraySource)outputSource).shift(subIndexToShift, delta); } - // endregion Shifts - - // region Processing - - @Override - public void processChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { - final Context ctx = (Context) updateContext; - ctx.storeWorkingChunk(valuesChunk); - for (int ii = 0; ii < valuesChunk.size(); ii++) { - push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); - ctx.outputValues.get().set(ii, result(ctx)); - } - outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues.get(), inputKeys); - } - - // endregion } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java index e495c11e73b..a7c68886349 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java @@ -8,6 +8,7 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.sized.SizedIntChunk; import io.deephaven.engine.rowset.*; @@ -32,21 +33,44 @@ public abstract class BaseWindowedIntUpdateByOperator extends UpdateByWindowedOp // region extra-fields // endregion extra-fields - protected class Context extends UpdateWindowedContext { - public final ChunkSink.FillFromContext fillContext; - public final SizedIntChunk outputValues; + protected abstract class Context extends UpdateByWindowedOperator.Context { + public final ChunkSink.FillFromContext outputFillContext; + public final WritableIntChunk outputValues; public int curVal = NULL_INT; protected Context(final int chunkSize) { - this.fillContext = outputSource.makeFillFromContext(chunkSize); - this.outputValues = new SizedIntChunk<>(chunkSize); + this.outputFillContext = outputSource.makeFillFromContext(chunkSize); + this.outputValues = WritableIntChunk.makeWritableChunk(chunkSize); + } + + @Override + public void setValuesChunk(@NotNull final Chunk valuesChunk) {} + + @Override + public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} + + @Override + public void writeToOutputChunk(int outIdx) { + outputValues.set(outIdx, curVal); + } + + @Override + public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { + outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); } @Override public void close() { + super.close(); outputValues.close(); - fillContext.close(); + outputFillContext.close(); + } + + @Override + public void reset() { + curVal = NULL_INT; + nullCount = 0; } } @@ -77,23 +101,9 @@ public BaseWindowedIntUpdateByOperator(@NotNull final MatchPair pair, // endregion constructor } - @Override - public void reset(UpdateContext context) { - final Context ctx = (Context)context; - ctx.curVal = NULL_INT; - ctx.nullCount = 0; - } - // region extra-methods // endregion extra-methods - - @NotNull - @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { - return new Context(chunkSize); - } - @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); @@ -109,23 +119,6 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d } // endregion Shifts - // region Processing - @Override - public void processChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { - final Context ctx = (Context) updateContext; - for (int ii = 0; ii < valuesChunk.size(); ii++) { - push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); - ctx.outputValues.get().set(ii, ctx.curVal); - } - outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues.get(), inputKeys); - } - // endregion - @NotNull @Override public Map> getOutputColumns() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java index 7448cfd9367..7439501faee 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java @@ -8,6 +8,7 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.WritableLongChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.sized.SizedLongChunk; import io.deephaven.engine.rowset.*; @@ -32,21 +33,44 @@ public abstract class BaseWindowedLongUpdateByOperator extends UpdateByWindowedO // region extra-fields // endregion extra-fields - protected class Context extends UpdateWindowedContext { - public final ChunkSink.FillFromContext fillContext; - public final SizedLongChunk outputValues; + protected abstract class Context extends UpdateByWindowedOperator.Context { + public final ChunkSink.FillFromContext outputFillContext; + public final WritableLongChunk outputValues; public long curVal = NULL_LONG; protected Context(final int chunkSize) { - this.fillContext = outputSource.makeFillFromContext(chunkSize); - this.outputValues = new SizedLongChunk<>(chunkSize); + this.outputFillContext = outputSource.makeFillFromContext(chunkSize); + this.outputValues = WritableLongChunk.makeWritableChunk(chunkSize); + } + + @Override + public void setValuesChunk(@NotNull final Chunk valuesChunk) {} + + @Override + public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} + + @Override + public void writeToOutputChunk(int outIdx) { + outputValues.set(outIdx, curVal); + } + + @Override + public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { + outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); } @Override public void close() { + super.close(); outputValues.close(); - fillContext.close(); + outputFillContext.close(); + } + + @Override + public void reset() { + curVal = NULL_LONG; + nullCount = 0; } } @@ -77,23 +101,9 @@ public BaseWindowedLongUpdateByOperator(@NotNull final MatchPair pair, // endregion constructor } - @Override - public void reset(UpdateContext context) { - final Context ctx = (Context)context; - ctx.curVal = NULL_LONG; - ctx.nullCount = 0; - } - // region extra-methods // endregion extra-methods - - @NotNull - @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { - return new Context(chunkSize); - } - @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); @@ -109,23 +119,6 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d } // endregion Shifts - // region Processing - @Override - public void processChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { - final Context ctx = (Context) updateContext; - for (int ii = 0; ii < valuesChunk.size(); ii++) { - push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); - ctx.outputValues.get().set(ii, ctx.curVal); - } - outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues.get(), inputKeys); - } - // endregion - @NotNull @Override public Map> getOutputColumns() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java index 86ed3e4b897..32d3ad35ced 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java @@ -8,6 +8,7 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.sized.SizedObjectChunk; import io.deephaven.engine.rowset.*; @@ -33,21 +34,44 @@ public abstract class BaseWindowedObjectUpdateByOperator extends UpdateByWind private final Class colType; // endregion extra-fields - protected class Context extends UpdateWindowedContext { - public final ChunkSink.FillFromContext fillContext; - public final SizedObjectChunk outputValues; + protected abstract class Context extends UpdateByWindowedOperator.Context { + public final ChunkSink.FillFromContext outputFillContext; + public final WritableObjectChunk outputValues; public T curVal = null; protected Context(final int chunkSize) { - this.fillContext = outputSource.makeFillFromContext(chunkSize); - this.outputValues = new SizedObjectChunk<>(chunkSize); + this.outputFillContext = outputSource.makeFillFromContext(chunkSize); + this.outputValues = WritableObjectChunk.makeWritableChunk(chunkSize); + } + + @Override + public void setValuesChunk(@NotNull final Chunk valuesChunk) {} + + @Override + public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} + + @Override + public void writeToOutputChunk(int outIdx) { + outputValues.set(outIdx, curVal); + } + + @Override + public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { + outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); } @Override public void close() { + super.close(); outputValues.close(); - fillContext.close(); + outputFillContext.close(); + } + + @Override + public void reset() { + curVal = null; + nullCount = 0; } } @@ -80,23 +104,9 @@ public BaseWindowedObjectUpdateByOperator(@NotNull final MatchPair pair, // endregion constructor } - @Override - public void reset(UpdateContext context) { - final Context ctx = (Context)context; - ctx.curVal = null; - ctx.nullCount = 0; - } - // region extra-methods // endregion extra-methods - - @NotNull - @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { - return new Context(chunkSize); - } - @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); @@ -112,23 +122,6 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d } // endregion Shifts - // region Processing - @Override - public void processChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { - final Context ctx = (Context) updateContext; - for (int ii = 0; ii < valuesChunk.size(); ii++) { - push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); - ctx.outputValues.get().set(ii, ctx.curVal); - } - outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues.get(), inputKeys); - } - // endregion - @NotNull @Override public Map> getOutputColumns() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java index ef09d222af5..0bfcd5f9e93 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java @@ -8,6 +8,7 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.WritableShortChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.sized.SizedShortChunk; import io.deephaven.engine.rowset.*; @@ -32,21 +33,44 @@ public abstract class BaseWindowedShortUpdateByOperator extends UpdateByWindowed // region extra-fields // endregion extra-fields - protected class Context extends UpdateWindowedContext { - public final ChunkSink.FillFromContext fillContext; - public final SizedShortChunk outputValues; + protected abstract class Context extends UpdateByWindowedOperator.Context { + public final ChunkSink.FillFromContext outputFillContext; + public final WritableShortChunk outputValues; public short curVal = NULL_SHORT; protected Context(final int chunkSize) { - this.fillContext = outputSource.makeFillFromContext(chunkSize); - this.outputValues = new SizedShortChunk<>(chunkSize); + this.outputFillContext = outputSource.makeFillFromContext(chunkSize); + this.outputValues = WritableShortChunk.makeWritableChunk(chunkSize); + } + + @Override + public void setValuesChunk(@NotNull final Chunk valuesChunk) {} + + @Override + public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} + + @Override + public void writeToOutputChunk(int outIdx) { + outputValues.set(outIdx, curVal); + } + + @Override + public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { + outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); } @Override public void close() { + super.close(); outputValues.close(); - fillContext.close(); + outputFillContext.close(); + } + + @Override + public void reset() { + curVal = NULL_SHORT; + nullCount = 0; } } @@ -77,23 +101,9 @@ public BaseWindowedShortUpdateByOperator(@NotNull final MatchPair pair, // endregion constructor } - @Override - public void reset(UpdateContext context) { - final Context ctx = (Context)context; - ctx.curVal = NULL_SHORT; - ctx.nullCount = 0; - } - // region extra-methods // endregion extra-methods - - @NotNull - @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { - return new Context(chunkSize); - } - @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); @@ -109,23 +119,6 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d } // endregion Shifts - // region Processing - @Override - public void processChunk(@NotNull final UpdateContext updateContext, - @NotNull final RowSequence inputKeys, - @Nullable final LongChunk keyChunk, - @Nullable final LongChunk posChunk, - @Nullable final Chunk valuesChunk, - @Nullable final LongChunk timestampValuesChunk) { - final Context ctx = (Context) updateContext; - for (int ii = 0; ii < valuesChunk.size(); ii++) { - push(ctx, keyChunk == null ? NULL_ROW_KEY : keyChunk.get(ii), ii); - ctx.outputValues.get().set(ii, ctx.curVal); - } - outputSource.fillFromChunk(ctx.fillContext, ctx.outputValues.get(), inputKeys); - } - // endregion - @NotNull @Override public Map> getOutputColumns() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java index 3cc1b07cf7e..83313540444 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java @@ -283,7 +283,7 @@ public int remaining() { } public void clear() { - head = tail = 0; + head = tail = capacity; dirtyIndices.clear(); allDirty = false; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java index f686788da53..e2c9b3c2df8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java @@ -278,7 +278,7 @@ public int remaining() { } public void clear() { - head = tail = 0; + head = tail = capacity; dirtyIndices.clear(); allDirty = false; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java index 0b49e5b73d5..24c8d8d5965 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java @@ -30,9 +30,29 @@ protected Context(int chunkSize) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { byteValueChunk = valuesChunk.asByteChunk(); } + + @Override + public void push(long key, int pos) { + // read the value from the values chunk + final byte currentVal = byteValueChunk.get(pos); + + if (curVal == NULL_BYTE) { + curVal = currentVal; + } else if (currentVal != NULL_BYTE) { + if ((isMax && currentVal > curVal) || + (!isMax && currentVal < curVal)) { + curVal = currentVal; + } + } + } + + @Override + public void reset() { + curVal = NULL_BYTE; + } } public ByteCumMinMaxOperator(@NotNull final MatchPair pair, @@ -52,23 +72,4 @@ public ByteCumMinMaxOperator(@NotNull final MatchPair pair, public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } - - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - // read the value from the values chunk - final byte currentVal = ctx.byteValueChunk.get(pos); - - if (ctx.curVal == NULL_BYTE) { - ctx.curVal = currentVal; - } else if (currentVal != NULL_BYTE) { - if ((isMax && currentVal > ctx.curVal) || - (!isMax && currentVal < ctx.curVal)) { - ctx.curVal = currentVal; - } - } - } - // region extra-methods - // endregion extra-methods } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java index 7f449d25770..077dd6daff0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java @@ -30,9 +30,29 @@ protected Context(int chunkSize) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { doubleValueChunk = valuesChunk.asDoubleChunk(); } + + @Override + public void push(long key, int pos) { + // read the value from the values chunk + final double currentVal = doubleValueChunk.get(pos); + + if(curVal == NULL_DOUBLE) { + curVal = currentVal; + } else if(currentVal != NULL_DOUBLE) { + if ((isMax && currentVal > curVal) || + (!isMax && currentVal < curVal)) { + curVal = currentVal; + } + } + } + + @Override + public void reset() { + curVal = NULL_DOUBLE; + } } public DoubleCumMinMaxOperator(@NotNull final MatchPair pair, @@ -52,23 +72,4 @@ public DoubleCumMinMaxOperator(@NotNull final MatchPair pair, public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } - - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - // read the value from the values chunk - final double currentVal = ctx.doubleValueChunk.get(pos); - - if(ctx.curVal == NULL_DOUBLE) { - ctx.curVal = currentVal; - } else if(currentVal != NULL_DOUBLE) { - if ((isMax && currentVal > ctx.curVal) || - (!isMax && currentVal < ctx.curVal)) { - ctx.curVal = currentVal; - } - } - } - // region extra-methods - // endregion extra-methods } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java index caf9c09db11..f0840064945 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java @@ -25,9 +25,29 @@ protected Context(int chunkSize) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { floatValueChunk = valuesChunk.asFloatChunk(); } + + @Override + public void push(long key, int pos) { + // read the value from the values chunk + final float currentVal = floatValueChunk.get(pos); + + if(curVal == NULL_FLOAT) { + curVal = currentVal; + } else if(currentVal != NULL_FLOAT) { + if ((isMax && currentVal > curVal) || + (!isMax && currentVal < curVal)) { + curVal = currentVal; + } + } + } + + @Override + public void reset() { + curVal = NULL_FLOAT; + } } public FloatCumMinMaxOperator(@NotNull final MatchPair pair, @@ -47,23 +67,4 @@ public FloatCumMinMaxOperator(@NotNull final MatchPair pair, public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } - - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - // read the value from the values chunk - final float currentVal = ctx.floatValueChunk.get(pos); - - if(ctx.curVal == NULL_FLOAT) { - ctx.curVal = currentVal; - } else if(currentVal != NULL_FLOAT) { - if ((isMax && currentVal > ctx.curVal) || - (!isMax && currentVal < ctx.curVal)) { - ctx.curVal = currentVal; - } - } - } - // region extra-methods - // endregion extra-methods } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java index b86a9cc5bca..364e5075b53 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java @@ -30,9 +30,29 @@ protected Context(int chunkSize) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { intValueChunk = valuesChunk.asIntChunk(); } + + @Override + public void push(long key, int pos) { + // read the value from the values chunk + final int currentVal = intValueChunk.get(pos); + + if (curVal == NULL_INT) { + curVal = currentVal; + } else if (currentVal != NULL_INT) { + if ((isMax && currentVal > curVal) || + (!isMax && currentVal < curVal)) { + curVal = currentVal; + } + } + } + + @Override + public void reset() { + curVal = NULL_INT; + } } public IntCumMinMaxOperator(@NotNull final MatchPair pair, @@ -52,23 +72,4 @@ public IntCumMinMaxOperator(@NotNull final MatchPair pair, public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } - - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - // read the value from the values chunk - final int currentVal = ctx.intValueChunk.get(pos); - - if (ctx.curVal == NULL_INT) { - ctx.curVal = currentVal; - } else if (currentVal != NULL_INT) { - if ((isMax && currentVal > ctx.curVal) || - (!isMax && currentVal < ctx.curVal)) { - ctx.curVal = currentVal; - } - } - } - // region extra-methods - // endregion extra-methods } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java index 5582cbe3c26..fd203e71e72 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java @@ -38,9 +38,29 @@ protected Context(int chunkSize) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { longValueChunk = valuesChunk.asLongChunk(); } + + @Override + public void push(long key, int pos) { + // read the value from the values chunk + final long currentVal = longValueChunk.get(pos); + + if (curVal == NULL_LONG) { + curVal = currentVal; + } else if (currentVal != NULL_LONG) { + if ((isMax && currentVal > curVal) || + (!isMax && currentVal < curVal)) { + curVal = currentVal; + } + } + } + + @Override + public void reset() { + curVal = NULL_LONG; + } } public LongCumMinMaxOperator(@NotNull final MatchPair pair, @@ -62,34 +82,4 @@ public LongCumMinMaxOperator(@NotNull final MatchPair pair, public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } - - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - // read the value from the values chunk - final long currentVal = ctx.longValueChunk.get(pos); - - if (ctx.curVal == NULL_LONG) { - ctx.curVal = currentVal; - } else if (currentVal != NULL_LONG) { - if ((isMax && currentVal > ctx.curVal) || - (!isMax && currentVal < ctx.curVal)) { - ctx.curVal = currentVal; - } - } - } - // region extra-methods - @NotNull - @Override - public Map> getOutputColumns() { - final ColumnSource actualOutput; - if(type == DateTime.class) { - actualOutput = ReinterpretUtils.longToDateTimeSource(outputSource); - } else { - actualOutput = outputSource; - } - return Collections.singletonMap(pair.leftColumn, actualOutput); - } - // endregion extra-methods } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java index a17c8a00c53..229041b7e5d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java @@ -25,9 +25,29 @@ protected Context(int chunkSize) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { shortValueChunk = valuesChunk.asShortChunk(); } + + @Override + public void push(long key, int pos) { + // read the value from the values chunk + final short currentVal = shortValueChunk.get(pos); + + if (curVal == NULL_SHORT) { + curVal = currentVal; + } else if (currentVal != NULL_SHORT) { + if ((isMax && currentVal > curVal) || + (!isMax && currentVal < curVal)) { + curVal = currentVal; + } + } + } + + @Override + public void reset() { + curVal = NULL_SHORT; + } } public ShortCumMinMaxOperator(@NotNull final MatchPair pair, @@ -47,23 +67,4 @@ public ShortCumMinMaxOperator(@NotNull final MatchPair pair, public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } - - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - // read the value from the values chunk - final short currentVal = ctx.shortValueChunk.get(pos); - - if (ctx.curVal == NULL_SHORT) { - ctx.curVal = currentVal; - } else if (currentVal != NULL_SHORT) { - if ((isMax && currentVal > ctx.curVal) || - (!isMax && currentVal < ctx.curVal)) { - ctx.curVal = currentVal; - } - } - } - // region extra-methods - // endregion extra-methods } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java index 6518fde111f..2cccd699dd5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java @@ -12,6 +12,8 @@ import java.math.BigDecimal; import java.math.MathContext; +import static io.deephaven.util.QueryConstants.NULL_FLOAT; + public final class BigDecimalCumProdOperator extends BaseObjectUpdateByOperator { @NotNull private final MathContext mathContext; @@ -24,9 +26,29 @@ protected Context(int chunkSize) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { objectValueChunk = valuesChunk.asObjectChunk(); } + + @Override + public void push(long key, int pos) { + // read the value from the values chunk + final BigDecimal currentVal = objectValueChunk.get(pos); + + final boolean isCurrentNull = currentVal == null; + if (curVal == null) { + curVal = isCurrentNull ? null : currentVal; + } else { + if (!isCurrentNull) { + curVal = curVal.multiply(objectValueChunk.get(pos), mathContext); + } + } + } + + @Override + public void reset() { + curVal = null; + } } public BigDecimalCumProdOperator(@NotNull final MatchPair inputPair, @@ -41,21 +63,4 @@ public BigDecimalCumProdOperator(@NotNull final MatchPair inputPair, public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } - - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - // read the value from the values chunk - final BigDecimal currentVal = ctx.objectValueChunk.get(pos); - - final boolean isCurrentNull = currentVal == null; - if (ctx.curVal == null) { - ctx.curVal = isCurrentNull ? null : currentVal; - } else { - if (!isCurrentNull) { - ctx.curVal = ctx.curVal.multiply(ctx.objectValueChunk.get(pos), mathContext); - } - } - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java index 164a1478bd7..d89ced41ab2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java @@ -11,6 +11,8 @@ import java.math.BigInteger; +import static io.deephaven.util.QueryConstants.NULL_FLOAT; + public final class BigIntegerCumProdOperator extends BaseObjectUpdateByOperator { protected class Context extends BaseObjectUpdateByOperator.Context { public ObjectChunk objectValueChunk; @@ -20,9 +22,29 @@ protected Context(int chunkSize) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { objectValueChunk = valuesChunk.asObjectChunk(); } + + @Override + public void push(long key, int pos) { + // read the value from the values chunk + final BigInteger currentVal = objectValueChunk.get(pos); + + final boolean isCurrentNull = currentVal == null; + if(curVal == null) { + curVal = isCurrentNull ? null : currentVal; + } else { + if(!isCurrentNull) { + curVal = curVal.multiply(objectValueChunk.get(pos)); + } + } + } + + @Override + public void reset() { + curVal = null; + } } public BigIntegerCumProdOperator(@NotNull final MatchPair inputPair, @@ -35,21 +57,4 @@ public BigIntegerCumProdOperator(@NotNull final MatchPair inputPair, public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } - - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - // read the value from the values chunk - final BigInteger currentVal = ctx.objectValueChunk.get(pos); - - final boolean isCurrentNull = currentVal == null; - if(ctx.curVal == null) { - ctx.curVal = isCurrentNull ? null : currentVal; - } else { - if(!isCurrentNull) { - ctx.curVal = ctx.curVal.multiply(ctx.objectValueChunk.get(pos)); - } - } - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java index 74e5abbfe29..9f0e0f20ae3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java @@ -14,8 +14,7 @@ import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; import org.jetbrains.annotations.NotNull; -import static io.deephaven.util.QueryConstants.NULL_LONG; -import static io.deephaven.util.QueryConstants.NULL_BYTE; +import static io.deephaven.util.QueryConstants.*; public class ByteCumProdOperator extends BaseLongUpdateByOperator { // region extra-fields @@ -29,9 +28,26 @@ protected Context(int chunkSize) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { byteValueChunk = valuesChunk.asByteChunk(); } + + @Override + public void push(long key, int pos) { + // read the value from the values chunk + final byte currentVal = byteValueChunk.get(pos); + + if (curVal == NULL_LONG) { + curVal = currentVal == NULL_BYTE ? NULL_LONG : currentVal; + } else if (currentVal != NULL_BYTE) { + curVal *= currentVal; + } + } + + @Override + public void reset() { + curVal = NULL_BYTE; + } } public ByteCumProdOperator(@NotNull final MatchPair pair, @@ -49,18 +65,4 @@ public ByteCumProdOperator(@NotNull final MatchPair pair, public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } - - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - // read the value from the values chunk - final byte currentVal = ctx.byteValueChunk.get(pos); - - if (ctx.curVal == NULL_LONG) { - ctx.curVal = currentVal == NULL_BYTE ? NULL_LONG : currentVal; - } else if (currentVal != NULL_BYTE) { - ctx.curVal *= currentVal; - } - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java index f08058a7281..e66d648c0ca 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java @@ -15,6 +15,7 @@ import org.jetbrains.annotations.NotNull; import static io.deephaven.util.QueryConstants.NULL_DOUBLE; +import static io.deephaven.util.QueryConstants.NULL_SHORT; public class DoubleCumProdOperator extends BaseDoubleUpdateByOperator { // region extra-fields @@ -28,9 +29,26 @@ protected Context(int chunkSize) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { doubleValueChunk = valuesChunk.asDoubleChunk(); } + + @Override + public void push(long key, int pos) { + // read the value from the values chunk + final double currentVal = doubleValueChunk.get(pos); + + if (curVal == NULL_DOUBLE) { + curVal = currentVal; + } else if (currentVal != NULL_DOUBLE) { + curVal *= currentVal; + } + } + + @Override + public void reset() { + curVal = NULL_DOUBLE; + } } public DoubleCumProdOperator(@NotNull final MatchPair pair, @@ -49,17 +67,5 @@ public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSourc return new Context(chunkSize); } - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - // read the value from the values chunk - final double currentVal = ctx.doubleValueChunk.get(pos); - if (ctx.curVal == NULL_DOUBLE) { - ctx.curVal = currentVal; - } else if (currentVal != NULL_DOUBLE) { - ctx.curVal *= currentVal; - } - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java index 049d1181b25..21ec47b8c67 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java @@ -10,6 +10,7 @@ import org.jetbrains.annotations.NotNull; import static io.deephaven.util.QueryConstants.NULL_FLOAT; +import static io.deephaven.util.QueryConstants.NULL_SHORT; public class FloatCumProdOperator extends BaseFloatUpdateByOperator { // region extra-fields @@ -23,9 +24,26 @@ protected Context(int chunkSize) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { floatValueChunk = valuesChunk.asFloatChunk(); } + + @Override + public void push(long key, int pos) { + // read the value from the values chunk + final float currentVal = floatValueChunk.get(pos); + + if (curVal == NULL_FLOAT) { + curVal = currentVal; + } else if (currentVal != NULL_FLOAT) { + curVal *= currentVal; + } + } + + @Override + public void reset() { + curVal = NULL_FLOAT; + } } public FloatCumProdOperator(@NotNull final MatchPair pair, @@ -44,17 +62,5 @@ public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSourc return new Context(chunkSize); } - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - // read the value from the values chunk - final float currentVal = ctx.floatValueChunk.get(pos); - if (ctx.curVal == NULL_FLOAT) { - ctx.curVal = currentVal; - } else if (currentVal != NULL_FLOAT) { - ctx.curVal *= currentVal; - } - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java index 2834a8e14f2..c5f3a2e7b5f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java @@ -14,8 +14,7 @@ import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; import org.jetbrains.annotations.NotNull; -import static io.deephaven.util.QueryConstants.NULL_LONG; -import static io.deephaven.util.QueryConstants.NULL_INT; +import static io.deephaven.util.QueryConstants.*; public class IntCumProdOperator extends BaseLongUpdateByOperator { // region extra-fields @@ -29,9 +28,26 @@ protected Context(int chunkSize) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { intValueChunk = valuesChunk.asIntChunk(); } + + @Override + public void push(long key, int pos) { + // read the value from the values chunk + final int currentVal = intValueChunk.get(pos); + + if (curVal == NULL_LONG) { + curVal = currentVal == NULL_INT ? NULL_LONG : currentVal; + } else if (currentVal != NULL_INT) { + curVal *= currentVal; + } + } + + @Override + public void reset() { + curVal = NULL_INT; + } } public IntCumProdOperator(@NotNull final MatchPair pair, @@ -49,18 +65,4 @@ public IntCumProdOperator(@NotNull final MatchPair pair, public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } - - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - // read the value from the values chunk - final int currentVal = ctx.intValueChunk.get(pos); - - if (ctx.curVal == NULL_LONG) { - ctx.curVal = currentVal == NULL_INT ? NULL_LONG : currentVal; - } else if (currentVal != NULL_INT) { - ctx.curVal *= currentVal; - } - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java index 2c16628292a..8e89a58fe03 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java @@ -14,8 +14,7 @@ import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; import org.jetbrains.annotations.NotNull; -import static io.deephaven.util.QueryConstants.NULL_LONG; -import static io.deephaven.util.QueryConstants.NULL_LONG; +import static io.deephaven.util.QueryConstants.*; public class LongCumProdOperator extends BaseLongUpdateByOperator { // region extra-fields @@ -29,9 +28,26 @@ protected Context(int chunkSize) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { longValueChunk = valuesChunk.asLongChunk(); } + + @Override + public void push(long key, int pos) { + // read the value from the values chunk + final long currentVal = longValueChunk.get(pos); + + if (curVal == NULL_LONG) { + curVal = currentVal == NULL_LONG ? NULL_LONG : currentVal; + } else if (currentVal != NULL_LONG) { + curVal *= currentVal; + } + } + + @Override + public void reset() { + curVal = NULL_LONG; + } } public LongCumProdOperator(@NotNull final MatchPair pair, @@ -49,18 +65,4 @@ public LongCumProdOperator(@NotNull final MatchPair pair, public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } - - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - // read the value from the values chunk - final long currentVal = ctx.longValueChunk.get(pos); - - if (ctx.curVal == NULL_LONG) { - ctx.curVal = currentVal == NULL_LONG ? NULL_LONG : currentVal; - } else if (currentVal != NULL_LONG) { - ctx.curVal *= currentVal; - } - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java index 7858b8f1f5b..55f5cce77ec 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java @@ -9,8 +9,7 @@ import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; import org.jetbrains.annotations.NotNull; -import static io.deephaven.util.QueryConstants.NULL_LONG; -import static io.deephaven.util.QueryConstants.NULL_SHORT; +import static io.deephaven.util.QueryConstants.*; public class ShortCumProdOperator extends BaseLongUpdateByOperator { // region extra-fields @@ -24,9 +23,26 @@ protected Context(int chunkSize) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { shortValueChunk = valuesChunk.asShortChunk(); } + + @Override + public void push(long key, int pos) { + // read the value from the values chunk + final short currentVal = shortValueChunk.get(pos); + + if (curVal == NULL_LONG) { + curVal = currentVal == NULL_SHORT ? NULL_LONG : currentVal; + } else if (currentVal != NULL_SHORT) { + curVal *= currentVal; + } + } + + @Override + public void reset() { + curVal = NULL_SHORT; + } } public ShortCumProdOperator(@NotNull final MatchPair pair, @@ -44,18 +60,4 @@ public ShortCumProdOperator(@NotNull final MatchPair pair, public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } - - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - // read the value from the values chunk - final short currentVal = ctx.shortValueChunk.get(pos); - - if (ctx.curVal == NULL_LONG) { - ctx.curVal = currentVal == NULL_SHORT ? NULL_LONG : currentVal; - } else if (currentVal != NULL_SHORT) { - ctx.curVal *= currentVal; - } - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java index aa8b3a12321..d2cc480f38d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java @@ -1,6 +1,7 @@ package io.deephaven.engine.table.impl.updateby.rollingsum; import io.deephaven.api.updateby.OperationControl; +import io.deephaven.base.RingBuffer; import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.*; @@ -11,6 +12,7 @@ import org.jetbrains.annotations.Nullable; import java.math.BigDecimal; +import java.math.BigInteger; import java.math.MathContext; public final class BigDecimalRollingSumOperator extends BaseWindowedObjectUpdateByOperator { @@ -18,15 +20,62 @@ public final class BigDecimalRollingSumOperator extends BaseWindowedObjectUpdate private final MathContext mathContext; protected class Context extends BaseWindowedObjectUpdateByOperator.Context { - public ObjectChunk objectInfluencerValuesChunk; + protected ObjectChunk objectInfluencerValuesChunk; + protected RingBuffer objectWindowValues; protected Context(final int chunkSize) { super(chunkSize); + objectWindowValues = new RingBuffer<>(512); } @Override - public void storeInfluencerValuesChunk(@NotNull final Chunk influencerValuesChunk) { - objectInfluencerValuesChunk = influencerValuesChunk.asObjectChunk(); + public void close() { + super.close(); + objectWindowValues = null; + } + + + @Override + public void setValuesChunk(@NotNull final Chunk valuesChunk) { + objectInfluencerValuesChunk = valuesChunk.asObjectChunk(); + } + + @Override + public void push(long key, int pos) { + BigDecimal val = objectInfluencerValuesChunk.get(pos); + objectWindowValues.add(val); + + // increase the running sum + if (val != null) { + if (curVal == null) { + curVal = val; + } else { + curVal = curVal.add(val, mathContext); + } + } else { + nullCount++; + } + } + + @Override + public void pop() { + BigDecimal val = val = objectWindowValues.remove(); + + // reduce the running sum + if (val != null) { + curVal = curVal.subtract(val, mathContext); + } else { + nullCount--; + } + } + + @Override + public void writeToOutputChunk(int outIdx) { + if (objectWindowValues.size() == nullCount) { + outputValues.set(outIdx, null); + } else { + outputValues.set(outIdx, curVal); + } } } @@ -50,33 +99,5 @@ public BigDecimalRollingSumOperator(@NotNull final MatchPair pair, } @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - BigDecimal val = ctx.objectInfluencerValuesChunk.get(pos); - - // increase the running sum - if (val != null) { - if (ctx.curVal == null) { - ctx.curVal = val; - } else { - ctx.curVal = ctx.curVal.add(val, mathContext); - } - } else { - ctx.nullCount++; - } - } - - @Override - public void pop(UpdateContext context) { - final Context ctx = (Context) context; - int pos = ctx.windowIndices.front(); - BigDecimal val = ctx.objectInfluencerValuesChunk.get(pos); - - // reduce the running sum - if (val != null) { - ctx.curVal = ctx.curVal.subtract(val, mathContext); - } else { - ctx.nullCount--; - } - } + public void initializeUpdate(@NotNull UpdateContext context) {} } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java index 8f9d69ecc4d..77eb403010d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java @@ -1,6 +1,7 @@ package io.deephaven.engine.table.impl.updateby.rollingsum; import io.deephaven.api.updateby.OperationControl; +import io.deephaven.base.RingBuffer; import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.*; @@ -12,17 +13,66 @@ import java.math.BigInteger; +import static io.deephaven.util.QueryConstants.NULL_DOUBLE; + public final class BigIntegerRollingSumOperator extends BaseWindowedObjectUpdateByOperator { protected class Context extends BaseWindowedObjectUpdateByOperator.Context { - public ObjectChunk objectInfluencerValuesChunk; + protected ObjectChunk objectInfluencerValuesChunk; + protected RingBuffer objectWindowValues; protected Context(final int chunkSize) { super(chunkSize); + objectWindowValues = new RingBuffer<>(512); + } + + @Override + public void close() { + super.close(); + objectWindowValues = null; + } + + + @Override + public void setValuesChunk(@NotNull final Chunk valuesChunk) { + objectInfluencerValuesChunk = valuesChunk.asObjectChunk(); + } + + @Override + public void push(long key, int pos) { + BigInteger val = objectInfluencerValuesChunk.get(pos); + objectWindowValues.add(val); + + // increase the running sum + if (val != null) { + if (curVal == null) { + curVal = val; + } else { + curVal = curVal.add(val); + } + } else { + nullCount++; + } + } + + @Override + public void pop() { + BigInteger val = objectWindowValues.remove(); + + // reduce the running sum + if (val != null) { + curVal = curVal.subtract(val); + } else { + nullCount--; + } } @Override - public void storeInfluencerValuesChunk(@NotNull final Chunk influencerValuesChunk) { - objectInfluencerValuesChunk = influencerValuesChunk.asObjectChunk(); + public void writeToOutputChunk(int outIdx) { + if (objectWindowValues.size() == nullCount) { + outputValues.set(outIdx, null); + } else { + outputValues.set(outIdx, curVal); + } } } @@ -48,39 +98,6 @@ public BigIntegerRollingSumOperator(@NotNull final MatchPair pair, } @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - BigInteger val = ctx.objectInfluencerValuesChunk.get(pos); - - // increase the running sum - if (val != null) { - if (ctx.curVal == null) { - ctx.curVal = val; - } else { - ctx.curVal = ctx.curVal.add(val); - } - } else { - ctx.nullCount++; - } - } - - @Override - public void pop(UpdateContext context) { - final Context ctx = (Context) context; - int pos = ctx.windowIndices.front(); - BigInteger val = ctx.objectInfluencerValuesChunk.get(pos); - - // reduce the running sum - if (val != null) { - ctx.curVal = ctx.curVal.subtract(val); - } else { - ctx.nullCount--; - } - } - - @Override - public void reset(UpdateContext context) { - final Context ctx = (Context) context; - ctx.curVal = null; + public void initializeUpdate(@NotNull UpdateContext context) { } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java index 0fb3ac0a71d..74d7c7383b4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java @@ -6,10 +6,10 @@ package io.deephaven.engine.table.impl.updateby.rollingsum; import io.deephaven.api.updateby.OperationControl; +import io.deephaven.base.ringbuffer.ByteRingBuffer; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ByteChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; @@ -17,8 +17,7 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.NULL_LONG; -import static io.deephaven.util.QueryConstants.NULL_BYTE; +import static io.deephaven.util.QueryConstants.*; public class ByteRollingSumOperator extends BaseWindowedLongUpdateByOperator { // region extra-fields @@ -26,15 +25,63 @@ public class ByteRollingSumOperator extends BaseWindowedLongUpdateByOperator { // endregion extra-fields protected class Context extends BaseWindowedLongUpdateByOperator.Context { - public ByteChunk byteInfluencerValuesChunk; + protected ByteChunk byteInfluencerValuesChunk; + protected ByteRingBuffer byteWindowValues; + protected Context(int chunkSize) { super(chunkSize); + byteWindowValues = new ByteRingBuffer(512, true); + } + + @Override + public void close() { + super.close(); + byteWindowValues = null; } + @Override - public void storeInfluencerValuesChunk(@NotNull final Chunk influencerValuesChunk) { - byteInfluencerValuesChunk = influencerValuesChunk.asByteChunk(); + public void setValuesChunk(@NotNull final Chunk valuesChunk) { + byteInfluencerValuesChunk = valuesChunk.asByteChunk(); + } + + @Override + public void push(long key, int pos) { + byte val = byteInfluencerValuesChunk.get(pos); + byteWindowValues.add(val); + + // increase the running sum + if (val != NULL_BYTE) { + if (curVal == NULL_LONG) { + curVal = val; + } else { + curVal += val; + } + } else { + nullCount++; + } + } + + @Override + public void pop() { + byte val = byteWindowValues.remove(); + + // reduce the running sum + if (val != NULL_BYTE) { + curVal -= val; + } else { + nullCount--; + } + } + + @Override + public void writeToOutputChunk(int outIdx) { + if (byteWindowValues.size() == nullCount) { + outputValues.set(outIdx, NULL_LONG); + } else { + outputValues.set(outIdx, curVal); + } } } @@ -62,33 +109,6 @@ public ByteRollingSumOperator(@NotNull final MatchPair pair, } @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - byte val = ctx.byteInfluencerValuesChunk.get(pos); - - // increase the running sum - if (val != NULL_BYTE) { - if (ctx.curVal == NULL_LONG) { - ctx.curVal = val; - } else { - ctx.curVal += val; - } - } else { - ctx.nullCount++; - } - } - - @Override - public void pop(UpdateContext context) { - final Context ctx = (Context) context; - int pos = ctx.windowIndices.front(); - byte val = ctx.byteInfluencerValuesChunk.get(pos); - - // reduce the running sum - if (val != NULL_BYTE) { - ctx.curVal -= val; - } else { - ctx.nullCount--; - } + public void initializeUpdate(@NotNull UpdateContext context) { } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java index c9587ac2e8a..610b63d4d0f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java @@ -25,19 +25,61 @@ public class DoubleRollingSumOperator extends BaseWindowedDoubleUpdateByOperator // endregion extra-fields protected class Context extends BaseWindowedDoubleUpdateByOperator.Context { - public DoubleChunk doubleInfluencerValuesChunk; - - public PairwiseDoubleRingBuffer pairwiseSum; + protected DoubleChunk doubleInfluencerValuesChunk; + protected PairwiseDoubleRingBuffer doublePairwiseSum; protected Context(final int chunkSize) { super(chunkSize); - this.pairwiseSum = new PairwiseDoubleRingBuffer(64, 0.0f, Double::sum); + doublePairwiseSum = new PairwiseDoubleRingBuffer(64, 0.0f, new PairwiseDoubleRingBuffer.DoubleFunction() { + @Override + public double apply(double a, double b) { + if (a == NULL_DOUBLE) { + return b; + } else if (b == NULL_DOUBLE) { + return a; + } + return a + b; + } + }); } @Override public void close() { super.close(); - this.pairwiseSum.close(); + doublePairwiseSum.close(); + } + + @Override + public void setValuesChunk(@NotNull final Chunk valuesChunk) { + doubleInfluencerValuesChunk = valuesChunk.asDoubleChunk(); + } + + @Override + public void push(long key, int pos) { + double val = doubleInfluencerValuesChunk.get(pos); + + doublePairwiseSum.push(val); + if (val == NULL_DOUBLE) { + nullCount++; + } + } + + @Override + public void pop() { + double val = doublePairwiseSum.pop(); + + if (val == NULL_DOUBLE) { + nullCount--; + } + } + + @Override + public void writeToOutputChunk(int outIdx) { + if (doublePairwiseSum.size() == nullCount) { + outputValues.set(outIdx, NULL_DOUBLE); + } else { + outputValues.set(outIdx, doublePairwiseSum.evaluate()); + } } } @@ -63,45 +105,7 @@ public DoubleRollingSumOperator(@NotNull final MatchPair pair, } @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - double val = ctx.doubleInfluencerValuesChunk.get(pos); - - if (val != NULL_DOUBLE) { - ctx.pairwiseSum.push(val); - } else { - ctx.pairwiseSum.pushEmptyValue(); - ctx.nullCount++; - } - } - - @Override - public void pop(UpdateContext context) { - final Context ctx = (Context) context; - int pos = ctx.windowIndices.front(); - double val = ctx.doubleInfluencerValuesChunk.get(pos); - - if (val == NULL_DOUBLE) { - ctx.nullCount--; - } - ctx.pairwiseSum.pop(); - } - - @Override - public void reset(UpdateContext context) { - final Context ctx = (Context) context; - // take this opportunity to clear the pairwise structure - ctx.pairwiseSum.clear(); - ctx.nullCount = 0; - } - - @Override - public double result(UpdateContext context) { - final Context ctx = (Context) context; - if (ctx.pairwiseSum.size() == ctx.nullCount) { - return NULL_DOUBLE; - } - return ctx.pairwiseSum.evaluate(); + public void initializeUpdate(@NotNull UpdateContext context) { } @NotNull diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java index a870fd76784..f89134a70f0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java @@ -20,19 +20,61 @@ public class FloatRollingSumOperator extends BaseWindowedFloatUpdateByOperator { // endregion extra-fields protected class Context extends BaseWindowedFloatUpdateByOperator.Context { - public FloatChunk floatInfluencerValuesChunk; - - public PairwiseFloatRingBuffer pairwiseSum; + protected FloatChunk floatInfluencerValuesChunk; + protected PairwiseFloatRingBuffer floatPairwiseSum; protected Context(final int chunkSize) { super(chunkSize); - this.pairwiseSum = new PairwiseFloatRingBuffer(64, 0.0f, Float::sum); + floatPairwiseSum = new PairwiseFloatRingBuffer(64, 0.0f, new PairwiseFloatRingBuffer.FloatFunction() { + @Override + public float apply(float a, float b) { + if (a == NULL_FLOAT) { + return b; + } else if (b == NULL_FLOAT) { + return a; + } + return a + b; + } + }); } @Override public void close() { super.close(); - this.pairwiseSum.close(); + floatPairwiseSum.close(); + } + + @Override + public void setValuesChunk(@NotNull final Chunk valuesChunk) { + floatInfluencerValuesChunk = valuesChunk.asFloatChunk(); + } + + @Override + public void push(long key, int pos) { + float val = floatInfluencerValuesChunk.get(pos); + + floatPairwiseSum.push(val); + if (val == NULL_FLOAT) { + nullCount++; + } + } + + @Override + public void pop() { + float val = floatPairwiseSum.pop(); + + if (val == NULL_FLOAT) { + nullCount--; + } + } + + @Override + public void writeToOutputChunk(int outIdx) { + if (floatPairwiseSum.size() == nullCount) { + outputValues.set(outIdx, NULL_FLOAT); + } else { + outputValues.set(outIdx, floatPairwiseSum.evaluate()); + } } } @@ -58,45 +100,7 @@ public FloatRollingSumOperator(@NotNull final MatchPair pair, } @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - float val = ctx.floatInfluencerValuesChunk.get(pos); - - if (val != NULL_FLOAT) { - ctx.pairwiseSum.push(val); - } else { - ctx.pairwiseSum.pushEmptyValue(); - ctx.nullCount++; - } - } - - @Override - public void pop(UpdateContext context) { - final Context ctx = (Context) context; - int pos = ctx.windowIndices.front(); - float val = ctx.floatInfluencerValuesChunk.get(pos); - - if (val == NULL_FLOAT) { - ctx.nullCount--; - } - ctx.pairwiseSum.pop(); - } - - @Override - public void reset(UpdateContext context) { - final Context ctx = (Context) context; - // take this opportunity to clear the pairwise structure - ctx.pairwiseSum.clear(); - ctx.nullCount = 0; - } - - @Override - public float result(UpdateContext context) { - final Context ctx = (Context) context; - if (ctx.pairwiseSum.size() == ctx.nullCount) { - return NULL_FLOAT; - } - return ctx.pairwiseSum.evaluate(); + public void initializeUpdate(@NotNull UpdateContext context) { } @NotNull diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java index 08c8a0b51b3..3f3ae69f2c8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java @@ -6,10 +6,10 @@ package io.deephaven.engine.table.impl.updateby.rollingsum; import io.deephaven.api.updateby.OperationControl; +import io.deephaven.base.ringbuffer.IntRingBuffer; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; @@ -17,23 +17,70 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.NULL_LONG; -import static io.deephaven.util.QueryConstants.NULL_INT; +import static io.deephaven.util.QueryConstants.*; public class IntRollingSumOperator extends BaseWindowedLongUpdateByOperator { // region extra-fields // endregion extra-fields protected class Context extends BaseWindowedLongUpdateByOperator.Context { - public IntChunk intInfluencerValuesChunk; + protected IntChunk intInfluencerValuesChunk; + protected IntRingBuffer intWindowValues; + protected Context(int chunkSize) { super(chunkSize); + intWindowValues = new IntRingBuffer(512, true); + } + + @Override + public void close() { + super.close(); + intWindowValues = null; } + @Override - public void storeInfluencerValuesChunk(@NotNull final Chunk influencerValuesChunk) { - intInfluencerValuesChunk = influencerValuesChunk.asIntChunk(); + public void setValuesChunk(@NotNull final Chunk valuesChunk) { + intInfluencerValuesChunk = valuesChunk.asIntChunk(); + } + + @Override + public void push(long key, int pos) { + int val = intInfluencerValuesChunk.get(pos); + intWindowValues.add(val); + + // increase the running sum + if (val != NULL_INT) { + if (curVal == NULL_LONG) { + curVal = val; + } else { + curVal += val; + } + } else { + nullCount++; + } + } + + @Override + public void pop() { + int val = intWindowValues.remove(); + + // reduce the running sum + if (val != NULL_INT) { + curVal -= val; + } else { + nullCount--; + } + } + + @Override + public void writeToOutputChunk(int outIdx) { + if (intWindowValues.size() == nullCount) { + outputValues.set(outIdx, NULL_LONG); + } else { + outputValues.set(outIdx, curVal); + } } } @@ -59,33 +106,6 @@ public IntRollingSumOperator(@NotNull final MatchPair pair, } @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - int val = ctx.intInfluencerValuesChunk.get(pos); - - // increase the running sum - if (val != NULL_INT) { - if (ctx.curVal == NULL_LONG) { - ctx.curVal = val; - } else { - ctx.curVal += val; - } - } else { - ctx.nullCount++; - } - } - - @Override - public void pop(UpdateContext context) { - final Context ctx = (Context) context; - int pos = ctx.windowIndices.front(); - int val = ctx.intInfluencerValuesChunk.get(pos); - - // reduce the running sum - if (val != NULL_INT) { - ctx.curVal -= val; - } else { - ctx.nullCount--; - } + public void initializeUpdate(@NotNull UpdateContext context) { } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java index d5bef5a9a33..061c35917b6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java @@ -6,10 +6,10 @@ package io.deephaven.engine.table.impl.updateby.rollingsum; import io.deephaven.api.updateby.OperationControl; +import io.deephaven.base.ringbuffer.LongRingBuffer; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; @@ -17,23 +17,70 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.NULL_LONG; -import static io.deephaven.util.QueryConstants.NULL_LONG; +import static io.deephaven.util.QueryConstants.*; public class LongRollingSumOperator extends BaseWindowedLongUpdateByOperator { // region extra-fields // endregion extra-fields protected class Context extends BaseWindowedLongUpdateByOperator.Context { - public LongChunk longInfluencerValuesChunk; + protected LongChunk longInfluencerValuesChunk; + protected LongRingBuffer longWindowValues; + protected Context(int chunkSize) { super(chunkSize); + longWindowValues = new LongRingBuffer(512, true); + } + + @Override + public void close() { + super.close(); + longWindowValues = null; } + @Override - public void storeInfluencerValuesChunk(@NotNull final Chunk influencerValuesChunk) { - longInfluencerValuesChunk = influencerValuesChunk.asLongChunk(); + public void setValuesChunk(@NotNull final Chunk valuesChunk) { + longInfluencerValuesChunk = valuesChunk.asLongChunk(); + } + + @Override + public void push(long key, int pos) { + long val = longInfluencerValuesChunk.get(pos); + longWindowValues.add(val); + + // increase the running sum + if (val != NULL_LONG) { + if (curVal == NULL_LONG) { + curVal = val; + } else { + curVal += val; + } + } else { + nullCount++; + } + } + + @Override + public void pop() { + long val = longWindowValues.remove(); + + // reduce the running sum + if (val != NULL_LONG) { + curVal -= val; + } else { + nullCount--; + } + } + + @Override + public void writeToOutputChunk(int outIdx) { + if (longWindowValues.size() == nullCount) { + outputValues.set(outIdx, NULL_LONG); + } else { + outputValues.set(outIdx, curVal); + } } } @@ -59,33 +106,6 @@ public LongRollingSumOperator(@NotNull final MatchPair pair, } @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - long val = ctx.longInfluencerValuesChunk.get(pos); - - // increase the running sum - if (val != NULL_LONG) { - if (ctx.curVal == NULL_LONG) { - ctx.curVal = val; - } else { - ctx.curVal += val; - } - } else { - ctx.nullCount++; - } - } - - @Override - public void pop(UpdateContext context) { - final Context ctx = (Context) context; - int pos = ctx.windowIndices.front(); - long val = ctx.longInfluencerValuesChunk.get(pos); - - // reduce the running sum - if (val != NULL_LONG) { - ctx.curVal -= val; - } else { - ctx.nullCount--; - } + public void initializeUpdate(@NotNull UpdateContext context) { } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java index fbb80611ea6..a588064a37e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java @@ -1,10 +1,10 @@ package io.deephaven.engine.table.impl.updateby.rollingsum; import io.deephaven.api.updateby.OperationControl; +import io.deephaven.base.ringbuffer.ShortRingBuffer; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ShortChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; @@ -12,23 +12,70 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.NULL_LONG; -import static io.deephaven.util.QueryConstants.NULL_SHORT; +import static io.deephaven.util.QueryConstants.*; public class ShortRollingSumOperator extends BaseWindowedLongUpdateByOperator { // region extra-fields // endregion extra-fields protected class Context extends BaseWindowedLongUpdateByOperator.Context { - public ShortChunk shortInfluencerValuesChunk; + protected ShortChunk shortInfluencerValuesChunk; + protected ShortRingBuffer shortWindowValues; + protected Context(int chunkSize) { super(chunkSize); + shortWindowValues = new ShortRingBuffer(512, true); + } + + @Override + public void close() { + super.close(); + shortWindowValues = null; } + @Override - public void storeInfluencerValuesChunk(@NotNull final Chunk influencerValuesChunk) { - shortInfluencerValuesChunk = influencerValuesChunk.asShortChunk(); + public void setValuesChunk(@NotNull final Chunk valuesChunk) { + shortInfluencerValuesChunk = valuesChunk.asShortChunk(); + } + + @Override + public void push(long key, int pos) { + short val = shortInfluencerValuesChunk.get(pos); + shortWindowValues.add(val); + + // increase the running sum + if (val != NULL_SHORT) { + if (curVal == NULL_LONG) { + curVal = val; + } else { + curVal += val; + } + } else { + nullCount++; + } + } + + @Override + public void pop() { + short val = shortWindowValues.remove(); + + // reduce the running sum + if (val != NULL_SHORT) { + curVal -= val; + } else { + nullCount--; + } + } + + @Override + public void writeToOutputChunk(int outIdx) { + if (shortWindowValues.size() == nullCount) { + outputValues.set(outIdx, NULL_LONG); + } else { + outputValues.set(outIdx, curVal); + } } } @@ -54,33 +101,6 @@ public ShortRollingSumOperator(@NotNull final MatchPair pair, } @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - short val = ctx.shortInfluencerValuesChunk.get(pos); - - // increase the running sum - if (val != NULL_SHORT) { - if (ctx.curVal == NULL_LONG) { - ctx.curVal = val; - } else { - ctx.curVal += val; - } - } else { - ctx.nullCount++; - } - } - - @Override - public void pop(UpdateContext context) { - final Context ctx = (Context) context; - int pos = ctx.windowIndices.front(); - short val = ctx.shortInfluencerValuesChunk.get(pos); - - // reduce the running sum - if (val != NULL_SHORT) { - ctx.curVal -= val; - } else { - ctx.nullCount--; - } + public void initializeUpdate(@NotNull UpdateContext context) { } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java index baa42c3d826..406e3fffb0a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java @@ -12,6 +12,8 @@ import java.math.BigDecimal; import java.math.MathContext; +import static io.deephaven.util.QueryConstants.NULL_CHAR; + public final class BigDecimalCumSumOperator extends BaseObjectUpdateByOperator { @NotNull private final MathContext mathContext; @@ -24,9 +26,30 @@ protected Context(int chunkSize) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { objectValueChunk = valuesChunk.asObjectChunk(); } + + + @Override + public void push(long key, int pos) { + // read the value from the values chunk + final BigDecimal currentVal = objectValueChunk.get(pos); + + final boolean isCurrentNull = currentVal == null; + if (curVal == null) { + curVal = isCurrentNull ? null : currentVal; + } else { + if (!isCurrentNull) { + curVal = curVal.add(objectValueChunk.get(pos), mathContext); + } + } + } + + @Override + public void reset() { + curVal = null; + } } public BigDecimalCumSumOperator(@NotNull final MatchPair inputPair, @@ -41,21 +64,4 @@ public BigDecimalCumSumOperator(@NotNull final MatchPair inputPair, public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } - - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - // read the value from the values chunk - final BigDecimal currentVal = ctx.objectValueChunk.get(pos); - - final boolean isCurrentNull = currentVal == null; - if (ctx.curVal == null) { - ctx.curVal = isCurrentNull ? null : currentVal; - } else { - if (!isCurrentNull) { - ctx.curVal = ctx.curVal.add(ctx.objectValueChunk.get(pos), mathContext); - } - } - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java index cb795e0fe9a..42fe393b34f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java @@ -11,6 +11,8 @@ import java.math.BigInteger; +import static io.deephaven.util.QueryConstants.NULL_CHAR; + public final class BigIntegerCumSumOperator extends BaseObjectUpdateByOperator { protected class Context extends BaseObjectUpdateByOperator.Context { public ObjectChunk objectValueChunk; @@ -20,9 +22,29 @@ protected Context(int chunkSize) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { objectValueChunk = valuesChunk.asObjectChunk(); } + + @Override + public void push(long key, int pos) { + // read the value from the values chunk + final BigInteger currentVal = objectValueChunk.get(pos); + + final boolean isCurrentNull = currentVal == null; + if(curVal == null) { + curVal = isCurrentNull ? null : currentVal; + } else { + if(!isCurrentNull) { + curVal = curVal.add(objectValueChunk.get(pos)); + } + } + } + + @Override + public void reset() { + curVal = null; + } } public BigIntegerCumSumOperator(@NotNull final MatchPair inputPair, @@ -35,21 +57,4 @@ public BigIntegerCumSumOperator(@NotNull final MatchPair inputPair, public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } - - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - // read the value from the values chunk - final BigInteger currentVal = ctx.objectValueChunk.get(pos); - - final boolean isCurrentNull = currentVal == null; - if(ctx.curVal == null) { - ctx.curVal = isCurrentNull ? null : currentVal; - } else { - if(!isCurrentNull) { - ctx.curVal = ctx.curVal.add(ctx.objectValueChunk.get(pos)); - } - } - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java index 8820dc500f3..529adfff101 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java @@ -28,9 +28,26 @@ protected Context(int chunkSize) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { byteValueChunk = valuesChunk.asByteChunk(); } + + @Override + public void push(long key, int pos) { + // read the value from the values chunk + final byte currentVal = byteValueChunk.get(pos); + + if(curVal == NULL_LONG) { + curVal = currentVal == NULL_BYTE ? NULL_LONG : currentVal; + } else if (currentVal != NULL_BYTE) { + curVal += currentVal; + } + } + + @Override + public void reset() { + curVal = NULL_LONG; + } } public ByteCumSumOperator(@NotNull final MatchPair pair, @@ -50,18 +67,4 @@ public ByteCumSumOperator(@NotNull final MatchPair pair, public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } - - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - // read the value from the values chunk - final byte currentVal = ctx.byteValueChunk.get(pos); - - if(ctx.curVal == NULL_LONG) { - ctx.curVal = currentVal == NULL_BYTE ? NULL_LONG : currentVal; - } else if (currentVal != NULL_BYTE) { - ctx.curVal += currentVal; - } - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java index 9bc75559a19..ff3d2e89df0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java @@ -28,9 +28,26 @@ protected Context(int chunkSize) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { doubleValueChunk = valuesChunk.asDoubleChunk(); } + + @Override + public void push(long key, int pos) { + // read the value from the values chunk + final double currentVal = doubleValueChunk.get(pos); + + if (curVal == NULL_DOUBLE) { + curVal = currentVal; + } else if (currentVal != NULL_DOUBLE) { + curVal += currentVal; + } + } + + @Override + public void reset() { + curVal = NULL_DOUBLE; + } } public DoubleCumSumOperator(@NotNull final MatchPair pair, @@ -48,18 +65,4 @@ public DoubleCumSumOperator(@NotNull final MatchPair pair, public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } - - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - // read the value from the values chunk - final double currentVal = ctx.doubleValueChunk.get(pos); - - if (ctx.curVal == NULL_DOUBLE) { - ctx.curVal = currentVal; - } else if (currentVal != NULL_DOUBLE) { - ctx.curVal += currentVal; - } - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java index 80ed891ea77..404715f106c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java @@ -23,9 +23,26 @@ protected Context(int chunkSize) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { floatValueChunk = valuesChunk.asFloatChunk(); } + + @Override + public void push(long key, int pos) { + // read the value from the values chunk + final float currentVal = floatValueChunk.get(pos); + + if (curVal == NULL_FLOAT) { + curVal = currentVal; + } else if (currentVal != NULL_FLOAT) { + curVal += currentVal; + } + } + + @Override + public void reset() { + curVal = NULL_FLOAT; + } } public FloatCumSumOperator(@NotNull final MatchPair pair, @@ -43,18 +60,4 @@ public FloatCumSumOperator(@NotNull final MatchPair pair, public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } - - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - // read the value from the values chunk - final float currentVal = ctx.floatValueChunk.get(pos); - - if (ctx.curVal == NULL_FLOAT) { - ctx.curVal = currentVal; - } else if (currentVal != NULL_FLOAT) { - ctx.curVal += currentVal; - } - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java index 8ac45493840..ad4c9bf73b7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java @@ -27,9 +27,26 @@ protected Context(int chunkSize) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { intValueChunk = valuesChunk.asIntChunk(); } + + @Override + public void push(long key, int pos) { + // read the value from the values chunk + final int currentVal = intValueChunk.get(pos); + + if(curVal == NULL_LONG) { + curVal = currentVal == NULL_INT ? NULL_LONG : currentVal; + } else if (currentVal != NULL_INT) { + curVal += currentVal; + } + } + + @Override + public void reset() { + curVal = NULL_LONG; + } } public IntCumSumOperator(@NotNull final MatchPair pair, @@ -47,18 +64,4 @@ public IntCumSumOperator(@NotNull final MatchPair pair, public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } - - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - // read the value from the values chunk - final int currentVal = ctx.intValueChunk.get(pos); - - if(ctx.curVal == NULL_LONG) { - ctx.curVal = currentVal == NULL_INT ? NULL_LONG : currentVal; - } else if (currentVal != NULL_INT) { - ctx.curVal += currentVal; - } - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java index 8faf7124fcd..7e024136889 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java @@ -27,9 +27,26 @@ protected Context(int chunkSize) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { longValueChunk = valuesChunk.asLongChunk(); } + + @Override + public void push(long key, int pos) { + // read the value from the values chunk + final long currentVal = longValueChunk.get(pos); + + if(curVal == NULL_LONG) { + curVal = currentVal == NULL_LONG ? NULL_LONG : currentVal; + } else if (currentVal != NULL_LONG) { + curVal += currentVal; + } + } + + @Override + public void reset() { + curVal = NULL_LONG; + } } public LongCumSumOperator(@NotNull final MatchPair pair, @@ -47,18 +64,4 @@ public LongCumSumOperator(@NotNull final MatchPair pair, public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } - - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - // read the value from the values chunk - final long currentVal = ctx.longValueChunk.get(pos); - - if(ctx.curVal == NULL_LONG) { - ctx.curVal = currentVal == NULL_LONG ? NULL_LONG : currentVal; - } else if (currentVal != NULL_LONG) { - ctx.curVal += currentVal; - } - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java index fa8b9a24715..0d324653055 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java @@ -22,9 +22,26 @@ protected Context(int chunkSize) { } @Override - public void storeValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { shortValueChunk = valuesChunk.asShortChunk(); } + + @Override + public void push(long key, int pos) { + // read the value from the values chunk + final short currentVal = shortValueChunk.get(pos); + + if(curVal == NULL_LONG) { + curVal = currentVal == NULL_SHORT ? NULL_LONG : currentVal; + } else if (currentVal != NULL_SHORT) { + curVal += currentVal; + } + } + + @Override + public void reset() { + curVal = NULL_LONG; + } } public ShortCumSumOperator(@NotNull final MatchPair pair, @@ -42,18 +59,4 @@ public ShortCumSumOperator(@NotNull final MatchPair pair, public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { return new Context(chunkSize); } - - @Override - public void push(UpdateContext context, long key, int pos) { - final Context ctx = (Context) context; - - // read the value from the values chunk - final short currentVal = ctx.shortValueChunk.get(pos); - - if(ctx.curVal == NULL_LONG) { - ctx.curVal = currentVal == NULL_SHORT ? NULL_LONG : currentVal; - } else if (currentVal != NULL_SHORT) { - ctx.curVal += currentVal; - } - } } diff --git a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java index 0f95aa217a6..fd50e63e5fa 100644 --- a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java +++ b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java @@ -257,6 +257,7 @@ private static void fixupInteger(String intResult) throws IOException { "WritableIntegerChunk", "WritableIntChunk", "IntegerChunk", "IntChunk", "getInteger", "getInt", + "IntegerRingBuffer", "IntRingBuffer", "SizedIntegerChunk", "SizedIntChunk"); if (intResult.contains("Integer")) { FileUtils.writeLines(new File(intResult.replaceAll("Integer", "Int")), lines); From f3e9eae23523274495b1134d934641fa3b17692c Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 11 Oct 2022 16:52:33 -0700 Subject: [PATCH 028/123] Addressed some PR comments, more to do --- .../base/ringbuffer/ByteRingBuffer.java | 13 +++--- .../base/ringbuffer/CharRingBuffer.java | 13 +++--- .../base/ringbuffer/DoubleRingBuffer.java | 13 +++--- .../base/ringbuffer/FloatRingBuffer.java | 13 +++--- .../base/ringbuffer/IntRingBuffer.java | 13 +++--- .../base/ringbuffer/LongRingBuffer.java | 13 +++--- .../base/ringbuffer/ObjectRingBuffer.java | 13 +++--- .../base/ringbuffer/ShortRingBuffer.java | 13 +++--- .../base/ringbuffer/ByteRingBufferTest.java | 27 +++++++++++ .../base/ringbuffer/CharRingBufferTest.java | 27 +++++++++++ .../base/ringbuffer/DoubleRingBufferTest.java | 27 +++++++++++ .../base/ringbuffer/FloatRingBufferTest.java | 27 +++++++++++ .../base/ringbuffer/IntRingBufferTest.java | 27 +++++++++++ .../base/ringbuffer/LongRingBufferTest.java | 27 +++++++++++ .../base/ringbuffer/ObjectRingBufferTest.java | 27 +++++++++++ .../base/ringbuffer/ShortRingBufferTest.java | 27 +++++++++++ .../impl/BucketedPartitionedUpdateBy.java | 10 ++--- .../deephaven/engine/table/impl/UpdateBy.java | 29 ++++++++---- .../table/impl/updateby/UpdateByWindow.java | 45 ++++++++++++------- .../deephaven/engine/liveness/Liveness.java | 2 +- .../client/impl/UpdateByBuilder.java | 14 +++--- .../src/main/resources/dh-tests.prop | 7 +-- 22 files changed, 344 insertions(+), 83 deletions(-) diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java index f83655e9ba8..2c7644a4afa 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java @@ -181,11 +181,14 @@ public void remove() { } public byte[] getAll() { - int n = size(), h = head; - byte[] result = new byte[n]; - for (int i = 0; i < n; ++i) { - result[i] = storage[h]; - h = (h + 1) % storage.length; + byte[] result = new byte[size()]; + if (result.length > 0) { + if (tail > head) { + System.arraycopy(storage, head, result, 0, tail - head); + } else { + System.arraycopy(storage, head, result, 0, storage.length - head); + System.arraycopy(storage, 0, result, storage.length - head, tail); + } } return result; } diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java index 56a698a09fe..34d1060f38d 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java @@ -176,11 +176,14 @@ public void remove() { } public char[] getAll() { - int n = size(), h = head; - char[] result = new char[n]; - for (int i = 0; i < n; ++i) { - result[i] = storage[h]; - h = (h + 1) % storage.length; + char[] result = new char[size()]; + if (result.length > 0) { + if (tail > head) { + System.arraycopy(storage, head, result, 0, tail - head); + } else { + System.arraycopy(storage, head, result, 0, storage.length - head); + System.arraycopy(storage, 0, result, storage.length - head, tail); + } } return result; } diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java index f8f3f14ea0a..e39f94e46e6 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java @@ -181,11 +181,14 @@ public void remove() { } public double[] getAll() { - int n = size(), h = head; - double[] result = new double[n]; - for (int i = 0; i < n; ++i) { - result[i] = storage[h]; - h = (h + 1) % storage.length; + double[] result = new double[size()]; + if (result.length > 0) { + if (tail > head) { + System.arraycopy(storage, head, result, 0, tail - head); + } else { + System.arraycopy(storage, head, result, 0, storage.length - head); + System.arraycopy(storage, 0, result, storage.length - head, tail); + } } return result; } diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java index cb1ad5f4e1d..b62b4a7d36b 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java @@ -181,11 +181,14 @@ public void remove() { } public float[] getAll() { - int n = size(), h = head; - float[] result = new float[n]; - for (int i = 0; i < n; ++i) { - result[i] = storage[h]; - h = (h + 1) % storage.length; + float[] result = new float[size()]; + if (result.length > 0) { + if (tail > head) { + System.arraycopy(storage, head, result, 0, tail - head); + } else { + System.arraycopy(storage, head, result, 0, storage.length - head); + System.arraycopy(storage, 0, result, storage.length - head, tail); + } } return result; } diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java index 98012f97a9e..89218f977df 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java @@ -181,11 +181,14 @@ public void remove() { } public int[] getAll() { - int n = size(), h = head; - int[] result = new int[n]; - for (int i = 0; i < n; ++i) { - result[i] = storage[h]; - h = (h + 1) % storage.length; + int[] result = new int[size()]; + if (result.length > 0) { + if (tail > head) { + System.arraycopy(storage, head, result, 0, tail - head); + } else { + System.arraycopy(storage, head, result, 0, storage.length - head); + System.arraycopy(storage, 0, result, storage.length - head, tail); + } } return result; } diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java index 3f447516632..9011c0eef97 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java @@ -181,11 +181,14 @@ public void remove() { } public long[] getAll() { - int n = size(), h = head; - long[] result = new long[n]; - for (int i = 0; i < n; ++i) { - result[i] = storage[h]; - h = (h + 1) % storage.length; + long[] result = new long[size()]; + if (result.length > 0) { + if (tail > head) { + System.arraycopy(storage, head, result, 0, tail - head); + } else { + System.arraycopy(storage, head, result, 0, storage.length - head); + System.arraycopy(storage, 0, result, storage.length - head, tail); + } } return result; } diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/ObjectRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/ObjectRingBuffer.java index d3c6b6910a3..9c690c31666 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/ObjectRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/ObjectRingBuffer.java @@ -181,11 +181,14 @@ public void remove() { } public Object[] getAll() { - int n = size(), h = head; - Object[] result = new Object[n]; - for (int i = 0; i < n; ++i) { - result[i] = storage[h]; - h = (h + 1) % storage.length; + Object[] result = new Object[size()]; + if (result.length > 0) { + if (tail > head) { + System.arraycopy(storage, head, result, 0, tail - head); + } else { + System.arraycopy(storage, head, result, 0, storage.length - head); + System.arraycopy(storage, 0, result, storage.length - head, tail); + } } return result; } diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java index 03f7a6095f3..dcf75d3e6c7 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java @@ -181,11 +181,14 @@ public void remove() { } public short[] getAll() { - int n = size(), h = head; - short[] result = new short[n]; - for (int i = 0; i < n; ++i) { - result[i] = storage[h]; - h = (h + 1) % storage.length; + short[] result = new short[size()]; + if (result.length > 0) { + if (tail > head) { + System.arraycopy(storage, head, result, 0, tail - head); + } else { + System.arraycopy(storage, head, result, 0, storage.length - head); + System.arraycopy(storage, 0, result, storage.length - head, tail); + } } return result; } diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/ByteRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/ByteRingBufferTest.java index 87a89a104d1..f66ab828251 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/ByteRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/ByteRingBufferTest.java @@ -78,6 +78,14 @@ private void assertRemove(ByteRingBuffer rb, int expectedSize, byte expectedHead } } + private void assertContents(ByteRingBuffer rb, byte[] expectedData) { + final byte[] data = rb.getAll(); + assertEquals(data.length, expectedData.length); + for (int ii = 0; ii < data.length; ii++) { + assertEquals(data[ii], expectedData[ii]); + } + } + byte A = 'A'; byte B = 'B'; byte C = 'C'; @@ -94,19 +102,35 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); + assertContents(rb, new byte[] { A,B,C }); assertFull(rb); assertRemove(rb, 3, A); + assertContents(rb, new byte[] { B,C }); + assertRemove(rb, 2, B); + assertContents(rb, new byte[] { C }); + assertRemove(rb, 1, C); + assertContents(rb, new byte[0]); assertEmpty(rb); assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); + assertContents(rb, new byte[] { A,B }); + assertRemove(rb, 2, A); + assertContents(rb, new byte[] { B }); + assertAdd(rb, C, 2, B); + assertContents(rb, new byte[] { B,C }); + assertRemove(rb, 2, B); + assertContents(rb, new byte[] { C }); + assertRemove(rb, 1, C); + assertContents(rb, new byte[0]); + assertEmpty(rb); assertAdd(rb, A, 1, A); @@ -132,11 +156,13 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); + assertContents(rb, new byte[] { A,B,C }); assertFull(rb); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); assertAdd(rb, F, 6, A); + assertContents(rb, new byte[] { A,B,C,D,E,F }); assertRemove(rb, 6, A); assertRemove(rb, 5, B); @@ -144,6 +170,7 @@ public void testAddRemove() { assertRemove(rb, 3, D); assertRemove(rb, 2, E); assertRemove(rb, 1, F); + assertContents(rb, new byte[0]); assertEmpty(rb); } diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java index 80a98c13546..29dba185b62 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java @@ -73,6 +73,14 @@ private void assertRemove(CharRingBuffer rb, int expectedSize, char expectedHead } } + private void assertContents(CharRingBuffer rb, char[] expectedData) { + final char[] data = rb.getAll(); + assertEquals(data.length, expectedData.length); + for (int ii = 0; ii < data.length; ii++) { + assertEquals(data[ii], expectedData[ii]); + } + } + char A = 'A'; char B = 'B'; char C = 'C'; @@ -89,19 +97,35 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); + assertContents(rb, new char[] { A,B,C }); assertFull(rb); assertRemove(rb, 3, A); + assertContents(rb, new char[] { B,C }); + assertRemove(rb, 2, B); + assertContents(rb, new char[] { C }); + assertRemove(rb, 1, C); + assertContents(rb, new char[0]); assertEmpty(rb); assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); + assertContents(rb, new char[] { A,B }); + assertRemove(rb, 2, A); + assertContents(rb, new char[] { B }); + assertAdd(rb, C, 2, B); + assertContents(rb, new char[] { B,C }); + assertRemove(rb, 2, B); + assertContents(rb, new char[] { C }); + assertRemove(rb, 1, C); + assertContents(rb, new char[0]); + assertEmpty(rb); assertAdd(rb, A, 1, A); @@ -127,11 +151,13 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); + assertContents(rb, new char[] { A,B,C }); assertFull(rb); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); assertAdd(rb, F, 6, A); + assertContents(rb, new char[] { A,B,C,D,E,F }); assertRemove(rb, 6, A); assertRemove(rb, 5, B); @@ -139,6 +165,7 @@ public void testAddRemove() { assertRemove(rb, 3, D); assertRemove(rb, 2, E); assertRemove(rb, 1, F); + assertContents(rb, new char[0]); assertEmpty(rb); } diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/DoubleRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/DoubleRingBufferTest.java index 214d9c1a284..3b673e2fa2d 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/DoubleRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/DoubleRingBufferTest.java @@ -78,6 +78,14 @@ private void assertRemove(DoubleRingBuffer rb, int expectedSize, double expected } } + private void assertContents(DoubleRingBuffer rb, double[] expectedData) { + final double[] data = rb.getAll(); + assertEquals(data.length, expectedData.length); + for (int ii = 0; ii < data.length; ii++) { + assertEquals(data[ii], expectedData[ii]); + } + } + double A = 'A'; double B = 'B'; double C = 'C'; @@ -94,19 +102,35 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); + assertContents(rb, new double[] { A,B,C }); assertFull(rb); assertRemove(rb, 3, A); + assertContents(rb, new double[] { B,C }); + assertRemove(rb, 2, B); + assertContents(rb, new double[] { C }); + assertRemove(rb, 1, C); + assertContents(rb, new double[0]); assertEmpty(rb); assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); + assertContents(rb, new double[] { A,B }); + assertRemove(rb, 2, A); + assertContents(rb, new double[] { B }); + assertAdd(rb, C, 2, B); + assertContents(rb, new double[] { B,C }); + assertRemove(rb, 2, B); + assertContents(rb, new double[] { C }); + assertRemove(rb, 1, C); + assertContents(rb, new double[0]); + assertEmpty(rb); assertAdd(rb, A, 1, A); @@ -132,11 +156,13 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); + assertContents(rb, new double[] { A,B,C }); assertFull(rb); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); assertAdd(rb, F, 6, A); + assertContents(rb, new double[] { A,B,C,D,E,F }); assertRemove(rb, 6, A); assertRemove(rb, 5, B); @@ -144,6 +170,7 @@ public void testAddRemove() { assertRemove(rb, 3, D); assertRemove(rb, 2, E); assertRemove(rb, 1, F); + assertContents(rb, new double[0]); assertEmpty(rb); } diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/FloatRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/FloatRingBufferTest.java index 0eed8d1965c..f9b2dceb277 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/FloatRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/FloatRingBufferTest.java @@ -78,6 +78,14 @@ private void assertRemove(FloatRingBuffer rb, int expectedSize, float expectedHe } } + private void assertContents(FloatRingBuffer rb, float[] expectedData) { + final float[] data = rb.getAll(); + assertEquals(data.length, expectedData.length); + for (int ii = 0; ii < data.length; ii++) { + assertEquals(data[ii], expectedData[ii]); + } + } + float A = 'A'; float B = 'B'; float C = 'C'; @@ -94,19 +102,35 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); + assertContents(rb, new float[] { A,B,C }); assertFull(rb); assertRemove(rb, 3, A); + assertContents(rb, new float[] { B,C }); + assertRemove(rb, 2, B); + assertContents(rb, new float[] { C }); + assertRemove(rb, 1, C); + assertContents(rb, new float[0]); assertEmpty(rb); assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); + assertContents(rb, new float[] { A,B }); + assertRemove(rb, 2, A); + assertContents(rb, new float[] { B }); + assertAdd(rb, C, 2, B); + assertContents(rb, new float[] { B,C }); + assertRemove(rb, 2, B); + assertContents(rb, new float[] { C }); + assertRemove(rb, 1, C); + assertContents(rb, new float[0]); + assertEmpty(rb); assertAdd(rb, A, 1, A); @@ -132,11 +156,13 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); + assertContents(rb, new float[] { A,B,C }); assertFull(rb); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); assertAdd(rb, F, 6, A); + assertContents(rb, new float[] { A,B,C,D,E,F }); assertRemove(rb, 6, A); assertRemove(rb, 5, B); @@ -144,6 +170,7 @@ public void testAddRemove() { assertRemove(rb, 3, D); assertRemove(rb, 2, E); assertRemove(rb, 1, F); + assertContents(rb, new float[0]); assertEmpty(rb); } diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/IntRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/IntRingBufferTest.java index 3c798cfb97f..1099b07be12 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/IntRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/IntRingBufferTest.java @@ -78,6 +78,14 @@ private void assertRemove(IntRingBuffer rb, int expectedSize, int expectedHead) } } + private void assertContents(IntRingBuffer rb, int[] expectedData) { + final int[] data = rb.getAll(); + assertEquals(data.length, expectedData.length); + for (int ii = 0; ii < data.length; ii++) { + assertEquals(data[ii], expectedData[ii]); + } + } + int A = 'A'; int B = 'B'; int C = 'C'; @@ -94,19 +102,35 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); + assertContents(rb, new int[] { A,B,C }); assertFull(rb); assertRemove(rb, 3, A); + assertContents(rb, new int[] { B,C }); + assertRemove(rb, 2, B); + assertContents(rb, new int[] { C }); + assertRemove(rb, 1, C); + assertContents(rb, new int[0]); assertEmpty(rb); assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); + assertContents(rb, new int[] { A,B }); + assertRemove(rb, 2, A); + assertContents(rb, new int[] { B }); + assertAdd(rb, C, 2, B); + assertContents(rb, new int[] { B,C }); + assertRemove(rb, 2, B); + assertContents(rb, new int[] { C }); + assertRemove(rb, 1, C); + assertContents(rb, new int[0]); + assertEmpty(rb); assertAdd(rb, A, 1, A); @@ -132,11 +156,13 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); + assertContents(rb, new int[] { A,B,C }); assertFull(rb); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); assertAdd(rb, F, 6, A); + assertContents(rb, new int[] { A,B,C,D,E,F }); assertRemove(rb, 6, A); assertRemove(rb, 5, B); @@ -144,6 +170,7 @@ public void testAddRemove() { assertRemove(rb, 3, D); assertRemove(rb, 2, E); assertRemove(rb, 1, F); + assertContents(rb, new int[0]); assertEmpty(rb); } diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/LongRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/LongRingBufferTest.java index 8f94e96d03c..8619a66916e 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/LongRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/LongRingBufferTest.java @@ -78,6 +78,14 @@ private void assertRemove(LongRingBuffer rb, int expectedSize, long expectedHead } } + private void assertContents(LongRingBuffer rb, long[] expectedData) { + final long[] data = rb.getAll(); + assertEquals(data.length, expectedData.length); + for (int ii = 0; ii < data.length; ii++) { + assertEquals(data[ii], expectedData[ii]); + } + } + long A = 'A'; long B = 'B'; long C = 'C'; @@ -94,19 +102,35 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); + assertContents(rb, new long[] { A,B,C }); assertFull(rb); assertRemove(rb, 3, A); + assertContents(rb, new long[] { B,C }); + assertRemove(rb, 2, B); + assertContents(rb, new long[] { C }); + assertRemove(rb, 1, C); + assertContents(rb, new long[0]); assertEmpty(rb); assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); + assertContents(rb, new long[] { A,B }); + assertRemove(rb, 2, A); + assertContents(rb, new long[] { B }); + assertAdd(rb, C, 2, B); + assertContents(rb, new long[] { B,C }); + assertRemove(rb, 2, B); + assertContents(rb, new long[] { C }); + assertRemove(rb, 1, C); + assertContents(rb, new long[0]); + assertEmpty(rb); assertAdd(rb, A, 1, A); @@ -132,11 +156,13 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); + assertContents(rb, new long[] { A,B,C }); assertFull(rb); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); assertAdd(rb, F, 6, A); + assertContents(rb, new long[] { A,B,C,D,E,F }); assertRemove(rb, 6, A); assertRemove(rb, 5, B); @@ -144,6 +170,7 @@ public void testAddRemove() { assertRemove(rb, 3, D); assertRemove(rb, 2, E); assertRemove(rb, 1, F); + assertContents(rb, new long[0]); assertEmpty(rb); } diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/ObjectRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/ObjectRingBufferTest.java index 266f5ba2b49..5ca257383fa 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/ObjectRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/ObjectRingBufferTest.java @@ -78,6 +78,14 @@ private void assertRemove(ObjectRingBuffer rb, int expectedSize, Object expected } } + private void assertContents(ObjectRingBuffer rb, Object[] expectedData) { + final Object[] data = rb.getAll(); + assertEquals(data.length, expectedData.length); + for (int ii = 0; ii < data.length; ii++) { + assertEquals(data[ii], expectedData[ii]); + } + } + Object A = 'A'; Object B = 'B'; Object C = 'C'; @@ -94,19 +102,35 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); + assertContents(rb, new Object[] { A,B,C }); assertFull(rb); assertRemove(rb, 3, A); + assertContents(rb, new Object[] { B,C }); + assertRemove(rb, 2, B); + assertContents(rb, new Object[] { C }); + assertRemove(rb, 1, C); + assertContents(rb, new Object[0]); assertEmpty(rb); assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); + assertContents(rb, new Object[] { A,B }); + assertRemove(rb, 2, A); + assertContents(rb, new Object[] { B }); + assertAdd(rb, C, 2, B); + assertContents(rb, new Object[] { B,C }); + assertRemove(rb, 2, B); + assertContents(rb, new Object[] { C }); + assertRemove(rb, 1, C); + assertContents(rb, new Object[0]); + assertEmpty(rb); assertAdd(rb, A, 1, A); @@ -132,11 +156,13 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); + assertContents(rb, new Object[] { A,B,C }); assertFull(rb); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); assertAdd(rb, F, 6, A); + assertContents(rb, new Object[] { A,B,C,D,E,F }); assertRemove(rb, 6, A); assertRemove(rb, 5, B); @@ -144,6 +170,7 @@ public void testAddRemove() { assertRemove(rb, 3, D); assertRemove(rb, 2, E); assertRemove(rb, 1, F); + assertContents(rb, new Object[0]); assertEmpty(rb); } diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/ShortRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/ShortRingBufferTest.java index e5843b22e5d..0de586c5339 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/ShortRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/ShortRingBufferTest.java @@ -78,6 +78,14 @@ private void assertRemove(ShortRingBuffer rb, int expectedSize, short expectedHe } } + private void assertContents(ShortRingBuffer rb, short[] expectedData) { + final short[] data = rb.getAll(); + assertEquals(data.length, expectedData.length); + for (int ii = 0; ii < data.length; ii++) { + assertEquals(data[ii], expectedData[ii]); + } + } + short A = 'A'; short B = 'B'; short C = 'C'; @@ -94,19 +102,35 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); + assertContents(rb, new short[] { A,B,C }); assertFull(rb); assertRemove(rb, 3, A); + assertContents(rb, new short[] { B,C }); + assertRemove(rb, 2, B); + assertContents(rb, new short[] { C }); + assertRemove(rb, 1, C); + assertContents(rb, new short[0]); assertEmpty(rb); assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); + assertContents(rb, new short[] { A,B }); + assertRemove(rb, 2, A); + assertContents(rb, new short[] { B }); + assertAdd(rb, C, 2, B); + assertContents(rb, new short[] { B,C }); + assertRemove(rb, 2, B); + assertContents(rb, new short[] { C }); + assertRemove(rb, 1, C); + assertContents(rb, new short[0]); + assertEmpty(rb); assertAdd(rb, A, 1, A); @@ -132,11 +156,13 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); + assertContents(rb, new short[] { A,B,C }); assertFull(rb); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); assertAdd(rb, F, 6, A); + assertContents(rb, new short[] { A,B,C,D,E,F }); assertRemove(rb, 6, A); assertRemove(rb, 5, B); @@ -144,6 +170,7 @@ public void testAddRemove() { assertRemove(rb, 3, D); assertRemove(rb, 2, E); assertRemove(rb, 1, F); + assertContents(rb, new short[0]); assertEmpty(rb); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java index 0fa085fb84a..6fbb407c3ec 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java @@ -17,7 +17,7 @@ import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; /** - * An implementation of {@link UpdateBy} dedicated to zero key computation. + * An implementation of {@link UpdateBy} dedicated to bucketed computation. */ class BucketedPartitionedUpdateBy extends UpdateBy { private final BucketedPartitionedUpdateByListener listener; @@ -25,12 +25,13 @@ class BucketedPartitionedUpdateBy extends UpdateBy { private final QueryTable resultTable; /** - * Perform an updateBy without any key columns. + * Perform a bucketed updateBy using {@code byColumns} as the keys * * @param description the operation description * @param source the source table * @param ops the operations to perform * @param resultSources the result sources + * @param byColumns the columns to use for the bucket keys * @param redirContext the row redirection shared context * @param control the control object. * @return the result table @@ -111,7 +112,7 @@ public void onUpdate(@NotNull final TableUpdate upstream) { RowSetFactory.empty(), RowSetFactory.empty(), RowSetShiftData.EMPTY, - ModifiedColumnSet.ALL); + ModifiedColumnSet.EMPTY); redirContext.processUpdateForRedirection(fakeUpdate, source.getRowSet()); } @@ -191,9 +192,8 @@ protected void process() { if (lr.getModified().isNonempty()) { modifiedRowSet.insert(lr.getModified()); } - // Transform any untouched modified columns to the output. + // always transform, ZeroKey listener sets this independently of the modified rowset lr.modifiedColumnsTransformer.transform(lr.getModifiedColumnSet(), downstream.modifiedColumnSet); - }); // should not include actual adds as modifies modifiedRowSet.remove(downstream.added); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java index 9b9d5ca80e0..863116f0890 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java @@ -23,7 +23,6 @@ import org.jetbrains.annotations.Nullable; import java.util.*; -import java.util.stream.Stream; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; @@ -163,14 +162,28 @@ protected UpdateBy(@NotNull final UpdateByOperator[] operators, TIntObjectHashMap windowHashToOperatorIndicesMap = new TIntObjectHashMap<>(); for (int opIdx = 0; opIdx < operators.length; opIdx++) { - // get the hash - final int hash = UpdateByWindow.hashCodeFromOperator(operators[opIdx]); - - // add this if not found - if (!windowHashToOperatorIndicesMap.containsKey(hash)) { - windowHashToOperatorIndicesMap.put(hash, new TIntArrayList()); + int hash = UpdateByWindow.hashCodeFromOperator(operators[opIdx]); + boolean added = false; + + // rudimentary linear probing for collisions + while (!added) { + if (!windowHashToOperatorIndicesMap.containsKey(hash)) { + // does not exist, can add immediately + windowHashToOperatorIndicesMap.put(hash, new TIntArrayList()); + windowHashToOperatorIndicesMap.get(hash).add(opIdx); + added = true; + } else { + final int existingOpIdx = windowHashToOperatorIndicesMap.get(hash).get(0); + if (UpdateByWindow.isEquivalentWindow(operators[existingOpIdx], operators[opIdx])) { + // no collision, can add immediately + windowHashToOperatorIndicesMap.get(hash).add(opIdx); + added = true; + } else { + // there is a collision, increment hash and try again + hash++; + } + } } - windowHashToOperatorIndicesMap.get(hash).add(opIdx); } // store the operator information into the windows diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java index 6dc29babca3..382a17336c2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java @@ -1,31 +1,22 @@ package io.deephaven.engine.table.impl.updateby; -import io.deephaven.base.ringbuffer.LongRingBuffer; -import io.deephaven.base.verify.Assert; -import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.ModifiedColumnSet; import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.impl.QueryTable; -import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.util.SafeCloseable; -import org.apache.commons.lang3.mutable.MutableLong; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import java.util.*; -import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; -import static io.deephaven.util.QueryConstants.NULL_LONG; - public abstract class UpdateByWindow { @Nullable protected final String timestampColumnName; @@ -252,16 +243,14 @@ public String getTimestampColumnName() { protected static int hashCode(boolean windowed, @Nullable String timestampColumnName, long prevUnits, long fwdUnits) { - // treat all cumulative as identical, even if they rely on timestamps + // treat all cumulative ops as identical, even if they rely on timestamps if (!windowed) { - return Boolean.hashCode(windowed); + return Boolean.hashCode(false); } - // windowed are unique per timestamp column and window-size - int hash = Boolean.hashCode(windowed); - if (timestampColumnName != null) { - hash = 31 * hash + timestampColumnName.hashCode(); - } + // windowed ops are unique per type (ticks/time-based) and window dimensions + int hash = Boolean.hashCode(true); + hash = 31 * hash + Boolean.hashCode(timestampColumnName != null); hash = 31 * hash + Long.hashCode(prevUnits); hash = 31 * hash + Long.hashCode(fwdUnits); return hash; @@ -273,4 +262,28 @@ public static int hashCodeFromOperator(final UpdateByOperator op) { op.getPrevWindowUnits(), op.getPrevWindowUnits()); } + + public static boolean isEquivalentWindow(final UpdateByOperator opA, final UpdateByOperator opB) { + final boolean aWindowed = opA instanceof UpdateByWindowedOperator; + final boolean bWindowed = opB instanceof UpdateByWindowedOperator; + + // equivalent if both are cumulative, not equivalent if only one is cumulative + if (!aWindowed && !bWindowed) { + return true; + } else if (!aWindowed) { + return false; + } else if (!bWindowed) { + return false; + } + + final boolean aTimeWindowed = opA.getTimestampColumnName() != null; + final boolean bTimeWindowed = opB.getTimestampColumnName() != null; + + // must have same time/tick base to be equivalent + if (aTimeWindowed != bTimeWindowed) { + return false; + } + return opA.getPrevWindowUnits() == opB.getPrevWindowUnits() && + opB.getFwdWindowUnits() == opB.getFwdWindowUnits(); + } } diff --git a/engine/updategraph/src/main/java/io/deephaven/engine/liveness/Liveness.java b/engine/updategraph/src/main/java/io/deephaven/engine/liveness/Liveness.java index 39a5628860b..9b6476ff4e8 100644 --- a/engine/updategraph/src/main/java/io/deephaven/engine/liveness/Liveness.java +++ b/engine/updategraph/src/main/java/io/deephaven/engine/liveness/Liveness.java @@ -34,7 +34,7 @@ public final class Liveness { Configuration.getInstance().getBooleanWithDefault("Liveness.heapDump", false); static final boolean CLEANUP_LOG_ENABLED = - Configuration.getInstance().getBooleanWithDefault("Liveness.cleanupLogEnabled", false); + Configuration.getInstance().getBooleanWithDefault("Liveness.cleanupLogEnabled", true); private static final long OUTSTANDING_COUNT_LOG_INTERVAL_MILLIS = 1000L; diff --git a/java-client/session/src/main/java/io/deephaven/client/impl/UpdateByBuilder.java b/java-client/session/src/main/java/io/deephaven/client/impl/UpdateByBuilder.java index 12265dd6f62..a3816ab21cb 100644 --- a/java-client/session/src/main/java/io/deephaven/client/impl/UpdateByBuilder.java +++ b/java-client/session/src/main/java/io/deephaven/client/impl/UpdateByBuilder.java @@ -8,13 +8,7 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.api.updateby.UpdateByControl; import io.deephaven.api.updateby.UpdateByOperation; -import io.deephaven.api.updateby.spec.CumMinMaxSpec; -import io.deephaven.api.updateby.spec.CumProdSpec; -import io.deephaven.api.updateby.spec.CumSumSpec; -import io.deephaven.api.updateby.spec.EmaSpec; -import io.deephaven.api.updateby.spec.FillBySpec; -import io.deephaven.api.updateby.spec.TimeScale; -import io.deephaven.api.updateby.spec.UpdateBySpec; +import io.deephaven.api.updateby.spec.*; import io.deephaven.proto.backplane.grpc.UpdateByRequest; import io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn; import io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByCumulativeMax; @@ -157,6 +151,12 @@ public UpdateByColumn.UpdateBySpec visit(CumProdSpec p) { .setProduct(UpdateByCumulativeProduct.getDefaultInstance()) .build(); } + + // TODO: add this correctly to `table.proto` + @Override + public UpdateByColumn.UpdateBySpec visit(RollingSumSpec rs) { + return null; + } } static UpdateByColumn.UpdateBySpec adapt(UpdateBySpec spec) { diff --git a/props/test-configs/src/main/resources/dh-tests.prop b/props/test-configs/src/main/resources/dh-tests.prop index e1472e396d3..c9be3d9d596 100644 --- a/props/test-configs/src/main/resources/dh-tests.prop +++ b/props/test-configs/src/main/resources/dh-tests.prop @@ -94,9 +94,4 @@ BarrageMessageProducer.subscriptionGrowthEnabled=true BarrageMessageProducer.targetSnapshotPercentage=0.1 BarrageMessageProducer.minSnapshotCellCount=50 BarrageMessageProducer.maxSnapshotCellCount=50 -BarrageStreamGenerator.batchSize=4 - -#BarrageTable.debug=true -#BarrageMessageProducer.debug=true - -ReleaseTracker.captureStackTraces=true \ No newline at end of file +BarrageStreamGenerator.batchSize=4 \ No newline at end of file From 818292639906f7ed1ef87c4f3054d2178a71db67 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 11 Oct 2022 17:56:23 -0700 Subject: [PATCH 029/123] Add'l PR comments addressed and spotless applied --- .../base/ringbuffer/ByteRingBufferTest.java | 18 ++++++------ .../base/ringbuffer/CharRingBufferTest.java | 18 ++++++------ .../base/ringbuffer/DoubleRingBufferTest.java | 18 ++++++------ .../base/ringbuffer/FloatRingBufferTest.java | 18 ++++++------ .../base/ringbuffer/IntRingBufferTest.java | 18 ++++++------ .../base/ringbuffer/LongRingBufferTest.java | 18 ++++++------ .../base/ringbuffer/ObjectRingBufferTest.java | 18 ++++++------ .../base/ringbuffer/ShortRingBufferTest.java | 18 ++++++------ .../engine/table/impl/ZeroKeyUpdateBy.java | 14 +++++---- .../updateby/UpdateByWindowCumulative.java | 22 ++++++++------ .../impl/updateby/UpdateByWindowTicks.java | 3 +- .../impl/updateby/UpdateByWindowTime.java | 3 +- .../updateby/ema/BigDecimalEMAOperator.java | 2 +- .../updateby/ema/BigIntegerEMAOperator.java | 2 +- .../internal/PairwiseDoubleRingBuffer.java | 29 +++++++++++-------- .../internal/PairwiseFloatRingBuffer.java | 29 +++++++++++-------- 16 files changed, 131 insertions(+), 117 deletions(-) diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/ByteRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/ByteRingBufferTest.java index f66ab828251..f67ba44d124 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/ByteRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/ByteRingBufferTest.java @@ -102,14 +102,14 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertContents(rb, new byte[] { A,B,C }); + assertContents(rb, new byte[] {A, B, C}); assertFull(rb); assertRemove(rb, 3, A); - assertContents(rb, new byte[] { B,C }); + assertContents(rb, new byte[] {B, C}); assertRemove(rb, 2, B); - assertContents(rb, new byte[] { C }); + assertContents(rb, new byte[] {C}); assertRemove(rb, 1, C); assertContents(rb, new byte[0]); @@ -117,16 +117,16 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); - assertContents(rb, new byte[] { A,B }); + assertContents(rb, new byte[] {A, B}); assertRemove(rb, 2, A); - assertContents(rb, new byte[] { B }); + assertContents(rb, new byte[] {B}); assertAdd(rb, C, 2, B); - assertContents(rb, new byte[] { B,C }); + assertContents(rb, new byte[] {B, C}); assertRemove(rb, 2, B); - assertContents(rb, new byte[] { C }); + assertContents(rb, new byte[] {C}); assertRemove(rb, 1, C); assertContents(rb, new byte[0]); @@ -156,13 +156,13 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertContents(rb, new byte[] { A,B,C }); + assertContents(rb, new byte[] {A, B, C}); assertFull(rb); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); assertAdd(rb, F, 6, A); - assertContents(rb, new byte[] { A,B,C,D,E,F }); + assertContents(rb, new byte[] {A, B, C, D, E, F}); assertRemove(rb, 6, A); assertRemove(rb, 5, B); diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java index 29dba185b62..c968881bb3e 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java @@ -97,14 +97,14 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertContents(rb, new char[] { A,B,C }); + assertContents(rb, new char[] {A, B, C}); assertFull(rb); assertRemove(rb, 3, A); - assertContents(rb, new char[] { B,C }); + assertContents(rb, new char[] {B, C}); assertRemove(rb, 2, B); - assertContents(rb, new char[] { C }); + assertContents(rb, new char[] {C}); assertRemove(rb, 1, C); assertContents(rb, new char[0]); @@ -112,16 +112,16 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); - assertContents(rb, new char[] { A,B }); + assertContents(rb, new char[] {A, B}); assertRemove(rb, 2, A); - assertContents(rb, new char[] { B }); + assertContents(rb, new char[] {B}); assertAdd(rb, C, 2, B); - assertContents(rb, new char[] { B,C }); + assertContents(rb, new char[] {B, C}); assertRemove(rb, 2, B); - assertContents(rb, new char[] { C }); + assertContents(rb, new char[] {C}); assertRemove(rb, 1, C); assertContents(rb, new char[0]); @@ -151,13 +151,13 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertContents(rb, new char[] { A,B,C }); + assertContents(rb, new char[] {A, B, C}); assertFull(rb); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); assertAdd(rb, F, 6, A); - assertContents(rb, new char[] { A,B,C,D,E,F }); + assertContents(rb, new char[] {A, B, C, D, E, F}); assertRemove(rb, 6, A); assertRemove(rb, 5, B); diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/DoubleRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/DoubleRingBufferTest.java index 3b673e2fa2d..494341d7015 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/DoubleRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/DoubleRingBufferTest.java @@ -102,14 +102,14 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertContents(rb, new double[] { A,B,C }); + assertContents(rb, new double[] {A, B, C}); assertFull(rb); assertRemove(rb, 3, A); - assertContents(rb, new double[] { B,C }); + assertContents(rb, new double[] {B, C}); assertRemove(rb, 2, B); - assertContents(rb, new double[] { C }); + assertContents(rb, new double[] {C}); assertRemove(rb, 1, C); assertContents(rb, new double[0]); @@ -117,16 +117,16 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); - assertContents(rb, new double[] { A,B }); + assertContents(rb, new double[] {A, B}); assertRemove(rb, 2, A); - assertContents(rb, new double[] { B }); + assertContents(rb, new double[] {B}); assertAdd(rb, C, 2, B); - assertContents(rb, new double[] { B,C }); + assertContents(rb, new double[] {B, C}); assertRemove(rb, 2, B); - assertContents(rb, new double[] { C }); + assertContents(rb, new double[] {C}); assertRemove(rb, 1, C); assertContents(rb, new double[0]); @@ -156,13 +156,13 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertContents(rb, new double[] { A,B,C }); + assertContents(rb, new double[] {A, B, C}); assertFull(rb); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); assertAdd(rb, F, 6, A); - assertContents(rb, new double[] { A,B,C,D,E,F }); + assertContents(rb, new double[] {A, B, C, D, E, F}); assertRemove(rb, 6, A); assertRemove(rb, 5, B); diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/FloatRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/FloatRingBufferTest.java index f9b2dceb277..cc6f13d78f0 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/FloatRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/FloatRingBufferTest.java @@ -102,14 +102,14 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertContents(rb, new float[] { A,B,C }); + assertContents(rb, new float[] {A, B, C}); assertFull(rb); assertRemove(rb, 3, A); - assertContents(rb, new float[] { B,C }); + assertContents(rb, new float[] {B, C}); assertRemove(rb, 2, B); - assertContents(rb, new float[] { C }); + assertContents(rb, new float[] {C}); assertRemove(rb, 1, C); assertContents(rb, new float[0]); @@ -117,16 +117,16 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); - assertContents(rb, new float[] { A,B }); + assertContents(rb, new float[] {A, B}); assertRemove(rb, 2, A); - assertContents(rb, new float[] { B }); + assertContents(rb, new float[] {B}); assertAdd(rb, C, 2, B); - assertContents(rb, new float[] { B,C }); + assertContents(rb, new float[] {B, C}); assertRemove(rb, 2, B); - assertContents(rb, new float[] { C }); + assertContents(rb, new float[] {C}); assertRemove(rb, 1, C); assertContents(rb, new float[0]); @@ -156,13 +156,13 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertContents(rb, new float[] { A,B,C }); + assertContents(rb, new float[] {A, B, C}); assertFull(rb); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); assertAdd(rb, F, 6, A); - assertContents(rb, new float[] { A,B,C,D,E,F }); + assertContents(rb, new float[] {A, B, C, D, E, F}); assertRemove(rb, 6, A); assertRemove(rb, 5, B); diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/IntRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/IntRingBufferTest.java index 1099b07be12..f0239fecdc8 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/IntRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/IntRingBufferTest.java @@ -102,14 +102,14 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertContents(rb, new int[] { A,B,C }); + assertContents(rb, new int[] {A, B, C}); assertFull(rb); assertRemove(rb, 3, A); - assertContents(rb, new int[] { B,C }); + assertContents(rb, new int[] {B, C}); assertRemove(rb, 2, B); - assertContents(rb, new int[] { C }); + assertContents(rb, new int[] {C}); assertRemove(rb, 1, C); assertContents(rb, new int[0]); @@ -117,16 +117,16 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); - assertContents(rb, new int[] { A,B }); + assertContents(rb, new int[] {A, B}); assertRemove(rb, 2, A); - assertContents(rb, new int[] { B }); + assertContents(rb, new int[] {B}); assertAdd(rb, C, 2, B); - assertContents(rb, new int[] { B,C }); + assertContents(rb, new int[] {B, C}); assertRemove(rb, 2, B); - assertContents(rb, new int[] { C }); + assertContents(rb, new int[] {C}); assertRemove(rb, 1, C); assertContents(rb, new int[0]); @@ -156,13 +156,13 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertContents(rb, new int[] { A,B,C }); + assertContents(rb, new int[] {A, B, C}); assertFull(rb); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); assertAdd(rb, F, 6, A); - assertContents(rb, new int[] { A,B,C,D,E,F }); + assertContents(rb, new int[] {A, B, C, D, E, F}); assertRemove(rb, 6, A); assertRemove(rb, 5, B); diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/LongRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/LongRingBufferTest.java index 8619a66916e..72e79963ce6 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/LongRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/LongRingBufferTest.java @@ -102,14 +102,14 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertContents(rb, new long[] { A,B,C }); + assertContents(rb, new long[] {A, B, C}); assertFull(rb); assertRemove(rb, 3, A); - assertContents(rb, new long[] { B,C }); + assertContents(rb, new long[] {B, C}); assertRemove(rb, 2, B); - assertContents(rb, new long[] { C }); + assertContents(rb, new long[] {C}); assertRemove(rb, 1, C); assertContents(rb, new long[0]); @@ -117,16 +117,16 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); - assertContents(rb, new long[] { A,B }); + assertContents(rb, new long[] {A, B}); assertRemove(rb, 2, A); - assertContents(rb, new long[] { B }); + assertContents(rb, new long[] {B}); assertAdd(rb, C, 2, B); - assertContents(rb, new long[] { B,C }); + assertContents(rb, new long[] {B, C}); assertRemove(rb, 2, B); - assertContents(rb, new long[] { C }); + assertContents(rb, new long[] {C}); assertRemove(rb, 1, C); assertContents(rb, new long[0]); @@ -156,13 +156,13 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertContents(rb, new long[] { A,B,C }); + assertContents(rb, new long[] {A, B, C}); assertFull(rb); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); assertAdd(rb, F, 6, A); - assertContents(rb, new long[] { A,B,C,D,E,F }); + assertContents(rb, new long[] {A, B, C, D, E, F}); assertRemove(rb, 6, A); assertRemove(rb, 5, B); diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/ObjectRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/ObjectRingBufferTest.java index 5ca257383fa..30f21d6c748 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/ObjectRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/ObjectRingBufferTest.java @@ -102,14 +102,14 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertContents(rb, new Object[] { A,B,C }); + assertContents(rb, new Object[] {A, B, C}); assertFull(rb); assertRemove(rb, 3, A); - assertContents(rb, new Object[] { B,C }); + assertContents(rb, new Object[] {B, C}); assertRemove(rb, 2, B); - assertContents(rb, new Object[] { C }); + assertContents(rb, new Object[] {C}); assertRemove(rb, 1, C); assertContents(rb, new Object[0]); @@ -117,16 +117,16 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); - assertContents(rb, new Object[] { A,B }); + assertContents(rb, new Object[] {A, B}); assertRemove(rb, 2, A); - assertContents(rb, new Object[] { B }); + assertContents(rb, new Object[] {B}); assertAdd(rb, C, 2, B); - assertContents(rb, new Object[] { B,C }); + assertContents(rb, new Object[] {B, C}); assertRemove(rb, 2, B); - assertContents(rb, new Object[] { C }); + assertContents(rb, new Object[] {C}); assertRemove(rb, 1, C); assertContents(rb, new Object[0]); @@ -156,13 +156,13 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertContents(rb, new Object[] { A,B,C }); + assertContents(rb, new Object[] {A, B, C}); assertFull(rb); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); assertAdd(rb, F, 6, A); - assertContents(rb, new Object[] { A,B,C,D,E,F }); + assertContents(rb, new Object[] {A, B, C, D, E, F}); assertRemove(rb, 6, A); assertRemove(rb, 5, B); diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/ShortRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/ShortRingBufferTest.java index 0de586c5339..9d1222fa7e0 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/ShortRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/ShortRingBufferTest.java @@ -102,14 +102,14 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertContents(rb, new short[] { A,B,C }); + assertContents(rb, new short[] {A, B, C}); assertFull(rb); assertRemove(rb, 3, A); - assertContents(rb, new short[] { B,C }); + assertContents(rb, new short[] {B, C}); assertRemove(rb, 2, B); - assertContents(rb, new short[] { C }); + assertContents(rb, new short[] {C}); assertRemove(rb, 1, C); assertContents(rb, new short[0]); @@ -117,16 +117,16 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); - assertContents(rb, new short[] { A,B }); + assertContents(rb, new short[] {A, B}); assertRemove(rb, 2, A); - assertContents(rb, new short[] { B }); + assertContents(rb, new short[] {B}); assertAdd(rb, C, 2, B); - assertContents(rb, new short[] { B,C }); + assertContents(rb, new short[] {B, C}); assertRemove(rb, 2, B); - assertContents(rb, new short[] { C }); + assertContents(rb, new short[] {C}); assertRemove(rb, 1, C); assertContents(rb, new short[0]); @@ -156,13 +156,13 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertContents(rb, new short[] { A,B,C }); + assertContents(rb, new short[] {A, B, C}); assertFull(rb); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); assertAdd(rb, F, 6, A); - assertContents(rb, new short[] { A,B,C,D,E,F }); + assertContents(rb, new short[] {A, B, C, D, E, F}); assertRemove(rb, 6, A); assertRemove(rb, 5, B); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java index 6fb17421b55..2cd2db4165d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java @@ -141,22 +141,24 @@ private void processUpdateForSsa(TableUpdate upstream) { upstream.modified().intSize() + Math.max(upstream.added().intSize(), upstream.removed().intSize()), (int) upstream.shifted().getEffectiveSize()); try (final RowSet fullPrevRowSet = source.getRowSet().copyPrev(); - final RowSet previousToShift = fullPrevRowSet.minus(restampRemovals); + final WritableRowSet previousToShift = fullPrevRowSet.minus(restampRemovals); final ColumnSource.GetContext getContext = timestampColumnSource.makeGetContext(size)) { + // no need to consider upstream removals + previousToShift.remove(upstream.removed()); + final RowSetShiftData.Iterator sit = upstream.shifted().applyIterator(); while (sit.hasNext()) { sit.next(); - try (final RowSet subRowSet = previousToShift.subSetByKeyRange(sit.beginRange(), sit.endRange()); - final RowSet rowSetToShift = subRowSet.minus(upstream.removed())) { - if (rowSetToShift.isEmpty()) { + try (final RowSet subRowSet = previousToShift.subSetByKeyRange(sit.beginRange(), sit.endRange())) { + if (subRowSet.isEmpty()) { continue; } final LongChunk shiftValues = - timestampColumnSource.getPrevChunk(getContext, rowSetToShift).asLongChunk(); + timestampColumnSource.getPrevChunk(getContext, subRowSet).asLongChunk(); - timestampSsa.applyShiftReverse(shiftValues, rowSetToShift.asRowKeyChunk(), sit.shiftDelta()); + timestampSsa.applyShiftReverse(shiftValues, subRowSet.asRowKeyChunk(), sit.shiftDelta()); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java index 9b922e2e71d..8c9c8184ca9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java @@ -57,8 +57,7 @@ public boolean computeAffectedRowsAndOperators(@NotNull final TableUpdate upstre // determine which operators are affected by this update boolean anyAffected = false; boolean allAffected = upstream.added().isNonempty() || - upstream.removed().isNonempty() || - upstream.shifted().nonempty(); + upstream.removed().isNonempty(); for (int opIdx = 0; opIdx < operators.length; opIdx++) { opAffected[opIdx] = allAffected @@ -93,25 +92,30 @@ public void processRows() { // find the key before the first affected row final long keyBefore; - try (final RowSet.SearchIterator sit = sourceRowSet.searchIterator()) { - keyBefore = sit.binarySearchValue( - (compareTo, ignored) -> Long.compare(affectedRows.firstRowKey() - 1, compareTo), 1); + try (final RowSet.SearchIterator rIt = sourceRowSet.reverseIterator()) { + rIt.advance(affectedRows.firstRowKey()); + if (rIt.hasNext()) { + keyBefore = rIt.nextLong(); + } else { + keyBefore = NULL_ROW_KEY; + } } // and preload that data for these operators for (int opIdx = 0; opIdx < operators.length; opIdx++) { if (opAffected[opIdx]) { UpdateByCumulativeOperator cumOp = (UpdateByCumulativeOperator) operators[opIdx]; - if (cumOp.getTimestampColumnName() == null || keyBefore == -1) { + if (cumOp.getTimestampColumnName() == null || keyBefore == NULL_ROW_KEY) { + // this operator doesn't care about timestamps or we know we are at the beginning of the rowset cumOp.initializeUpdate(opContext[opIdx], keyBefore, NULL_LONG); } else { + // this operator cares about timestamps, so make sure it is starting from a valid value and + // valid timestamp by moving backward until the conditions are met UpdateByCumulativeOperator.Context cumOpContext = (UpdateByCumulativeOperator.Context) opContext[opIdx]; - // make sure the time-based cumulative operators are starting from a valid value and timestamp long potentialResetTimestamp = timestampColumnSource.getLong(keyBefore); - if (potentialResetTimestamp == NULL_LONG || - !cumOpContext.isValueValid(keyBefore)) { + if (potentialResetTimestamp == NULL_LONG || !cumOpContext.isValueValid(keyBefore)) { try (final RowSet.SearchIterator rIt = sourceRowSet.reverseIterator()) { if (rIt.advance(keyBefore)) { while (rIt.hasNext()) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java index c38c7655857..8590c2b2d52 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java @@ -84,8 +84,7 @@ public boolean computeAffectedRowsAndOperators(@NotNull final TableUpdate upstre // determine which operators are affected by this update boolean anyAffected = false; boolean allAffected = upstream.added().isNonempty() || - upstream.removed().isNonempty() || - upstream.shifted().nonempty(); + upstream.removed().isNonempty(); for (int opIdx = 0; opIdx < operators.length; opIdx++) { opAffected[opIdx] = allAffected diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java index ee092c28a3a..4614d16c87f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java @@ -70,8 +70,7 @@ public boolean computeAffectedRowsAndOperators(@NotNull final TableUpdate upstre // determine which operators are affected by this update boolean anyAffected = false; boolean allAffected = upstream.added().isNonempty() || - upstream.removed().isNonempty() || - upstream.shifted().nonempty(); + upstream.removed().isNonempty(); for (int opIdx = 0; opIdx < operators.length; opIdx++) { opAffected[opIdx] = allAffected diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java index b9b5d920e8b..15c03583f85 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java @@ -71,7 +71,7 @@ public void push(long key, int pos) { } /** - * An operator that computes an EMA from a int column using an exponential decay function. + * An operator that computes an EMA from a BigDecimal column using an exponential decay function. * * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this ema diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java index 965f7632d29..3bc34e29da0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java @@ -68,7 +68,7 @@ public void push(long key, int pos) { } /** - * An operator that computes an EMA from a int column using an exponential decay function. + * An operator that computes an EMA from a BigInteger column using an exponential decay function. * * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this ema diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java index 83313540444..cf6de3cec26 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java @@ -16,15 +16,18 @@ import java.util.NoSuchElementException; /*** - * Store this data in the form of a binary tree where the latter half of the chunk is treated as a ring buffer and pairwise results of the `DoubleFunction` are stored in the parent nodes. We do lazy evaluation by maintaining a 'dirty' `BitSet` and compute the ultimate pairwise result only when requested by `evaluate()' + * Store this data in the form of a binary tree where the latter half of the chunk is treated as a ring buffer and + * pairwise results of the `DoubleFunction` are stored in the parent nodes. We do lazy evaluation by maintaining a + * 'dirty' index list and computing the ultimate pairwise result only when requested by `evaluate()' * - * To keep the parent-node finding math easy and consistent between the ring buffer and the computation tree, the binary tree is shifted by one index so the root (and final result of computation) ends up in index 1 (instead of 0 which is un-used) + * To keep the parent-node finding math easy and consistent between the ring buffer and the computation tree, the binary + * tree is shifted by one index so the root (and final result of computation) ends up in index 1 (instead of 0 which is + * un-used) */ public class PairwiseDoubleRingBuffer implements SafeCloseable { // use a sized double chunk for underlying storage private WritableDoubleChunk storageChunk; -// private final BitSet dirtyBits; private final TIntArrayList dirtyIndices; private boolean allDirty; @@ -51,11 +54,16 @@ public interface DoubleFunction { } /** - * Create a ring buffer for Double values that will perform pairwise evaluation of the internal data values using an efficient binary-tree implementation to compute only changed values. The buffer will grow exponentially as items are pushed into it but will not shrink as values are removed + * Create a ring buffer for double values that will perform pairwise evaluation of the internal data values using + * an efficient binary-tree implementation to compute only changed values. The buffer will grow exponentially as + * items are pushed into it but will not shrink as values are removed * * @param initialSize the minimum size for the structure to hold - * @param emptyVal an innocuous value that will not affect the user-provided function results. for example, 0.0f for performing addition/subtraction, 1.0f for performing multiplication/division - * @param pairwiseFunction the user provided function for evaluation, takes two double parameters and returns a double. This function will be applied repeatedly to pairs of data values until the final result is available + * @param emptyVal an innocuous value that will not affect the user-provided function results. for example, 0.0f + * for performing addition/subtraction, 1.0f for performing multiplication/division + * @param pairwiseFunction the user provided function for evaluation, takes two double parameters and returns a + * double. This function will be applied repeatedly to pairs of data values until the final + * result is available */ public PairwiseDoubleRingBuffer(int initialSize, double emptyVal, DoubleFunction pairwiseFunction) { // increase to next power of two @@ -72,7 +80,7 @@ public PairwiseDoubleRingBuffer(int initialSize, double emptyVal, DoubleFunction } private void evaluateRangeFast(int start, int end) { - // everything in this range needs recomputed + // everything in this range needs to be reevaluated for (int left = start & 0xFFFFFFFE; left < end; left += 2) { final int right = left + 1; final int parent = left / 2; @@ -81,7 +89,7 @@ private void evaluateRangeFast(int start, int end) { final double leftVal = storageChunk.get(left); final double rightVal = storageChunk.get(right); - // compute & stpre + // compute & store final double computeVal = pairwiseFunction.apply(leftVal, rightVal); storageChunk.set(parent, computeVal); @@ -99,12 +107,9 @@ public double evaluate() { evaluateRangeFast(head, chunkSize); evaluateRangeFast(capacity, tail); } - } else { - // sort so consecutive values are adjacent - dirtyIndices.sort(); } - // work through all the dirty bits from high to low until none remain + // work through all the dirty bits from high to low until none remain. int dirtyIndex = 0; while (dirtyIndex < dirtyIndices.size()) { final int left = dirtyIndices.get(dirtyIndex) & 0xFFFFFFFE; // clear the final bit to force evenness diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java index e2c9b3c2df8..2cd2e19ee3b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java @@ -11,15 +11,18 @@ import java.util.NoSuchElementException; /*** - * Store this data in the form of a binary tree where the latter half of the chunk is treated as a ring buffer and pairwise results of the `FloatFunction` are stored in the parent nodes. We do lazy evaluation by maintaining a 'dirty' `BitSet` and compute the ultimate pairwise result only when requested by `evaluate()' + * Store this data in the form of a binary tree where the latter half of the chunk is treated as a ring buffer and + * pairwise results of the `FloatFunction` are stored in the parent nodes. We do lazy evaluation by maintaining a + * 'dirty' index list and computing the ultimate pairwise result only when requested by `evaluate()' * - * To keep the parent-node finding math easy and consistent between the ring buffer and the computation tree, the binary tree is shifted by one index so the root (and final result of computation) ends up in index 1 (instead of 0 which is un-used) + * To keep the parent-node finding math easy and consistent between the ring buffer and the computation tree, the binary + * tree is shifted by one index so the root (and final result of computation) ends up in index 1 (instead of 0 which is + * un-used) */ public class PairwiseFloatRingBuffer implements SafeCloseable { // use a sized float chunk for underlying storage private WritableFloatChunk storageChunk; -// private final BitSet dirtyBits; private final TIntArrayList dirtyIndices; private boolean allDirty; @@ -46,11 +49,16 @@ public interface FloatFunction { } /** - * Create a ring buffer for Float values that will perform pairwise evaluation of the internal data values using an efficient binary-tree implementation to compute only changed values. The buffer will grow exponentially as items are pushed into it but will not shrink as values are removed + * Create a ring buffer for float values that will perform pairwise evaluation of the internal data values using + * an efficient binary-tree implementation to compute only changed values. The buffer will grow exponentially as + * items are pushed into it but will not shrink as values are removed * * @param initialSize the minimum size for the structure to hold - * @param emptyVal an innocuous value that will not affect the user-provided function results. for example, 0.0f for performing addition/subtraction, 1.0f for performing multiplication/division - * @param pairwiseFunction the user provided function for evaluation, takes two float parameters and returns a float. This function will be applied repeatedly to pairs of data values until the final result is available + * @param emptyVal an innocuous value that will not affect the user-provided function results. for example, 0.0f + * for performing addition/subtraction, 1.0f for performing multiplication/division + * @param pairwiseFunction the user provided function for evaluation, takes two float parameters and returns a + * float. This function will be applied repeatedly to pairs of data values until the final + * result is available */ public PairwiseFloatRingBuffer(int initialSize, float emptyVal, FloatFunction pairwiseFunction) { // increase to next power of two @@ -67,7 +75,7 @@ public PairwiseFloatRingBuffer(int initialSize, float emptyVal, FloatFunction pa } private void evaluateRangeFast(int start, int end) { - // everything in this range needs recomputed + // everything in this range needs to be reevaluated for (int left = start & 0xFFFFFFFE; left < end; left += 2) { final int right = left + 1; final int parent = left / 2; @@ -76,7 +84,7 @@ private void evaluateRangeFast(int start, int end) { final float leftVal = storageChunk.get(left); final float rightVal = storageChunk.get(right); - // compute & stpre + // compute & store final float computeVal = pairwiseFunction.apply(leftVal, rightVal); storageChunk.set(parent, computeVal); @@ -94,12 +102,9 @@ public float evaluate() { evaluateRangeFast(head, chunkSize); evaluateRangeFast(capacity, tail); } - } else { - // sort so consecutive values are adjacent - dirtyIndices.sort(); } - // work through all the dirty bits from high to low until none remain + // work through all the dirty bits from high to low until none remain. int dirtyIndex = 0; while (dirtyIndex < dirtyIndices.size()) { final int left = dirtyIndices.get(dirtyIndex) & 0xFFFFFFFE; // clear the final bit to force evenness From 2d223720fa0be1f77f0c44894d34dc287754b970 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Thu, 13 Oct 2022 13:23:06 -0700 Subject: [PATCH 030/123] Tests pass and a few bugs fixed --- .../deephaven/engine/table/impl/UpdateBy.java | 2 +- .../impl/UpdateByCumulativeOperator.java | 8 + .../table/impl/updateby/UpdateByWindow.java | 6 +- .../updateby/UpdateByWindowCumulative.java | 17 +- .../impl/updateby/UpdateByWindowTicks.java | 145 ++++++++++-------- .../impl/updateby/UpdateByWindowTime.java | 31 ++-- .../updateby/ema/BigDecimalEMAOperator.java | 87 +++++++---- .../updateby/ema/BigIntegerEMAOperator.java | 94 ++++++++---- .../updateby/ema/BigNumberEMAOperator.java | 2 + .../impl/updateby/ema/ByteEMAOperator.java | 96 ++++++++---- .../impl/updateby/ema/DoubleEMAOperator.java | 123 +++++++++------ .../impl/updateby/ema/FloatEMAOperator.java | 123 +++++++++------ .../impl/updateby/ema/IntEMAOperator.java | 96 ++++++++---- .../impl/updateby/ema/LongEMAOperator.java | 96 ++++++++---- .../impl/updateby/ema/ShortEMAOperator.java | 96 ++++++++---- .../internal/BaseByteUpdateByOperator.java | 20 +++ .../internal/BaseCharUpdateByOperator.java | 20 +++ .../internal/BaseDoubleUpdateByOperator.java | 20 +++ .../internal/BaseFloatUpdateByOperator.java | 20 +++ .../internal/BaseIntUpdateByOperator.java | 20 +++ .../internal/BaseLongUpdateByOperator.java | 20 +++ .../internal/BaseObjectUpdateByOperator.java | 20 +++ .../internal/BaseShortUpdateByOperator.java | 20 +++ .../BaseWindowedByteUpdateByOperator.java | 11 ++ .../BaseWindowedCharUpdateByOperator.java | 11 ++ .../BaseWindowedDoubleUpdateByOperator.java | 11 ++ .../BaseWindowedFloatUpdateByOperator.java | 11 ++ .../BaseWindowedIntUpdateByOperator.java | 11 ++ .../BaseWindowedLongUpdateByOperator.java | 11 ++ .../BaseWindowedObjectUpdateByOperator.java | 11 ++ .../BaseWindowedShortUpdateByOperator.java | 11 ++ .../BigDecimalRollingSumOperator.java | 3 - .../BigIntegerRollingSumOperator.java | 4 - .../rollingsum/ByteRollingSumOperator.java | 4 - .../rollingsum/DoubleRollingSumOperator.java | 4 - .../rollingsum/FloatRollingSumOperator.java | 4 - .../rollingsum/IntRollingSumOperator.java | 4 - .../rollingsum/LongRollingSumOperator.java | 4 - .../rollingsum/ShortRollingSumOperator.java | 4 - .../impl/updateby/TestUpdateByGeneral.java | 10 +- .../client/impl/UpdateBySpecBuilderTest.java | 13 +- .../api/updateby/spec/RollingSumSpec.java | 2 +- 42 files changed, 903 insertions(+), 423 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java index 863116f0890..37e3041dd30 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java @@ -165,7 +165,7 @@ protected UpdateBy(@NotNull final UpdateByOperator[] operators, int hash = UpdateByWindow.hashCodeFromOperator(operators[opIdx]); boolean added = false; - // rudimentary linear probing for collisions + // rudimentary collision detection and handling while (!added) { if (!windowHashToOperatorIndicesMap.containsKey(hash)) { // does not exist, can add immediately diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java index 55078719cce..933695ae723 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java @@ -2,8 +2,11 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.sized.SizedLongChunk; +import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.ChunkSink; import io.deephaven.engine.table.MatchPair; import io.deephaven.util.annotations.FinalDefault; @@ -42,6 +45,11 @@ public void close() {} public void pop() { throw new UnsupportedOperationException("Cumulative operators should never call pop()"); } + + public abstract void accumulate(RowSequence inputKeys, + WritableChunk valueChunk, + LongChunk tsChunk, + int len); } public UpdateByCumulativeOperator(@NotNull final MatchPair pair, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java index 382a17336c2..bcd4d1b011a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java @@ -83,10 +83,10 @@ public UpdateByWindowContext(final TrackingRowSet sourceRowSet, final ColumnSour this.opAffected = new boolean[operators.length]; this.opContext = new UpdateByOperator.UpdateContext[operators.length]; - this.inputSourceFillContexts = new ChunkSource.FillContext[operators.length]; - this.inputSourceChunkPopulated = new boolean[operators.length]; + this.inputSourceFillContexts = new ChunkSource.FillContext[inputSources.length]; + this.inputSourceChunkPopulated = new boolean[inputSources.length]; // noinspection unchecked - this.inputSourceChunks = new WritableChunk[operators.length]; + this.inputSourceChunks = new WritableChunk[inputSources.length]; this.chunkSize = chunkSize; this.initialStep = initialStep; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java index 8c9c8184ca9..6b3b73c06f0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java @@ -140,6 +140,7 @@ public void processRows() { timestampColumnSource == null ? null : timestampColumnSource.makeGetContext(chunkSize)) { while (it.hasMore()) { final RowSequence rs = it.getNextRowSequenceWithLength(chunkSize); + final int size = rs.intSize(); Arrays.fill(inputSourceChunkPopulated, false); // create the timestamp chunk if needed @@ -152,17 +153,13 @@ public void processRows() { // chunk prep prepareValuesChunkForSource(srcIdx, rs); - opContext[opIdx].setValuesChunk(inputSourceChunks[srcIdx]); - opContext[opIdx].setTimestampChunk(tsChunk); - // chunk processing - for (int ii = 0; ii < rs.size(); ii++) { - opContext[opIdx].push(NULL_ROW_KEY, ii); - opContext[opIdx].writeToOutputChunk(ii); - } - - // chunk output to column - opContext[opIdx].writeToOutputColumn(rs); + // make the specialized call for cumulative operators + ((UpdateByCumulativeOperator.Context) opContext[opIdx]).accumulate( + rs, + inputSourceChunks[srcIdx], + tsChunk, + size); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java index 8590c2b2d52..ece10b77f69 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java @@ -7,7 +7,9 @@ import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.impl.QueryTable; +import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.UpdateByOperator; +import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.util.datastructures.LongSizedDataStructure; import org.apache.commons.lang3.mutable.MutableLong; @@ -16,6 +18,9 @@ import java.util.Arrays; +import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; +import static io.deephaven.util.QueryConstants.NULL_LONG; + // this class is currently too big, should specialize into CumWindow, TickWindow, TimeWindow to simplify implementation public class UpdateByWindowTicks extends UpdateByWindow { protected final long prevUnits; @@ -169,69 +174,7 @@ private void loadNextInfluencerValueChunks() { nextInfluencerKey = influencerKeyChunk.get(nextInfluencerIndex); } - private void fillWindowTicks(long currentPos) { - // compute the head and tail positions (inclusive) - final long head = Math.max(0, currentPos - prevUnits + 1); - final long tail = Math.min(sourceRowSet.size() - 1, currentPos + fwdUnits); - - // pop out all values from the current window that are not in the new window - while (!currentWindowPositions.isEmpty() && currentWindowPositions.front() < head) { - // operator pop - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - opContext[opIdx].pop(); - } - } - currentWindowPositions.remove(); - } - - // if the window is empty, reset - if (currentWindowPositions.isEmpty()) { - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - opContext[opIdx].reset(); - } - } - } - - // skip values until they match the window (this can only happen on the initial addition of rows to the - // table, because we short-circuit the precise building of the influencer rows for efficiency) - while (nextInfluencerPos < head) { - nextInfluencerIndex++; - - if (nextInfluencerIndex < influencerPosChunk.size()) { - nextInfluencerPos = LongSizedDataStructure.intSize( - "updateBy window positions exceeded maximum size", - influencerPosChunk.get(nextInfluencerIndex)); - nextInfluencerKey = influencerKeyChunk.get(nextInfluencerIndex); - } else { - // try to bring in new data - loadNextInfluencerValueChunks(); - } - } - - // push matching values - while (nextInfluencerPos <= tail) { - // operator push - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - opContext[opIdx].push(nextInfluencerKey, nextInfluencerIndex); - } - } - currentWindowPositions.add(nextInfluencerPos); - nextInfluencerIndex++; - - if (nextInfluencerIndex < influencerPosChunk.size()) { - nextInfluencerPos = LongSizedDataStructure.intSize( - "updateBy window positions exceeded maximum size", - influencerPosChunk.get(nextInfluencerIndex)); - nextInfluencerKey = influencerKeyChunk.get(nextInfluencerIndex); - } else { - // try to bring in new data - loadNextInfluencerValueChunks(); - } - } - } + private void fillWindowTicks(long currentPos, long sourceRowSetSize) {} // this function process the affected rows chunkwise, but will call fillWindowTicks() for each // new row. fillWindowTicks() will advance the moving window (which is the same for all operators in this @@ -245,6 +188,14 @@ public void processRows() { modifiedBuilder = RowSetFactory.builderSequential(); } + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (opAffected[opIdx]) { + UpdateByWindowedOperator winOp = (UpdateByWindowedOperator) operators[opIdx]; + // call the specialized version of `intializeUpdate()` for these operators + winOp.initializeUpdate(opContext[opIdx]); + } + } + influencerIt = influencerRows.getRowSequenceIterator(); influencerPosIt = influencerPositions.getRowSequenceIterator(); @@ -254,19 +205,83 @@ public void processRows() { // load the first chunk of influencer values (fillWindowTicks() will call in future) loadNextInfluencerValueChunks(); + final long sourceRowSetSize = sourceRowSet.size(); + while (it.hasMore()) { final RowSequence rs = it.getNextRowSequenceWithLength(chunkSize); final RowSequence posRs = posIt.getNextRowSequenceWithLength(chunkSize); + final int size = rs.intSize(); final LongChunk posChunk = posRs.asRowKeyChunk(); // chunk processing - for (int ii = 0; ii < rs.size(); ii++) { + for (int ii = 0; ii < size; ii++) { // read the current position final long currentPos = posChunk.get(ii); // fill the operator windows (calls push/pop/reset as appropriate) - fillWindowTicks(currentPos); + // compute the head and tail positions (inclusive) + final long head = Math.max(0, currentPos - prevUnits + 1); + final long tail = Math.min(sourceRowSetSize - 1, currentPos + fwdUnits); + + // pop out all values from the current window that are not in the new window + while (!currentWindowPositions.isEmpty() && currentWindowPositions.front() < head) { + // operator pop + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (opAffected[opIdx]) { + opContext[opIdx].pop(); + } + } + currentWindowPositions.remove(); + } + + // if the window is empty, reset + if (currentWindowPositions.isEmpty()) { + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (opAffected[opIdx]) { + opContext[opIdx].reset(); + } + } + } + + // skip values until they match the window (this can only happen on the initial addition of rows + // to the + // table, because we short-circuit the precise building of the influencer rows for efficiency) + while (nextInfluencerPos < head) { + nextInfluencerIndex++; + + if (nextInfluencerIndex < influencerPosChunk.size()) { + nextInfluencerPos = LongSizedDataStructure.intSize( + "updateBy window positions exceeded maximum size", + influencerPosChunk.get(nextInfluencerIndex)); + nextInfluencerKey = influencerKeyChunk.get(nextInfluencerIndex); + } else { + // try to bring in new data + loadNextInfluencerValueChunks(); + } + } + + // push matching values + while (nextInfluencerPos <= tail) { + // operator push + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (opAffected[opIdx]) { + opContext[opIdx].push(nextInfluencerKey, nextInfluencerIndex); + } + } + currentWindowPositions.add(nextInfluencerPos); + nextInfluencerIndex++; + + if (nextInfluencerIndex < influencerPosChunk.size()) { + nextInfluencerPos = LongSizedDataStructure.intSize( + "updateBy window positions exceeded maximum size", + influencerPosChunk.get(nextInfluencerIndex)); + nextInfluencerKey = influencerKeyChunk.get(nextInfluencerIndex); + } else { + // try to bring in new data + loadNextInfluencerValueChunks(); + } + } // now the operators have seen the correct window data, write to the output chunk for (int opIdx = 0; opIdx < operators.length; opIdx++) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java index 4614d16c87f..80db5dd4a0c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java @@ -10,6 +10,7 @@ import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.impl.UpdateByOperator; +import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -90,16 +91,15 @@ public boolean computeAffectedRowsAndOperators(@NotNull final TableUpdate upstre // need a writable rowset WritableRowSet tmpAffected = computeAffectedRowsTime(sourceRowSet, changed, prevUnits, fwdUnits, - timestampColumnSource, timestampSsa); + timestampColumnSource, timestampSsa, false); // other rows can be affected by removes if (upstream.removed().isNonempty()) { try (final RowSet prev = sourceRowSet.copyPrev(); final WritableRowSet affectedByRemoves = computeAffectedRowsTime(prev, upstream.removed(), prevUnits, fwdUnits, - timestampColumnSource, timestampSsa)) { - // apply shifts to get back to pos-shift space - upstream.shifted().apply(affectedByRemoves); + timestampColumnSource, timestampSsa, true)) { + // we used the SSA (post-shift) to get these keys, no need to shift // retain only the rows that still exist in the sourceRowSet affectedByRemoves.retain(sourceRowSet); tmpAffected.insert(affectedByRemoves); @@ -110,7 +110,7 @@ public boolean computeAffectedRowsAndOperators(@NotNull final TableUpdate upstre // now get influencer rows for the affected rows influencerRows = computeInfluencerRowsTime(sourceRowSet, affectedRows, prevUnits, fwdUnits, - timestampColumnSource, timestampSsa); + timestampColumnSource, timestampSsa, false); makeOperatorContexts(); return true; @@ -215,6 +215,14 @@ public void processRows() { modifiedBuilder = RowSetFactory.builderSequential(); } + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (opAffected[opIdx]) { + UpdateByWindowedOperator winOp = (UpdateByWindowedOperator) operators[opIdx]; + // call the specialized version of `intializeUpdate()` for these operators + winOp.initializeUpdate(opContext[opIdx]); + } + } + influencerIt = influencerRows.getRowSequenceIterator(); try (final RowSequence.Iterator it = affectedRows.getRowSequenceIterator(); final ChunkSource.GetContext localTimestampContext = @@ -285,13 +293,16 @@ public UpdateByWindowContext makeWindowContext(final TrackingRowSet sourceRowSet } private static WritableRowSet computeAffectedRowsTime(final RowSet sourceSet, final RowSet subset, long revNanos, - long fwdNanos, final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa) { + long fwdNanos, final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa, + boolean usePrev) { // swap fwd/rev to get the affected windows - return computeInfluencerRowsTime(sourceSet, subset, fwdNanos, revNanos, timestampColumnSource, timestampSsa); + return computeInfluencerRowsTime(sourceSet, subset, fwdNanos, revNanos, timestampColumnSource, timestampSsa, + usePrev); } private static WritableRowSet computeInfluencerRowsTime(final RowSet sourceSet, final RowSet subset, long revNanos, - long fwdNanos, final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa) { + long fwdNanos, final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa, + boolean usePrev) { if (sourceSet.size() == subset.size()) { return sourceSet.copy(); } @@ -303,7 +314,9 @@ private static WritableRowSet computeInfluencerRowsTime(final RowSet sourceSet, LongSegmentedSortedArray.Iterator ssaIt = timestampSsa.iterator(false, false); while (it.hasMore() && ssaIt.hasNext()) { final RowSequence rs = it.getNextRowSequenceWithLength(chunkSize); - LongChunk timestamps = timestampColumnSource.getChunk(context, rs).asLongChunk(); + LongChunk timestamps = usePrev + ? timestampColumnSource.getPrevChunk(context, rs).asLongChunk() + : timestampColumnSource.getChunk(context, rs).asLongChunk(); for (int ii = 0; ii < rs.intSize(); ii++) { // if the timestamp of the row is null, it won't belong to any set and we can ignore it diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java index 15c03583f85..002f64fa32b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java @@ -4,7 +4,9 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.ObjectChunk; +import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; @@ -13,6 +15,7 @@ import org.jetbrains.annotations.Nullable; import java.math.BigDecimal; +import java.math.BigInteger; import static io.deephaven.util.QueryConstants.NULL_LONG; @@ -23,50 +26,74 @@ protected Context(int chunkSize, ColumnSource inputSource) { } @Override - public void push(long key, int pos) { - final BigDecimal input = objectValueChunk.get(pos); + public void accumulate(RowSequence inputKeys, + WritableChunk valueChunk, + LongChunk tsChunk, + int len) { + setValuesChunk(valueChunk); + setTimestampChunk(tsChunk); + + // chunk processing if (timestampColumnName == null) { // compute with ticks - if (input == null) { - handleBadData(this, true, false); - } else { - if (curVal == null) { - curVal = input; + for (int ii = 0; ii < len; ii++) { + // read the value from the values chunk + final BigDecimal input = objectValueChunk.get(ii); + if (input == null) { + handleBadData(this, true, false); } else { - curVal = curVal.multiply(alpha, control.bigValueContextOrDefault()) - .add(input.multiply(oneMinusAlpha, control.bigValueContextOrDefault()), - control.bigValueContextOrDefault()); + if (curVal == null) { + curVal = input; + } else { + curVal = curVal.multiply(alpha, control.bigValueContextOrDefault()) + .add(input.multiply(oneMinusAlpha, control.bigValueContextOrDefault()), + control.bigValueContextOrDefault()); + } } + outputValues.set(ii, curVal); } } else { // compute with time - final long timestamp = timestampValueChunk.get(pos); - final boolean isNull = input == null; - final boolean isNullTime = timestamp == NULL_LONG; - if (isNull || isNullTime) { - handleBadData(this, isNull, isNullTime); - } else { - if (curVal == null) { - curVal = input; - lastStamp = timestamp; + for (int ii = 0; ii < len; ii++) { + // read the value from the values chunk + final BigDecimal input = objectValueChunk.get(ii); + final long timestamp = tsChunk.get(ii); + final boolean isNull = input == null; + final boolean isNullTime = timestamp == NULL_LONG; + if (isNull || isNullTime) { + handleBadData(this, isNull, isNullTime); } else { - final long dt = timestamp - lastStamp; - if (dt <= 0) { - handleBadTime(this, dt); + if (curVal == null) { + curVal = input; + lastStamp = timestamp; } else { - // alpha is dynamic, based on time - BigDecimal alpha = BigDecimal.valueOf(Math.exp(-dt / timeScaleUnits)); - BigDecimal oneMinusAlpha = - BigDecimal.ONE.subtract(alpha, control.bigValueContextOrDefault()); + final long dt = timestamp - lastStamp; + if (dt <= 0) { + handleBadTime(this, dt); + } else { + // alpha is dynamic, based on time + BigDecimal alpha = BigDecimal.valueOf(Math.exp(-dt / timeScaleUnits)); + BigDecimal oneMinusAlpha = + BigDecimal.ONE.subtract(alpha, control.bigValueContextOrDefault()); - curVal = curVal.multiply(alpha, control.bigValueContextOrDefault()) - .add(input.multiply(oneMinusAlpha, control.bigValueContextOrDefault()), - control.bigValueContextOrDefault()); - lastStamp = timestamp; + curVal = curVal.multiply(alpha, control.bigValueContextOrDefault()) + .add(input.multiply(oneMinusAlpha, control.bigValueContextOrDefault()), + control.bigValueContextOrDefault()); + lastStamp = timestamp; + } } } + outputValues.set(ii, curVal); } } + + // chunk output to column + writeToOutputColumn(inputKeys); + } + + @Override + public void push(long key, int pos) { + throw new IllegalStateException("EMAOperator#push() is not used"); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java index 3bc34e29da0..2a9f6c5b7aa 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java @@ -1,6 +1,10 @@ package io.deephaven.engine.table.impl.updateby.ema; import io.deephaven.api.updateby.OperationControl; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.WritableChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; @@ -10,7 +14,7 @@ import java.math.BigDecimal; import java.math.BigInteger; -import static io.deephaven.util.QueryConstants.NULL_LONG; +import static io.deephaven.util.QueryConstants.*; public class BigIntegerEMAOperator extends BigNumberEMAOperator { public class Context extends BigNumberEMAOperator.Context { @@ -19,53 +23,77 @@ protected Context(int chunkSize, ColumnSource inputSource) { } @Override - public void push(long key, int pos) { - final BigInteger input = objectValueChunk.get(pos); + public void accumulate(RowSequence inputKeys, + WritableChunk valueChunk, + LongChunk tsChunk, + int len) { + setValuesChunk(valueChunk); + setTimestampChunk(tsChunk); + + // chunk processing if (timestampColumnName == null) { // compute with ticks - if (input == null) { - handleBadData(this, true, false); - } else { - final BigDecimal decimalInput = new BigDecimal(input, control.bigValueContextOrDefault()); - if(curVal == null) { - curVal = decimalInput; + for (int ii = 0; ii < len; ii++) { + // read the value from the values chunk + final BigInteger input = objectValueChunk.get(ii); + if (input == null) { + handleBadData(this, true, false); } else { - curVal = curVal.multiply(alpha, control.bigValueContextOrDefault()) - .add(decimalInput.multiply(oneMinusAlpha, control.bigValueContextOrDefault()), - control.bigValueContextOrDefault()); + final BigDecimal decimalInput = new BigDecimal(input, control.bigValueContextOrDefault()); + if (curVal == null) { + curVal = decimalInput; + } else { + curVal = curVal.multiply(alpha, control.bigValueContextOrDefault()) + .add(decimalInput.multiply(oneMinusAlpha, control.bigValueContextOrDefault()), + control.bigValueContextOrDefault()); + } } + outputValues.set(ii, curVal); } } else { // compute with time - final long timestamp = timestampValueChunk.get(pos); - final boolean isNull = input == null; - final boolean isNullTime = timestamp == NULL_LONG; - if (isNull || isNullTime) { - handleBadData(this, isNull, isNullTime); - } else { - final BigDecimal decimalInput = new BigDecimal(input, control.bigValueContextOrDefault()); - if(curVal == null) { - curVal = decimalInput; - lastStamp = timestamp; + for (int ii = 0; ii < len; ii++) { + // read the value from the values chunk + final BigInteger input = objectValueChunk.get(ii); + final long timestamp = tsChunk.get(ii); + final boolean isNull = input == null; + final boolean isNullTime = timestamp == NULL_LONG; + if (isNull || isNullTime) { + handleBadData(this, isNull, isNullTime); } else { - final long dt = timestamp - lastStamp; - if(dt <= 0) { - handleBadTime(this, dt); + final BigDecimal decimalInput = new BigDecimal(input, control.bigValueContextOrDefault()); + if (curVal == null) { + curVal = decimalInput; + lastStamp = timestamp; } else { - // alpha is dynamic, based on time - BigDecimal alpha = BigDecimal.valueOf(Math.exp(-dt / timeScaleUnits)); - BigDecimal oneMinusAlpha = BigDecimal.ONE.subtract(alpha, control.bigValueContextOrDefault()); + final long dt = timestamp - lastStamp; + if (dt <= 0) { + handleBadTime(this, dt); + } else { + // alpha is dynamic, based on time + BigDecimal alpha = BigDecimal.valueOf(Math.exp(-dt / timeScaleUnits)); + BigDecimal oneMinusAlpha = BigDecimal.ONE.subtract(alpha, control.bigValueContextOrDefault()); - curVal = curVal.multiply(alpha, control.bigValueContextOrDefault()) - .add(decimalInput.multiply(oneMinusAlpha, control.bigValueContextOrDefault()), - control.bigValueContextOrDefault()); - lastStamp = timestamp; + curVal = curVal.multiply(alpha, control.bigValueContextOrDefault()) + .add(decimalInput.multiply(oneMinusAlpha, control.bigValueContextOrDefault()), + control.bigValueContextOrDefault()); + lastStamp = timestamp; + } } } + outputValues.set(ii, curVal); } } + + // chunk output to column + writeToOutputColumn(inputKeys); } - } + + @Override + public void push(long key, int pos) { + throw new IllegalStateException("EMAOperator#push() is not used"); + } + } /** * An operator that computes an EMA from a BigInteger column using an exponential decay function. diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java index 893bf1b0215..11cb4134c6e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java @@ -38,6 +38,8 @@ protected Context(int chunkSize, ColumnSource inputSource) { this.valueSource = inputSource; } + + @Override public void setValuesChunk(@NotNull final Chunk valuesChunk) { objectValueChunk = valuesChunk.asObjectChunk(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java index bceb3a109e6..c415dcd05da 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java @@ -7,8 +7,11 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.ByteChunk; +import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; @@ -30,54 +33,79 @@ protected Context(int chunkSize, ColumnSource inputSource) { } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { - byteValueChunk = valuesChunk.asByteChunk(); - } - - @Override - public boolean isValueValid(long atKey) { - return valueSource.getByte(atKey) != NULL_BYTE; - } + public void accumulate(RowSequence inputKeys, + WritableChunk valueChunk, + LongChunk tsChunk, + int len) { + setValuesChunk(valueChunk); + setTimestampChunk(tsChunk); - @Override - public void push(long key, int pos) { - final byte input = byteValueChunk.get(pos); + // chunk processing if (timestampColumnName == null) { // compute with ticks - if(input == NULL_BYTE) { - handleBadData(this, true, false, false); - } else { - if(curVal == NULL_DOUBLE) { - curVal = input; + for (int ii = 0; ii < len; ii++) { + // read the value from the values chunk + final byte input = byteValueChunk.get(ii); + + if(input == NULL_BYTE) { + handleBadData(this, true, false, false); } else { - curVal = alpha * curVal + (oneMinusAlpha * input); + if(curVal == NULL_DOUBLE) { + curVal = input; + } else { + curVal = alpha * curVal + (oneMinusAlpha * input); + } } + outputValues.set(ii, curVal); } } else { // compute with time - final long timestamp = timestampValueChunk.get(pos); - //noinspection ConstantConditions - final boolean isNull = input == NULL_BYTE; - final boolean isNullTime = timestamp == NULL_LONG; - if(isNull || isNullTime) { - handleBadData(this, isNull, false, isNullTime); - } else { - if(curVal == NULL_DOUBLE) { - curVal = input; - lastStamp = timestamp; + for (int ii = 0; ii < len; ii++) { + // read the value from the values chunk + final byte input = byteValueChunk.get(ii); + final long timestamp = tsChunk.get(ii); + //noinspection ConstantConditions + final boolean isNull = input == NULL_BYTE; + final boolean isNullTime = timestamp == NULL_LONG; + if(isNull || isNullTime) { + handleBadData(this, isNull, false, isNullTime); } else { - final long dt = timestamp - lastStamp; - if(dt <= 0) { - handleBadTime(this, dt); - } else { - // alpha is dynamic, based on time - final double alpha = Math.exp(-dt / timeScaleUnits); - curVal = alpha * curVal + ((1 - alpha) * input); + if(curVal == NULL_DOUBLE) { + curVal = input; lastStamp = timestamp; + } else { + final long dt = timestamp - lastStamp; + if(dt <= 0) { + handleBadTime(this, dt); + } else { + // alpha is dynamic, based on time + final double alpha = Math.exp(-dt / timeScaleUnits); + curVal = alpha * curVal + ((1 - alpha) * input); + lastStamp = timestamp; + } } } + outputValues.set(ii, curVal); } } + + // chunk output to column + writeToOutputColumn(inputKeys); + } + + @Override + public void setValuesChunk(@NotNull final Chunk valuesChunk) { + byteValueChunk = valuesChunk.asByteChunk(); + } + + @Override + public boolean isValueValid(long atKey) { + return valueSource.getByte(atKey) != NULL_BYTE; + } + + @Override + public void push(long key, int pos) { + throw new IllegalStateException("EMAOperator#push() is not used"); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java index 2a06a1fe187..1c2448a0867 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java @@ -9,7 +9,10 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.DoubleChunk; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; @@ -29,6 +32,75 @@ protected Context(int chunkSize, ColumnSource inputSource) { this.valueSource = inputSource; } + @Override + public void accumulate(RowSequence inputKeys, + WritableChunk valueChunk, + LongChunk tsChunk, + int len) { + setValuesChunk(valueChunk); + setTimestampChunk(tsChunk); + + // chunk processing + if (timestampColumnName == null) { + // compute with ticks + for (int ii = 0; ii < len; ii++) { + // read the value from the values chunk + final double input = doubleValueChunk.get(ii); + final boolean isNull = input == NULL_DOUBLE; + final boolean isNan = Double.isNaN(input); + + if (isNull || isNan) { + handleBadData(this, isNull, isNan, false); + } else { + if (curVal == NULL_DOUBLE) { + curVal = input; + } else { + curVal = alpha * curVal + (oneMinusAlpha * input); + } + } + outputValues.set(ii, curVal); + } + } else { + // compute with time + for (int ii = 0; ii < len; ii++) { + // read the value from the values chunk + final double input = doubleValueChunk.get(ii); + final long timestamp = tsChunk.get(ii); + final boolean isNull = input == NULL_DOUBLE; + final boolean isNan = Double.isNaN(input); + final boolean isNullTime = timestamp == NULL_LONG; + // Handle bad data first + if (isNull || isNan || isNullTime) { + handleBadData(this, isNull, isNan, isNullTime); + } else if (curVal == NULL_DOUBLE) { + // If the data looks good, and we have a null ema, just accept the current value + curVal = input; + lastStamp = timestamp; + } else { + final boolean currentPoisoned = Double.isNaN(curVal); + if (currentPoisoned && lastStamp == NULL_LONG) { + // If the current EMA was a NaN, we should accept the first good timestamp so that + // we can handle reset behavior properly in the following else + lastStamp = timestamp; + } else { + final long dt = timestamp - lastStamp; + if (dt <= 0) { + handleBadTime(this, dt); + } else if (!currentPoisoned) { + final double alpha = Math.exp(-dt / timeScaleUnits); + curVal = alpha * curVal + ((1 - alpha) * input); + lastStamp = timestamp; + } + } + } + outputValues.set(ii, curVal); + } + } + + // chunk output to column + writeToOutputColumn(inputKeys); + } + @Override public void setValuesChunk(@NotNull final Chunk valuesChunk) { doubleValueChunk = valuesChunk.asDoubleChunk(); @@ -37,7 +109,7 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { @Override public boolean isValueValid(long atKey) { final double value = valueSource.getDouble(atKey); - if(value == NULL_DOUBLE) { + if (value == NULL_DOUBLE) { return false; } @@ -49,53 +121,8 @@ public boolean isValueValid(long atKey) { @Override public void push(long key, int pos) { - final double input = doubleValueChunk.get(pos); - final boolean isNull = input == NULL_DOUBLE; - final boolean isNan = Double.isNaN(input); - - if (timestampColumnName == null) { - // compute with ticks - if(isNull || isNan) { - handleBadData(this, isNull, isNan, false); - } else { - if(curVal == NULL_DOUBLE) { - curVal = input; - } else { - curVal = alpha * curVal + (oneMinusAlpha * input); - } - } - } else { - // compute with time - final long timestamp = timestampValueChunk.get(pos); - final boolean isNullTime = timestamp == NULL_LONG; - - - // Handle bad data first - if (isNull || isNan || isNullTime) { - handleBadData(this, isNull, isNan, isNullTime); - } else if (curVal == NULL_DOUBLE) { - // If the data looks good, and we have a null ema, just accept the current value - curVal = input; - lastStamp = timestamp; - } else { - final boolean currentPoisoned = Double.isNaN(curVal); - if (currentPoisoned && lastStamp == NULL_LONG) { - // If the current EMA was a NaN, we should accept the first good timestamp so that - // we can handle reset behavior properly in the following else - lastStamp = timestamp; - } else { - final long dt = timestamp - lastStamp; - if (dt <= 0) { - handleBadTime(this, dt); - } else if (!currentPoisoned) { - final double alpha = Math.exp(-dt / timeScaleUnits); - curVal = alpha * curVal + ((1 - alpha) * input); - lastStamp = timestamp; - } - } - } - } - } + throw new IllegalStateException("EMAOperator#push() is not used"); + } } /** diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java index dd25971cb46..d2afb17f338 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java @@ -4,7 +4,10 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.FloatChunk; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; @@ -24,6 +27,75 @@ protected Context(int chunkSize, ColumnSource inputSource) { this.valueSource = inputSource; } + @Override + public void accumulate(RowSequence inputKeys, + WritableChunk valueChunk, + LongChunk tsChunk, + int len) { + setValuesChunk(valueChunk); + setTimestampChunk(tsChunk); + + // chunk processing + if (timestampColumnName == null) { + // compute with ticks + for (int ii = 0; ii < len; ii++) { + // read the value from the values chunk + final float input = floatValueChunk.get(ii); + final boolean isNull = input == NULL_FLOAT; + final boolean isNan = Float.isNaN(input); + + if (isNull || isNan) { + handleBadData(this, isNull, isNan, false); + } else { + if (curVal == NULL_DOUBLE) { + curVal = input; + } else { + curVal = alpha * curVal + (oneMinusAlpha * input); + } + } + outputValues.set(ii, curVal); + } + } else { + // compute with time + for (int ii = 0; ii < len; ii++) { + // read the value from the values chunk + final float input = floatValueChunk.get(ii); + final long timestamp = tsChunk.get(ii); + final boolean isNull = input == NULL_FLOAT; + final boolean isNan = Float.isNaN(input); + final boolean isNullTime = timestamp == NULL_LONG; + // Handle bad data first + if (isNull || isNan || isNullTime) { + handleBadData(this, isNull, isNan, isNullTime); + } else if (curVal == NULL_DOUBLE) { + // If the data looks good, and we have a null ema, just accept the current value + curVal = input; + lastStamp = timestamp; + } else { + final boolean currentPoisoned = Double.isNaN(curVal); + if (currentPoisoned && lastStamp == NULL_LONG) { + // If the current EMA was a NaN, we should accept the first good timestamp so that + // we can handle reset behavior properly in the following else + lastStamp = timestamp; + } else { + final long dt = timestamp - lastStamp; + if (dt <= 0) { + handleBadTime(this, dt); + } else if (!currentPoisoned) { + final double alpha = Math.exp(-dt / timeScaleUnits); + curVal = alpha * curVal + ((1 - alpha) * input); + lastStamp = timestamp; + } + } + } + outputValues.set(ii, curVal); + } + } + + // chunk output to column + writeToOutputColumn(inputKeys); + } + @Override public void setValuesChunk(@NotNull final Chunk valuesChunk) { floatValueChunk = valuesChunk.asFloatChunk(); @@ -32,7 +104,7 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { @Override public boolean isValueValid(long atKey) { final float value = valueSource.getFloat(atKey); - if(value == NULL_FLOAT) { + if (value == NULL_FLOAT) { return false; } @@ -44,53 +116,8 @@ public boolean isValueValid(long atKey) { @Override public void push(long key, int pos) { - final float input = floatValueChunk.get(pos); - final boolean isNull = input == NULL_FLOAT; - final boolean isNan = Float.isNaN(input); - - if (timestampColumnName == null) { - // compute with ticks - if(isNull || isNan) { - handleBadData(this, isNull, isNan, false); - } else { - if(curVal == NULL_DOUBLE) { - curVal = input; - } else { - curVal = alpha * curVal + (oneMinusAlpha * input); - } - } - } else { - // compute with time - final long timestamp = timestampValueChunk.get(pos); - final boolean isNullTime = timestamp == NULL_LONG; - - - // Handle bad data first - if (isNull || isNan || isNullTime) { - handleBadData(this, isNull, isNan, isNullTime); - } else if (curVal == NULL_DOUBLE) { - // If the data looks good, and we have a null ema, just accept the current value - curVal = input; - lastStamp = timestamp; - } else { - final boolean currentPoisoned = Double.isNaN(curVal); - if (currentPoisoned && lastStamp == NULL_LONG) { - // If the current EMA was a NaN, we should accept the first good timestamp so that - // we can handle reset behavior properly in the following else - lastStamp = timestamp; - } else { - final long dt = timestamp - lastStamp; - if (dt <= 0) { - handleBadTime(this, dt); - } else if (!currentPoisoned) { - final double alpha = Math.exp(-dt / timeScaleUnits); - curVal = alpha * curVal + ((1 - alpha) * input); - lastStamp = timestamp; - } - } - } - } - } + throw new IllegalStateException("EMAOperator#push() is not used"); + } } /** diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java index df030e6a657..2b1a65969cd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java @@ -7,8 +7,11 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.IntChunk; +import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; @@ -30,54 +33,79 @@ protected Context(int chunkSize, ColumnSource inputSource) { } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { - intValueChunk = valuesChunk.asIntChunk(); - } - - @Override - public boolean isValueValid(long atKey) { - return valueSource.getInt(atKey) != NULL_INT; - } + public void accumulate(RowSequence inputKeys, + WritableChunk valueChunk, + LongChunk tsChunk, + int len) { + setValuesChunk(valueChunk); + setTimestampChunk(tsChunk); - @Override - public void push(long key, int pos) { - final int input = intValueChunk.get(pos); + // chunk processing if (timestampColumnName == null) { // compute with ticks - if(input == NULL_INT) { - handleBadData(this, true, false, false); - } else { - if(curVal == NULL_DOUBLE) { - curVal = input; + for (int ii = 0; ii < len; ii++) { + // read the value from the values chunk + final int input = intValueChunk.get(ii); + + if(input == NULL_INT) { + handleBadData(this, true, false, false); } else { - curVal = alpha * curVal + (oneMinusAlpha * input); + if(curVal == NULL_DOUBLE) { + curVal = input; + } else { + curVal = alpha * curVal + (oneMinusAlpha * input); + } } + outputValues.set(ii, curVal); } } else { // compute with time - final long timestamp = timestampValueChunk.get(pos); - //noinspection ConstantConditions - final boolean isNull = input == NULL_INT; - final boolean isNullTime = timestamp == NULL_LONG; - if(isNull || isNullTime) { - handleBadData(this, isNull, false, isNullTime); - } else { - if(curVal == NULL_DOUBLE) { - curVal = input; - lastStamp = timestamp; + for (int ii = 0; ii < len; ii++) { + // read the value from the values chunk + final int input = intValueChunk.get(ii); + final long timestamp = tsChunk.get(ii); + //noinspection ConstantConditions + final boolean isNull = input == NULL_INT; + final boolean isNullTime = timestamp == NULL_LONG; + if(isNull || isNullTime) { + handleBadData(this, isNull, false, isNullTime); } else { - final long dt = timestamp - lastStamp; - if(dt <= 0) { - handleBadTime(this, dt); - } else { - // alpha is dynamic, based on time - final double alpha = Math.exp(-dt / timeScaleUnits); - curVal = alpha * curVal + ((1 - alpha) * input); + if(curVal == NULL_DOUBLE) { + curVal = input; lastStamp = timestamp; + } else { + final long dt = timestamp - lastStamp; + if(dt <= 0) { + handleBadTime(this, dt); + } else { + // alpha is dynamic, based on time + final double alpha = Math.exp(-dt / timeScaleUnits); + curVal = alpha * curVal + ((1 - alpha) * input); + lastStamp = timestamp; + } } } + outputValues.set(ii, curVal); } } + + // chunk output to column + writeToOutputColumn(inputKeys); + } + + @Override + public void setValuesChunk(@NotNull final Chunk valuesChunk) { + intValueChunk = valuesChunk.asIntChunk(); + } + + @Override + public boolean isValueValid(long atKey) { + return valueSource.getInt(atKey) != NULL_INT; + } + + @Override + public void push(long key, int pos) { + throw new IllegalStateException("EMAOperator#push() is not used"); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java index 71bcc44b81b..516e237b0c4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java @@ -8,7 +8,10 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; @@ -30,54 +33,79 @@ protected Context(int chunkSize, ColumnSource inputSource) { } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { - longValueChunk = valuesChunk.asLongChunk(); - } - - @Override - public boolean isValueValid(long atKey) { - return valueSource.getLong(atKey) != NULL_LONG; - } + public void accumulate(RowSequence inputKeys, + WritableChunk valueChunk, + LongChunk tsChunk, + int len) { + setValuesChunk(valueChunk); + setTimestampChunk(tsChunk); - @Override - public void push(long key, int pos) { - final long input = longValueChunk.get(pos); + // chunk processing if (timestampColumnName == null) { // compute with ticks - if(input == NULL_LONG) { - handleBadData(this, true, false, false); - } else { - if(curVal == NULL_DOUBLE) { - curVal = input; + for (int ii = 0; ii < len; ii++) { + // read the value from the values chunk + final long input = longValueChunk.get(ii); + + if(input == NULL_LONG) { + handleBadData(this, true, false, false); } else { - curVal = alpha * curVal + (oneMinusAlpha * input); + if(curVal == NULL_DOUBLE) { + curVal = input; + } else { + curVal = alpha * curVal + (oneMinusAlpha * input); + } } + outputValues.set(ii, curVal); } } else { // compute with time - final long timestamp = timestampValueChunk.get(pos); - //noinspection ConstantConditions - final boolean isNull = input == NULL_LONG; - final boolean isNullTime = timestamp == NULL_LONG; - if(isNull || isNullTime) { - handleBadData(this, isNull, false, isNullTime); - } else { - if(curVal == NULL_DOUBLE) { - curVal = input; - lastStamp = timestamp; + for (int ii = 0; ii < len; ii++) { + // read the value from the values chunk + final long input = longValueChunk.get(ii); + final long timestamp = tsChunk.get(ii); + //noinspection ConstantConditions + final boolean isNull = input == NULL_LONG; + final boolean isNullTime = timestamp == NULL_LONG; + if(isNull || isNullTime) { + handleBadData(this, isNull, false, isNullTime); } else { - final long dt = timestamp - lastStamp; - if(dt <= 0) { - handleBadTime(this, dt); - } else { - // alpha is dynamic, based on time - final double alpha = Math.exp(-dt / timeScaleUnits); - curVal = alpha * curVal + ((1 - alpha) * input); + if(curVal == NULL_DOUBLE) { + curVal = input; lastStamp = timestamp; + } else { + final long dt = timestamp - lastStamp; + if(dt <= 0) { + handleBadTime(this, dt); + } else { + // alpha is dynamic, based on time + final double alpha = Math.exp(-dt / timeScaleUnits); + curVal = alpha * curVal + ((1 - alpha) * input); + lastStamp = timestamp; + } } } + outputValues.set(ii, curVal); } } + + // chunk output to column + writeToOutputColumn(inputKeys); + } + + @Override + public void setValuesChunk(@NotNull final Chunk valuesChunk) { + longValueChunk = valuesChunk.asLongChunk(); + } + + @Override + public boolean isValueValid(long atKey) { + return valueSource.getLong(atKey) != NULL_LONG; + } + + @Override + public void push(long key, int pos) { + throw new IllegalStateException("EMAOperator#push() is not used"); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java index c255a21977f..38fd0bcb8da 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java @@ -2,8 +2,11 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.ShortChunk; +import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; @@ -25,54 +28,79 @@ protected Context(int chunkSize, ColumnSource inputSource) { } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { - shortValueChunk = valuesChunk.asShortChunk(); - } - - @Override - public boolean isValueValid(long atKey) { - return valueSource.getShort(atKey) != NULL_SHORT; - } + public void accumulate(RowSequence inputKeys, + WritableChunk valueChunk, + LongChunk tsChunk, + int len) { + setValuesChunk(valueChunk); + setTimestampChunk(tsChunk); - @Override - public void push(long key, int pos) { - final short input = shortValueChunk.get(pos); + // chunk processing if (timestampColumnName == null) { // compute with ticks - if(input == NULL_SHORT) { - handleBadData(this, true, false, false); - } else { - if(curVal == NULL_DOUBLE) { - curVal = input; + for (int ii = 0; ii < len; ii++) { + // read the value from the values chunk + final short input = shortValueChunk.get(ii); + + if(input == NULL_SHORT) { + handleBadData(this, true, false, false); } else { - curVal = alpha * curVal + (oneMinusAlpha * input); + if(curVal == NULL_DOUBLE) { + curVal = input; + } else { + curVal = alpha * curVal + (oneMinusAlpha * input); + } } + outputValues.set(ii, curVal); } } else { // compute with time - final long timestamp = timestampValueChunk.get(pos); - //noinspection ConstantConditions - final boolean isNull = input == NULL_SHORT; - final boolean isNullTime = timestamp == NULL_LONG; - if(isNull || isNullTime) { - handleBadData(this, isNull, false, isNullTime); - } else { - if(curVal == NULL_DOUBLE) { - curVal = input; - lastStamp = timestamp; + for (int ii = 0; ii < len; ii++) { + // read the value from the values chunk + final short input = shortValueChunk.get(ii); + final long timestamp = tsChunk.get(ii); + //noinspection ConstantConditions + final boolean isNull = input == NULL_SHORT; + final boolean isNullTime = timestamp == NULL_LONG; + if(isNull || isNullTime) { + handleBadData(this, isNull, false, isNullTime); } else { - final long dt = timestamp - lastStamp; - if(dt <= 0) { - handleBadTime(this, dt); - } else { - // alpha is dynamic, based on time - final double alpha = Math.exp(-dt / timeScaleUnits); - curVal = alpha * curVal + ((1 - alpha) * input); + if(curVal == NULL_DOUBLE) { + curVal = input; lastStamp = timestamp; + } else { + final long dt = timestamp - lastStamp; + if(dt <= 0) { + handleBadTime(this, dt); + } else { + // alpha is dynamic, based on time + final double alpha = Math.exp(-dt / timeScaleUnits); + curVal = alpha * curVal + ((1 - alpha) * input); + lastStamp = timestamp; + } } } + outputValues.set(ii, curVal); } } + + // chunk output to column + writeToOutputColumn(inputKeys); + } + + @Override + public void setValuesChunk(@NotNull final Chunk valuesChunk) { + shortValueChunk = valuesChunk.asShortChunk(); + } + + @Override + public boolean isValueValid(long atKey) { + return valueSource.getShort(atKey) != NULL_SHORT; + } + + @Override + public void push(long key, int pos) { + throw new IllegalStateException("EMAOperator#push() is not used"); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java index 0c5a87d17d7..aefd387b4a2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java @@ -13,6 +13,7 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableByteChunk; +import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; @@ -51,6 +52,25 @@ protected Context(final int chunkSize) { this.outputValues = WritableByteChunk.makeWritableChunk(chunkSize); } + @Override + public void accumulate(RowSequence inputKeys, + WritableChunk valueChunk, + LongChunk tsChunk, + int len) { + + setValuesChunk(valueChunk); + setTimestampChunk(tsChunk); + + // chunk processing + for (int ii = 0; ii < len; ii++) { + push(NULL_ROW_KEY, ii); + writeToOutputChunk(ii); + } + + // chunk output to column + writeToOutputColumn(inputKeys); + } + @Override public void close() { super.close(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java index 0d251168aca..336f061ced2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java @@ -3,6 +3,7 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableCharChunk; +import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; @@ -40,6 +41,25 @@ protected Context(final int chunkSize) { this.outputValues = WritableCharChunk.makeWritableChunk(chunkSize); } + @Override + public void accumulate(RowSequence inputKeys, + WritableChunk valueChunk, + LongChunk tsChunk, + int len) { + + setValuesChunk(valueChunk); + setTimestampChunk(tsChunk); + + // chunk processing + for (int ii = 0; ii < len; ii++) { + push(NULL_ROW_KEY, ii); + writeToOutputChunk(ii); + } + + // chunk output to column + writeToOutputColumn(inputKeys); + } + @Override public void close() { super.close(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java index cd2c9153412..ec1524132cd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java @@ -8,6 +8,7 @@ import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.WritableDoubleChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.*; @@ -45,6 +46,25 @@ protected Context(final int chunkSize) { this.outputValues = WritableDoubleChunk.makeWritableChunk(chunkSize); } + @Override + public void accumulate(RowSequence inputKeys, + WritableChunk valueChunk, + LongChunk tsChunk, + int len) { + + setValuesChunk(valueChunk); + setTimestampChunk(tsChunk); + + // chunk processing + for (int ii = 0; ii < len; ii++) { + push(NULL_ROW_KEY, ii); + writeToOutputChunk(ii); + } + + // chunk output to column + writeToOutputColumn(inputKeys); + } + @Override public void close() { super.close(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java index cdfdf12b0b1..bf4ec7b9c3c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java @@ -3,6 +3,7 @@ import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.WritableFloatChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.*; @@ -40,6 +41,25 @@ protected Context(final int chunkSize) { this.outputValues = WritableFloatChunk.makeWritableChunk(chunkSize); } + @Override + public void accumulate(RowSequence inputKeys, + WritableChunk valueChunk, + LongChunk tsChunk, + int len) { + + setValuesChunk(valueChunk); + setTimestampChunk(tsChunk); + + // chunk processing + for (int ii = 0; ii < len; ii++) { + push(NULL_ROW_KEY, ii); + writeToOutputChunk(ii); + } + + // chunk output to column + writeToOutputColumn(inputKeys); + } + @Override public void close() { super.close(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java index f46caaadf2b..a03b6de66f0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java @@ -8,6 +8,7 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableIntChunk; +import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; @@ -45,6 +46,25 @@ protected Context(final int chunkSize) { this.outputValues = WritableIntChunk.makeWritableChunk(chunkSize); } + @Override + public void accumulate(RowSequence inputKeys, + WritableChunk valueChunk, + LongChunk tsChunk, + int len) { + + setValuesChunk(valueChunk); + setTimestampChunk(tsChunk); + + // chunk processing + for (int ii = 0; ii < len; ii++) { + push(NULL_ROW_KEY, ii); + writeToOutputChunk(ii); + } + + // chunk output to column + writeToOutputColumn(inputKeys); + } + @Override public void close() { super.close(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java index c921c5e227c..379c821817f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java @@ -8,6 +8,7 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableLongChunk; +import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; @@ -45,6 +46,25 @@ protected Context(final int chunkSize) { this.outputValues = WritableLongChunk.makeWritableChunk(chunkSize); } + @Override + public void accumulate(RowSequence inputKeys, + WritableChunk valueChunk, + LongChunk tsChunk, + int len) { + + setValuesChunk(valueChunk); + setTimestampChunk(tsChunk); + + // chunk processing + for (int ii = 0; ii < len; ii++) { + push(NULL_ROW_KEY, ii); + writeToOutputChunk(ii); + } + + // chunk output to column + writeToOutputColumn(inputKeys); + } + @Override public void close() { super.close(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java index 5661a536753..4d922066bc1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java @@ -8,6 +8,7 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableObjectChunk; +import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; @@ -46,6 +47,25 @@ protected Context(final int chunkSize) { this.outputValues = WritableObjectChunk.makeWritableChunk(chunkSize); } + @Override + public void accumulate(RowSequence inputKeys, + WritableChunk valueChunk, + LongChunk tsChunk, + int len) { + + setValuesChunk(valueChunk); + setTimestampChunk(tsChunk); + + // chunk processing + for (int ii = 0; ii < len; ii++) { + push(NULL_ROW_KEY, ii); + writeToOutputChunk(ii); + } + + // chunk output to column + writeToOutputColumn(inputKeys); + } + @Override public void close() { super.close(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java index c82b48fa107..e64845ccea2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java @@ -8,6 +8,7 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableShortChunk; +import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; @@ -45,6 +46,25 @@ protected Context(final int chunkSize) { this.outputValues = WritableShortChunk.makeWritableChunk(chunkSize); } + @Override + public void accumulate(RowSequence inputKeys, + WritableChunk valueChunk, + LongChunk tsChunk, + int len) { + + setValuesChunk(valueChunk); + setTimestampChunk(tsChunk); + + // chunk processing + for (int ii = 0; ii < len; ii++) { + push(NULL_ROW_KEY, ii); + writeToOutputChunk(ii); + } + + // chunk output to column + writeToOutputColumn(inputKeys); + } + @Override public void close() { super.close(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java index e5839f47134..4f165a0da20 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java @@ -123,6 +123,17 @@ protected WritableColumnSource makeDenseSource() { } // endregion extra-methods + @Override + public void initializeUpdate(@NotNull UpdateContext context) { + // If we're redirected we have to make sure we tell the output source it's actual size, or we're going + // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not + // need to do anything with capacity. + if(redirContext.isRedirected()) { + // The redirection index does not use the 0th index for some reason. + outputSource.ensureCapacity(redirContext.requiredCapacity()); + } + } + @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java index 9cb765e941b..ae68cbe8159 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java @@ -99,6 +99,17 @@ public BaseWindowedCharUpdateByOperator(@NotNull final MatchPair pair, // region extra-methods // endregion extra-methods + @Override + public void initializeUpdate(@NotNull UpdateContext context) { + // If we're redirected we have to make sure we tell the output source it's actual size, or we're going + // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not + // need to do anything with capacity. + if(redirContext.isRedirected()) { + // The redirection index does not use the 0th index for some reason. + outputSource.ensureCapacity(redirContext.requiredCapacity()); + } + } + @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java index 468d4bef0e6..d326849d512 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java @@ -103,6 +103,17 @@ public BaseWindowedDoubleUpdateByOperator(@NotNull final MatchPair pair, // region extra-methods // endregion extra-methods + @Override + public void initializeUpdate(@NotNull UpdateContext context) { + // If we're redirected we have to make sure we tell the output source it's actual size, or we're going + // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not + // need to do anything with capacity. + if(redirContext.isRedirected()) { + // The redirection index does not use the 0th index for some reason. + outputSource.ensureCapacity(redirContext.requiredCapacity()); + } + } + @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java index ea794d34671..24e4374eb94 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java @@ -98,6 +98,17 @@ public BaseWindowedFloatUpdateByOperator(@NotNull final MatchPair pair, // region extra-methods // endregion extra-methods + @Override + public void initializeUpdate(@NotNull UpdateContext context) { + // If we're redirected we have to make sure we tell the output source it's actual size, or we're going + // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not + // need to do anything with capacity. + if(redirContext.isRedirected()) { + // The redirection index does not use the 0th index for some reason. + outputSource.ensureCapacity(redirContext.requiredCapacity()); + } + } + @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java index a7c68886349..57d94dad329 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java @@ -104,6 +104,17 @@ public BaseWindowedIntUpdateByOperator(@NotNull final MatchPair pair, // region extra-methods // endregion extra-methods + @Override + public void initializeUpdate(@NotNull UpdateContext context) { + // If we're redirected we have to make sure we tell the output source it's actual size, or we're going + // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not + // need to do anything with capacity. + if(redirContext.isRedirected()) { + // The redirection index does not use the 0th index for some reason. + outputSource.ensureCapacity(redirContext.requiredCapacity()); + } + } + @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java index 7439501faee..e8921c4e0b7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java @@ -104,6 +104,17 @@ public BaseWindowedLongUpdateByOperator(@NotNull final MatchPair pair, // region extra-methods // endregion extra-methods + @Override + public void initializeUpdate(@NotNull UpdateContext context) { + // If we're redirected we have to make sure we tell the output source it's actual size, or we're going + // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not + // need to do anything with capacity. + if(redirContext.isRedirected()) { + // The redirection index does not use the 0th index for some reason. + outputSource.ensureCapacity(redirContext.requiredCapacity()); + } + } + @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java index 32d3ad35ced..ed51f997ae3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java @@ -107,6 +107,17 @@ public BaseWindowedObjectUpdateByOperator(@NotNull final MatchPair pair, // region extra-methods // endregion extra-methods + @Override + public void initializeUpdate(@NotNull UpdateContext context) { + // If we're redirected we have to make sure we tell the output source it's actual size, or we're going + // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not + // need to do anything with capacity. + if(redirContext.isRedirected()) { + // The redirection index does not use the 0th index for some reason. + outputSource.ensureCapacity(redirContext.requiredCapacity()); + } + } + @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java index 0bfcd5f9e93..ba2c1a1a29d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java @@ -104,6 +104,17 @@ public BaseWindowedShortUpdateByOperator(@NotNull final MatchPair pair, // region extra-methods // endregion extra-methods + @Override + public void initializeUpdate(@NotNull UpdateContext context) { + // If we're redirected we have to make sure we tell the output source it's actual size, or we're going + // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not + // need to do anything with capacity. + if(redirContext.isRedirected()) { + // The redirection index does not use the 0th index for some reason. + outputSource.ensureCapacity(redirContext.requiredCapacity()); + } + } + @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java index d2cc480f38d..7e8f3bb112c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java @@ -97,7 +97,4 @@ public BigDecimalRollingSumOperator(@NotNull final MatchPair pair, redirContext, BigDecimal.class); this.mathContext = mathContext; } - - @Override - public void initializeUpdate(@NotNull UpdateContext context) {} } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java index 77eb403010d..604898a6850 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java @@ -96,8 +96,4 @@ public BigIntegerRollingSumOperator(@NotNull final MatchPair pair, // region constructor // endregion constructor } - - @Override - public void initializeUpdate(@NotNull UpdateContext context) { - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java index 74d7c7383b4..7ed9fd6f66f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java @@ -107,8 +107,4 @@ public ByteRollingSumOperator(@NotNull final MatchPair pair, this.nullValue = nullValue; // endregion constructor } - - @Override - public void initializeUpdate(@NotNull UpdateContext context) { - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java index 610b63d4d0f..06410a09147 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java @@ -104,10 +104,6 @@ public DoubleRollingSumOperator(@NotNull final MatchPair pair, // endregion constructor } - @Override - public void initializeUpdate(@NotNull UpdateContext context) { - } - @NotNull @Override public Map> getOutputColumns() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java index f89134a70f0..845a1d3ba6a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java @@ -99,10 +99,6 @@ public FloatRollingSumOperator(@NotNull final MatchPair pair, // endregion constructor } - @Override - public void initializeUpdate(@NotNull UpdateContext context) { - } - @NotNull @Override public Map> getOutputColumns() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java index 3f3ae69f2c8..e2aa3b936d0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java @@ -104,8 +104,4 @@ public IntRollingSumOperator(@NotNull final MatchPair pair, // region constructor // endregion constructor } - - @Override - public void initializeUpdate(@NotNull UpdateContext context) { - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java index 061c35917b6..977543f8545 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java @@ -104,8 +104,4 @@ public LongRollingSumOperator(@NotNull final MatchPair pair, // region constructor // endregion constructor } - - @Override - public void initializeUpdate(@NotNull UpdateContext context) { - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java index a588064a37e..5e32f0b0c98 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java @@ -99,8 +99,4 @@ public ShortRollingSumOperator(@NotNull final MatchPair pair, // region constructor // endregion constructor } - - @Override - public void initializeUpdate(@NotNull UpdateContext context) { - } } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java index c573aea9788..3736d7cf4b9 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java @@ -14,6 +14,7 @@ import org.junit.Test; import org.junit.experimental.categories.Category; +import java.time.Duration; import java.util.*; import static io.deephaven.engine.table.impl.GenerateTableUpdates.generateAppends; @@ -45,6 +46,8 @@ public void testMixedAppendOnlyBucketed() { @Test public void testMixedGeneralZeroKey() { + doTestTicking(false, false, false, 20, 10, 12); + for (int size = 10; size <= 10000; size *= 10) { for (int seed = 10; seed < 20; seed++) { doTestTicking(seed > 15, false, false, 20, size, seed); @@ -78,8 +81,6 @@ private void doTestTicking(boolean redirected, boolean bucketed, boolean appendO final EvalNugget[] nuggets = new EvalNugget[] { new EvalNugget() { - Table base; - @Override protected Table e() { TableDefaults base = result.t; @@ -88,8 +89,13 @@ protected Table e() { } final String[] columnNamesArray = base.getDefinition().getColumnNamesArray(); + // NOTE: I can't include the float/double based Rolling final Collection clauses = List.of( UpdateByOperation.Fill(), + UpdateByOperation.RollingSum(100, 0, + makeOpColNames(columnNamesArray, "_rollsumticks", "Sym", "ts", "boolCol")), + UpdateByOperation.RollingSum("ts", Duration.ofMinutes(15), Duration.ofMinutes(0), + makeOpColNames(columnNamesArray, "_rollsumtime", "Sym", "ts", "boolCol")), UpdateByOperation.Ema(skipControl, "ts", 10 * MINUTE, makeOpColNames(columnNamesArray, "_ema", "Sym", "ts", "boolCol")), UpdateByOperation.CumSum(makeOpColNames(columnNamesArray, "_sum", "Sym", "ts")), diff --git a/java-client/session/src/test/java/io/deephaven/client/impl/UpdateBySpecBuilderTest.java b/java-client/session/src/test/java/io/deephaven/client/impl/UpdateBySpecBuilderTest.java index 42e04b0f201..35445cb3dd3 100644 --- a/java-client/session/src/test/java/io/deephaven/client/impl/UpdateBySpecBuilderTest.java +++ b/java-client/session/src/test/java/io/deephaven/client/impl/UpdateBySpecBuilderTest.java @@ -2,12 +2,7 @@ import io.deephaven.api.updateby.BadDataBehavior; import io.deephaven.api.updateby.OperationControl; -import io.deephaven.api.updateby.spec.CumMinMaxSpec; -import io.deephaven.api.updateby.spec.CumProdSpec; -import io.deephaven.api.updateby.spec.CumSumSpec; -import io.deephaven.api.updateby.spec.EmaSpec; -import io.deephaven.api.updateby.spec.FillBySpec; -import io.deephaven.api.updateby.spec.UpdateBySpec; +import io.deephaven.api.updateby.spec.*; import io.deephaven.api.updateby.spec.UpdateBySpec.Visitor; import io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn; import io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByCumulativeMax; @@ -72,6 +67,12 @@ public UpdateByColumn.UpdateBySpec visit(CumProdSpec p) { return UpdateByColumn.UpdateBySpec.newBuilder().setProduct(UpdateByCumulativeProduct.getDefaultInstance()) .build(); } + + // TODO: add this properly + @Override + public UpdateByColumn.UpdateBySpec visit(RollingSumSpec p) { + return null; + } } @Test diff --git a/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java b/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java index 27cf0ca85e4..a641d24e048 100644 --- a/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java +++ b/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java @@ -35,7 +35,7 @@ public static RollingSumSpec ofTime(final String timestampCol, Duration prevWind TimeScale.ofTime(timestampCol, fwdWindowDuration)); } - // general use contructors + // general use constructors public static RollingSumSpec of(TimeScale prevTimeScale) { return ImmutableRollingSumSpec.builder().prevTimeScale(prevTimeScale).build(); From 0c6a7264f5c27e6aa2bf53be8d42d4422526b655 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 14 Oct 2022 13:29:51 -0700 Subject: [PATCH 031/123] Refactored the windowed operators to chunk-process data --- .../impl/UpdateByCumulativeOperator.java | 2 +- .../engine/table/impl/UpdateByOperator.java | 6 +- .../table/impl/UpdateByWindowedOperator.java | 40 +-- .../engine/table/impl/ZeroKeyUpdateBy.java | 6 +- .../table/impl/updateby/UpdateByWindow.java | 56 ++-- .../updateby/UpdateByWindowCumulative.java | 35 ++- .../impl/updateby/UpdateByWindowTicks.java | 209 +++++++++----- .../impl/updateby/UpdateByWindowTime.java | 264 +++++++++++------- .../updateby/ema/BigDecimalEMAOperator.java | 6 +- .../updateby/ema/BigIntegerEMAOperator.java | 4 +- .../updateby/ema/BigNumberEMAOperator.java | 4 +- .../impl/updateby/ema/ByteEMAOperator.java | 7 +- .../impl/updateby/ema/DoubleEMAOperator.java | 6 +- .../impl/updateby/ema/FloatEMAOperator.java | 6 +- .../impl/updateby/ema/IntEMAOperator.java | 7 +- .../impl/updateby/ema/LongEMAOperator.java | 7 +- .../impl/updateby/ema/ShortEMAOperator.java | 7 +- .../updateby/fill/BooleanFillByOperator.java | 4 +- .../updateby/fill/ByteFillByOperator.java | 4 +- .../updateby/fill/CharFillByOperator.java | 4 +- .../updateby/fill/DoubleFillByOperator.java | 4 +- .../updateby/fill/FloatFillByOperator.java | 4 +- .../impl/updateby/fill/IntFillByOperator.java | 4 +- .../updateby/fill/LongFillByOperator.java | 4 +- .../updateby/fill/ObjectFillByOperator.java | 4 +- .../updateby/fill/ShortFillByOperator.java | 4 +- .../internal/BaseByteUpdateByOperator.java | 4 +- .../internal/BaseCharUpdateByOperator.java | 4 +- .../internal/BaseDoubleUpdateByOperator.java | 4 +- .../internal/BaseFloatUpdateByOperator.java | 4 +- .../internal/BaseIntUpdateByOperator.java | 4 +- .../internal/BaseLongUpdateByOperator.java | 4 +- .../internal/BaseObjectBinaryOperator.java | 4 +- .../internal/BaseObjectUpdateByOperator.java | 4 +- .../internal/BaseShortUpdateByOperator.java | 4 +- .../BaseWindowedByteUpdateByOperator.java | 37 ++- .../BaseWindowedCharUpdateByOperator.java | 37 ++- .../BaseWindowedDoubleUpdateByOperator.java | 36 ++- .../BaseWindowedFloatUpdateByOperator.java | 36 ++- .../BaseWindowedIntUpdateByOperator.java | 37 ++- .../BaseWindowedLongUpdateByOperator.java | 37 ++- .../BaseWindowedObjectUpdateByOperator.java | 37 ++- .../BaseWindowedShortUpdateByOperator.java | 37 ++- .../minmax/ByteCumMinMaxOperator.java | 4 +- .../minmax/DoubleCumMinMaxOperator.java | 4 +- .../minmax/FloatCumMinMaxOperator.java | 4 +- .../updateby/minmax/IntCumMinMaxOperator.java | 4 +- .../minmax/LongCumMinMaxOperator.java | 4 +- .../minmax/ShortCumMinMaxOperator.java | 4 +- .../prod/BigDecimalCumProdOperator.java | 4 +- .../prod/BigIntegerCumProdOperator.java | 4 +- .../updateby/prod/ByteCumProdOperator.java | 4 +- .../updateby/prod/DoubleCumProdOperator.java | 4 +- .../updateby/prod/FloatCumProdOperator.java | 4 +- .../updateby/prod/IntCumProdOperator.java | 4 +- .../updateby/prod/LongCumProdOperator.java | 4 +- .../updateby/prod/ShortCumProdOperator.java | 4 +- .../BigDecimalRollingSumOperator.java | 4 +- .../BigIntegerRollingSumOperator.java | 4 +- .../rollingsum/ByteRollingSumOperator.java | 4 +- .../rollingsum/DoubleRollingSumOperator.java | 4 +- .../rollingsum/FloatRollingSumOperator.java | 4 +- .../rollingsum/IntRollingSumOperator.java | 4 +- .../rollingsum/LongRollingSumOperator.java | 4 +- .../rollingsum/ShortRollingSumOperator.java | 4 +- .../sum/BigDecimalCumSumOperator.java | 4 +- .../sum/BigIntegerCumSumOperator.java | 4 +- .../impl/updateby/sum/ByteCumSumOperator.java | 4 +- .../updateby/sum/DoubleCumSumOperator.java | 4 +- .../updateby/sum/FloatCumSumOperator.java | 4 +- .../impl/updateby/sum/IntCumSumOperator.java | 4 +- .../impl/updateby/sum/LongCumSumOperator.java | 4 +- .../updateby/sum/ShortCumSumOperator.java | 4 +- .../replicators/ReplicateUpdateBy.java | 3 +- 74 files changed, 762 insertions(+), 399 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java index 933695ae723..5ebc93aa79c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java @@ -47,7 +47,7 @@ public void pop() { } public abstract void accumulate(RowSequence inputKeys, - WritableChunk valueChunk, + Chunk valueChunk, LongChunk tsChunk, int len); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java index 96ed27f22ed..594c0a72927 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java @@ -45,7 +45,7 @@ public interface UpdateByOperator { */ interface UpdateContext extends SafeCloseable { - void setValuesChunk(@NotNull final Chunk valuesChunk); + void setValuesChunk(@NotNull final Chunk valuesChunk); void setTimestampChunk(@NotNull final LongChunk valuesChunk); @@ -66,7 +66,7 @@ interface UpdateContext extends SafeCloseable { void pop(); /** - * TODO: update doc + * Write the current value for this row to the output chunk */ void writeToOutputChunk(int outIdx); @@ -77,7 +77,7 @@ interface UpdateContext extends SafeCloseable { void reset(); /** - * TODO: update doc + * Write the output chunk to the output column */ void writeToOutputColumn(@NotNull final RowSequence inputKeys); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java index 591787bbddd..af2baf88538 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java @@ -1,6 +1,10 @@ package io.deephaven.engine.table.impl; import io.deephaven.api.updateby.OperationControl; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.IntChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.MatchPair; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -19,37 +23,15 @@ public abstract class UpdateByWindowedOperator implements UpdateByOperator { public abstract class Context implements UpdateContext { public int nullCount = 0; - // - // protected long currentInfluencerKey; - // - // candidate data for the window - // - // // allocate some chunks for holding the key, position and timestamp data - // protected SizedLongChunk influencerKeyChunk; - // protected SizedLongChunk influencerPosChunk; - // protected SizedLongChunk influencerTimestampChunk; - // - // // for use with a ticking window - // protected RowSet affectedRowPositions; - // protected RowSet influencerPositions; - // - // protected long currentInfluencerPosOrTimestamp; - // protected int currentInfluencerIndex; - - // public abstract void loadInfluencerValueChunk(); - - @Override - public void close() { - // try (final SizedLongChunk ignoredChk1 = influencerKeyChunk; - // final SizedLongChunk ignoredChk2 = influencerPosChunk; - // final SizedLongChunk ignoredChk3 = influencerTimestampChunk; - // final RowSet ignoredRs3 = affectedRowPositions; - // final RowSet ignoredRs4 = influencerPositions; - // ) { - // } - } + public void close() {} + + public abstract void accumulate(RowSequence inputKeys, + Chunk influencerValueChunk, + IntChunk pushChunk, + IntChunk popChunk, + int len); } /** diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java index 2cd2db4165d..d0126c67834 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java @@ -258,10 +258,6 @@ private class UpdateContext implements SafeCloseable { @SuppressWarnings("resource") UpdateContext(@NotNull final TableUpdate upstream, final boolean isInitializeStep) { - final int updateSize = UpdateSizeCalculator.chunkSize(upstream, control.chunkCapacityOrDefault()); - - this.chunkSize = - UpdateSizeCalculator.chunkSize(updateSize, upstream.shifted(), control.chunkCapacityOrDefault()); this.windowContexts = new UpdateByWindow.UpdateByWindowContext[windows.length]; this.windowAffected = new boolean[windows.length]; @@ -273,7 +269,7 @@ private class UpdateContext implements SafeCloseable { inputSources, timestampColumnSource, timestampSsa, - chunkSize, + control.chunkCapacityOrDefault(), isInitializeStep); // compute the affected/influenced operators and rowset within this window diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java index bcd4d1b011a..35d5ee822b2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java @@ -1,5 +1,6 @@ package io.deephaven.engine.table.impl.updateby; +import io.deephaven.chunk.Chunk; import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.*; @@ -15,8 +16,6 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.util.*; - public abstract class UpdateByWindow { @Nullable protected final String timestampColumnName; @@ -52,15 +51,17 @@ public abstract class UpdateByWindowContext implements SafeCloseable { /** An array of ColumnSources for each underlying operator */ protected final ColumnSource[] inputSources; - /** An array of {@link ChunkSource.FillContext}s for each input column */ - protected final ChunkSource.FillContext[] inputSourceFillContexts; + /** An array of {@link ChunkSource.GetContext}s for each input column */ + protected final ChunkSource.GetContext[] inputSourceGetContexts; /** A set of chunks used to store working values */ - protected final WritableChunk[] inputSourceChunks; + protected final Chunk[] inputSourceChunks; /** An indicator of if each slot has been populated with data or not for this phase. */ protected final boolean[] inputSourceChunkPopulated; + protected final boolean initialStep; + /** the rows affected by this update */ protected RowSet affectedRows; /** the rows that contain values used to compute affected row values */ @@ -70,8 +71,7 @@ public abstract class UpdateByWindowContext implements SafeCloseable { protected RowSetBuilderSequential modifiedBuilder; protected RowSet newModified; - protected final int chunkSize; - protected final boolean initialStep; + protected int workingChunkSize; public UpdateByWindowContext(final TrackingRowSet sourceRowSet, final ColumnSource[] inputSources, @Nullable final ColumnSource timestampColumnSource, @@ -83,12 +83,12 @@ public UpdateByWindowContext(final TrackingRowSet sourceRowSet, final ColumnSour this.opAffected = new boolean[operators.length]; this.opContext = new UpdateByOperator.UpdateContext[operators.length]; - this.inputSourceFillContexts = new ChunkSource.FillContext[inputSources.length]; + this.inputSourceGetContexts = new ChunkSource.GetContext[inputSources.length]; this.inputSourceChunkPopulated = new boolean[inputSources.length]; // noinspection unchecked this.inputSourceChunks = new WritableChunk[inputSources.length]; - this.chunkSize = chunkSize; + this.workingChunkSize = chunkSize; this.initialStep = initialStep; } @@ -96,25 +96,7 @@ public UpdateByWindowContext(final TrackingRowSet sourceRowSet, final ColumnSour public abstract void processRows(); - protected void makeOperatorContexts() { - // use this to make which input sources are initialized - Arrays.fill(inputSourceChunkPopulated, false); - - // create contexts for the affected operators - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - // create the fill contexts for the input sources - int sourceSlot = operatorSourceSlots[opIdx]; - if (!inputSourceChunkPopulated[sourceSlot]) { - inputSourceChunks[sourceSlot] = - inputSources[sourceSlot].getChunkType().makeWritableChunk(chunkSize); - inputSourceFillContexts[sourceSlot] = inputSources[sourceSlot].makeFillContext(chunkSize); - inputSourceChunkPopulated[sourceSlot] = true; - } - opContext[opIdx] = operators[opIdx].makeUpdateContext(chunkSize, inputSources[sourceSlot]); - } - } - } + protected abstract void makeOperatorContexts(); public boolean anyModified() { return newModified != null && newModified.isNonempty(); @@ -141,11 +123,11 @@ public RowSet getInfluencerRows() { } protected void prepareValuesChunkForSource(final int srcIdx, final RowSequence rs) { + if (rs.isEmpty()) { + return; + } if (!inputSourceChunkPopulated[srcIdx]) { - inputSources[srcIdx].fillChunk( - inputSourceFillContexts[srcIdx], - inputSourceChunks[srcIdx], - rs); + inputSourceChunks[srcIdx] = inputSources[srcIdx].getChunk(inputSourceGetContexts[srcIdx], rs); inputSourceChunkPopulated[srcIdx] = true; } } @@ -163,13 +145,9 @@ public void close() { for (int opIdx = 0; opIdx < operators.length; opIdx++) { if (opAffected[opIdx]) { final int srcIdx = operatorSourceSlots[opIdx]; - if (inputSourceChunks[srcIdx] != null) { - - inputSourceChunks[srcIdx].close(); - inputSourceChunks[srcIdx] = null; - - inputSourceFillContexts[srcIdx].close(); - inputSourceFillContexts[srcIdx] = null; + if (inputSourceGetContexts[srcIdx] != null) { + inputSourceGetContexts[srcIdx].close(); + inputSourceGetContexts[srcIdx] = null; } opContext[opIdx].close(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java index 6b3b73c06f0..8252272716f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java @@ -1,23 +1,15 @@ package io.deephaven.engine.table.impl.updateby; -import io.deephaven.base.ringbuffer.LongRingBuffer; -import io.deephaven.base.verify.Assert; import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.ModifiedColumnSet; import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.impl.QueryTable; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.UpdateByOperator; -import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; -import io.deephaven.util.SafeCloseable; -import org.apache.commons.lang3.mutable.MutableLong; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -40,6 +32,28 @@ public void close() { super.close(); } + @Override + protected void makeOperatorContexts() { + // use this to make which input sources are initialized + Arrays.fill(inputSourceChunkPopulated, false); + + // working chunk size need not be larger than affectedRows.size() + workingChunkSize = Math.min(workingChunkSize, affectedRows.intSize()); + + // create contexts for the affected operators + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (opAffected[opIdx]) { + // create the fill contexts for the input sources + int sourceSlot = operatorSourceSlots[opIdx]; + if (!inputSourceChunkPopulated[sourceSlot]) { + inputSourceGetContexts[sourceSlot] = inputSources[sourceSlot].makeGetContext(workingChunkSize); + inputSourceChunkPopulated[sourceSlot] = true; + } + opContext[opIdx] = operators[opIdx].makeUpdateContext(workingChunkSize, inputSources[sourceSlot]); + } + } + } + @Override public boolean computeAffectedRowsAndOperators(@NotNull final TableUpdate upstream) { // all rows are affected on the initial step @@ -137,9 +151,10 @@ public void processRows() { try (final RowSequence.Iterator it = affectedRows.getRowSequenceIterator(); ChunkSource.GetContext tsGetCtx = - timestampColumnSource == null ? null : timestampColumnSource.makeGetContext(chunkSize)) { + timestampColumnSource == null ? null + : timestampColumnSource.makeGetContext(workingChunkSize)) { while (it.hasMore()) { - final RowSequence rs = it.getNextRowSequenceWithLength(chunkSize); + final RowSequence rs = it.getNextRowSequenceWithLength(workingChunkSize); final int size = rs.intSize(); Arrays.fill(inputSourceChunkPopulated, false); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java index ece10b77f69..71cb0aca4c5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java @@ -1,13 +1,15 @@ package io.deephaven.engine.table.impl.updateby; import io.deephaven.base.ringbuffer.IntRingBuffer; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.WritableIntChunk; +import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.impl.QueryTable; -import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; @@ -18,15 +20,14 @@ import java.util.Arrays; -import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; -import static io.deephaven.util.QueryConstants.NULL_LONG; - // this class is currently too big, should specialize into CumWindow, TickWindow, TimeWindow to simplify implementation public class UpdateByWindowTicks extends UpdateByWindow { protected final long prevUnits; protected final long fwdUnits; public class UpdateByWindowTicksContext extends UpdateByWindow.UpdateByWindowContext { + private static final int WINDOW_CHUNK_SIZE = 4096; + protected final IntRingBuffer currentWindowPositions; protected RowSet affectedRowPositions; @@ -40,6 +41,8 @@ public class UpdateByWindowTicksContext extends UpdateByWindow.UpdateByWindowCon protected RowSequence.Iterator influencerPosIt; protected LongChunk influencerPosChunk; protected LongChunk influencerKeyChunk; + protected long influencerPosChunkSize; + protected int currentGetContextSize; public UpdateByWindowTicksContext(final TrackingRowSet sourceRowSet, final ColumnSource[] inputSources, @Nullable final ColumnSource timestampColumnSource, @@ -49,7 +52,6 @@ public UpdateByWindowTicksContext(final TrackingRowSet sourceRowSet, final Colum currentWindowPositions = new IntRingBuffer(512, true); } - @Override public void close() { super.close(); @@ -64,6 +66,61 @@ public void close() { } } + @Override + protected void makeOperatorContexts() { + // use this to determine which input sources are initialized + Arrays.fill(inputSourceChunkPopulated, false); + + // create contexts for the affected operators + currentGetContextSize = WINDOW_CHUNK_SIZE; + + // working chunk size need not be larger than affectedRows.size() + workingChunkSize = Math.min(workingChunkSize, affectedRows.intSize()); + + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (opAffected[opIdx]) { + // create the fill contexts for the input sources + int sourceSlot = operatorSourceSlots[opIdx]; + if (!inputSourceChunkPopulated[sourceSlot]) { + inputSourceGetContexts[sourceSlot] = + inputSources[sourceSlot].makeGetContext(currentGetContextSize); + inputSourceChunkPopulated[sourceSlot] = true; + } + opContext[opIdx] = operators[opIdx].makeUpdateContext(workingChunkSize, inputSources[sourceSlot]); + } + } + } + + protected void ensureGetContextSize(long newSize) { + if (currentGetContextSize < newSize) { + long size = currentGetContextSize; + while (size < newSize) { + size *= 2; + } + currentGetContextSize = LongSizedDataStructure.intSize( + "ensureGetContextSize exceeded Integer.MAX_VALUE", + size); + + // use this to determine which input sources are initialized + Arrays.fill(inputSourceChunkPopulated, false); + + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (opAffected[opIdx]) { + int sourceSlot = operatorSourceSlots[opIdx]; + if (!inputSourceChunkPopulated[sourceSlot]) { + // close the existing context + inputSourceGetContexts[sourceSlot].close(); + + // create a new context of the larger size + inputSourceGetContexts[sourceSlot] = + inputSources[sourceSlot].makeGetContext(currentGetContextSize); + inputSourceChunkPopulated[sourceSlot] = true; + } + } + } + } + } + // windowed by time/ticks is more complex to compute: find all the changed rows and the rows that would // be affected by the changes (includes newly added rows) and need to be recomputed. Then include all // the rows that are affected by deletions (if any). After the affected rows have been identified, @@ -144,28 +201,27 @@ public boolean computeAffectedRowsAndOperators(@NotNull final TableUpdate upstre return true; } - private void loadNextInfluencerValueChunks() { + /*** + * This function takes care of loading/preparing the next set of influencer data, in this case we load the next + * chunk of key and position data and reset the index + */ + private void loadNextInfluencerChunks() { if (!influencerIt.hasMore()) { nextInfluencerPos = Integer.MAX_VALUE; nextInfluencerKey = Long.MAX_VALUE; return; } - final RowSequence influencerRs = influencerIt.getNextRowSequenceWithLength(chunkSize); + final RowSequence influencerRs = influencerIt.getNextRowSequenceWithLength(WINDOW_CHUNK_SIZE); influencerKeyChunk = influencerRs.asRowKeyChunk(); - final RowSequence influencePosRs = influencerPosIt.getNextRowSequenceWithLength(chunkSize); + final RowSequence influencePosRs = influencerPosIt.getNextRowSequenceWithLength(WINDOW_CHUNK_SIZE); influencerPosChunk = influencePosRs.asRowKeyChunk(); - Arrays.fill(inputSourceChunkPopulated, false); - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - final int srcIdx = operatorSourceSlots[opIdx]; - prepareValuesChunkForSource(srcIdx, influencerRs); + Assert.eqTrue(influencePosRs.lastRowKey() < Integer.MAX_VALUE, + "updateBy window positions exceeded maximum size"); - opContext[opIdx].setValuesChunk(inputSourceChunks[srcIdx]); - } - } + influencerPosChunkSize = influencerPosChunk.size(); nextInfluencerIndex = 0; nextInfluencerPos = LongSizedDataStructure.intSize( @@ -174,15 +230,16 @@ private void loadNextInfluencerValueChunks() { nextInfluencerKey = influencerKeyChunk.get(nextInfluencerIndex); } - private void fillWindowTicks(long currentPos, long sourceRowSetSize) {} - - // this function process the affected rows chunkwise, but will call fillWindowTicks() for each - // new row. fillWindowTicks() will advance the moving window (which is the same for all operators in this - // collection) and will call push/pop for each operator as it advances the window. - // - // We track the minimum amount of data needed, only the window position data. The downstream operators - // should manage local storage in a RingBuffer or other efficient structure - @Override + /*** + * This function process the affected rows chunkwise, and will advance the moving window (which is the same for + * all operators in this collection). For each row in the dataset the sliding window will adjust and + * instructions for pushing/popping data will be created for the operators. For each chunk of `affected` rows, + * we will identify exactly which `influencer` rows are needed and will provide those and the push/pop + * instructions to the operators. + * + * Downstream operators should manage local storage in a RingBuffer or other efficient structure since our pop() + * calls do not provide the popped data + */ public void processRows() { if (trackModifications) { modifiedBuilder = RowSetFactory.builderSequential(); @@ -200,107 +257,105 @@ public void processRows() { influencerPosIt = influencerPositions.getRowSequenceIterator(); try (final RowSequence.Iterator it = affectedRows.getRowSequenceIterator(); - final RowSequence.Iterator posIt = affectedRowPositions.getRowSequenceIterator()) { + final RowSequence.Iterator posIt = affectedRowPositions.getRowSequenceIterator(); + final WritableIntChunk pushChunk = + WritableIntChunk.makeWritableChunk(workingChunkSize); + final WritableIntChunk popChunk = + WritableIntChunk.makeWritableChunk(workingChunkSize)) { // load the first chunk of influencer values (fillWindowTicks() will call in future) - loadNextInfluencerValueChunks(); + loadNextInfluencerChunks(); final long sourceRowSetSize = sourceRowSet.size(); while (it.hasMore()) { - final RowSequence rs = it.getNextRowSequenceWithLength(chunkSize); - final RowSequence posRs = posIt.getNextRowSequenceWithLength(chunkSize); - final int size = rs.intSize(); + final RowSequence chunkRs = it.getNextRowSequenceWithLength(workingChunkSize); + final RowSequence chunkPosRs = posIt.getNextRowSequenceWithLength(workingChunkSize); + final int chunkRsSize = chunkRs.intSize(); - final LongChunk posChunk = posRs.asRowKeyChunk(); + final LongChunk posChunk = chunkPosRs.asRowKeyChunk(); + + // we are going to track all the influencer rows that affect this chunk of data + final RowSetBuilderSequential chunkInfluencerBuilder = RowSetFactory.builderSequential(); // chunk processing - for (int ii = 0; ii < size; ii++) { + for (int ii = 0; ii < chunkRsSize; ii++) { // read the current position final long currentPos = posChunk.get(ii); - // fill the operator windows (calls push/pop/reset as appropriate) // compute the head and tail positions (inclusive) final long head = Math.max(0, currentPos - prevUnits + 1); final long tail = Math.min(sourceRowSetSize - 1, currentPos + fwdUnits); // pop out all values from the current window that are not in the new window + int popCount = 0; while (!currentWindowPositions.isEmpty() && currentWindowPositions.front() < head) { - // operator pop - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - opContext[opIdx].pop(); - } - } currentWindowPositions.remove(); + popCount++; } - // if the window is empty, reset - if (currentWindowPositions.isEmpty()) { - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - opContext[opIdx].reset(); - } - } - } - - // skip values until they match the window (this can only happen on the initial addition of rows - // to the - // table, because we short-circuit the precise building of the influencer rows for efficiency) + // skip values until they match the window (this can only happen on initial addition of rows + // to the table, because we short-circuited the precise building of the influencer rows for + // efficiency) while (nextInfluencerPos < head) { nextInfluencerIndex++; - if (nextInfluencerIndex < influencerPosChunk.size()) { - nextInfluencerPos = LongSizedDataStructure.intSize( - "updateBy window positions exceeded maximum size", - influencerPosChunk.get(nextInfluencerIndex)); + if (nextInfluencerIndex < influencerPosChunkSize) { + nextInfluencerPos = (int) influencerPosChunk.get(nextInfluencerIndex); nextInfluencerKey = influencerKeyChunk.get(nextInfluencerIndex); } else { // try to bring in new data - loadNextInfluencerValueChunks(); + loadNextInfluencerChunks(); } } // push matching values + int pushCount = 0; while (nextInfluencerPos <= tail) { - // operator push - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - opContext[opIdx].push(nextInfluencerKey, nextInfluencerIndex); - } - } currentWindowPositions.add(nextInfluencerPos); + pushCount++; + // add this key to the needed set for this chunk + chunkInfluencerBuilder.appendKey(nextInfluencerKey); nextInfluencerIndex++; - if (nextInfluencerIndex < influencerPosChunk.size()) { - nextInfluencerPos = LongSizedDataStructure.intSize( - "updateBy window positions exceeded maximum size", - influencerPosChunk.get(nextInfluencerIndex)); + if (nextInfluencerIndex < influencerPosChunkSize) { + nextInfluencerPos = (int) influencerPosChunk.get(nextInfluencerIndex); nextInfluencerKey = influencerKeyChunk.get(nextInfluencerIndex); } else { // try to bring in new data - loadNextInfluencerValueChunks(); + loadNextInfluencerChunks(); } } - // now the operators have seen the correct window data, write to the output chunk + // write the push and pop counts to the chunks + popChunk.set(ii, popCount); + pushChunk.set(ii, pushCount); + } + + // execute the operators + try (final RowSet chunkInfluencerRs = chunkInfluencerBuilder.build()) { + ensureGetContextSize(chunkInfluencerRs.size()); + + Arrays.fill(inputSourceChunkPopulated, false); for (int opIdx = 0; opIdx < operators.length; opIdx++) { if (opAffected[opIdx]) { - opContext[opIdx].writeToOutputChunk(ii); + final int srcIdx = operatorSourceSlots[opIdx]; + prepareValuesChunkForSource(srcIdx, chunkInfluencerRs); + + // make the specialized call for windowed operators + ((UpdateByWindowedOperator.Context) opContext[opIdx]).accumulate( + chunkRs, + inputSourceChunks[srcIdx], + pushChunk, + popChunk, + chunkRsSize); } } } - // chunk output to column - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - opContext[opIdx].writeToOutputColumn(rs); - } - } - // all these rows were modified if (modifiedBuilder != null) { - modifiedBuilder.appendRowSequence(rs); + modifiedBuilder.appendRowSequence(chunkRs); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java index 80db5dd4a0c..a9da8a0899d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java @@ -3,6 +3,7 @@ import io.deephaven.base.ringbuffer.LongRingBuffer; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; @@ -12,6 +13,7 @@ import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; +import io.deephaven.util.datastructures.LongSizedDataStructure; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -25,6 +27,8 @@ public class UpdateByWindowTime extends UpdateByWindow { protected final long fwdUnits; public class UpdateByWindowTimeContext extends UpdateByWindowContext { + private static final int WINDOW_CHUNK_SIZE = 4096; + protected final ChunkSource.GetContext influencerTimestampContext; protected final LongRingBuffer currentWindowTimestamps; @@ -35,13 +39,15 @@ public class UpdateByWindowTimeContext extends UpdateByWindowContext { protected RowSequence.Iterator influencerIt; protected LongChunk influencerKeyChunk; protected LongChunk influencerTimestampChunk; + protected long influencerTimestampChunkSize; + protected int currentGetContextSize; public UpdateByWindowTimeContext(final TrackingRowSet sourceRowSet, final ColumnSource[] inputSources, @NotNull final ColumnSource timestampColumnSource, @Nullable final LongSegmentedSortedArray timestampSsa, final int chunkSize, final boolean initialStep) { super(sourceRowSet, inputSources, timestampColumnSource, timestampSsa, chunkSize, initialStep); - influencerTimestampContext = timestampColumnSource.makeGetContext(chunkSize); + influencerTimestampContext = timestampColumnSource.makeGetContext(WINDOW_CHUNK_SIZE); currentWindowTimestamps = new LongRingBuffer(512, true); } @@ -54,6 +60,66 @@ public void close() { } } + @Override + protected void makeOperatorContexts() { + // use this to make which input sources are initialized + Arrays.fill(inputSourceChunkPopulated, false); + + // create contexts for the affected operators + currentGetContextSize = WINDOW_CHUNK_SIZE; + + // working chunk size need not be larger than affectedRows.size() + workingChunkSize = Math.min(workingChunkSize, affectedRows.intSize()); + + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (opAffected[opIdx]) { + // create the fill contexts for the input sources + int sourceSlot = operatorSourceSlots[opIdx]; + if (!inputSourceChunkPopulated[sourceSlot]) { + // we are going to grab all the influencer rows as one chunk, make sure it's large enough + inputSourceGetContexts[sourceSlot] = + inputSources[sourceSlot].makeGetContext(currentGetContextSize); + inputSourceChunkPopulated[sourceSlot] = true; + } + opContext[opIdx] = operators[opIdx].makeUpdateContext(workingChunkSize, inputSources[sourceSlot]); + } + } + } + + protected void ensureGetContextSize(long newSize) { + if (currentGetContextSize < newSize) { + long size = currentGetContextSize; + while (size < newSize) { + size *= 2; + } + currentGetContextSize = LongSizedDataStructure.intSize( + "ensureGetContextSize exceeded Integer.MAX_VALUE", + size); + + // use this to determine which input sources are initialized + Arrays.fill(inputSourceChunkPopulated, false); + + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (opAffected[opIdx]) { + int sourceSlot = operatorSourceSlots[opIdx]; + if (!inputSourceChunkPopulated[sourceSlot]) { + // close the existing context + inputSourceGetContexts[sourceSlot].close(); + + // create a new context of the larger size + inputSourceGetContexts[sourceSlot] = + inputSources[sourceSlot].makeGetContext(currentGetContextSize); + inputSourceChunkPopulated[sourceSlot] = true; + } + } + } + } + } + + // windowed by time/ticks is more complex to compute: find all the changed rows and the rows that would + // be affected by the changes (includes newly added rows) and need to be recomputed. Then include all + // the rows that are affected by deletions (if any). After the affected rows have been identified, + // determine which rows will be needed to compute new values for the affected rows (influencer rows) @Override public boolean computeAffectedRowsAndOperators(@NotNull final TableUpdate upstream) { // all rows are affected on the initial step @@ -116,99 +182,39 @@ public boolean computeAffectedRowsAndOperators(@NotNull final TableUpdate upstre return true; } - private void loadNextInfluencerValueChunks() { + /*** + * This function takes care of loading/preparing the next set of influencer data, in this case we load the next + * chunk of key and position data and reset the index + */ + private void loadNextInfluencerChunks() { if (!influencerIt.hasMore()) { nextInfluencerTimestamp = Long.MAX_VALUE; nextInfluencerKey = Long.MAX_VALUE; return; } - final RowSequence influencerRs = influencerIt.getNextRowSequenceWithLength(chunkSize); + final RowSequence influencerRs = influencerIt.getNextRowSequenceWithLength(WINDOW_CHUNK_SIZE); influencerKeyChunk = influencerRs.asRowKeyChunk(); influencerTimestampChunk = timestampColumnSource.getChunk(influencerTimestampContext, influencerRs).asLongChunk(); - Arrays.fill(inputSourceChunkPopulated, false); - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - final int srcIdx = operatorSourceSlots[opIdx]; - prepareValuesChunkForSource(srcIdx, influencerRs); - - opContext[opIdx].setValuesChunk(inputSourceChunks[srcIdx]); - } - } + influencerTimestampChunkSize = influencerTimestampChunk.size(); nextInfluencerIndex = 0; nextInfluencerTimestamp = influencerTimestampChunk.get(nextInfluencerIndex); nextInfluencerKey = influencerKeyChunk.get(nextInfluencerIndex); } - public void fillWindowTime(long currentTimestamp) { - // compute the head and tail positions (inclusive) - final long head = currentTimestamp - prevUnits; - final long tail = currentTimestamp + fwdUnits; - - // pop out all values from the current window that are not in the new window - while (!currentWindowTimestamps.isEmpty() && currentWindowTimestamps.front() < head) { - // operator pop - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - opContext[opIdx].pop(); - } - } - currentWindowTimestamps.remove(); - } - - // if the window is empty, reset - if (currentWindowTimestamps.isEmpty()) { - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - opContext[opIdx].reset(); - } - } - } - - // skip values until they match the window (this can only happen on the initial addition of rows to the - // table, because we short-circuit the precise building of the influencer rows for efficiency) - while (nextInfluencerTimestamp < head) { - nextInfluencerIndex++; - - if (nextInfluencerIndex < influencerTimestampChunk.size()) { - nextInfluencerTimestamp = influencerTimestampChunk.get(nextInfluencerIndex); - nextInfluencerKey = influencerKeyChunk.get(nextInfluencerIndex); - } else { - // try to bring in new data - loadNextInfluencerValueChunks(); - } - } - - // push matching values - while (nextInfluencerTimestamp <= tail) { - // operator push - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - opContext[opIdx].push(nextInfluencerKey, nextInfluencerIndex); - } - } - currentWindowTimestamps.add(nextInfluencerTimestamp); - nextInfluencerIndex++; - - if (nextInfluencerIndex < influencerTimestampChunk.size()) { - nextInfluencerTimestamp = influencerTimestampChunk.get(nextInfluencerIndex); - nextInfluencerKey = influencerKeyChunk.get(nextInfluencerIndex); - } else { - // try to bring in new data - loadNextInfluencerValueChunks(); - } - } - } - - // overview: this function process the affected rows chunkwise, but will call fillWindowTime() for each - // new row. fillWindowTime() will advance the moving window (which is the same for all operators in this - // collection) and will call push/pop for each operator as it advances the window. - // - // We track the minimum amount of data needed, only the window timestamp data. The downstream operators - // should manage local storage in a RingBuffer or other efficient structure + /*** + * This function process the affected rows chunkwise, and will advance the moving window (which is the same for + * all operators in this collection). For each row in the dataset the sliding window will adjust and + * instructions for pushing/popping data will be created for the operators. For each chunk of `affected` rows, + * we will identify exactly which `influencer` rows are needed and will provide those and the push/pop + * instructions to the operators. + * + * Downstream operators should manage local storage in a RingBuffer or other efficient structure since our pop() + * calls do not provide the popped data + */ @Override public void processRows() { if (trackModifications) { @@ -224,47 +230,109 @@ public void processRows() { } influencerIt = influencerRows.getRowSequenceIterator(); + try (final RowSequence.Iterator it = affectedRows.getRowSequenceIterator(); final ChunkSource.GetContext localTimestampContext = - timestampColumnSource.makeGetContext(chunkSize)) { + timestampColumnSource.makeGetContext(workingChunkSize); + final WritableIntChunk pushChunk = + WritableIntChunk.makeWritableChunk(workingChunkSize); + final WritableIntChunk popChunk = + WritableIntChunk.makeWritableChunk(workingChunkSize)) { // load the first chunk of influencer values (fillWindowTime() will call in future) - loadNextInfluencerValueChunks(); + loadNextInfluencerChunks(); while (it.hasMore()) { - final RowSequence rs = it.getNextRowSequenceWithLength(chunkSize); + final RowSequence chunkRs = it.getNextRowSequenceWithLength(workingChunkSize); + final int chunkRsSize = chunkRs.intSize(); - // just a note, we did not put null values into our SSA and our influencer rowset is built using the + // NOTE: we did not put null values into our SSA and our influencer rowset is built using the // SSA. there should be no null timestamps considered in the rolling windows final LongChunk timestampChunk = - timestampColumnSource.getChunk(localTimestampContext, rs).asLongChunk(); + timestampColumnSource.getChunk(localTimestampContext, chunkRs).asLongChunk(); + + // we are going to track all the influencer rows that affect this chunk of data + final RowSetBuilderSequential chunkInfluencerBuilder = RowSetFactory.builderSequential(); // chunk processing - for (int ii = 0; ii < rs.size(); ii++) { + for (int ii = 0; ii < chunkRsSize; ii++) { // read the current position - final long ts = timestampChunk.get(ii); + final long currentTimestamp = timestampChunk.get(ii); - // fill the operator windows (calls push/pop/reset as appropriate) - fillWindowTime(ts); + // compute the head and tail positions (inclusive) + final long head = currentTimestamp - prevUnits; + final long tail = currentTimestamp + fwdUnits; - // now the operators have seen the correct window data, write to the output chunk - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - opContext[opIdx].writeToOutputChunk(ii); + // pop out all values from the current window that are not in the new window + int popCount = 0; + while (!currentWindowTimestamps.isEmpty() && currentWindowTimestamps.front() < head) { + currentWindowTimestamps.remove(); + popCount++; + } + + + // skip values until they match the window (this can only happen on the initial addition of rows + // to the table, because we short-circuited the precise building of the influencer rows for + // efficiency) + while (nextInfluencerTimestamp < head) { + nextInfluencerIndex++; + + if (nextInfluencerIndex < influencerTimestampChunkSize) { + nextInfluencerTimestamp = influencerTimestampChunk.get(nextInfluencerIndex); + nextInfluencerKey = influencerKeyChunk.get(nextInfluencerIndex); + } else { + // try to bring in new data + loadNextInfluencerChunks(); } } + + // push matching values + int pushCount = 0; + while (nextInfluencerTimestamp <= tail) { + currentWindowTimestamps.add(nextInfluencerTimestamp); + pushCount++; + // add this key to the needed set for this chunk + chunkInfluencerBuilder.appendKey(nextInfluencerKey); + nextInfluencerIndex++; + + if (nextInfluencerIndex < influencerTimestampChunkSize) { + nextInfluencerTimestamp = influencerTimestampChunk.get(nextInfluencerIndex); + nextInfluencerKey = influencerKeyChunk.get(nextInfluencerIndex); + } else { + // try to bring in new data + loadNextInfluencerChunks(); + } + } + + // write the push and pop counts to the chunks + popChunk.set(ii, popCount); + pushChunk.set(ii, pushCount); } - // chunk output to column - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - opContext[opIdx].writeToOutputColumn(rs); + // execute the operators + try (final RowSet chunkInfluencerRs = chunkInfluencerBuilder.build()) { + ensureGetContextSize(chunkInfluencerRs.size()); + + Arrays.fill(inputSourceChunkPopulated, false); + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (opAffected[opIdx]) { + final int srcIdx = operatorSourceSlots[opIdx]; + prepareValuesChunkForSource(srcIdx, chunkInfluencerRs); + + // make the specialized call for windowed operators + ((UpdateByWindowedOperator.Context) opContext[opIdx]).accumulate( + chunkRs, + inputSourceChunks[srcIdx], + pushChunk, + popChunk, + chunkRsSize); + } } } // all these rows were modified if (modifiedBuilder != null) { - modifiedBuilder.appendRowSequence(rs); + modifiedBuilder.appendRowSequence(chunkRs); } } } @@ -314,11 +382,13 @@ private static WritableRowSet computeInfluencerRowsTime(final RowSet sourceSet, LongSegmentedSortedArray.Iterator ssaIt = timestampSsa.iterator(false, false); while (it.hasMore() && ssaIt.hasNext()) { final RowSequence rs = it.getNextRowSequenceWithLength(chunkSize); + final int rsSize = rs.intSize(); + LongChunk timestamps = usePrev ? timestampColumnSource.getPrevChunk(context, rs).asLongChunk() : timestampColumnSource.getChunk(context, rs).asLongChunk(); - for (int ii = 0; ii < rs.intSize(); ii++) { + for (int ii = 0; ii < rsSize; ii++) { // if the timestamp of the row is null, it won't belong to any set and we can ignore it // completely final long ts = timestamps.get(ii); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java index 002f64fa32b..3f67a141e25 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java @@ -3,19 +3,15 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.ObjectChunk; -import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import java.math.BigDecimal; -import java.math.BigInteger; import static io.deephaven.util.QueryConstants.NULL_LONG; @@ -27,7 +23,7 @@ protected Context(int chunkSize, ColumnSource inputSource) { @Override public void accumulate(RowSequence inputKeys, - WritableChunk valueChunk, + Chunk valueChunk, LongChunk tsChunk, int len) { setValuesChunk(valueChunk); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java index 2a9f6c5b7aa..0d79bb6fca6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java @@ -1,8 +1,8 @@ package io.deephaven.engine.table.impl.updateby.ema; import io.deephaven.api.updateby.OperationControl; +import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.ColumnSource; @@ -24,7 +24,7 @@ protected Context(int chunkSize, ColumnSource inputSource) { @Override public void accumulate(RowSequence inputKeys, - WritableChunk valueChunk, + Chunk valueChunk, LongChunk tsChunk, int len) { setValuesChunk(valueChunk); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java index 11cb4134c6e..63d1efecc1d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java @@ -29,7 +29,7 @@ public abstract class BigNumberEMAOperator extends BaseObjectUpdateByOperator public abstract class Context extends BaseObjectUpdateByOperator.Context { protected final ColumnSource valueSource; public LongChunk timestampValueChunk; - public ObjectChunk objectValueChunk; + public ObjectChunk objectValueChunk; long lastStamp = NULL_LONG; @@ -41,7 +41,7 @@ protected Context(int chunkSize, ColumnSource inputSource) { @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { objectValueChunk = valuesChunk.asObjectChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java index c415dcd05da..081e818c9e9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java @@ -9,7 +9,6 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.ByteChunk; -import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.ColumnSource; @@ -25,7 +24,7 @@ public class ByteEMAOperator extends BasePrimitiveEMAOperator { protected class Context extends BasePrimitiveEMAOperator.Context { public final ColumnSource valueSource; - public ByteChunk byteValueChunk; + public ByteChunk byteValueChunk; protected Context(int chunkSize, ColumnSource inputSource) { super(chunkSize); @@ -34,7 +33,7 @@ protected Context(int chunkSize, ColumnSource inputSource) { @Override public void accumulate(RowSequence inputKeys, - WritableChunk valueChunk, + Chunk valueChunk, LongChunk tsChunk, int len) { setValuesChunk(valueChunk); @@ -94,7 +93,7 @@ public void accumulate(RowSequence inputKeys, } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { byteValueChunk = valuesChunk.asByteChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java index 1c2448a0867..8534e4585f2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java @@ -25,7 +25,7 @@ public class DoubleEMAOperator extends BasePrimitiveEMAOperator { protected class Context extends BasePrimitiveEMAOperator.Context { private final ColumnSource valueSource; - public DoubleChunk doubleValueChunk; + public DoubleChunk doubleValueChunk; protected Context(int chunkSize, ColumnSource inputSource) { super(chunkSize); @@ -34,7 +34,7 @@ protected Context(int chunkSize, ColumnSource inputSource) { @Override public void accumulate(RowSequence inputKeys, - WritableChunk valueChunk, + Chunk valueChunk, LongChunk tsChunk, int len) { setValuesChunk(valueChunk); @@ -102,7 +102,7 @@ public void accumulate(RowSequence inputKeys, } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { doubleValueChunk = valuesChunk.asDoubleChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java index d2afb17f338..24646b44aa6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java @@ -20,7 +20,7 @@ public class FloatEMAOperator extends BasePrimitiveEMAOperator { protected class Context extends BasePrimitiveEMAOperator.Context { private final ColumnSource valueSource; - public FloatChunk floatValueChunk; + public FloatChunk floatValueChunk; protected Context(int chunkSize, ColumnSource inputSource) { super(chunkSize); @@ -29,7 +29,7 @@ protected Context(int chunkSize, ColumnSource inputSource) { @Override public void accumulate(RowSequence inputKeys, - WritableChunk valueChunk, + Chunk valueChunk, LongChunk tsChunk, int len) { setValuesChunk(valueChunk); @@ -97,7 +97,7 @@ public void accumulate(RowSequence inputKeys, } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { floatValueChunk = valuesChunk.asFloatChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java index 2b1a65969cd..4ca064e0abb 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java @@ -9,7 +9,6 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.IntChunk; -import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.ColumnSource; @@ -25,7 +24,7 @@ public class IntEMAOperator extends BasePrimitiveEMAOperator { protected class Context extends BasePrimitiveEMAOperator.Context { public final ColumnSource valueSource; - public IntChunk intValueChunk; + public IntChunk intValueChunk; protected Context(int chunkSize, ColumnSource inputSource) { super(chunkSize); @@ -34,7 +33,7 @@ protected Context(int chunkSize, ColumnSource inputSource) { @Override public void accumulate(RowSequence inputKeys, - WritableChunk valueChunk, + Chunk valueChunk, LongChunk tsChunk, int len) { setValuesChunk(valueChunk); @@ -94,7 +93,7 @@ public void accumulate(RowSequence inputKeys, } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { intValueChunk = valuesChunk.asIntChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java index 516e237b0c4..291d9d455fc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java @@ -9,7 +9,6 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.ColumnSource; @@ -25,7 +24,7 @@ public class LongEMAOperator extends BasePrimitiveEMAOperator { protected class Context extends BasePrimitiveEMAOperator.Context { public final ColumnSource valueSource; - public LongChunk longValueChunk; + public LongChunk longValueChunk; protected Context(int chunkSize, ColumnSource inputSource) { super(chunkSize); @@ -34,7 +33,7 @@ protected Context(int chunkSize, ColumnSource inputSource) { @Override public void accumulate(RowSequence inputKeys, - WritableChunk valueChunk, + Chunk valueChunk, LongChunk tsChunk, int len) { setValuesChunk(valueChunk); @@ -94,7 +93,7 @@ public void accumulate(RowSequence inputKeys, } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { longValueChunk = valuesChunk.asLongChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java index 38fd0bcb8da..288b097e1a8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java @@ -4,7 +4,6 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.ShortChunk; -import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.ColumnSource; @@ -20,7 +19,7 @@ public class ShortEMAOperator extends BasePrimitiveEMAOperator { protected class Context extends BasePrimitiveEMAOperator.Context { public final ColumnSource valueSource; - public ShortChunk shortValueChunk; + public ShortChunk shortValueChunk; protected Context(int chunkSize, ColumnSource inputSource) { super(chunkSize); @@ -29,7 +28,7 @@ protected Context(int chunkSize, ColumnSource inputSource) { @Override public void accumulate(RowSequence inputKeys, - WritableChunk valueChunk, + Chunk valueChunk, LongChunk tsChunk, int len) { setValuesChunk(valueChunk); @@ -89,7 +88,7 @@ public void accumulate(RowSequence inputKeys, } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { shortValueChunk = valuesChunk.asShortChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java index b28a5f74c71..c86da007d5f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java @@ -28,14 +28,14 @@ public class BooleanFillByOperator extends BaseByteUpdateByOperator { // endregion extra-fields protected class Context extends BaseByteUpdateByOperator.Context { - public ByteChunk booleanValueChunk; + public ByteChunk booleanValueChunk; protected Context(int chunkSize) { super(chunkSize); } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { booleanValueChunk = valuesChunk.asByteChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java index 8cdd33dd894..ec2710e0375 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java @@ -21,14 +21,14 @@ public class ByteFillByOperator extends BaseByteUpdateByOperator { // endregion extra-fields protected class Context extends BaseByteUpdateByOperator.Context { - public ByteChunk byteValueChunk; + public ByteChunk byteValueChunk; protected Context(int chunkSize) { super(chunkSize); } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { byteValueChunk = valuesChunk.asByteChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java index 0d84441d629..5a8dafcb5e7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java @@ -16,14 +16,14 @@ public class CharFillByOperator extends BaseCharUpdateByOperator { // endregion extra-fields protected class Context extends BaseCharUpdateByOperator.Context { - public CharChunk charValueChunk; + public CharChunk charValueChunk; protected Context(int chunkSize) { super(chunkSize); } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { charValueChunk = valuesChunk.asCharChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java index 78ddd49ccf9..5835131f08d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java @@ -21,14 +21,14 @@ public class DoubleFillByOperator extends BaseDoubleUpdateByOperator { // endregion extra-fields protected class Context extends BaseDoubleUpdateByOperator.Context { - public DoubleChunk doubleValueChunk; + public DoubleChunk doubleValueChunk; protected Context(int chunkSize) { super(chunkSize); } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { doubleValueChunk = valuesChunk.asDoubleChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java index 9de8bc4e639..7b6a449d58a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java @@ -21,14 +21,14 @@ public class FloatFillByOperator extends BaseFloatUpdateByOperator { // endregion extra-fields protected class Context extends BaseFloatUpdateByOperator.Context { - public FloatChunk floatValueChunk; + public FloatChunk floatValueChunk; protected Context(int chunkSize) { super(chunkSize); } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { floatValueChunk = valuesChunk.asFloatChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java index c770d3776e3..7a98e68a3cd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java @@ -21,14 +21,14 @@ public class IntFillByOperator extends BaseIntUpdateByOperator { // endregion extra-fields protected class Context extends BaseIntUpdateByOperator.Context { - public IntChunk intValueChunk; + public IntChunk intValueChunk; protected Context(int chunkSize) { super(chunkSize); } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { intValueChunk = valuesChunk.asIntChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java index 36e6a50e10a..33144692320 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java @@ -29,14 +29,14 @@ public class LongFillByOperator extends BaseLongUpdateByOperator { // endregion extra-fields protected class Context extends BaseLongUpdateByOperator.Context { - public LongChunk longValueChunk; + public LongChunk longValueChunk; protected Context(int chunkSize) { super(chunkSize); } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { longValueChunk = valuesChunk.asLongChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java index d18af2a20ef..41a50ec9e42 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java @@ -20,14 +20,14 @@ public class ObjectFillByOperator extends BaseObjectUpdateByOperator { // endregion extra-fields protected class Context extends BaseObjectUpdateByOperator.Context { - public ObjectChunk ObjectValueChunk; + public ObjectChunk ObjectValueChunk; protected Context(int chunkSize) { super(chunkSize); } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { ObjectValueChunk = valuesChunk.asObjectChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java index 696cad5280a..9f6e57a2187 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java @@ -21,14 +21,14 @@ public class ShortFillByOperator extends BaseShortUpdateByOperator { // endregion extra-fields protected class Context extends BaseShortUpdateByOperator.Context { - public ShortChunk shortValueChunk; + public ShortChunk shortValueChunk; protected Context(int chunkSize) { super(chunkSize); } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { shortValueChunk = valuesChunk.asShortChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java index aefd387b4a2..2bfdcc84e00 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java @@ -54,7 +54,7 @@ protected Context(final int chunkSize) { @Override public void accumulate(RowSequence inputKeys, - WritableChunk valueChunk, + Chunk valueChunk, LongChunk tsChunk, int len) { @@ -79,7 +79,7 @@ public void close() { } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) {} + public void setValuesChunk(@NotNull final Chunk valuesChunk) {} @Override public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java index 336f061ced2..3cf56dc925c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java @@ -43,7 +43,7 @@ protected Context(final int chunkSize) { @Override public void accumulate(RowSequence inputKeys, - WritableChunk valueChunk, + Chunk valueChunk, LongChunk tsChunk, int len) { @@ -68,7 +68,7 @@ public void close() { } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) {} + public void setValuesChunk(@NotNull final Chunk valuesChunk) {} @Override public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java index ec1524132cd..6f0cd5cb990 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java @@ -48,7 +48,7 @@ protected Context(final int chunkSize) { @Override public void accumulate(RowSequence inputKeys, - WritableChunk valueChunk, + Chunk valueChunk, LongChunk tsChunk, int len) { @@ -73,7 +73,7 @@ public void close() { } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) {} + public void setValuesChunk(@NotNull final Chunk valuesChunk) {} @Override public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java index bf4ec7b9c3c..f656cfc046b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java @@ -43,7 +43,7 @@ protected Context(final int chunkSize) { @Override public void accumulate(RowSequence inputKeys, - WritableChunk valueChunk, + Chunk valueChunk, LongChunk tsChunk, int len) { @@ -68,7 +68,7 @@ public void close() { } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) {} + public void setValuesChunk(@NotNull final Chunk valuesChunk) {} @Override public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java index a03b6de66f0..ecfa73a9037 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java @@ -48,7 +48,7 @@ protected Context(final int chunkSize) { @Override public void accumulate(RowSequence inputKeys, - WritableChunk valueChunk, + Chunk valueChunk, LongChunk tsChunk, int len) { @@ -73,7 +73,7 @@ public void close() { } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) {} + public void setValuesChunk(@NotNull final Chunk valuesChunk) {} @Override public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java index 379c821817f..ffff7b03b29 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java @@ -48,7 +48,7 @@ protected Context(final int chunkSize) { @Override public void accumulate(RowSequence inputKeys, - WritableChunk valueChunk, + Chunk valueChunk, LongChunk tsChunk, int len) { @@ -73,7 +73,7 @@ public void close() { } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) {} + public void setValuesChunk(@NotNull final Chunk valuesChunk) {} @Override public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java index 10881dd0bbe..ac3fe9d5ed6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java @@ -17,14 +17,14 @@ public abstract class BaseObjectBinaryOperator extends BaseObjectUpdateByOperator { protected class Context extends BaseObjectUpdateByOperator.Context { - public ObjectChunk objectValueChunk; + public ObjectChunk objectValueChunk; protected Context(int chunkSize) { super(chunkSize); } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { objectValueChunk = valuesChunk.asObjectChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java index 4d922066bc1..a937d0dd446 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java @@ -49,7 +49,7 @@ protected Context(final int chunkSize) { @Override public void accumulate(RowSequence inputKeys, - WritableChunk valueChunk, + Chunk valueChunk, LongChunk tsChunk, int len) { @@ -74,7 +74,7 @@ public void close() { } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) {} + public void setValuesChunk(@NotNull final Chunk valuesChunk) {} @Override public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java index e64845ccea2..efcaf0f4fe6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java @@ -48,7 +48,7 @@ protected Context(final int chunkSize) { @Override public void accumulate(RowSequence inputKeys, - WritableChunk valueChunk, + Chunk valueChunk, LongChunk tsChunk, int len) { @@ -73,7 +73,7 @@ public void close() { } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) {} + public void setValuesChunk(@NotNull final Chunk valuesChunk) {} @Override public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java index 4f165a0da20..f7f70930197 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java @@ -12,6 +12,7 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableByteChunk; import io.deephaven.chunk.attributes.Values; @@ -51,7 +52,41 @@ protected Context(final int chunkSize) { } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) {} + public void accumulate(RowSequence inputKeys, + Chunk influencerValueChunk, + IntChunk pushChunk, + IntChunk popChunk, + int len) { + + setValuesChunk(influencerValueChunk); + int pushIndex = 0; + + // chunk processing + for (int ii = 0; ii < len; ii++) { + final int pushCount = pushChunk.get(ii); + final int popCount = popChunk.get(ii); + + // pop for this row + for (int count = 0; count < popCount; count++) { + pop(); + } + + // push for this row + for (int count = 0; count < pushCount; count++) { + push(NULL_ROW_KEY, pushIndex + count); + } + pushIndex += pushCount; + + // write the results to the output chunk + writeToOutputChunk(ii); + } + + // chunk output to column + writeToOutputColumn(inputKeys); + } + + @Override + public void setValuesChunk(@NotNull final Chunk valuesChunk) {} @Override public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java index ae68cbe8159..eba5ec7ca1b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java @@ -2,6 +2,7 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableCharChunk; import io.deephaven.chunk.attributes.Values; @@ -40,7 +41,41 @@ protected Context(final int chunkSize) { } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) {} + public void accumulate(RowSequence inputKeys, + Chunk influencerValueChunk, + IntChunk pushChunk, + IntChunk popChunk, + int len) { + + setValuesChunk(influencerValueChunk); + int pushIndex = 0; + + // chunk processing + for (int ii = 0; ii < len; ii++) { + final int pushCount = pushChunk.get(ii); + final int popCount = popChunk.get(ii); + + // pop for this row + for (int count = 0; count < popCount; count++) { + pop(); + } + + // push for this row + for (int count = 0; count < pushCount; count++) { + push(NULL_ROW_KEY, pushIndex + count); + } + pushIndex += pushCount; + + // write the results to the output chunk + writeToOutputChunk(ii); + } + + // chunk output to column + writeToOutputColumn(inputKeys); + } + + @Override + public void setValuesChunk(@NotNull final Chunk valuesChunk) {} @Override public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java index d326849d512..ee406b77b8c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java @@ -41,10 +41,42 @@ protected Context(final int chunkSize) { this.outputValues = WritableDoubleChunk.makeWritableChunk(chunkSize); } - public void storeWorkingChunk(@NotNull final Chunk valuesChunk) {} + @Override + public void accumulate(RowSequence inputKeys, + Chunk influencerValueChunk, + IntChunk pushChunk, + IntChunk popChunk, + int len) { + + setValuesChunk(influencerValueChunk); + int pushIndex = 0; + + // chunk processing + for (int ii = 0; ii < len; ii++) { + final int pushCount = pushChunk.get(ii); + final int popCount = popChunk.get(ii); + + // pop for this row + for (int count = 0; count < popCount; count++) { + pop(); + } + + // push for this row + for (int count = 0; count < pushCount; count++) { + push(NULL_ROW_KEY, pushIndex + count); + } + pushIndex += pushCount; + + // write the results to the output chunk + writeToOutputChunk(ii); + } + + // chunk output to column + writeToOutputColumn(inputKeys); + } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) {} + public void setValuesChunk(@NotNull final Chunk valuesChunk) {} @Override public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java index 24e4374eb94..cd69b57e48e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java @@ -36,10 +36,42 @@ protected Context(final int chunkSize) { this.outputValues = WritableFloatChunk.makeWritableChunk(chunkSize); } - public void storeWorkingChunk(@NotNull final Chunk valuesChunk) {} + @Override + public void accumulate(RowSequence inputKeys, + Chunk influencerValueChunk, + IntChunk pushChunk, + IntChunk popChunk, + int len) { + + setValuesChunk(influencerValueChunk); + int pushIndex = 0; + + // chunk processing + for (int ii = 0; ii < len; ii++) { + final int pushCount = pushChunk.get(ii); + final int popCount = popChunk.get(ii); + + // pop for this row + for (int count = 0; count < popCount; count++) { + pop(); + } + + // push for this row + for (int count = 0; count < pushCount; count++) { + push(NULL_ROW_KEY, pushIndex + count); + } + pushIndex += pushCount; + + // write the results to the output chunk + writeToOutputChunk(ii); + } + + // chunk output to column + writeToOutputColumn(inputKeys); + } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) {} + public void setValuesChunk(@NotNull final Chunk valuesChunk) {} @Override public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java index 57d94dad329..9e77ab4e060 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java @@ -7,6 +7,7 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.attributes.Values; @@ -45,7 +46,41 @@ protected Context(final int chunkSize) { } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) {} + public void accumulate(RowSequence inputKeys, + Chunk influencerValueChunk, + IntChunk pushChunk, + IntChunk popChunk, + int len) { + + setValuesChunk(influencerValueChunk); + int pushIndex = 0; + + // chunk processing + for (int ii = 0; ii < len; ii++) { + final int pushCount = pushChunk.get(ii); + final int popCount = popChunk.get(ii); + + // pop for this row + for (int count = 0; count < popCount; count++) { + pop(); + } + + // push for this row + for (int count = 0; count < pushCount; count++) { + push(NULL_ROW_KEY, pushIndex + count); + } + pushIndex += pushCount; + + // write the results to the output chunk + writeToOutputChunk(ii); + } + + // chunk output to column + writeToOutputColumn(inputKeys); + } + + @Override + public void setValuesChunk(@NotNull final Chunk valuesChunk) {} @Override public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java index e8921c4e0b7..1872ccb5a4e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java @@ -7,6 +7,7 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableLongChunk; import io.deephaven.chunk.attributes.Values; @@ -45,7 +46,41 @@ protected Context(final int chunkSize) { } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) {} + public void accumulate(RowSequence inputKeys, + Chunk influencerValueChunk, + IntChunk pushChunk, + IntChunk popChunk, + int len) { + + setValuesChunk(influencerValueChunk); + int pushIndex = 0; + + // chunk processing + for (int ii = 0; ii < len; ii++) { + final int pushCount = pushChunk.get(ii); + final int popCount = popChunk.get(ii); + + // pop for this row + for (int count = 0; count < popCount; count++) { + pop(); + } + + // push for this row + for (int count = 0; count < pushCount; count++) { + push(NULL_ROW_KEY, pushIndex + count); + } + pushIndex += pushCount; + + // write the results to the output chunk + writeToOutputChunk(ii); + } + + // chunk output to column + writeToOutputColumn(inputKeys); + } + + @Override + public void setValuesChunk(@NotNull final Chunk valuesChunk) {} @Override public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java index ed51f997ae3..6dd4e96ca32 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java @@ -7,6 +7,7 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.attributes.Values; @@ -46,7 +47,41 @@ protected Context(final int chunkSize) { } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) {} + public void accumulate(RowSequence inputKeys, + Chunk influencerValueChunk, + IntChunk pushChunk, + IntChunk popChunk, + int len) { + + setValuesChunk(influencerValueChunk); + int pushIndex = 0; + + // chunk processing + for (int ii = 0; ii < len; ii++) { + final int pushCount = pushChunk.get(ii); + final int popCount = popChunk.get(ii); + + // pop for this row + for (int count = 0; count < popCount; count++) { + pop(); + } + + // push for this row + for (int count = 0; count < pushCount; count++) { + push(NULL_ROW_KEY, pushIndex + count); + } + pushIndex += pushCount; + + // write the results to the output chunk + writeToOutputChunk(ii); + } + + // chunk output to column + writeToOutputColumn(inputKeys); + } + + @Override + public void setValuesChunk(@NotNull final Chunk valuesChunk) {} @Override public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java index ba2c1a1a29d..53ecca43c05 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java @@ -7,6 +7,7 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableShortChunk; import io.deephaven.chunk.attributes.Values; @@ -45,7 +46,41 @@ protected Context(final int chunkSize) { } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) {} + public void accumulate(RowSequence inputKeys, + Chunk influencerValueChunk, + IntChunk pushChunk, + IntChunk popChunk, + int len) { + + setValuesChunk(influencerValueChunk); + int pushIndex = 0; + + // chunk processing + for (int ii = 0; ii < len; ii++) { + final int pushCount = pushChunk.get(ii); + final int popCount = popChunk.get(ii); + + // pop for this row + for (int count = 0; count < popCount; count++) { + pop(); + } + + // push for this row + for (int count = 0; count < pushCount; count++) { + push(NULL_ROW_KEY, pushIndex + count); + } + pushIndex += pushCount; + + // write the results to the output chunk + writeToOutputChunk(ii); + } + + // chunk output to column + writeToOutputColumn(inputKeys); + } + + @Override + public void setValuesChunk(@NotNull final Chunk valuesChunk) {} @Override public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java index 24c8d8d5965..57bafea113f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java @@ -23,14 +23,14 @@ public class ByteCumMinMaxOperator extends BaseByteUpdateByOperator { // endregion extra-fields protected class Context extends BaseByteUpdateByOperator.Context { - public ByteChunk byteValueChunk; + public ByteChunk byteValueChunk; protected Context(int chunkSize) { super(chunkSize); } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { byteValueChunk = valuesChunk.asByteChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java index 077dd6daff0..77987c81f33 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java @@ -23,14 +23,14 @@ public class DoubleCumMinMaxOperator extends BaseDoubleUpdateByOperator { // endregion extra-fields protected class Context extends BaseDoubleUpdateByOperator.Context { - public DoubleChunk doubleValueChunk; + public DoubleChunk doubleValueChunk; protected Context(int chunkSize) { super(chunkSize); } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { doubleValueChunk = valuesChunk.asDoubleChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java index f0840064945..099f80b2218 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java @@ -18,14 +18,14 @@ public class FloatCumMinMaxOperator extends BaseFloatUpdateByOperator { // endregion extra-fields protected class Context extends BaseFloatUpdateByOperator.Context { - public FloatChunk floatValueChunk; + public FloatChunk floatValueChunk; protected Context(int chunkSize) { super(chunkSize); } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { floatValueChunk = valuesChunk.asFloatChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java index 364e5075b53..42ef98e1e94 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java @@ -23,14 +23,14 @@ public class IntCumMinMaxOperator extends BaseIntUpdateByOperator { // endregion extra-fields protected class Context extends BaseIntUpdateByOperator.Context { - public IntChunk intValueChunk; + public IntChunk intValueChunk; protected Context(int chunkSize) { super(chunkSize); } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { intValueChunk = valuesChunk.asIntChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java index fd203e71e72..f3a6e178873 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java @@ -31,14 +31,14 @@ public class LongCumMinMaxOperator extends BaseLongUpdateByOperator { // endregion extra-fields protected class Context extends BaseLongUpdateByOperator.Context { - public LongChunk longValueChunk; + public LongChunk longValueChunk; protected Context(int chunkSize) { super(chunkSize); } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { longValueChunk = valuesChunk.asLongChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java index 229041b7e5d..51bf5a95952 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java @@ -18,14 +18,14 @@ public class ShortCumMinMaxOperator extends BaseShortUpdateByOperator { // endregion extra-fields protected class Context extends BaseShortUpdateByOperator.Context { - public ShortChunk shortValueChunk; + public ShortChunk shortValueChunk; protected Context(int chunkSize) { super(chunkSize); } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { shortValueChunk = valuesChunk.asShortChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java index 2cccd699dd5..aad5262ffa6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java @@ -19,14 +19,14 @@ public final class BigDecimalCumProdOperator extends BaseObjectUpdateByOperator< private final MathContext mathContext; protected class Context extends BaseObjectUpdateByOperator.Context { - public ObjectChunk objectValueChunk; + public ObjectChunk objectValueChunk; protected Context(int chunkSize) { super(chunkSize); } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { objectValueChunk = valuesChunk.asObjectChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java index d89ced41ab2..f168b360ad3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java @@ -15,14 +15,14 @@ public final class BigIntegerCumProdOperator extends BaseObjectUpdateByOperator { protected class Context extends BaseObjectUpdateByOperator.Context { - public ObjectChunk objectValueChunk; + public ObjectChunk objectValueChunk; protected Context(int chunkSize) { super(chunkSize); } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { objectValueChunk = valuesChunk.asObjectChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java index 9f0e0f20ae3..cfe11d83891 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java @@ -21,14 +21,14 @@ public class ByteCumProdOperator extends BaseLongUpdateByOperator { // endregion extra-fields protected class Context extends BaseLongUpdateByOperator.Context { - public ByteChunk byteValueChunk; + public ByteChunk byteValueChunk; protected Context(int chunkSize) { super(chunkSize); } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { byteValueChunk = valuesChunk.asByteChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java index e66d648c0ca..1e36c21be4f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java @@ -22,14 +22,14 @@ public class DoubleCumProdOperator extends BaseDoubleUpdateByOperator { // endregion extra-fields protected class Context extends BaseDoubleUpdateByOperator.Context { - public DoubleChunk doubleValueChunk; + public DoubleChunk doubleValueChunk; protected Context(int chunkSize) { super(chunkSize); } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { doubleValueChunk = valuesChunk.asDoubleChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java index 21ec47b8c67..24ce0d12b44 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java @@ -17,14 +17,14 @@ public class FloatCumProdOperator extends BaseFloatUpdateByOperator { // endregion extra-fields protected class Context extends BaseFloatUpdateByOperator.Context { - public FloatChunk floatValueChunk; + public FloatChunk floatValueChunk; protected Context(int chunkSize) { super(chunkSize); } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { floatValueChunk = valuesChunk.asFloatChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java index c5f3a2e7b5f..51f8b5a9000 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java @@ -21,14 +21,14 @@ public class IntCumProdOperator extends BaseLongUpdateByOperator { // endregion extra-fields protected class Context extends BaseLongUpdateByOperator.Context { - public IntChunk intValueChunk; + public IntChunk intValueChunk; protected Context(int chunkSize) { super(chunkSize); } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { intValueChunk = valuesChunk.asIntChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java index 8e89a58fe03..88d010f6600 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java @@ -21,14 +21,14 @@ public class LongCumProdOperator extends BaseLongUpdateByOperator { // endregion extra-fields protected class Context extends BaseLongUpdateByOperator.Context { - public LongChunk longValueChunk; + public LongChunk longValueChunk; protected Context(int chunkSize) { super(chunkSize); } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { longValueChunk = valuesChunk.asLongChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java index 55f5cce77ec..93e0f448491 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java @@ -16,14 +16,14 @@ public class ShortCumProdOperator extends BaseLongUpdateByOperator { // endregion extra-fields protected class Context extends BaseLongUpdateByOperator.Context { - public ShortChunk shortValueChunk; + public ShortChunk shortValueChunk; protected Context(int chunkSize) { super(chunkSize); } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { shortValueChunk = valuesChunk.asShortChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java index 7e8f3bb112c..92b79230efc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java @@ -20,7 +20,7 @@ public final class BigDecimalRollingSumOperator extends BaseWindowedObjectUpdate private final MathContext mathContext; protected class Context extends BaseWindowedObjectUpdateByOperator.Context { - protected ObjectChunk objectInfluencerValuesChunk; + protected ObjectChunk objectInfluencerValuesChunk; protected RingBuffer objectWindowValues; protected Context(final int chunkSize) { @@ -36,7 +36,7 @@ public void close() { @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { objectInfluencerValuesChunk = valuesChunk.asObjectChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java index 604898a6850..01cd2c6d163 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java @@ -17,7 +17,7 @@ public final class BigIntegerRollingSumOperator extends BaseWindowedObjectUpdateByOperator { protected class Context extends BaseWindowedObjectUpdateByOperator.Context { - protected ObjectChunk objectInfluencerValuesChunk; + protected ObjectChunk objectInfluencerValuesChunk; protected RingBuffer objectWindowValues; protected Context(final int chunkSize) { @@ -33,7 +33,7 @@ public void close() { @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { objectInfluencerValuesChunk = valuesChunk.asObjectChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java index 7ed9fd6f66f..b2ffb04df4f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java @@ -25,7 +25,7 @@ public class ByteRollingSumOperator extends BaseWindowedLongUpdateByOperator { // endregion extra-fields protected class Context extends BaseWindowedLongUpdateByOperator.Context { - protected ByteChunk byteInfluencerValuesChunk; + protected ByteChunk byteInfluencerValuesChunk; protected ByteRingBuffer byteWindowValues; @@ -42,7 +42,7 @@ public void close() { @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { byteInfluencerValuesChunk = valuesChunk.asByteChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java index 06410a09147..65652154a5a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java @@ -25,7 +25,7 @@ public class DoubleRollingSumOperator extends BaseWindowedDoubleUpdateByOperator // endregion extra-fields protected class Context extends BaseWindowedDoubleUpdateByOperator.Context { - protected DoubleChunk doubleInfluencerValuesChunk; + protected DoubleChunk doubleInfluencerValuesChunk; protected PairwiseDoubleRingBuffer doublePairwiseSum; protected Context(final int chunkSize) { @@ -50,7 +50,7 @@ public void close() { } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { doubleInfluencerValuesChunk = valuesChunk.asDoubleChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java index 845a1d3ba6a..a321c099168 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java @@ -20,7 +20,7 @@ public class FloatRollingSumOperator extends BaseWindowedFloatUpdateByOperator { // endregion extra-fields protected class Context extends BaseWindowedFloatUpdateByOperator.Context { - protected FloatChunk floatInfluencerValuesChunk; + protected FloatChunk floatInfluencerValuesChunk; protected PairwiseFloatRingBuffer floatPairwiseSum; protected Context(final int chunkSize) { @@ -45,7 +45,7 @@ public void close() { } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { floatInfluencerValuesChunk = valuesChunk.asFloatChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java index e2aa3b936d0..540068537fc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java @@ -24,7 +24,7 @@ public class IntRollingSumOperator extends BaseWindowedLongUpdateByOperator { // endregion extra-fields protected class Context extends BaseWindowedLongUpdateByOperator.Context { - protected IntChunk intInfluencerValuesChunk; + protected IntChunk intInfluencerValuesChunk; protected IntRingBuffer intWindowValues; @@ -41,7 +41,7 @@ public void close() { @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { intInfluencerValuesChunk = valuesChunk.asIntChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java index 977543f8545..f6bf5e147b3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java @@ -24,7 +24,7 @@ public class LongRollingSumOperator extends BaseWindowedLongUpdateByOperator { // endregion extra-fields protected class Context extends BaseWindowedLongUpdateByOperator.Context { - protected LongChunk longInfluencerValuesChunk; + protected LongChunk longInfluencerValuesChunk; protected LongRingBuffer longWindowValues; @@ -41,7 +41,7 @@ public void close() { @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { longInfluencerValuesChunk = valuesChunk.asLongChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java index 5e32f0b0c98..50b1d35410b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java @@ -19,7 +19,7 @@ public class ShortRollingSumOperator extends BaseWindowedLongUpdateByOperator { // endregion extra-fields protected class Context extends BaseWindowedLongUpdateByOperator.Context { - protected ShortChunk shortInfluencerValuesChunk; + protected ShortChunk shortInfluencerValuesChunk; protected ShortRingBuffer shortWindowValues; @@ -36,7 +36,7 @@ public void close() { @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { shortInfluencerValuesChunk = valuesChunk.asShortChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java index 406e3fffb0a..579b01a4a07 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java @@ -19,14 +19,14 @@ public final class BigDecimalCumSumOperator extends BaseObjectUpdateByOperator.Context { - public ObjectChunk objectValueChunk; + public ObjectChunk objectValueChunk; protected Context(int chunkSize) { super(chunkSize); } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { objectValueChunk = valuesChunk.asObjectChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java index 42fe393b34f..2ed92475971 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java @@ -15,14 +15,14 @@ public final class BigIntegerCumSumOperator extends BaseObjectUpdateByOperator { protected class Context extends BaseObjectUpdateByOperator.Context { - public ObjectChunk objectValueChunk; + public ObjectChunk objectValueChunk; protected Context(int chunkSize) { super(chunkSize); } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { objectValueChunk = valuesChunk.asObjectChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java index 529adfff101..f815f252f6c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java @@ -21,14 +21,14 @@ public class ByteCumSumOperator extends BaseLongUpdateByOperator { // endregion extra-fields protected class Context extends BaseLongUpdateByOperator.Context { - public ByteChunk byteValueChunk; + public ByteChunk byteValueChunk; protected Context(int chunkSize) { super(chunkSize); } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { byteValueChunk = valuesChunk.asByteChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java index ff3d2e89df0..6b57eb6abf0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java @@ -21,14 +21,14 @@ public class DoubleCumSumOperator extends BaseDoubleUpdateByOperator { // endregion extra-fields protected class Context extends BaseDoubleUpdateByOperator.Context { - public DoubleChunk doubleValueChunk; + public DoubleChunk doubleValueChunk; protected Context(int chunkSize) { super(chunkSize); } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { doubleValueChunk = valuesChunk.asDoubleChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java index 404715f106c..ba15dedde91 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java @@ -16,14 +16,14 @@ public class FloatCumSumOperator extends BaseFloatUpdateByOperator { // endregion extra-fields protected class Context extends BaseFloatUpdateByOperator.Context { - public FloatChunk floatValueChunk; + public FloatChunk floatValueChunk; protected Context(int chunkSize) { super(chunkSize); } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { floatValueChunk = valuesChunk.asFloatChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java index ad4c9bf73b7..4c666aa0c2f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java @@ -20,14 +20,14 @@ public class IntCumSumOperator extends BaseLongUpdateByOperator { // endregion extra-fields protected class Context extends BaseLongUpdateByOperator.Context { - public IntChunk intValueChunk; + public IntChunk intValueChunk; protected Context(int chunkSize) { super(chunkSize); } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { intValueChunk = valuesChunk.asIntChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java index 7e024136889..bad965e6ae2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java @@ -20,14 +20,14 @@ public class LongCumSumOperator extends BaseLongUpdateByOperator { // endregion extra-fields protected class Context extends BaseLongUpdateByOperator.Context { - public LongChunk longValueChunk; + public LongChunk longValueChunk; protected Context(int chunkSize) { super(chunkSize); } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { longValueChunk = valuesChunk.asLongChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java index 0d324653055..708ddbedb81 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java @@ -15,14 +15,14 @@ public class ShortCumSumOperator extends BaseLongUpdateByOperator { // endregion extra-fields protected class Context extends BaseLongUpdateByOperator.Context { - public ShortChunk shortValueChunk; + public ShortChunk shortValueChunk; protected Context(int chunkSize) { super(chunkSize); } @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) { + public void setValuesChunk(@NotNull final Chunk valuesChunk) { shortValueChunk = valuesChunk.asShortChunk(); } diff --git a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java index fd50e63e5fa..47bf51a3261 100644 --- a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java +++ b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java @@ -36,7 +36,8 @@ public static void main(String[] args) throws IOException { fixupStandardObject(objectResult, "ObjectFillByOperator", false, "super\\(fillPair, new String\\[\\] \\{ fillPair.rightColumn \\}, redirContext\\);", "super(fillPair, new String[] { fillPair.rightColumn }, redirContext, colType);", - " BaseObjectUpdateByOperator", " BaseObjectUpdateByOperator"); + " BaseObjectUpdateByOperator", " BaseObjectUpdateByOperator", + "public ObjectChunk Date: Wed, 19 Oct 2022 09:12:01 -0700 Subject: [PATCH 032/123] Enabled multi-column input operators, lightened ZeroKey creation --- .../impl/BucketedPartitionedUpdateBy.java | 18 +- .../deephaven/engine/table/impl/UpdateBy.java | 155 ++++++++++-------- .../impl/UpdateByCumulativeOperator.java | 6 +- .../engine/table/impl/UpdateByOperator.java | 10 +- .../table/impl/UpdateByWindowedOperator.java | 7 +- .../engine/table/impl/ZeroKeyUpdateBy.java | 31 +++- .../table/impl/updateby/UpdateByWindow.java | 63 ++++--- .../updateby/UpdateByWindowCumulative.java | 52 +++--- .../impl/updateby/UpdateByWindowTicks.java | 75 +++++---- .../impl/updateby/UpdateByWindowTime.java | 62 +++---- .../updateby/ema/BigDecimalEMAOperator.java | 12 +- .../updateby/ema/BigIntegerEMAOperator.java | 12 +- .../updateby/ema/BigNumberEMAOperator.java | 6 +- .../impl/updateby/ema/ByteEMAOperator.java | 12 +- .../impl/updateby/ema/DoubleEMAOperator.java | 12 +- .../impl/updateby/ema/FloatEMAOperator.java | 12 +- .../impl/updateby/ema/IntEMAOperator.java | 12 +- .../impl/updateby/ema/LongEMAOperator.java | 12 +- .../impl/updateby/ema/ShortEMAOperator.java | 12 +- .../updateby/fill/BooleanFillByOperator.java | 2 +- .../updateby/fill/ByteFillByOperator.java | 2 +- .../updateby/fill/CharFillByOperator.java | 2 +- .../updateby/fill/DoubleFillByOperator.java | 2 +- .../updateby/fill/FloatFillByOperator.java | 2 +- .../impl/updateby/fill/IntFillByOperator.java | 2 +- .../updateby/fill/LongFillByOperator.java | 2 +- .../updateby/fill/ObjectFillByOperator.java | 2 +- .../updateby/fill/ShortFillByOperator.java | 2 +- .../internal/BaseByteUpdateByOperator.java | 4 +- .../internal/BaseCharUpdateByOperator.java | 4 +- .../internal/BaseDoubleUpdateByOperator.java | 4 +- .../internal/BaseFloatUpdateByOperator.java | 4 +- .../internal/BaseIntUpdateByOperator.java | 4 +- .../internal/BaseLongUpdateByOperator.java | 4 +- .../internal/BaseObjectBinaryOperator.java | 2 +- .../internal/BaseObjectUpdateByOperator.java | 4 +- .../internal/BaseShortUpdateByOperator.java | 4 +- .../BaseWindowedByteUpdateByOperator.java | 4 +- .../BaseWindowedCharUpdateByOperator.java | 4 +- .../BaseWindowedDoubleUpdateByOperator.java | 4 +- .../BaseWindowedFloatUpdateByOperator.java | 4 +- .../BaseWindowedIntUpdateByOperator.java | 4 +- .../BaseWindowedLongUpdateByOperator.java | 4 +- .../BaseWindowedObjectUpdateByOperator.java | 4 +- .../BaseWindowedShortUpdateByOperator.java | 4 +- .../minmax/ByteCumMinMaxOperator.java | 2 +- .../minmax/DoubleCumMinMaxOperator.java | 2 +- .../minmax/FloatCumMinMaxOperator.java | 2 +- .../updateby/minmax/IntCumMinMaxOperator.java | 2 +- .../minmax/LongCumMinMaxOperator.java | 2 +- .../minmax/ShortCumMinMaxOperator.java | 2 +- .../prod/BigDecimalCumProdOperator.java | 2 +- .../prod/BigIntegerCumProdOperator.java | 2 +- .../updateby/prod/ByteCumProdOperator.java | 2 +- .../updateby/prod/DoubleCumProdOperator.java | 2 +- .../updateby/prod/FloatCumProdOperator.java | 2 +- .../updateby/prod/IntCumProdOperator.java | 2 +- .../updateby/prod/LongCumProdOperator.java | 2 +- .../updateby/prod/ShortCumProdOperator.java | 2 +- .../BigDecimalRollingSumOperator.java | 2 +- .../BigIntegerRollingSumOperator.java | 2 +- .../rollingsum/ByteRollingSumOperator.java | 2 +- .../rollingsum/DoubleRollingSumOperator.java | 2 +- .../rollingsum/FloatRollingSumOperator.java | 2 +- .../rollingsum/IntRollingSumOperator.java | 2 +- .../rollingsum/LongRollingSumOperator.java | 2 +- .../rollingsum/ShortRollingSumOperator.java | 2 +- .../sum/BigDecimalCumSumOperator.java | 2 +- .../sum/BigIntegerCumSumOperator.java | 2 +- .../impl/updateby/sum/ByteCumSumOperator.java | 2 +- .../updateby/sum/DoubleCumSumOperator.java | 2 +- .../updateby/sum/FloatCumSumOperator.java | 2 +- .../impl/updateby/sum/IntCumSumOperator.java | 2 +- .../impl/updateby/sum/LongCumSumOperator.java | 2 +- .../updateby/sum/ShortCumSumOperator.java | 2 +- 75 files changed, 401 insertions(+), 324 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java index 6fbb407c3ec..c5cc64ecddd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java @@ -39,6 +39,9 @@ class BucketedPartitionedUpdateBy extends UpdateBy { public static Table compute(@NotNull final String description, @NotNull final QueryTable source, @NotNull final UpdateByOperator[] ops, + @NotNull final UpdateByWindow[] windows, + @NotNull final ColumnSource[] inputSources, + @NotNull final int[][] operatorInputSourceSlots, @NotNull final Map> resultSources, @NotNull final Collection byColumns, @NotNull final UpdateByRedirectionContext redirContext, @@ -46,6 +49,9 @@ public static Table compute(@NotNull final String description, final BucketedPartitionedUpdateBy updateBy = new BucketedPartitionedUpdateBy(description, ops, + windows, + inputSources, + operatorInputSourceSlots, source, resultSources, byColumns, @@ -57,12 +63,15 @@ public static Table compute(@NotNull final String description, protected BucketedPartitionedUpdateBy(@NotNull final String description, @NotNull final UpdateByOperator[] operators, + @NotNull final UpdateByWindow[] windows, + @NotNull final ColumnSource[] inputSources, + @NotNull final int[][] operatorInputSourceSlots, @NotNull final QueryTable source, @NotNull final Map> resultSources, @NotNull final Collection byColumns, @NotNull final UpdateByRedirectionContext redirContext, @NotNull final UpdateByControl control) { - super(operators, source, redirContext, control); + super(operators, windows, inputSources, operatorInputSourceSlots, source, redirContext, control); // create a source-listener that will listen to the source updates and apply the shifts to the output columns final QueryTable sourceListenerTable = new QueryTable(source.getRowSet(), source.getColumnSourceMap()); @@ -96,9 +105,9 @@ public void onUpdate(@NotNull final TableUpdate upstream) { // create a listener and recorder for the source table as first entry BucketedPartitionedUpdateByListenerRecorder recorder = - new BucketedPartitionedUpdateByListenerRecorder(description, source, resultTable); + new BucketedPartitionedUpdateByListenerRecorder(description, sourceListenerTable, resultTable); recorder.setMergedListener(listener); - source.listenForUpdates(recorder); + sourceListenerTable.listenForUpdates(recorder); recorders.offerLast(recorder); } else { @@ -123,6 +132,9 @@ public void onUpdate(@NotNull final TableUpdate upstream) { description, (QueryTable) t, operators, + windows, + inputSources, + operatorInputSourceSlots, resultSources, redirContext, control, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java index 37e3041dd30..46f64c7014c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java @@ -23,6 +23,7 @@ import org.jetbrains.annotations.Nullable; import java.util.*; +import java.util.stream.Collectors; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; @@ -31,9 +32,10 @@ */ public abstract class UpdateBy { protected final ColumnSource[] inputSources; - protected final int[] inputSourceSlots; - protected final UpdateByWindow[] windows; + // some columns will have multiple inputs, such as time-based and Weighted computations + protected final int[][] operatorInputSourceSlots; protected final UpdateByOperator[] operators; + protected final UpdateByWindow[] windows; protected final QueryTable source; protected final UpdateByRedirectionContext redirContext; @@ -127,6 +129,9 @@ public void close() { } protected UpdateBy(@NotNull final UpdateByOperator[] operators, + @NotNull final UpdateByWindow[] windows, + @NotNull final ColumnSource[] inputSources, + @NotNull final int[][] operatorInputSourceSlots, @NotNull final QueryTable source, @NotNull final UpdateByRedirectionContext redirContext, UpdateByControl control) { @@ -139,70 +144,9 @@ protected UpdateBy(@NotNull final UpdateByOperator[] operators, this.source = source; this.operators = operators; - - // the next bit is complicated but the goal is simple. We don't want to have duplicate input column sources, so - // we will store each one only once in inputSources and setup some mapping from the opIdx to the input column. - // noinspection unchecked - inputSources = new ColumnSource[operators.length]; - inputSourceSlots = new int[operators.length]; - final TObjectIntHashMap> sourceToSlotMap = new TObjectIntHashMap<>(); - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - final ColumnSource input = source.getColumnSource(operators[opIdx].getInputColumnName()); - final int maybeExistingSlot = sourceToSlotMap.get(input); - if (maybeExistingSlot == sourceToSlotMap.getNoEntryValue()) { - inputSourceSlots[opIdx] = opIdx; - sourceToSlotMap.put(input, opIdx); - inputSources[opIdx] = ReinterpretUtils.maybeConvertToPrimitive(input); - } else { - inputSourceSlots[opIdx] = maybeExistingSlot; - } - } - - // now we want to divide the operators into similar windows for efficient processing - TIntObjectHashMap windowHashToOperatorIndicesMap = new TIntObjectHashMap<>(); - - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - int hash = UpdateByWindow.hashCodeFromOperator(operators[opIdx]); - boolean added = false; - - // rudimentary collision detection and handling - while (!added) { - if (!windowHashToOperatorIndicesMap.containsKey(hash)) { - // does not exist, can add immediately - windowHashToOperatorIndicesMap.put(hash, new TIntArrayList()); - windowHashToOperatorIndicesMap.get(hash).add(opIdx); - added = true; - } else { - final int existingOpIdx = windowHashToOperatorIndicesMap.get(hash).get(0); - if (UpdateByWindow.isEquivalentWindow(operators[existingOpIdx], operators[opIdx])) { - // no collision, can add immediately - windowHashToOperatorIndicesMap.get(hash).add(opIdx); - added = true; - } else { - // there is a collision, increment hash and try again - hash++; - } - } - } - } - - // store the operator information into the windows - this.windows = new UpdateByWindow[windowHashToOperatorIndicesMap.size()]; - final MutableInt winIdx = new MutableInt(0); - - windowHashToOperatorIndicesMap.forEachEntry((final int hash, final TIntArrayList opIndices) -> { - final UpdateByOperator[] windowOperators = - Arrays.stream(opIndices.toArray()) - .mapToObj(idx -> operators[idx]) - .toArray(UpdateByOperator[]::new); - final int[] windowOperatorSourceSlots = - Arrays.stream(opIndices.toArray()) - .map(idx -> inputSourceSlots[idx]) - .toArray(); - this.windows[winIdx.getAndIncrement()] = - UpdateByWindow.createFromOperatorArray(windowOperators, windowOperatorSourceSlots); - return true; - }); + this.windows = windows; + this.inputSources = inputSources; + this.operatorInputSourceSlots = operatorInputSourceSlots; } // region UpdateBy implementation @@ -280,6 +224,79 @@ public static Table updateBy(@NotNull final QueryTable source, final UpdateByOperator[] opArr = ops.toArray(UpdateByOperator.ZERO_LENGTH_OP_ARRAY); + // the next bit is complicated but the goal is simple. We don't want to have duplicate input column sources, so + // we will store each one only once in inputSources and setup some mapping from the opIdx to the input column. + // noinspection unchecked + + final ArrayList> inputSourceList = new ArrayList<>(); + final int[][] operatorInputSourceSlotArr = new int[opArr.length][]; + final TObjectIntHashMap> sourceToSlotMap = new TObjectIntHashMap<>(); + + for (int opIdx = 0; opIdx < opArr.length; opIdx++) { + final String[] inputColumnNames = opArr[opIdx].getInputColumnNames(); + for (int colIdx = 0; colIdx < inputColumnNames.length; colIdx++) { + final ColumnSource input = source.getColumnSource(inputColumnNames[colIdx]); + final int maybeExistingSlot = sourceToSlotMap.get(input); + // add a new entry for this operator + operatorInputSourceSlotArr[opIdx] = new int[inputColumnNames.length]; + if (maybeExistingSlot == sourceToSlotMap.getNoEntryValue()) { + int srcIdx = inputSourceList.size(); + // create a new input source and map the operator to it + inputSourceList.add(ReinterpretUtils.maybeConvertToPrimitive(input)); + sourceToSlotMap.put(input, srcIdx); + operatorInputSourceSlotArr[opIdx][colIdx] = srcIdx; + } else { + operatorInputSourceSlotArr[opIdx][colIdx] = maybeExistingSlot; + } + } + } + final ColumnSource[] inputSourceArr = inputSourceList.toArray(new ColumnSource[0]); + + // now we want to divide the operators into similar windows for efficient processing + TIntObjectHashMap windowHashToOperatorIndicesMap = new TIntObjectHashMap<>(); + + for (int opIdx = 0; opIdx < opArr.length; opIdx++) { + int hash = UpdateByWindow.hashCodeFromOperator(opArr[opIdx]); + boolean added = false; + + // rudimentary collision detection and handling + while (!added) { + if (!windowHashToOperatorIndicesMap.containsKey(hash)) { + // does not exist, can add immediately + windowHashToOperatorIndicesMap.put(hash, new TIntArrayList()); + windowHashToOperatorIndicesMap.get(hash).add(opIdx); + added = true; + } else { + final int existingOpIdx = windowHashToOperatorIndicesMap.get(hash).get(0); + if (UpdateByWindow.isEquivalentWindow(opArr[existingOpIdx], opArr[opIdx])) { + // no collision, can add immediately + windowHashToOperatorIndicesMap.get(hash).add(opIdx); + added = true; + } else { + // there is a collision, increment hash and try again + hash++; + } + } + } + } + // store the operators into the windows + final UpdateByWindow[] windowArr = new UpdateByWindow[windowHashToOperatorIndicesMap.size()]; + final MutableInt winIdx = new MutableInt(0); + + windowHashToOperatorIndicesMap.forEachEntry((final int hash, final TIntArrayList opIndices) -> { + final UpdateByOperator[] windowOperators = new UpdateByOperator[opIndices.size()]; + final int[][] windowOperatorSourceSlots = new int[opIndices.size()][]; + + for (int ii = 0; ii < opIndices.size(); ii++) { + final int opIdx = opIndices.get(ii); + windowOperators[ii] = opArr[opIdx]; + windowOperatorSourceSlots[ii] = operatorInputSourceSlotArr[opIdx]; + } + windowArr[winIdx.getAndIncrement()] = + UpdateByWindow.createFromOperatorArray(windowOperators, windowOperatorSourceSlots); + return true; + }); + // noinspection rawtypes final Map> resultSources = new LinkedHashMap<>(source.getColumnSourceMap()); resultSources.putAll(opResultSources); @@ -290,6 +307,9 @@ public static Table updateBy(@NotNull final QueryTable source, descriptionBuilder.toString(), source, opArr, + windowArr, + inputSourceArr, + operatorInputSourceSlotArr, resultSources, ctx, control, @@ -323,6 +343,9 @@ public static Table updateBy(@NotNull final QueryTable source, descriptionBuilder.toString(), source, opArr, + windowArr, + inputSourceArr, + operatorInputSourceSlotArr, resultSources, byColumns, ctx, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java index 5ebc93aa79c..35b57c3ee3c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java @@ -47,7 +47,7 @@ public void pop() { } public abstract void accumulate(RowSequence inputKeys, - Chunk valueChunk, + Chunk[] valueChunkArr, LongChunk tsChunk, int len); } @@ -87,8 +87,8 @@ public long getFwdWindowUnits() { @NotNull @Override - public String getInputColumnName() { - return pair.rightColumn; + public String[] getInputColumnNames() { + return new String[] {pair.rightColumn}; } @NotNull diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java index 594c0a72927..1c8174ed878 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java @@ -83,12 +83,12 @@ interface UpdateContext extends SafeCloseable { } /** - * Get the name of the input column this operator depends on. + * Get the name of the input column(s) this operator depends on. * * @return the name of the input column */ @NotNull - String getInputColumnName(); + String[] getInputColumnNames(); /** * Get the name of the timestamp column this operator depends on. @@ -142,14 +142,14 @@ interface UpdateContext extends SafeCloseable { void startTrackingPrev(); /** - * Make an {@link UpdateContext} suitable for use with non-bucketed updates. + * Make an {@link UpdateContext} suitable for use with updates. * * @param chunkSize The expected size of chunks that will be provided during the update, - * @param inputSource The column source that contains the input values needed by the operator + * @param inputSourceArr The column sources that contains the input values needed by the operator * @return a new context */ @NotNull - UpdateContext makeUpdateContext(final int chunkSize, ColumnSource inputSource); + UpdateContext makeUpdateContext(final int chunkSize, ColumnSource[] inputSourceArr); /** * Perform and bookkeeping required at the end of a single part of the update. This is always preceded with a call diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java index af2baf88538..6741231c0a2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java @@ -28,7 +28,7 @@ public abstract class Context implements UpdateContext { public void close() {} public abstract void accumulate(RowSequence inputKeys, - Chunk influencerValueChunk, + Chunk influencerValueChunkArr[], IntChunk pushChunk, IntChunk popChunk, int len); @@ -81,10 +81,11 @@ public long getFwdWindowUnits() { return forwardTimeScaleUnits; } + /*** Mostly will be a single input column, but some accept multiple input (WAvg for example) */ @NotNull @Override - public String getInputColumnName() { - return pair.rightColumn; + public String[] getInputColumnNames() { + return new String[] {pair.rightColumn}; } @NotNull diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java index d0126c67834..b1a289057be 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java @@ -11,7 +11,6 @@ import io.deephaven.engine.table.impl.sources.ReinterpretUtils; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.updateby.UpdateByWindow; -import io.deephaven.engine.table.impl.util.UpdateSizeCalculator; import io.deephaven.util.SafeCloseable; import org.apache.commons.lang3.mutable.MutableLong; import org.jetbrains.annotations.NotNull; @@ -35,6 +34,9 @@ class ZeroKeyUpdateBy extends UpdateBy { final ColumnSource timestampColumnSource; final ModifiedColumnSet timestampColumnSet; + // individual output modifiedColumnSets for the operators + protected final ModifiedColumnSet[][] windowOperatorOutputModifiedColumnSets; + /** * Perform an updateBy without any key columns. * @@ -49,12 +51,18 @@ class ZeroKeyUpdateBy extends UpdateBy { public static Table compute(@NotNull final String description, @NotNull final QueryTable source, @NotNull final UpdateByOperator[] ops, + @NotNull final UpdateByWindow[] windows, + @NotNull final ColumnSource[] inputSources, + @NotNull final int[][] operatorInputSourceSlots, @NotNull final Map> resultSources, @NotNull final UpdateByRedirectionContext redirContext, @NotNull final UpdateByControl control, final boolean applyShifts) { + final QueryTable result = new QueryTable(source.getRowSet(), resultSources); - final ZeroKeyUpdateBy updateBy = new ZeroKeyUpdateBy(ops, source, redirContext, control, applyShifts); + final ZeroKeyUpdateBy updateBy = new ZeroKeyUpdateBy(ops, windows, inputSources, operatorInputSourceSlots, + source, redirContext, control, applyShifts); + updateBy.doInitialAdditions(); if (source.isRefreshing()) { @@ -67,11 +75,14 @@ public static Table compute(@NotNull final String description, } protected ZeroKeyUpdateBy(@NotNull final UpdateByOperator[] operators, + @NotNull final UpdateByWindow[] windows, + @NotNull final ColumnSource[] inputSources, + @NotNull final int[][] operatorInputSourceSlots, @NotNull final QueryTable source, @NotNull final UpdateByRedirectionContext redirContext, @NotNull final UpdateByControl control, final boolean applyShifts) { - super(operators, source, redirContext, control); + super(operators, windows, inputSources, operatorInputSourceSlots, source, redirContext, control); // do we need a timestamp SSA? this.timestampColumnName = Arrays.stream(operators) @@ -90,6 +101,8 @@ protected ZeroKeyUpdateBy(@NotNull final UpdateByOperator[] operators, this.timestampColumnSet = null; } this.applyShifts = applyShifts; + this.windowOperatorOutputModifiedColumnSets = new ModifiedColumnSet[windows.length][]; + } ZeroKeyUpdateByListener newListener(@NotNull final String description, @NotNull final QueryTable result) { @@ -331,8 +344,15 @@ public ZeroKeyUpdateByListener(@Nullable String description, for (int ii = 0; ii < windows.length; ii++) { windows[ii].startTrackingModifications(source, result); - } + final UpdateByOperator[] windowOps = windows[ii].getOperators(); + windowOperatorOutputModifiedColumnSets[ii] = new ModifiedColumnSet[windowOps.length]; + // these must be created here and relate to the local result table + for (int winOpIdx = 0; winOpIdx < windowOps.length; winOpIdx++) { + windowOperatorOutputModifiedColumnSets[ii][winOpIdx] = + result.newModifiedColumnSet(windowOps[winOpIdx].getOutputColumnNames()); + } + } this.transformer = source.newModifiedColumnSetTransformer(result, source.getDefinition().getColumnNamesArray()); } @@ -407,7 +427,8 @@ public void onUpdate(TableUpdate upstream) { // set the modified columns if any operators made changes (add/rem/modify) for (int winIdx = 0; winIdx < windows.length; winIdx++) { if (ctx.windowAffected[winIdx]) { - ctx.windowContexts[winIdx].updateOutputModifiedColumnSet(downstream.modifiedColumnSet); + ctx.windowContexts[winIdx].updateOutputModifiedColumnSet(downstream.modifiedColumnSet, + windowOperatorOutputModifiedColumnSets[winIdx]); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java index 35d5ee822b2..f2331bcc8e3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java @@ -19,14 +19,12 @@ public abstract class UpdateByWindow { @Nullable protected final String timestampColumnName; - // store the operators for this window protected final UpdateByOperator[] operators; // store the index in the {@link UpdateBy.inputSources} - protected final int[] operatorSourceSlots; - // individual input/output modifiedColumnSets for the operators + protected final int[][] operatorInputSourceSlots; + // individual input modifiedColumnSets for the operators protected final ModifiedColumnSet[] operatorInputModifiedColumnSets; - protected final ModifiedColumnSet[] operatorOutputModifiedColumnSets; protected boolean trackModifications; @@ -106,7 +104,8 @@ public RowSet getModifiedRows() { return newModified; } - public void updateOutputModifiedColumnSet(ModifiedColumnSet outputModifiedColumnSet) { + public void updateOutputModifiedColumnSet(ModifiedColumnSet outputModifiedColumnSet, + ModifiedColumnSet[] operatorOutputModifiedColumnSets) { for (int opIdx = 0; opIdx < operators.length; opIdx++) { if (opAffected[opIdx]) { outputModifiedColumnSet.setAll(operatorOutputModifiedColumnSets[opIdx]); @@ -144,14 +143,15 @@ public void close() { } for (int opIdx = 0; opIdx < operators.length; opIdx++) { if (opAffected[opIdx]) { - final int srcIdx = operatorSourceSlots[opIdx]; - if (inputSourceGetContexts[srcIdx] != null) { - inputSourceGetContexts[srcIdx].close(); - inputSourceGetContexts[srcIdx] = null; - } opContext[opIdx].close(); } } + for (int srcIdx = 0; srcIdx < inputSources.length; srcIdx++) { + if (inputSourceGetContexts[srcIdx] != null) { + inputSourceGetContexts[srcIdx].close(); + inputSourceGetContexts[srcIdx] = null; + } + } } } @@ -167,24 +167,21 @@ public void startTrackingModifications(@NotNull final QueryTable source, @NotNul for (int opIdx = 0; opIdx < operators.length; opIdx++) { operatorInputModifiedColumnSets[opIdx] = source.newModifiedColumnSet(operators[opIdx].getAffectingColumnNames()); - operatorOutputModifiedColumnSets[opIdx] = - result.newModifiedColumnSet(operators[opIdx].getOutputColumnNames()); } } - protected UpdateByWindow(UpdateByOperator[] operators, int[] operatorSourceSlots, + protected UpdateByWindow(UpdateByOperator[] operators, int[][] operatorInputSourceSlots, @Nullable String timestampColumnName) { this.operators = operators; - this.operatorSourceSlots = operatorSourceSlots; + this.operatorInputSourceSlots = operatorInputSourceSlots; this.timestampColumnName = timestampColumnName; operatorInputModifiedColumnSets = new ModifiedColumnSet[operators.length]; - operatorOutputModifiedColumnSets = new ModifiedColumnSet[operators.length]; trackModifications = false; } public static UpdateByWindow createFromOperatorArray(final UpdateByOperator[] operators, - final int[] operatorSourceSlots) { + final int[][] operatorSourceSlots) { // review operators to extract timestamp column (if one exists) String timestampColumnName = null; for (UpdateByOperator operator : operators) { @@ -219,15 +216,27 @@ public String getTimestampColumnName() { return timestampColumnName; } - protected static int hashCode(boolean windowed, @Nullable String timestampColumnName, long prevUnits, + public UpdateByOperator[] getOperators() { + return operators; + } + + protected static int hashCode(boolean windowed, @NotNull String[] inputColumnNames, + @Nullable String timestampColumnName, long prevUnits, long fwdUnits) { - // treat all cumulative ops as identical, even if they rely on timestamps + + // hash the input column names + int hash = 0; + for (String s : inputColumnNames) { + hash = 31 * hash + s.hashCode(); + } + + // treat all cumulative ops with the same input columns as identical, even if they rely on timestamps if (!windowed) { - return Boolean.hashCode(false); + return 31 * hash + Boolean.hashCode(false); } // windowed ops are unique per type (ticks/time-based) and window dimensions - int hash = Boolean.hashCode(true); + hash = 31 * hash + Boolean.hashCode(true); hash = 31 * hash + Boolean.hashCode(timestampColumnName != null); hash = 31 * hash + Long.hashCode(prevUnits); hash = 31 * hash + Long.hashCode(fwdUnits); @@ -236,6 +245,7 @@ protected static int hashCode(boolean windowed, @Nullable String timestampColumn public static int hashCodeFromOperator(final UpdateByOperator op) { return hashCode(op instanceof UpdateByWindowedOperator, + op.getInputColumnNames(), op.getTimestampColumnName(), op.getPrevWindowUnits(), op.getPrevWindowUnits()); @@ -245,6 +255,19 @@ public static boolean isEquivalentWindow(final UpdateByOperator opA, final Updat final boolean aWindowed = opA instanceof UpdateByWindowedOperator; final boolean bWindowed = opB instanceof UpdateByWindowedOperator; + // verify input columns + String[] opAInput = opA.getInputColumnNames(); + String[] opBInput = opB.getInputColumnNames(); + + if (opAInput.length != opBInput.length) { + return false; + } + for (int ii = 0; ii < opAInput.length; ii++) { + if (!opAInput[ii].equals(opBInput[ii])) { + return false; + } + } + // equivalent if both are cumulative, not equivalent if only one is cumulative if (!aWindowed && !bWindowed) { return true; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java index 8252272716f..7667c7cecc8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java @@ -1,5 +1,6 @@ package io.deephaven.engine.table.impl.updateby; +import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.*; @@ -44,12 +45,18 @@ protected void makeOperatorContexts() { for (int opIdx = 0; opIdx < operators.length; opIdx++) { if (opAffected[opIdx]) { // create the fill contexts for the input sources - int sourceSlot = operatorSourceSlots[opIdx]; - if (!inputSourceChunkPopulated[sourceSlot]) { - inputSourceGetContexts[sourceSlot] = inputSources[sourceSlot].makeGetContext(workingChunkSize); - inputSourceChunkPopulated[sourceSlot] = true; + final int[] sourceIndices = operatorInputSourceSlots[opIdx]; + final ColumnSource[] inputSourceArr = new ColumnSource[sourceIndices.length]; + for (int ii = 0; ii < sourceIndices.length; ii++) { + int sourceSlot = sourceIndices[ii]; + if (!inputSourceChunkPopulated[sourceSlot]) { + inputSourceGetContexts[sourceSlot] = + inputSources[sourceSlot].makeGetContext(workingChunkSize); + inputSourceChunkPopulated[sourceSlot] = true; + } + inputSourceArr[ii] = inputSources[sourceSlot]; } - opContext[opIdx] = operators[opIdx].makeUpdateContext(workingChunkSize, inputSources[sourceSlot]); + opContext[opIdx] = operators[opIdx].makeUpdateContext(workingChunkSize, inputSourceArr); } } } @@ -164,15 +171,20 @@ public void processRows() { for (int opIdx = 0; opIdx < operators.length; opIdx++) { if (opAffected[opIdx]) { - final int srcIdx = operatorSourceSlots[opIdx]; - - // chunk prep - prepareValuesChunkForSource(srcIdx, rs); + // prep the chunk array needed by the accumulate call + final int[] srcIndices = operatorInputSourceSlots[opIdx]; + Chunk[] chunkArr = new Chunk[srcIndices.length]; + for (int ii = 0; ii < srcIndices.length; ii++) { + int srcIdx = srcIndices[ii]; + // chunk prep + prepareValuesChunkForSource(srcIdx, rs); + chunkArr[ii] = inputSourceChunks[srcIdx]; + } // make the specialized call for cumulative operators ((UpdateByCumulativeOperator.Context) opContext[opIdx]).accumulate( rs, - inputSourceChunks[srcIdx], + chunkArr, tsChunk, size); } @@ -209,16 +221,6 @@ public UpdateByWindowContext makeWindowContext(final TrackingRowSet sourceRowSet isInitializeStep); } - public void startTrackingModifications(@NotNull final QueryTable source, @NotNull final QueryTable result) { - trackModifications = true; - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - operatorInputModifiedColumnSets[opIdx] = - source.newModifiedColumnSet(operators[opIdx].getAffectingColumnNames()); - operatorOutputModifiedColumnSets[opIdx] = - result.newModifiedColumnSet(operators[opIdx].getOutputColumnNames()); - } - } - /** * Find the smallest valued key that participated in the upstream {@link TableUpdate}. * @@ -293,16 +295,8 @@ private static long smallestAffectedKey(@NotNull final RowSet added, return smallestModifiedKey; } - public UpdateByWindowCumulative(UpdateByOperator[] operators, int[] operatorSourceSlots, + public UpdateByWindowCumulative(UpdateByOperator[] operators, int[][] operatorSourceSlots, @Nullable String timestampColumnName) { super(operators, operatorSourceSlots, timestampColumnName); } - - @Override - public int hashCode() { - return hashCode(false, - timestampColumnName, - 0L, - 0L); - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java index 71cb0aca4c5..1ffc1856392 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java @@ -2,6 +2,7 @@ import io.deephaven.base.ringbuffer.IntRingBuffer; import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.attributes.Values; @@ -79,14 +80,21 @@ protected void makeOperatorContexts() { for (int opIdx = 0; opIdx < operators.length; opIdx++) { if (opAffected[opIdx]) { - // create the fill contexts for the input sources - int sourceSlot = operatorSourceSlots[opIdx]; - if (!inputSourceChunkPopulated[sourceSlot]) { - inputSourceGetContexts[sourceSlot] = - inputSources[sourceSlot].makeGetContext(currentGetContextSize); - inputSourceChunkPopulated[sourceSlot] = true; + if (opAffected[opIdx]) { + // create the fill contexts for the input sources + final int[] sourceIndices = operatorInputSourceSlots[opIdx]; + final ColumnSource[] inputSourceArr = new ColumnSource[sourceIndices.length]; + for (int ii = 0; ii < sourceIndices.length; ii++) { + int sourceSlot = sourceIndices[ii]; + if (!inputSourceChunkPopulated[sourceSlot]) { + inputSourceGetContexts[sourceSlot] = + inputSources[sourceSlot].makeGetContext(currentGetContextSize); + inputSourceChunkPopulated[sourceSlot] = true; + } + inputSourceArr[ii] = inputSources[sourceSlot]; + } + opContext[opIdx] = operators[opIdx].makeUpdateContext(workingChunkSize, inputSourceArr); } - opContext[opIdx] = operators[opIdx].makeUpdateContext(workingChunkSize, inputSources[sourceSlot]); } } } @@ -106,15 +114,17 @@ protected void ensureGetContextSize(long newSize) { for (int opIdx = 0; opIdx < operators.length; opIdx++) { if (opAffected[opIdx]) { - int sourceSlot = operatorSourceSlots[opIdx]; - if (!inputSourceChunkPopulated[sourceSlot]) { - // close the existing context - inputSourceGetContexts[sourceSlot].close(); - - // create a new context of the larger size - inputSourceGetContexts[sourceSlot] = - inputSources[sourceSlot].makeGetContext(currentGetContextSize); - inputSourceChunkPopulated[sourceSlot] = true; + final int[] sourceIndices = operatorInputSourceSlots[opIdx]; + for (int sourceSlot : sourceIndices) { + if (!inputSourceChunkPopulated[sourceSlot]) { + // close the existing context + inputSourceGetContexts[sourceSlot].close(); + + // create a new context of the larger size + inputSourceGetContexts[sourceSlot] = + inputSources[sourceSlot].makeGetContext(currentGetContextSize); + inputSourceChunkPopulated[sourceSlot] = true; + } } } } @@ -339,13 +349,20 @@ public void processRows() { Arrays.fill(inputSourceChunkPopulated, false); for (int opIdx = 0; opIdx < operators.length; opIdx++) { if (opAffected[opIdx]) { - final int srcIdx = operatorSourceSlots[opIdx]; - prepareValuesChunkForSource(srcIdx, chunkInfluencerRs); + // prep the chunk array needed by the accumulate call + final int[] srcIndices = operatorInputSourceSlots[opIdx]; + Chunk[] chunkArr = new Chunk[srcIndices.length]; + for (int ii = 0; ii < srcIndices.length; ii++) { + int srcIdx = srcIndices[ii]; + // chunk prep + prepareValuesChunkForSource(srcIdx, chunkInfluencerRs); + chunkArr[ii] = inputSourceChunks[srcIdx]; + } // make the specialized call for windowed operators ((UpdateByWindowedOperator.Context) opContext[opIdx]).accumulate( chunkRs, - inputSourceChunks[srcIdx], + chunkArr, pushChunk, popChunk, chunkRsSize); @@ -384,16 +401,6 @@ public UpdateByWindowContext makeWindowContext(final TrackingRowSet sourceRowSet isInitializeStep); } - public void startTrackingModifications(@NotNull final QueryTable source, @NotNull final QueryTable result) { - trackModifications = true; - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - operatorInputModifiedColumnSets[opIdx] = - source.newModifiedColumnSet(operators[opIdx].getAffectingColumnNames()); - operatorOutputModifiedColumnSets[opIdx] = - result.newModifiedColumnSet(operators[opIdx].getOutputColumnNames()); - } - } - private static WritableRowSet computeAffectedRowsTicks(final RowSet sourceSet, final RowSet subset, final RowSet invertedSubSet, long revTicks, long fwdTicks) { // swap fwd/rev to get the influencer windows @@ -423,17 +430,9 @@ private static WritableRowSet computeInfluencerRowsTicks(final RowSet sourceSet, } } - UpdateByWindowTicks(UpdateByOperator[] operators, int[] operatorSourceSlots, long prevUnits, long fwdUnits) { + UpdateByWindowTicks(UpdateByOperator[] operators, int[][] operatorSourceSlots, long prevUnits, long fwdUnits) { super(operators, operatorSourceSlots, null); this.prevUnits = prevUnits; this.fwdUnits = fwdUnits; } - - @Override - public int hashCode() { - return hashCode(true, - null, - prevUnits, - fwdUnits); - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java index a9da8a0899d..8591023812b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java @@ -2,6 +2,7 @@ import io.deephaven.base.ringbuffer.LongRingBuffer; import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.attributes.Values; @@ -74,14 +75,18 @@ protected void makeOperatorContexts() { for (int opIdx = 0; opIdx < operators.length; opIdx++) { if (opAffected[opIdx]) { // create the fill contexts for the input sources - int sourceSlot = operatorSourceSlots[opIdx]; - if (!inputSourceChunkPopulated[sourceSlot]) { - // we are going to grab all the influencer rows as one chunk, make sure it's large enough - inputSourceGetContexts[sourceSlot] = - inputSources[sourceSlot].makeGetContext(currentGetContextSize); - inputSourceChunkPopulated[sourceSlot] = true; + final int[] sourceIndices = operatorInputSourceSlots[opIdx]; + final ColumnSource[] inputSourceArr = new ColumnSource[sourceIndices.length]; + for (int ii = 0; ii < sourceIndices.length; ii++) { + int sourceSlot = sourceIndices[ii]; + if (!inputSourceChunkPopulated[sourceSlot]) { + inputSourceGetContexts[sourceSlot] = + inputSources[sourceSlot].makeGetContext(currentGetContextSize); + inputSourceChunkPopulated[sourceSlot] = true; + } + inputSourceArr[ii] = inputSources[sourceSlot]; } - opContext[opIdx] = operators[opIdx].makeUpdateContext(workingChunkSize, inputSources[sourceSlot]); + opContext[opIdx] = operators[opIdx].makeUpdateContext(workingChunkSize, inputSourceArr); } } } @@ -101,15 +106,17 @@ protected void ensureGetContextSize(long newSize) { for (int opIdx = 0; opIdx < operators.length; opIdx++) { if (opAffected[opIdx]) { - int sourceSlot = operatorSourceSlots[opIdx]; - if (!inputSourceChunkPopulated[sourceSlot]) { - // close the existing context - inputSourceGetContexts[sourceSlot].close(); - - // create a new context of the larger size - inputSourceGetContexts[sourceSlot] = - inputSources[sourceSlot].makeGetContext(currentGetContextSize); - inputSourceChunkPopulated[sourceSlot] = true; + final int[] sourceIndices = operatorInputSourceSlots[opIdx]; + for (int sourceSlot : sourceIndices) { + if (!inputSourceChunkPopulated[sourceSlot]) { + // close the existing context + inputSourceGetContexts[sourceSlot].close(); + + // create a new context of the larger size + inputSourceGetContexts[sourceSlot] = + inputSources[sourceSlot].makeGetContext(currentGetContextSize); + inputSourceChunkPopulated[sourceSlot] = true; + } } } } @@ -316,13 +323,20 @@ public void processRows() { Arrays.fill(inputSourceChunkPopulated, false); for (int opIdx = 0; opIdx < operators.length; opIdx++) { if (opAffected[opIdx]) { - final int srcIdx = operatorSourceSlots[opIdx]; - prepareValuesChunkForSource(srcIdx, chunkInfluencerRs); + // prep the chunk array needed by the accumulate call + final int[] srcIndices = operatorInputSourceSlots[opIdx]; + Chunk[] chunkArr = new Chunk[srcIndices.length]; + for (int ii = 0; ii < srcIndices.length; ii++) { + int srcIdx = srcIndices[ii]; + // chunk prep + prepareValuesChunkForSource(srcIdx, chunkInfluencerRs); + chunkArr[ii] = inputSourceChunks[srcIdx]; + } // make the specialized call for windowed operators ((UpdateByWindowedOperator.Context) opContext[opIdx]).accumulate( chunkRs, - inputSourceChunks[srcIdx], + chunkArr, pushChunk, popChunk, chunkRsSize); @@ -426,18 +440,10 @@ private static WritableRowSet computeInfluencerRowsTime(final RowSet sourceSet, } } - UpdateByWindowTime(UpdateByOperator[] operators, int[] operatorSourceSlots, @Nullable String timestampColumnName, + UpdateByWindowTime(UpdateByOperator[] operators, int[][] operatorSourceSlots, @Nullable String timestampColumnName, long prevUnits, long fwdUnits) { super(operators, operatorSourceSlots, timestampColumnName); this.prevUnits = prevUnits; this.fwdUnits = fwdUnits; } - - @Override - public int hashCode() { - return hashCode(true, - timestampColumnName, - prevUnits, - fwdUnits); - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java index 3f67a141e25..8549c65a2d8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java @@ -17,16 +17,16 @@ public class BigDecimalEMAOperator extends BigNumberEMAOperator { public class Context extends BigNumberEMAOperator.Context { - protected Context(int chunkSize, ColumnSource inputSource) { - super(chunkSize, inputSource); + protected Context(int chunkSize, ColumnSource[] inputSourceArr) { + super(chunkSize, inputSourceArr); } @Override public void accumulate(RowSequence inputKeys, - Chunk valueChunk, + Chunk[] valueChunkArr, LongChunk tsChunk, int len) { - setValuesChunk(valueChunk); + setValuesChunk(valueChunkArr[0]); setTimestampChunk(tsChunk); // chunk processing @@ -116,7 +116,7 @@ public BigDecimalEMAOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, ColumnSource inputSource) { - return new Context(chunkSize, inputSource); + public UpdateContext makeUpdateContext(final int chunkSize, ColumnSource[] inputSourceArr) { + return new Context(chunkSize, inputSourceArr); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java index 0d79bb6fca6..cb0d687eab8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java @@ -18,16 +18,16 @@ public class BigIntegerEMAOperator extends BigNumberEMAOperator { public class Context extends BigNumberEMAOperator.Context { - protected Context(int chunkSize, ColumnSource inputSource) { - super(chunkSize, inputSource); + protected Context(int chunkSize, ColumnSource[] inputSourceArr) { + super(chunkSize, inputSourceArr); } @Override public void accumulate(RowSequence inputKeys, - Chunk valueChunk, + Chunk[] valueChunkArr, LongChunk tsChunk, int len) { - setValuesChunk(valueChunk); + setValuesChunk(valueChunkArr[0]); setTimestampChunk(tsChunk); // chunk processing @@ -119,7 +119,7 @@ public BigIntegerEMAOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, ColumnSource inputSource) { - return new Context(chunkSize, inputSource); + public UpdateContext makeUpdateContext(final int chunkSize, ColumnSource[] inputSourceArr) { + return new Context(chunkSize, inputSourceArr); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java index 63d1efecc1d..ab366c05a85 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java @@ -33,13 +33,11 @@ public abstract class Context extends BaseObjectUpdateByOperator.Con long lastStamp = NULL_LONG; - protected Context(int chunkSize, ColumnSource inputSource) { + protected Context(int chunkSize, ColumnSource[] inputSourceArr) { super(chunkSize); - this.valueSource = inputSource; + this.valueSource = inputSourceArr[0]; } - - @Override public void setValuesChunk(@NotNull final Chunk valuesChunk) { objectValueChunk = valuesChunk.asObjectChunk(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java index 081e818c9e9..5df8c92ee1f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java @@ -26,17 +26,17 @@ protected class Context extends BasePrimitiveEMAOperator.Context { public ByteChunk byteValueChunk; - protected Context(int chunkSize, ColumnSource inputSource) { + protected Context(int chunkSize, ColumnSource inputSourceArr[]) { super(chunkSize); - this.valueSource = inputSource; + this.valueSource = inputSourceArr[0]; } @Override public void accumulate(RowSequence inputKeys, - Chunk valueChunk, + Chunk[] valueChunkArr, LongChunk tsChunk, int len) { - setValuesChunk(valueChunk); + setValuesChunk(valueChunkArr[0]); setTimestampChunk(tsChunk); // chunk processing @@ -133,7 +133,7 @@ public ByteEMAOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { - return new Context(chunkSize, inputSource); + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + return new Context(chunkSize, inputSourceArr); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java index 8534e4585f2..df5bc51dac0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java @@ -27,17 +27,17 @@ protected class Context extends BasePrimitiveEMAOperator.Context { public DoubleChunk doubleValueChunk; - protected Context(int chunkSize, ColumnSource inputSource) { + protected Context(int chunkSize, ColumnSource[] inputSourceArr) { super(chunkSize); - this.valueSource = inputSource; + this.valueSource = inputSourceArr[0]; } @Override public void accumulate(RowSequence inputKeys, - Chunk valueChunk, + Chunk[] valueChunkArr, LongChunk tsChunk, int len) { - setValuesChunk(valueChunk); + setValuesChunk(valueChunkArr[0]); setTimestampChunk(tsChunk); // chunk processing @@ -150,7 +150,7 @@ public DoubleEMAOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { - return new Context(chunkSize, inputSource); + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + return new Context(chunkSize, inputSourceArr); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java index 24646b44aa6..6c45ab51e05 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java @@ -22,17 +22,17 @@ protected class Context extends BasePrimitiveEMAOperator.Context { public FloatChunk floatValueChunk; - protected Context(int chunkSize, ColumnSource inputSource) { + protected Context(int chunkSize, ColumnSource[] inputSourceArr) { super(chunkSize); - this.valueSource = inputSource; + this.valueSource = inputSourceArr[0]; } @Override public void accumulate(RowSequence inputKeys, - Chunk valueChunk, + Chunk[] valueChunkArr, LongChunk tsChunk, int len) { - setValuesChunk(valueChunk); + setValuesChunk(valueChunkArr[0]); setTimestampChunk(tsChunk); // chunk processing @@ -145,7 +145,7 @@ public FloatEMAOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { - return new Context(chunkSize, inputSource); + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + return new Context(chunkSize, inputSourceArr); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java index 4ca064e0abb..439dee4904b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java @@ -26,17 +26,17 @@ protected class Context extends BasePrimitiveEMAOperator.Context { public IntChunk intValueChunk; - protected Context(int chunkSize, ColumnSource inputSource) { + protected Context(int chunkSize, ColumnSource inputSourceArr[]) { super(chunkSize); - this.valueSource = inputSource; + this.valueSource = inputSourceArr[0]; } @Override public void accumulate(RowSequence inputKeys, - Chunk valueChunk, + Chunk[] valueChunkArr, LongChunk tsChunk, int len) { - setValuesChunk(valueChunk); + setValuesChunk(valueChunkArr[0]); setTimestampChunk(tsChunk); // chunk processing @@ -133,7 +133,7 @@ public IntEMAOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { - return new Context(chunkSize, inputSource); + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + return new Context(chunkSize, inputSourceArr); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java index 291d9d455fc..738d2293edf 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java @@ -26,17 +26,17 @@ protected class Context extends BasePrimitiveEMAOperator.Context { public LongChunk longValueChunk; - protected Context(int chunkSize, ColumnSource inputSource) { + protected Context(int chunkSize, ColumnSource inputSourceArr[]) { super(chunkSize); - this.valueSource = inputSource; + this.valueSource = inputSourceArr[0]; } @Override public void accumulate(RowSequence inputKeys, - Chunk valueChunk, + Chunk[] valueChunkArr, LongChunk tsChunk, int len) { - setValuesChunk(valueChunk); + setValuesChunk(valueChunkArr[0]); setTimestampChunk(tsChunk); // chunk processing @@ -133,7 +133,7 @@ public LongEMAOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { - return new Context(chunkSize, inputSource); + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + return new Context(chunkSize, inputSourceArr); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java index 288b097e1a8..aec0ad682f5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java @@ -21,17 +21,17 @@ protected class Context extends BasePrimitiveEMAOperator.Context { public ShortChunk shortValueChunk; - protected Context(int chunkSize, ColumnSource inputSource) { + protected Context(int chunkSize, ColumnSource inputSourceArr[]) { super(chunkSize); - this.valueSource = inputSource; + this.valueSource = inputSourceArr[0]; } @Override public void accumulate(RowSequence inputKeys, - Chunk valueChunk, + Chunk[] valueChunkArr, LongChunk tsChunk, int len) { - setValuesChunk(valueChunk); + setValuesChunk(valueChunkArr[0]); setTimestampChunk(tsChunk); // chunk processing @@ -128,7 +128,7 @@ public ShortEMAOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { - return new Context(chunkSize, inputSource); + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + return new Context(chunkSize, inputSourceArr); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java index c86da007d5f..503495a4c9b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java @@ -65,7 +65,7 @@ public BooleanFillByOperator(@NotNull final MatchPair fillPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java index ec2710e0375..15a6d327f5e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java @@ -58,7 +58,7 @@ public ByteFillByOperator(@NotNull final MatchPair fillPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java index 5a8dafcb5e7..464635953dc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java @@ -53,7 +53,7 @@ public CharFillByOperator(@NotNull final MatchPair fillPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java index 5835131f08d..2d5f9f2bbbf 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java @@ -58,7 +58,7 @@ public DoubleFillByOperator(@NotNull final MatchPair fillPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java index 7b6a449d58a..b9997b58a94 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java @@ -58,7 +58,7 @@ public FloatFillByOperator(@NotNull final MatchPair fillPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java index 7a98e68a3cd..3357e446667 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java @@ -58,7 +58,7 @@ public IntFillByOperator(@NotNull final MatchPair fillPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java index 33144692320..81aef44e612 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java @@ -68,7 +68,7 @@ public LongFillByOperator(@NotNull final MatchPair fillPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java index 41a50ec9e42..f9bc4b13366 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java @@ -58,7 +58,7 @@ public ObjectFillByOperator(@NotNull final MatchPair fillPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java index 9f6e57a2187..53c2a1d86c1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java @@ -58,7 +58,7 @@ public ShortFillByOperator(@NotNull final MatchPair fillPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java index 2bfdcc84e00..2a175012052 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java @@ -54,11 +54,11 @@ protected Context(final int chunkSize) { @Override public void accumulate(RowSequence inputKeys, - Chunk valueChunk, + Chunk[] valueChunkArr, LongChunk tsChunk, int len) { - setValuesChunk(valueChunk); + setValuesChunk(valueChunkArr[0]); setTimestampChunk(tsChunk); // chunk processing diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java index 3cf56dc925c..a8160575f78 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java @@ -43,11 +43,11 @@ protected Context(final int chunkSize) { @Override public void accumulate(RowSequence inputKeys, - Chunk valueChunk, + Chunk[] valueChunkArr, LongChunk tsChunk, int len) { - setValuesChunk(valueChunk); + setValuesChunk(valueChunkArr[0]); setTimestampChunk(tsChunk); // chunk processing diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java index 6f0cd5cb990..ce8c02175c4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java @@ -48,11 +48,11 @@ protected Context(final int chunkSize) { @Override public void accumulate(RowSequence inputKeys, - Chunk valueChunk, + Chunk valueChunkArr[], LongChunk tsChunk, int len) { - setValuesChunk(valueChunk); + setValuesChunk(valueChunkArr[0]); setTimestampChunk(tsChunk); // chunk processing diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java index f656cfc046b..d4e60cdad4a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java @@ -43,11 +43,11 @@ protected Context(final int chunkSize) { @Override public void accumulate(RowSequence inputKeys, - Chunk valueChunk, + Chunk valueChunkArr[], LongChunk tsChunk, int len) { - setValuesChunk(valueChunk); + setValuesChunk(valueChunkArr[0]); setTimestampChunk(tsChunk); // chunk processing diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java index ecfa73a9037..0cf0b7f84f7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java @@ -48,11 +48,11 @@ protected Context(final int chunkSize) { @Override public void accumulate(RowSequence inputKeys, - Chunk valueChunk, + Chunk[] valueChunkArr, LongChunk tsChunk, int len) { - setValuesChunk(valueChunk); + setValuesChunk(valueChunkArr[0]); setTimestampChunk(tsChunk); // chunk processing diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java index ffff7b03b29..4f3137b4934 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java @@ -48,11 +48,11 @@ protected Context(final int chunkSize) { @Override public void accumulate(RowSequence inputKeys, - Chunk valueChunk, + Chunk[] valueChunkArr, LongChunk tsChunk, int len) { - setValuesChunk(valueChunk); + setValuesChunk(valueChunkArr[0]); setTimestampChunk(tsChunk); // chunk processing diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java index ac3fe9d5ed6..b583ca2017f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java @@ -56,7 +56,7 @@ public BaseObjectBinaryOperator(@NotNull final Class type, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java index a937d0dd446..eda8d882975 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java @@ -49,11 +49,11 @@ protected Context(final int chunkSize) { @Override public void accumulate(RowSequence inputKeys, - Chunk valueChunk, + Chunk[] valueChunkArr, LongChunk tsChunk, int len) { - setValuesChunk(valueChunk); + setValuesChunk(valueChunkArr[0]); setTimestampChunk(tsChunk); // chunk processing diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java index efcaf0f4fe6..c5249cf5996 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java @@ -48,11 +48,11 @@ protected Context(final int chunkSize) { @Override public void accumulate(RowSequence inputKeys, - Chunk valueChunk, + Chunk[] valueChunkArr, LongChunk tsChunk, int len) { - setValuesChunk(valueChunk); + setValuesChunk(valueChunkArr[0]); setTimestampChunk(tsChunk); // chunk processing diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java index f7f70930197..0ba15f36b63 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java @@ -53,12 +53,12 @@ protected Context(final int chunkSize) { @Override public void accumulate(RowSequence inputKeys, - Chunk influencerValueChunk, + Chunk[] influencerValueChunkArr, IntChunk pushChunk, IntChunk popChunk, int len) { - setValuesChunk(influencerValueChunk); + setValuesChunk(influencerValueChunkArr[0]); int pushIndex = 0; // chunk processing diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java index eba5ec7ca1b..f40aca44d72 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java @@ -42,12 +42,12 @@ protected Context(final int chunkSize) { @Override public void accumulate(RowSequence inputKeys, - Chunk influencerValueChunk, + Chunk[] influencerValueChunkArr, IntChunk pushChunk, IntChunk popChunk, int len) { - setValuesChunk(influencerValueChunk); + setValuesChunk(influencerValueChunkArr[0]); int pushIndex = 0; // chunk processing diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java index ee406b77b8c..ddf9d5ac5cc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java @@ -43,12 +43,12 @@ protected Context(final int chunkSize) { @Override public void accumulate(RowSequence inputKeys, - Chunk influencerValueChunk, + Chunk influencerValueChunkArr[], IntChunk pushChunk, IntChunk popChunk, int len) { - setValuesChunk(influencerValueChunk); + setValuesChunk(influencerValueChunkArr[0]); int pushIndex = 0; // chunk processing diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java index cd69b57e48e..109f3243aef 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java @@ -38,12 +38,12 @@ protected Context(final int chunkSize) { @Override public void accumulate(RowSequence inputKeys, - Chunk influencerValueChunk, + Chunk influencerValueChunkArr[], IntChunk pushChunk, IntChunk popChunk, int len) { - setValuesChunk(influencerValueChunk); + setValuesChunk(influencerValueChunkArr[0]); int pushIndex = 0; // chunk processing diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java index 9e77ab4e060..b5078b1c6e6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java @@ -47,12 +47,12 @@ protected Context(final int chunkSize) { @Override public void accumulate(RowSequence inputKeys, - Chunk influencerValueChunk, + Chunk[] influencerValueChunkArr, IntChunk pushChunk, IntChunk popChunk, int len) { - setValuesChunk(influencerValueChunk); + setValuesChunk(influencerValueChunkArr[0]); int pushIndex = 0; // chunk processing diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java index 1872ccb5a4e..2da5ea9625e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java @@ -47,12 +47,12 @@ protected Context(final int chunkSize) { @Override public void accumulate(RowSequence inputKeys, - Chunk influencerValueChunk, + Chunk[] influencerValueChunkArr, IntChunk pushChunk, IntChunk popChunk, int len) { - setValuesChunk(influencerValueChunk); + setValuesChunk(influencerValueChunkArr[0]); int pushIndex = 0; // chunk processing diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java index 6dd4e96ca32..f90cedbb6d6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java @@ -48,12 +48,12 @@ protected Context(final int chunkSize) { @Override public void accumulate(RowSequence inputKeys, - Chunk influencerValueChunk, + Chunk[] influencerValueChunkArr, IntChunk pushChunk, IntChunk popChunk, int len) { - setValuesChunk(influencerValueChunk); + setValuesChunk(influencerValueChunkArr[0]); int pushIndex = 0; // chunk processing diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java index 53ecca43c05..9ec0db50578 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java @@ -47,12 +47,12 @@ protected Context(final int chunkSize) { @Override public void accumulate(RowSequence inputKeys, - Chunk influencerValueChunk, + Chunk[] influencerValueChunkArr, IntChunk pushChunk, IntChunk popChunk, int len) { - setValuesChunk(influencerValueChunk); + setValuesChunk(influencerValueChunkArr[0]); int pushIndex = 0; // chunk processing diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java index 57bafea113f..9617e72a141 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java @@ -69,7 +69,7 @@ public ByteCumMinMaxOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java index 77987c81f33..0d6094fbd7c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java @@ -69,7 +69,7 @@ public DoubleCumMinMaxOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java index 099f80b2218..5c734ff4587 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java @@ -64,7 +64,7 @@ public FloatCumMinMaxOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java index 42ef98e1e94..90bb79faa3f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java @@ -69,7 +69,7 @@ public IntCumMinMaxOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java index f3a6e178873..238ed7f0729 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java @@ -79,7 +79,7 @@ public LongCumMinMaxOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java index 51bf5a95952..702435af3a4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java @@ -64,7 +64,7 @@ public ShortCumMinMaxOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java index aad5262ffa6..eb68dcda425 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java @@ -60,7 +60,7 @@ public BigDecimalCumProdOperator(@NotNull final MatchPair inputPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java index f168b360ad3..5fff158db4b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java @@ -54,7 +54,7 @@ public BigIntegerCumProdOperator(@NotNull final MatchPair inputPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java index cfe11d83891..e6ed53d76a6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java @@ -62,7 +62,7 @@ public ByteCumProdOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java index 1e36c21be4f..c7d3c118c41 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java @@ -63,7 +63,7 @@ public DoubleCumProdOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java index 24ce0d12b44..ea9f07220c9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java @@ -58,7 +58,7 @@ public FloatCumProdOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java index 51f8b5a9000..a7e7caefe59 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java @@ -62,7 +62,7 @@ public IntCumProdOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java index 88d010f6600..9cc48fbe6ef 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java @@ -62,7 +62,7 @@ public LongCumProdOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java index 93e0f448491..ce3154f1062 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java @@ -57,7 +57,7 @@ public ShortCumProdOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java index 92b79230efc..8f75aaeaab8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java @@ -81,7 +81,7 @@ public void writeToOutputChunk(int outIdx) { @NotNull @Override - public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize, ColumnSource inputSource) { + public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java index 01cd2c6d163..c754e9c5d97 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java @@ -78,7 +78,7 @@ public void writeToOutputChunk(int outIdx) { @NotNull @Override - public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize, ColumnSource inputSource) { + public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java index b2ffb04df4f..d4cb7ceac5e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java @@ -87,7 +87,7 @@ public void writeToOutputChunk(int outIdx) { @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(final int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java index 65652154a5a..058f2d5c354 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java @@ -85,7 +85,7 @@ public void writeToOutputChunk(int outIdx) { @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(final int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java index a321c099168..f7ecfe15e94 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java @@ -80,7 +80,7 @@ public void writeToOutputChunk(int outIdx) { @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(final int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java index 540068537fc..a80aa7004d7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java @@ -86,7 +86,7 @@ public void writeToOutputChunk(int outIdx) { @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(final int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java index f6bf5e147b3..9a32bc743e2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java @@ -86,7 +86,7 @@ public void writeToOutputChunk(int outIdx) { @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(final int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java index 50b1d35410b..2a17ca9460b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java @@ -81,7 +81,7 @@ public void writeToOutputChunk(int outIdx) { @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(final int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java index 579b01a4a07..ddf5b39a206 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java @@ -61,7 +61,7 @@ public BigDecimalCumSumOperator(@NotNull final MatchPair inputPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java index 2ed92475971..35982a5cc13 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java @@ -54,7 +54,7 @@ public BigIntegerCumSumOperator(@NotNull final MatchPair inputPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java index f815f252f6c..e50c9acf804 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java @@ -64,7 +64,7 @@ public ByteCumSumOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java index 6b57eb6abf0..965e88f6d88 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java @@ -62,7 +62,7 @@ public DoubleCumSumOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java index ba15dedde91..32e41f00e9c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java @@ -57,7 +57,7 @@ public FloatCumSumOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java index 4c666aa0c2f..70527d6d5dd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java @@ -61,7 +61,7 @@ public IntCumSumOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java index bad965e6ae2..157d815150c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java @@ -61,7 +61,7 @@ public LongCumSumOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java index 708ddbedb81..8fa409c1b35 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java @@ -56,7 +56,7 @@ public ShortCumSumOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource inputSource) { + public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { return new Context(chunkSize); } } From f09651f8894b556f1ef5153a0677a00aac19e5d4 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 25 Oct 2022 16:32:59 -0700 Subject: [PATCH 033/123] WIP commit, tests passing again --- .../impl/BucketedPartitionedUpdateBy.java | 223 ------- .../BucketedPartitionedUpdateByManager.java | 161 +++++ .../deephaven/engine/table/impl/UpdateBy.java | 251 +++++++- .../table/impl/UpdateByBucketHelper.java | 310 +++++++++ .../impl/UpdateByCumulativeOperator.java | 53 +- .../engine/table/impl/UpdateByOperator.java | 124 +++- .../table/impl/UpdateByOperatorFactory.java | 20 +- .../table/impl/UpdateByWindowedOperator.java | 57 +- .../engine/table/impl/ZeroKeyUpdateBy.java | 439 ------------- .../table/impl/ZeroKeyUpdateByManager.java | 260 ++++++++ .../table/impl/updateby/UpdateByWindow.java | 145 +++-- .../updateby/UpdateByWindowCumulative.java | 312 +++++---- .../impl/updateby/UpdateByWindowTicks.java | 587 +++++++++-------- .../impl/updateby/UpdateByWindowTime.java | 597 +++++++++--------- 14 files changed, 1872 insertions(+), 1667 deletions(-) delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateByManager.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java deleted file mode 100644 index c5cc64ecddd..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateBy.java +++ /dev/null @@ -1,223 +0,0 @@ -package io.deephaven.engine.table.impl; - -import io.deephaven.api.ColumnName; -import io.deephaven.api.updateby.UpdateByControl; -import io.deephaven.engine.rowset.*; -import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.sources.UnionSourceManager; -import io.deephaven.engine.table.impl.updateby.UpdateByWindow; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; -import io.deephaven.util.datastructures.linked.IntrusiveDoublyLinkedNode; -import org.apache.commons.lang3.mutable.MutableLong; -import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; - -import java.util.*; - -import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; - -/** - * An implementation of {@link UpdateBy} dedicated to bucketed computation. - */ -class BucketedPartitionedUpdateBy extends UpdateBy { - private final BucketedPartitionedUpdateByListener listener; - private final LinkedList recorders; - private final QueryTable resultTable; - - /** - * Perform a bucketed updateBy using {@code byColumns} as the keys - * - * @param description the operation description - * @param source the source table - * @param ops the operations to perform - * @param resultSources the result sources - * @param byColumns the columns to use for the bucket keys - * @param redirContext the row redirection shared context - * @param control the control object. - * @return the result table - */ - public static Table compute(@NotNull final String description, - @NotNull final QueryTable source, - @NotNull final UpdateByOperator[] ops, - @NotNull final UpdateByWindow[] windows, - @NotNull final ColumnSource[] inputSources, - @NotNull final int[][] operatorInputSourceSlots, - @NotNull final Map> resultSources, - @NotNull final Collection byColumns, - @NotNull final UpdateByRedirectionContext redirContext, - @NotNull final UpdateByControl control) { - - final BucketedPartitionedUpdateBy updateBy = new BucketedPartitionedUpdateBy(description, - ops, - windows, - inputSources, - operatorInputSourceSlots, - source, - resultSources, - byColumns, - redirContext, - control); - - return updateBy.resultTable; - } - - protected BucketedPartitionedUpdateBy(@NotNull final String description, - @NotNull final UpdateByOperator[] operators, - @NotNull final UpdateByWindow[] windows, - @NotNull final ColumnSource[] inputSources, - @NotNull final int[][] operatorInputSourceSlots, - @NotNull final QueryTable source, - @NotNull final Map> resultSources, - @NotNull final Collection byColumns, - @NotNull final UpdateByRedirectionContext redirContext, - @NotNull final UpdateByControl control) { - super(operators, windows, inputSources, operatorInputSourceSlots, source, redirContext, control); - - // create a source-listener that will listen to the source updates and apply the shifts to the output columns - final QueryTable sourceListenerTable = new QueryTable(source.getRowSet(), source.getColumnSourceMap()); - - // this table will always have the rowset of the source - resultTable = new QueryTable(source.getRowSet(), resultSources); - - if (source.isRefreshing()) { - source.listenForUpdates(new BaseTable.ListenerImpl("", source, sourceListenerTable) { - @Override - public void onUpdate(@NotNull final TableUpdate upstream) { - if (redirContext.isRedirected()) { - redirContext.processUpdateForRedirection(upstream, source.getRowSet()); - } else if (upstream.shifted().nonempty()) { - try (final RowSet prevIdx = source.getRowSet().copyPrev()) { - upstream.shifted().apply((begin, end, delta) -> { - try (final RowSet subRowSet = prevIdx.subSetByKeyRange(begin, end)) { - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - operators[opIdx].applyOutputShift(subRowSet, delta); - } - } - }); - } - } - super.onUpdate(upstream); - } - }); - - recorders = new LinkedList<>(); - listener = newListener(description); - - // create a listener and recorder for the source table as first entry - BucketedPartitionedUpdateByListenerRecorder recorder = - new BucketedPartitionedUpdateByListenerRecorder(description, sourceListenerTable, resultTable); - recorder.setMergedListener(listener); - sourceListenerTable.listenForUpdates(recorder); - - recorders.offerLast(recorder); - } else { - listener = null; - recorders = null; - } - - if (redirContext.isRedirected()) { - // make a dummy update to generate the initial row keys - final TableUpdateImpl fakeUpdate = new TableUpdateImpl(source.getRowSet(), - RowSetFactory.empty(), - RowSetFactory.empty(), - RowSetShiftData.EMPTY, - ModifiedColumnSet.EMPTY); - redirContext.processUpdateForRedirection(fakeUpdate, source.getRowSet()); - } - - final PartitionedTable pt = sourceListenerTable.partitionedAggBy(List.of(), true, null, byColumns); - final PartitionedTable transformed = pt.transform(t -> { - // create the table - Table newTable = ZeroKeyUpdateBy.compute( - description, - (QueryTable) t, - operators, - windows, - inputSources, - operatorInputSourceSlots, - resultSources, - redirContext, - control, - false); - - if (listener != null) { - BucketedPartitionedUpdateByListenerRecorder recorder = - new BucketedPartitionedUpdateByListenerRecorder(description, newTable, resultTable); - recorder.setMergedListener(listener); - newTable.listenForUpdates(recorder); - - // add the listener only while synchronized - synchronized (recorders) { - recorders.offerLast(recorder); - } - } - - // return the table - return newTable; - }); - resultTable.addParentReference(transformed); - } - - BucketedPartitionedUpdateByListener newListener(@NotNull final String description) { - return new BucketedPartitionedUpdateByListener(description); - } - - private final class BucketedPartitionedUpdateByListenerRecorder extends ListenerRecorder { - - private final ModifiedColumnSet.Transformer modifiedColumnsTransformer; - - BucketedPartitionedUpdateByListenerRecorder(@NotNull String description, @NotNull final Table constituent, - @NotNull final Table dependent) { - super(description, constituent, dependent); - modifiedColumnsTransformer = ((QueryTable) constituent).newModifiedColumnSetTransformer( - (QueryTable) dependent, constituent.getDefinition().getColumnNamesArray()); - } - } - - /** - * The Listener for apply to the constituent table updates - */ - class BucketedPartitionedUpdateByListener extends MergedListener { - public BucketedPartitionedUpdateByListener(@Nullable String description) { - super(recorders, List.of(), description, resultTable); - } - - @Override - protected void process() { - final TableUpdateImpl downstream = new TableUpdateImpl(); - - // get the adds/removes/shifts from the first (source) entry, make a copy since TableUpdateImpl#reset will - // close them with the upstream update - ListenerRecorder sourceRecorder = recorders.peekFirst(); - downstream.added = sourceRecorder.getAdded().copy(); - downstream.removed = sourceRecorder.getRemoved().copy(); - downstream.shifted = sourceRecorder.getShifted(); - - // union the modifies from all the tables (including source) - downstream.modifiedColumnSet = resultTable.getModifiedColumnSetForUpdates(); - downstream.modifiedColumnSet.clear(); - - WritableRowSet modifiedRowSet = RowSetFactory.empty(); - downstream.modified = modifiedRowSet; - - recorders.forEach(lr -> { - if (lr.getModified().isNonempty()) { - modifiedRowSet.insert(lr.getModified()); - } - // always transform, ZeroKey listener sets this independently of the modified rowset - lr.modifiedColumnsTransformer.transform(lr.getModifiedColumnSet(), downstream.modifiedColumnSet); - }); - // should not include actual adds as modifies - modifiedRowSet.remove(downstream.added); - - result.notifyListeners(downstream); - } - - @Override - protected boolean canExecute(final long step) { - synchronized (recorders) { - return recorders.stream().allMatch(lr -> lr.satisfied(step)); - } - } - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java new file mode 100644 index 00000000000..f8277170efc --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java @@ -0,0 +1,161 @@ +package io.deephaven.engine.table.impl; + +import io.deephaven.api.ColumnName; +import io.deephaven.api.updateby.UpdateByControl; +import io.deephaven.engine.rowset.*; +import io.deephaven.engine.table.*; +import io.deephaven.engine.table.impl.updateby.UpdateByWindow; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.util.*; + +/** + * An implementation of {@link UpdateBy} dedicated to bucketed computation. + */ +class BucketedPartitionedUpdateByManager extends UpdateBy { + /** + * Perform a bucketed updateBy using {@code byColumns} as the keys + * + * @param description the operation description + * @param source the source table + * @param ops the operations to perform + * @param resultSources the result sources + * @param byColumns the columns to use for the bucket keys + * @param redirContext the row redirection shared context + * @param control the control object. + * @return the result table + */ + public static Table compute(@NotNull final String description, + @NotNull final QueryTable source, + @NotNull final UpdateByOperator[] ops, + @NotNull final UpdateByWindow[] windows, + @NotNull final ColumnSource[] inputSources, + @NotNull final int[][] operatorInputSourceSlots, + @NotNull final Map> resultSources, + @NotNull final Collection byColumns, + @Nullable final String timestampColumnName, + @NotNull final UpdateByRedirectionContext redirContext, + @NotNull final UpdateByControl control) { + + final BucketedPartitionedUpdateByManager updateBy = new BucketedPartitionedUpdateByManager(description, + ops, + windows, + inputSources, + operatorInputSourceSlots, + source, + resultSources, + byColumns, + timestampColumnName, + redirContext, + control); + + return updateBy.result; + } + + protected BucketedPartitionedUpdateByManager(@NotNull final String description, + @NotNull final UpdateByOperator[] operators, + @NotNull final UpdateByWindow[] windows, + @NotNull final ColumnSource[] inputSources, + @NotNull final int[][] operatorInputSourceSlots, + @NotNull final QueryTable source, + @NotNull final Map> resultSources, + @NotNull final Collection byColumns, + @Nullable final String timestampColumnName, + @NotNull final UpdateByRedirectionContext redirContext, + @NotNull final UpdateByControl control) { + super(description, source, operators, windows, inputSources, operatorInputSourceSlots, resultSources, + timestampColumnName, redirContext, control); + + // this table will always have the rowset of the source + result = new QueryTable(source.getRowSet(), resultSources); + + final PartitionedTable pt; + if (source.isRefreshing()) { + // this is a refreshing source, we will need a listener and recorders + recorders = new LinkedList<>(); + listener = newListener(description); + + // create an intermediate table that will listen to source updates and shift output columns + final QueryTable shiftApplyTable = new QueryTable(source.getRowSet(), source.getColumnSourceMap()); + + source.listenForUpdates(new BaseTable.ListenerImpl("", source, shiftApplyTable) { + @Override + public void onUpdate(@NotNull final TableUpdate upstream) { + shiftOutputColumns(upstream); + super.onUpdate(upstream); + } + }); + + // create a recorder instance sourced from the shifting table + ListenerRecorder shiftRecorder = new ListenerRecorder(description, shiftApplyTable, result); + shiftRecorder.setMergedListener(listener); + shiftApplyTable.listenForUpdates(shiftRecorder); + result.addParentReference(listener); + recorders.offerLast(shiftRecorder); + + // create input and output modified column sets + for (UpdateByOperator op : operators) { + op.createInputModifiedColumnSet(shiftApplyTable); + op.createOutputModifiedColumnSet(result); + } + pt = shiftApplyTable.partitionedAggBy(List.of(), true, null, byColumns); + } else { + // no shifting will be needed, can create directly from source + pt = source.partitionedAggBy(List.of(), true, null, byColumns); + + // create input modified column sets only + for (UpdateByOperator op : operators) { + op.createInputModifiedColumnSet(source); + } + } + + if (redirContext.isRedirected()) { + // make a dummy update to generate the initial row keys + final TableUpdateImpl fakeUpdate = new TableUpdateImpl(source.getRowSet(), + RowSetFactory.empty(), + RowSetFactory.empty(), + RowSetShiftData.EMPTY, + ModifiedColumnSet.EMPTY); + redirContext.processUpdateForRedirection(fakeUpdate, source.getRowSet()); + } + + final PartitionedTable transformed = pt.transform(t -> { + UpdateByBucketHelper updateBy = new UpdateByBucketHelper( + description, + (QueryTable) t, + operators, + windows, + inputSources, + operatorInputSourceSlots, + resultSources, + timestampColumnName, + redirContext, + control); + + if (listener != null) { + ListenerRecorder recorder = new ListenerRecorder(description, updateBy.result, result); + recorder.setMergedListener(listener); + updateBy.result.listenForUpdates(recorder); + + // add the listener only while synchronized + synchronized (recorders) { + recorders.offerLast(recorder); + } + } + // add this to the bucket list + synchronized (buckets) { + buckets.offerLast(updateBy); + } + // return the table + return updateBy.result; + }); + + result.addParentReference(transformed); + + // do the actual computations + UpdateByBucketHelper[] dirtyBuckets = buckets.toArray(UpdateByBucketHelper[]::new); + processBuckets(dirtyBuckets, true); + finalizeBuckets(dirtyBuckets); + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java index 46f64c7014c..967c57c3f7b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java @@ -10,6 +10,7 @@ import io.deephaven.engine.exceptions.UncheckedTableException; import io.deephaven.engine.rowset.*; import io.deephaven.engine.table.*; +import io.deephaven.engine.table.impl.sources.FillUnordered; import io.deephaven.engine.table.impl.sources.LongSparseArraySource; import io.deephaven.engine.table.impl.sources.ReinterpretUtils; import io.deephaven.engine.table.impl.sources.sparse.SparseConstants; @@ -17,13 +18,14 @@ import io.deephaven.engine.table.impl.util.InverseRowRedirectionImpl; import io.deephaven.engine.table.impl.util.LongColumnSourceWritableRowRedirection; import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.util.SafeCloseable; import org.apache.commons.lang3.mutable.MutableInt; import org.apache.commons.lang3.mutable.MutableLong; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import java.util.*; -import java.util.stream.Collectors; +import java.util.concurrent.atomic.AtomicInteger; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; @@ -37,10 +39,56 @@ public abstract class UpdateBy { protected final UpdateByOperator[] operators; protected final UpdateByWindow[] windows; protected final QueryTable source; - protected final UpdateByRedirectionContext redirContext; - protected final UpdateByControl control; + protected final String timestampColumnName; + + protected final LinkedList buckets; + + // column-caching management + + /** Whether caching benefits this UpdateBy operation */ + protected final boolean inputCacheNeeded; + /** Whether caching benefits this input source */ + protected final boolean[] inputSourceCacheNeeded; + /** The cached input sources for this update */ + protected final CachedInputSource[] cachedInputSources; + + /** The output table for this UpdateBy operation */ + protected QueryTable result; + protected LinkedList recorders; + protected UpdateByListener listener; + + protected static class CachedInputSource implements SafeCloseable { + public boolean isPopulated; + public final AtomicInteger referenceCount; + public WritableRowSet rowSet; + public ColumnSource[] source; + + public CachedInputSource() { + this.referenceCount = new AtomicInteger(1); + isPopulated = false; + } + + public void addReference() { + referenceCount.incrementAndGet(); + } + + public void addRowSet(final RowSet inputRowSet) { + if (rowSet == null) { + rowSet = inputRowSet.copy(); + } else { + // testing shows this union-ing is very efficient + rowSet.insert(inputRowSet); + } + } + + @Override + public void close() { + try (final RowSet ignored = rowSet) { + } + } + } public static class UpdateByRedirectionContext implements Context { @Nullable @@ -128,15 +176,16 @@ public void close() { } } - protected UpdateBy(@NotNull final UpdateByOperator[] operators, + protected UpdateBy(@NotNull final String description, + @NotNull final QueryTable source, + @NotNull final UpdateByOperator[] operators, @NotNull final UpdateByWindow[] windows, @NotNull final ColumnSource[] inputSources, @NotNull final int[][] operatorInputSourceSlots, - @NotNull final QueryTable source, + @NotNull final Map> resultSources, + @Nullable String timestampColumnName, @NotNull final UpdateByRedirectionContext redirContext, - UpdateByControl control) { - this.control = control; - this.redirContext = redirContext; + @NotNull final UpdateByControl control) { if (operators.length == 0) { throw new IllegalArgumentException("At least one operator must be specified"); @@ -147,6 +196,136 @@ protected UpdateBy(@NotNull final UpdateByOperator[] operators, this.windows = windows; this.inputSources = inputSources; this.operatorInputSourceSlots = operatorInputSourceSlots; + this.timestampColumnName = timestampColumnName; + this.redirContext = redirContext; + this.control = control; + + this.inputSourceCacheNeeded = new boolean[inputSources.length]; + this.cachedInputSources = new CachedInputSource[inputSources.length]; + + boolean cacheNeeded = false; + for (int ii = 0; ii < inputSources.length; ii++) { + inputSourceCacheNeeded[ii] = !FillUnordered.providesFillUnordered(inputSources[ii]); + cacheNeeded |= inputSourceCacheNeeded[ii]; + } + this.inputCacheNeeded = cacheNeeded; + + buckets = new LinkedList<>(); + } + + protected void shiftOutputColumns(TableUpdate upstream) { + if (redirContext.isRedirected()) { + redirContext.processUpdateForRedirection(upstream, source.getRowSet()); + } else if (upstream.shifted().nonempty()) { + try (final RowSet prevIdx = source.getRowSet().copyPrev()) { + upstream.shifted().apply((begin, end, delta) -> { + try (final RowSet subRowSet = prevIdx.subSetByKeyRange(begin, end)) { + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + operators[opIdx].applyOutputShift(subRowSet, delta); + } + } + }); + } + } + } + + protected void processBuckets(UpdateByBucketHelper[] dirtyBuckets, final boolean initialStep) { + // if (inputCacheNeeded) { + // computeCachedColumnContents(buckets, initialStep); + // } + + // let's start processing windows + for (int winIdx = 0; winIdx < windows.length; winIdx++) { + // maybeCacheInputSources(); + for (UpdateByBucketHelper bucket : dirtyBuckets) { + bucket.processWindow(winIdx, inputSources, initialStep); + } + // maybeReleaseInputSources() + } + } + + protected void finalizeBuckets(UpdateByBucketHelper[] dirtyBuckets) { + for (UpdateByBucketHelper bucket : dirtyBuckets) { + bucket.finalizeUpdate(); + } + } + + /** + * The Listener for apply to the constituent table updates + */ + class UpdateByListener extends MergedListener { + public UpdateByListener(@Nullable String description) { + super(UpdateBy.this.recorders, List.of(), description, UpdateBy.this.result); + } + + @Override + protected void process() { + UpdateByBucketHelper[] dirtyBuckets = buckets.stream().filter(b -> b.isDirty()) + .toArray(UpdateByBucketHelper[]::new); + + // do the actual computations + processBuckets(dirtyBuckets, false); + + final TableUpdateImpl downstream = new TableUpdateImpl(); + + // get the adds/removes/shifts from the first (source) entry, make a copy since TableUpdateImpl#reset will + // close them with the upstream update + ListenerRecorder sourceRecorder = recorders.peekFirst(); + downstream.added = sourceRecorder.getAdded().copy(); + downstream.removed = sourceRecorder.getRemoved().copy(); + downstream.shifted = sourceRecorder.getShifted(); + + // union the modifies from all the tables (including source) + downstream.modifiedColumnSet = result.getModifiedColumnSetForUpdates(); + downstream.modifiedColumnSet.clear(); + + WritableRowSet modifiedRowSet = RowSetFactory.empty(); + downstream.modified = modifiedRowSet; + + if (sourceRecorder.getModified().isNonempty()) { + modifiedRowSet.insert(sourceRecorder.getModified()); + downstream.modifiedColumnSet.setAll(sourceRecorder.getModifiedColumnSet()); + } + + for (UpdateByBucketHelper bucket : dirtyBuckets) { + // retrieve the modified row and column sets from the windows + for (int winIdx = 0; winIdx < windows.length; winIdx++) { + UpdateByWindow win = windows[winIdx]; + UpdateByWindow.UpdateByWindowContext winCtx = bucket.windowContexts[winIdx]; + + if (win.isWindowDirty(winCtx)) { + // add the window modified rows to this set + modifiedRowSet.insert(win.getModifiedRows(winCtx)); + // add the modified output column sets to the downstream set + final UpdateByOperator[] winOps = win.getOperators(); + for (int winOpIdx = 0; winOpIdx < winOps.length; winOpIdx++) { + if (win.isOperatorDirty(winCtx, winOpIdx)) { + // these were created directly from the result output columns so no transformer needed + downstream.modifiedColumnSet.setAll(winOps[winOpIdx].outputModifiedColumnSet); + } + } + } + } + + } + // should not include upstream adds as modifies + modifiedRowSet.remove(downstream.added); + + finalizeBuckets(dirtyBuckets); + + result.notifyListeners(downstream); + } + + @Override + protected boolean canExecute(final long step) { + synchronized (recorders) { + return recorders.stream().allMatch(lr -> lr.satisfied(step)); + } + } + } + + public UpdateByListener newListener(@NotNull final String description) { + return new UpdateByListener(description); } // region UpdateBy implementation @@ -193,37 +372,54 @@ public static Table updateBy(@NotNull final QueryTable source, rowRedirection = null; } - // create an UpdateByRedirectionContext for use by the UpdateBy objects - UpdateByRedirectionContext ctx = new UpdateByRedirectionContext(rowRedirection); + // create an UpdateByRedirectionContext for use by the UpdateByBucketHelper objects + UpdateByRedirectionContext redirContext = new UpdateByRedirectionContext(rowRedirection); - // TODO(deephaven-core#2693): Improve UpdateBy implementation for ColumnName + // TODO(deephaven-core#2693): Improve UpdateByBucketHelper implementation for ColumnName // generate a MatchPair array for use by the existing algorithm MatchPair[] pairs = MatchPair.fromPairs(byColumns); final UpdateByOperatorFactory updateByOperatorFactory = - new UpdateByOperatorFactory(source, pairs, ctx, control); + new UpdateByOperatorFactory(source, pairs, redirContext, control); final Collection ops = updateByOperatorFactory.getOperators(clauses); + final UpdateByOperator[] opArr = ops.toArray(UpdateByOperator.ZERO_LENGTH_OP_ARRAY); + + if (opArr.length == 0) { + throw new IllegalArgumentException("At least one operator must be specified"); + } + final StringBuilder descriptionBuilder = new StringBuilder("updateBy(ops={") .append(updateByOperatorFactory.describe(clauses)) .append("}"); + String timestampColumnName = null; final Set problems = new LinkedHashSet<>(); // noinspection rawtypes final Map> opResultSources = new LinkedHashMap<>(); - ops.forEach(op -> op.getOutputColumns().forEach((name, col) -> { - if (opResultSources.putIfAbsent(name, col) != null) { - problems.add(name); + for (int opIdx = 0; opIdx < opArr.length; opIdx++) { + final UpdateByOperator op = opArr[opIdx]; + op.getOutputColumns().forEach((name, col) -> { + if (opResultSources.putIfAbsent(name, col) != null) { + problems.add(name); + } + }); + // verify zero or one timestamp column names + if (op.getTimestampColumnName() != null) { + if (timestampColumnName == null) { + timestampColumnName = op.getTimestampColumnName(); + } else { + if (!timestampColumnName.equals(op.getTimestampColumnName())) { + throw new UncheckedTableException( + "Cannot reference more than one timestamp source on a single UpdateByBucketHelper operation {" + + + timestampColumnName + ", " + op.getTimestampColumnName() + "}"); + } + } } - })); - if (!problems.isEmpty()) { - throw new UncheckedTableException("Multiple Operators tried to produce the same output columns {" + - String.join(", ", problems) + "}"); } - final UpdateByOperator[] opArr = ops.toArray(UpdateByOperator.ZERO_LENGTH_OP_ARRAY); - // the next bit is complicated but the goal is simple. We don't want to have duplicate input column sources, so // we will store each one only once in inputSources and setup some mapping from the opIdx to the input column. // noinspection unchecked @@ -303,7 +499,7 @@ public static Table updateBy(@NotNull final QueryTable source, if (pairs.length == 0) { descriptionBuilder.append(")"); - Table ret = ZeroKeyUpdateBy.compute( + Table ret = ZeroKeyUpdateByManager.compute( descriptionBuilder.toString(), source, opArr, @@ -311,9 +507,9 @@ public static Table updateBy(@NotNull final QueryTable source, inputSourceArr, operatorInputSourceSlotArr, resultSources, - ctx, - control, - true); + timestampColumnName, + redirContext, + control); if (source.isRefreshing()) { // start tracking previous values @@ -339,7 +535,7 @@ public static Table updateBy(@NotNull final QueryTable source, String.join(", ", problems) + "}"); } - Table ret = BucketedPartitionedUpdateBy.compute( + Table ret = BucketedPartitionedUpdateByManager.compute( descriptionBuilder.toString(), source, opArr, @@ -348,7 +544,8 @@ public static Table updateBy(@NotNull final QueryTable source, operatorInputSourceSlotArr, resultSources, byColumns, - ctx, + timestampColumnName, + redirContext, control); if (source.isRefreshing()) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java new file mode 100644 index 00000000000..486f98e62f9 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java @@ -0,0 +1,310 @@ +package io.deephaven.engine.table.impl; + +import io.deephaven.api.updateby.UpdateByControl; +import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.*; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.*; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.table.*; +import io.deephaven.engine.table.impl.sources.ReinterpretUtils; +import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; +import io.deephaven.engine.table.impl.updateby.UpdateByWindow; +import org.apache.commons.lang3.mutable.MutableLong; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.util.Map; + +import static io.deephaven.util.QueryConstants.NULL_LONG; + +/** + * An implementation of {@link UpdateBy} dedicated to zero key computation. + */ +class UpdateByBucketHelper { + protected final ColumnSource[] inputSources; + // some columns will have multiple inputs, such as time-based and Weighted computations + final int[][] operatorInputSourceSlots; + final UpdateByOperator[] operators; + final UpdateByWindow[] windows; + final QueryTable source; + final UpdateBy.UpdateByRedirectionContext redirContext; + final UpdateByControl control; + final QueryTable result; + + /** An array of {@link UpdateByWindow.UpdateByWindowContext}s for each window */ + final UpdateByWindow.UpdateByWindowContext[] windowContexts; + + /** store timestamp data in an Ssa (if needed) */ + final String timestampColumnName; + final LongSegmentedSortedArray timestampSsa; + final ColumnSource timestampColumnSource; + final ModifiedColumnSet timestampColumnSet; + + /** Indicates this bucket needs to be processed (at least window and operator are dirty) */ + boolean isDirty; + + /** + * Perform an updateBy without any key columns. + * + * @param source the source table + * @param operators, the operations to perform + * @param resultSources the result sources + * @param redirContext the row redirection shared context + * @param control the control object. + * @return the result table + */ + + protected UpdateByBucketHelper(@NotNull final String description, + @NotNull final QueryTable source, + @NotNull final UpdateByOperator[] operators, + @NotNull final UpdateByWindow[] windows, + @NotNull final ColumnSource[] inputSources, + @NotNull final int[][] operatorInputSourceSlots, + @NotNull final Map> resultSources, + @Nullable String timestampColumnName, + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, + @NotNull final UpdateByControl control) { + + this.source = source; + this.operators = operators; + this.windows = windows; + this.inputSources = inputSources; + this.operatorInputSourceSlots = operatorInputSourceSlots; + this.redirContext = redirContext; + this.control = control; + + result = new QueryTable(source.getRowSet(), resultSources); + + // do we need a timestamp SSA? + this.timestampColumnName = timestampColumnName; + if (timestampColumnName != null) { + this.timestampSsa = new LongSegmentedSortedArray(4096); + this.timestampColumnSource = + ReinterpretUtils.maybeConvertToPrimitive(source.getColumnSource(this.timestampColumnName)); + this.timestampColumnSet = source.newModifiedColumnSet(timestampColumnName); + } else { + this.timestampSsa = null; + this.timestampColumnSource = null; + this.timestampColumnSet = null; + } + + this.windowContexts = new UpdateByWindow.UpdateByWindowContext[windows.length]; + + // make a fake update with the initial rows of the table + final TableUpdateImpl initialUpdate = new TableUpdateImpl(source.getRowSet(), + RowSetFactory.empty(), + RowSetFactory.empty(), + RowSetShiftData.EMPTY, + ModifiedColumnSet.EMPTY); + + prepareForUpdate(initialUpdate, true); + + if (source.isRefreshing()) { + final ZeroKeyUpdateByListener listener = newListener(description, result); + source.listenForUpdates(listener); + result.addParentReference(listener); + } + } + + ZeroKeyUpdateByListener newListener(@NotNull final String description, @NotNull final QueryTable result) { + return new ZeroKeyUpdateByListener(description, source); + } + + private void processUpdateForSsa(TableUpdate upstream) { + final boolean stampModified = upstream.modifiedColumnSet().containsAny(timestampColumnSet); + + final RowSet restampRemovals; + final RowSet restampAdditions; + + // modifies are remove + add operations + if (stampModified) { + restampAdditions = upstream.added().union(upstream.modified()); + restampRemovals = upstream.removed().union(upstream.getModifiedPreShift()); + } else { + restampAdditions = upstream.added(); + restampRemovals = upstream.removed(); + } + + // removes + if (restampRemovals.isNonempty()) { + final int size = (int) Math.min(restampRemovals.size(), 4096); + try (final RowSequence.Iterator it = restampRemovals.getRowSequenceIterator(); + final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(size); + final WritableLongChunk ssaValues = WritableLongChunk.makeWritableChunk(size); + final WritableLongChunk ssaKeys = WritableLongChunk.makeWritableChunk(size)) { + + MutableLong lastTimestamp = new MutableLong(NULL_LONG); + while (it.hasMore()) { + RowSequence chunkRs = it.getNextRowSequenceWithLength(4096); + + // get the chunks for values and keys + LongChunk valuesChunk = + timestampColumnSource.getPrevChunk(context, chunkRs).asLongChunk(); + LongChunk keysChunk = chunkRs.asRowKeyChunk(); + + // push only non-null values/keys into the Ssa + fillChunkWithNonNull(keysChunk, valuesChunk, ssaKeys, ssaValues, lastTimestamp); + timestampSsa.remove(ssaValues, ssaKeys); + } + } + } + + // shifts + if (upstream.shifted().nonempty()) { + final int size = Math.max( + upstream.modified().intSize() + Math.max(upstream.added().intSize(), upstream.removed().intSize()), + (int) upstream.shifted().getEffectiveSize()); + try (final RowSet fullPrevRowSet = source.getRowSet().copyPrev(); + final WritableRowSet previousToShift = fullPrevRowSet.minus(restampRemovals); + final ColumnSource.GetContext getContext = timestampColumnSource.makeGetContext(size)) { + + // no need to consider upstream removals + previousToShift.remove(upstream.removed()); + + final RowSetShiftData.Iterator sit = upstream.shifted().applyIterator(); + while (sit.hasNext()) { + sit.next(); + try (final RowSet subRowSet = previousToShift.subSetByKeyRange(sit.beginRange(), sit.endRange())) { + if (subRowSet.isEmpty()) { + continue; + } + + final LongChunk shiftValues = + timestampColumnSource.getPrevChunk(getContext, subRowSet).asLongChunk(); + + timestampSsa.applyShiftReverse(shiftValues, subRowSet.asRowKeyChunk(), sit.shiftDelta()); + } + } + } + } + + // adds + if (restampAdditions.isNonempty()) { + final int size = (int) Math.min(restampAdditions.size(), 4096); + try (final RowSequence.Iterator it = restampAdditions.getRowSequenceIterator(); + final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(size); + final WritableLongChunk ssaValues = WritableLongChunk.makeWritableChunk(size); + final WritableLongChunk ssaKeys = WritableLongChunk.makeWritableChunk(size)) { + MutableLong lastTimestamp = new MutableLong(NULL_LONG); + while (it.hasMore()) { + RowSequence chunkRs = it.getNextRowSequenceWithLength(4096); + + // get the chunks for values and keys + LongChunk valuesChunk = + timestampColumnSource.getChunk(context, chunkRs).asLongChunk(); + LongChunk keysChunk = chunkRs.asRowKeyChunk(); + + // push only non-null values/keys into the Ssa + fillChunkWithNonNull(keysChunk, valuesChunk, ssaKeys, ssaValues, lastTimestamp); + timestampSsa.insert(ssaValues, ssaKeys); + } + } + } + } + + /** + * helper function to fill a LongChunk while skipping values that are NULL_LONG. Used to populate an SSA from a + * source containing null values + */ + private void fillChunkWithNonNull(LongChunk keysChunk, LongChunk valuesChunk, + WritableLongChunk ssaKeys, WritableLongChunk ssaValues, + MutableLong lastTimestamp) { + // reset the insertion chunks + ssaValues.setSize(0); + ssaKeys.setSize(0); + + // add only non-null timestamps to this Ssa + for (int i = 0; i < valuesChunk.size(); i++) { + long ts = valuesChunk.get(i); + if (ts < lastTimestamp.longValue()) { + throw (new IllegalStateException( + "updateBy time-based operators require non-descending timestamp values")); + } + if (ts != NULL_LONG) { + ssaValues.add(ts); + ssaKeys.add(keysChunk.get(i)); + } + } + } + + /** + * Calling this function will prepare this bucket for computation, including making a + * {@link UpdateByWindow.UpdateByWindowContext} for each window and computing the affected and influencer rowsets + * for each window + * + * @param upstream The incoming update for which to prepare + * @param initialStep Whether this update is part of the initial creation of the bucket + */ + public void prepareForUpdate(final TableUpdate upstream, final boolean initialStep) { + Assert.eqFalse(isDirty, "UpdateBy bucket was marekd dirty before processing an update"); + + // add all the SSA data + if (timestampColumnName != null) { + processUpdateForSsa(upstream); + } + + final TrackingRowSet sourceRowSet = source.getRowSet(); + + // create context for each window + for (int winIdx = 0; winIdx < windows.length; winIdx++) { + windowContexts[winIdx] = windows[winIdx].makeWindowContext( + sourceRowSet, + inputSources, + timestampColumnSource, + timestampSsa, + control.chunkCapacityOrDefault(), + initialStep); + + // compute the affected/influenced operators and rowsets within this window + windows[winIdx].computeAffectedRowsAndOperators(windowContexts[winIdx], upstream); + + isDirty |= windows[winIdx].isWindowDirty(windowContexts[winIdx]); + } + + if (!isDirty) { + // we will never use these contexts, so clean them up now + finalizeUpdate(); + } + } + + public boolean isDirty() { + return isDirty; + } + + public void processWindow(final int winIdx, + final ColumnSource[] inputSources, + final boolean initialStep) { + // call the window.process() with the correct context for this bucket + if (windows[winIdx].isWindowDirty(windowContexts[winIdx])) { + windows[winIdx].processRows(windowContexts[winIdx], inputSources, initialStep); + } + } + + public void finalizeUpdate() { + for (int winIdx = 0; winIdx < windows.length; winIdx++) { + windowContexts[winIdx].close(); + windowContexts[winIdx] = null; + } + isDirty = false; + } + + /** + * The Listener for apply an upstream {@link InstrumentedTableUpdateListenerAdapter#onUpdate(TableUpdate) update} + */ + class ZeroKeyUpdateByListener extends InstrumentedTableUpdateListenerAdapter { + public ZeroKeyUpdateByListener(@Nullable String description, + @NotNull final QueryTable source) { + super(description, source, false); + } + + @Override + public void onUpdate(TableUpdate upstream) { + prepareForUpdate(upstream, false); + + // pass the update unchanged, just increment the ref count + TableUpdate downstream = upstream.acquire(); + result.notifyListeners(downstream); + } + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java index 35b57c3ee3c..ed01590b8de 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java @@ -15,17 +15,7 @@ import static io.deephaven.util.QueryConstants.NULL_LONG; -public abstract class UpdateByCumulativeOperator implements UpdateByOperator { - protected final MatchPair pair; - protected final String[] affectingColumns; - - protected final UpdateBy.UpdateByRedirectionContext redirContext; - - // these will be used by the timestamp-aware operators (EMA for example) - protected OperationControl control; - protected long timeScaleUnits; - protected String timestampColumnName; - +public abstract class UpdateByCumulativeOperator extends UpdateByOperator { public abstract class Context implements UpdateContext { public long curTimestamp; @@ -55,12 +45,7 @@ public abstract void accumulate(RowSequence inputKeys, public UpdateByCumulativeOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { - this.pair = pair; - this.affectingColumns = affectingColumns; - this.redirContext = redirContext; - - this.timeScaleUnits = 0L; - this.timestampColumnName = null; + super(pair, affectingColumns, null, null, 0L, 0L, redirContext); } abstract public void initializeUpdate(@NotNull final UpdateContext context, final long firstUnmodifiedKey, @@ -68,38 +53,4 @@ abstract public void initializeUpdate(@NotNull final UpdateContext context, fina @Override public void finishUpdate(@NotNull final UpdateContext context) {} - - @Override - public String getTimestampColumnName() { - return timestampColumnName; - } - - @Override - public long getPrevWindowUnits() { - return timeScaleUnits; - } - - /** cumulative operators do not have a forward-looking window */ - @Override - public long getFwdWindowUnits() { - return 0L; - } - - @NotNull - @Override - public String[] getInputColumnNames() { - return new String[] {pair.rightColumn}; - } - - @NotNull - @Override - public String[] getAffectingColumnNames() { - return affectingColumns; - } - - @NotNull - @Override - public String[] getOutputColumnNames() { - return new String[] {pair.leftColumn}; - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java index 1c8174ed878..d4b6a93320d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java @@ -1,15 +1,15 @@ package io.deephaven.engine.table.impl; +import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; -import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.ModifiedColumnSet; import io.deephaven.engine.table.Table; -import io.deephaven.engine.table.impl.updateby.UpdateByWindow; import io.deephaven.util.SafeCloseable; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -23,27 +23,36 @@ *
                *
              1. Reprocess *
                  - *
                • {@link #initializeFor(UpdateContext, RowSet, UpdateBy.UpdateType)}
                • - *
                • {@link #reprocessChunkBucketed(UpdateContext, RowSequence, Chunk, LongChunk, IntChunk, IntChunk, IntChunk)}
                • - *
                • {@link #finishFor(UpdateContext, UpdateBy.UpdateType)}
                • + *
                • {@link #initializeUpdate(UpdateContext, RowSet, UpdateBy.UpdateType)}
                • + *
                • {@link #UpdateByWindow.processRows}
                • + *
                • {@link #finishUpdate(UpdateContext)}
                • *
                *
              2. *
              - * - *

              - * Additionally, implementations are responsible for notifying the update model if any rows have been modified beyond - * what was passed through in the upstream update via the {@link #anyModified(UpdateContext)} and - * {@link #getAdditionalModifications(UpdateContext)} methods - *

              */ -public interface UpdateByOperator { - UpdateByWindow[] ZERO_LENGTH_WINDOW_ARRAY = new UpdateByWindow[0]; - UpdateByOperator[] ZERO_LENGTH_OP_ARRAY = new UpdateByOperator[0]; +public abstract class UpdateByOperator { + public static UpdateByOperator[] ZERO_LENGTH_OP_ARRAY = new UpdateByOperator[0]; + + protected final MatchPair pair; + protected final String[] affectingColumns; + protected final UpdateBy.UpdateByRedirectionContext redirContext; + + + // these will be used by the timestamp-aware operators (EMA for example) + protected OperationControl control; + protected long reverseTimeScaleUnits; + protected long forwardTimeScaleUnits; + protected String timestampColumnName; + + // individual input modifiedColumnSet for this operator + protected ModifiedColumnSet inputModifiedColumnSet; + // individual output modifiedColumnSet for this operators + protected ModifiedColumnSet outputModifiedColumnSet; /** * A context item for use with updateBy operators */ - interface UpdateContext extends SafeCloseable { + public interface UpdateContext extends SafeCloseable { void setValuesChunk(@NotNull final Chunk valuesChunk); @@ -82,13 +91,32 @@ interface UpdateContext extends SafeCloseable { void writeToOutputColumn(@NotNull final RowSequence inputKeys); } + + public UpdateByOperator(@NotNull final MatchPair pair, + @NotNull final String[] affectingColumns, + @Nullable final OperationControl control, + @Nullable final String timestampColumnName, + final long reverseTimeScaleUnits, + final long forwardTimeScaleUnits, + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { + this.pair = pair; + this.affectingColumns = affectingColumns; + this.redirContext = redirContext; + this.timestampColumnName = timestampColumnName; + this.control = control; + this.reverseTimeScaleUnits = reverseTimeScaleUnits; + this.forwardTimeScaleUnits = forwardTimeScaleUnits; + } + /** * Get the name of the input column(s) this operator depends on. * * @return the name of the input column */ @NotNull - String[] getInputColumnNames(); + public String[] getInputColumnNames() { + return new String[] {pair.rightColumn}; + } /** * Get the name of the timestamp column this operator depends on. @@ -96,21 +124,27 @@ interface UpdateContext extends SafeCloseable { * @return the name of the input column */ @Nullable - String getTimestampColumnName(); + public String getTimestampColumnName() { + return timestampColumnName; + } /** * Get the value of the backward-looking window (might be nanos or ticks). * * @return the name of the input column */ - long getPrevWindowUnits(); + public long getPrevWindowUnits() { + return reverseTimeScaleUnits; + } /** * Get the value of the forward-looking window (might be nanos or ticks). * * @return the name of the input column */ - long getFwdWindowUnits(); + public long getFwdWindowUnits() { + return forwardTimeScaleUnits; + } /** * Get an array of column names that, when modified, affect the result of this computation. @@ -118,7 +152,9 @@ interface UpdateContext extends SafeCloseable { * @return an array of column names that affect this operator. */ @NotNull - String[] getAffectingColumnNames(); + public String[] getAffectingColumnNames() { + return affectingColumns; + } /** * Get an array of the output column names. @@ -126,7 +162,9 @@ interface UpdateContext extends SafeCloseable { * @return the output column names. */ @NotNull - String[] getOutputColumnNames(); + public String[] getOutputColumnNames() { + return new String[] {pair.leftColumn}; + } /** * Get a map of outputName to output {@link ColumnSource} for this operation. @@ -134,12 +172,12 @@ interface UpdateContext extends SafeCloseable { * @return a map of output column name to output column source */ @NotNull - Map> getOutputColumns(); + public abstract Map> getOutputColumns(); /** * Indicate that the operation should start tracking previous values for ticking updates. */ - void startTrackingPrev(); + public abstract void startTrackingPrev(); /** * Make an {@link UpdateContext} suitable for use with updates. @@ -149,7 +187,7 @@ interface UpdateContext extends SafeCloseable { * @return a new context */ @NotNull - UpdateContext makeUpdateContext(final int chunkSize, ColumnSource[] inputSourceArr); + public abstract UpdateContext makeUpdateContext(final int chunkSize, ColumnSource[] inputSourceArr); /** * Perform and bookkeeping required at the end of a single part of the update. This is always preceded with a call @@ -157,11 +195,43 @@ interface UpdateContext extends SafeCloseable { * * @param context the context object */ - void finishUpdate(@NotNull final UpdateContext context); + public abstract void finishUpdate(@NotNull final UpdateContext context); /** * Apply a shift to the operation. * */ - void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta); + public abstract void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta); + + /** + * Create the modified column set for the input columns of this operator. + * + */ + public void createInputModifiedColumnSet(@NotNull final QueryTable source) { + inputModifiedColumnSet = source.newModifiedColumnSet(getAffectingColumnNames()); + } + + /** + * Create the modified column set for the output columns from this operator. + * + */ + public void createOutputModifiedColumnSet(@NotNull final QueryTable result) { + outputModifiedColumnSet = result.newModifiedColumnSet(getOutputColumnNames()); + } + + /** + * Return the modified column set for the input columns of this operator. + * + */ + public ModifiedColumnSet getInputModifiedColumnSet() { + return inputModifiedColumnSet; + } + + /** + * Return the modified column set for the output columns from this operator. + * + */ + public ModifiedColumnSet getOutputModifiedColumnSet() { + return outputModifiedColumnSet; + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java index b0ea8ddee63..c2e8dd78e52 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java @@ -2,6 +2,7 @@ import io.deephaven.api.agg.Pair; import io.deephaven.api.updateby.ColumnUpdateOperation; +import io.deephaven.api.updateby.OperationControl; import io.deephaven.api.updateby.UpdateByControl; import io.deephaven.api.updateby.UpdateByOperation; import io.deephaven.api.updateby.spec.*; @@ -221,32 +222,33 @@ private UpdateByOperator makeEmaOperator(@NotNull final MatchPair pair, affectingColumns = new String[] {ema.timeScale().timestampCol(), pair.rightColumn}; } - // use the correct units from the EmaSpec (depending on was Time or Tick based) + // use the correct units from the EmaSpec (depending on if Time or Tick based) final long timeScaleUnits = ema.timeScale().timescaleUnits(); + final OperationControl control = ema.controlOrDefault(); if (csType == byte.class || csType == Byte.class) { - return new ByteEMAOperator(pair, affectingColumns, ema.controlOrDefault(), + return new ByteEMAOperator(pair, affectingColumns, control, ema.timeScale().timestampCol(), timeScaleUnits, redirContext); } else if (csType == short.class || csType == Short.class) { - return new ShortEMAOperator(pair, affectingColumns, ema.controlOrDefault(), + return new ShortEMAOperator(pair, affectingColumns, control, ema.timeScale().timestampCol(), timeScaleUnits, redirContext); } else if (csType == int.class || csType == Integer.class) { - return new IntEMAOperator(pair, affectingColumns, ema.controlOrDefault(), + return new IntEMAOperator(pair, affectingColumns, control, ema.timeScale().timestampCol(), timeScaleUnits, redirContext); } else if (csType == long.class || csType == Long.class) { - return new LongEMAOperator(pair, affectingColumns, ema.controlOrDefault(), + return new LongEMAOperator(pair, affectingColumns, control, ema.timeScale().timestampCol(), timeScaleUnits, redirContext); } else if (csType == float.class || csType == Float.class) { - return new FloatEMAOperator(pair, affectingColumns, ema.controlOrDefault(), + return new FloatEMAOperator(pair, affectingColumns, control, ema.timeScale().timestampCol(), timeScaleUnits, redirContext); } else if (csType == double.class || csType == Double.class) { - return new DoubleEMAOperator(pair, affectingColumns, ema.controlOrDefault(), + return new DoubleEMAOperator(pair, affectingColumns, control, ema.timeScale().timestampCol(), timeScaleUnits, redirContext); } else if (csType == BigDecimal.class) { - return new BigDecimalEMAOperator(pair, affectingColumns, ema.controlOrDefault(), + return new BigDecimalEMAOperator(pair, affectingColumns, control, ema.timeScale().timestampCol(), timeScaleUnits, redirContext); } else if (csType == BigInteger.class) { - return new BigIntegerEMAOperator(pair, affectingColumns, ema.controlOrDefault(), + return new BigIntegerEMAOperator(pair, affectingColumns, control, ema.timeScale().timestampCol(), timeScaleUnits, redirContext); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java index 6741231c0a2..1e611633a47 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java @@ -9,17 +9,7 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -public abstract class UpdateByWindowedOperator implements UpdateByOperator { - protected final MatchPair pair; - protected final String[] affectingColumns; - - protected UpdateBy.UpdateByRedirectionContext redirContext; - - protected final OperationControl control; - protected final String timestampColumnName; - - protected final long reverseTimeScaleUnits; - protected final long forwardTimeScaleUnits; +public abstract class UpdateByWindowedOperator extends UpdateByOperator { public abstract class Context implements UpdateContext { public int nullCount = 0; @@ -50,53 +40,12 @@ public UpdateByWindowedOperator(@NotNull final MatchPair pair, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { - this.pair = pair; - this.affectingColumns = affectingColumns; - this.control = control; - this.timestampColumnName = timestampColumnName; - this.reverseTimeScaleUnits = reverseTimeScaleUnits; - this.forwardTimeScaleUnits = forwardTimeScaleUnits; - this.redirContext = redirContext; + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, + redirContext); } public abstract void initializeUpdate(@NotNull final UpdateContext context); @Override public void finishUpdate(@NotNull final UpdateContext context) {} - - @Override - public String getTimestampColumnName() { - return this.timestampColumnName; - } - - /*** Get the value of the backward-looking window (might be nanos or ticks) */ - @Override - public long getPrevWindowUnits() { - return reverseTimeScaleUnits; - } - - /*** Get the value of the forward-looking window (might be nanos or ticks) */ - @Override - public long getFwdWindowUnits() { - return forwardTimeScaleUnits; - } - - /*** Mostly will be a single input column, but some accept multiple input (WAvg for example) */ - @NotNull - @Override - public String[] getInputColumnNames() { - return new String[] {pair.rightColumn}; - } - - @NotNull - @Override - public String[] getAffectingColumnNames() { - return affectingColumns; - } - - @NotNull - @Override - public String[] getOutputColumnNames() { - return new String[] {pair.leftColumn}; - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java deleted file mode 100644 index b1a289057be..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateBy.java +++ /dev/null @@ -1,439 +0,0 @@ -package io.deephaven.engine.table.impl; - -import gnu.trove.set.hash.TLongHashSet; -import io.deephaven.api.updateby.UpdateByControl; -import io.deephaven.base.verify.Assert; -import io.deephaven.chunk.*; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.sources.ReinterpretUtils; -import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; -import io.deephaven.engine.table.impl.updateby.UpdateByWindow; -import io.deephaven.util.SafeCloseable; -import org.apache.commons.lang3.mutable.MutableLong; -import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; - -import java.util.Arrays; -import java.util.Map; - -import static io.deephaven.util.QueryConstants.NULL_LONG; - -/** - * An implementation of {@link UpdateBy} dedicated to zero key computation. - */ -class ZeroKeyUpdateBy extends UpdateBy { - /** Apply shifts to operator outputs? */ - final boolean applyShifts; - - /** store timestamp data in an Ssa (if needed) */ - final String timestampColumnName; - final LongSegmentedSortedArray timestampSsa; - final ColumnSource timestampColumnSource; - final ModifiedColumnSet timestampColumnSet; - - // individual output modifiedColumnSets for the operators - protected final ModifiedColumnSet[][] windowOperatorOutputModifiedColumnSets; - - /** - * Perform an updateBy without any key columns. - * - * @param description the operation description - * @param source the source table - * @param ops the operations to perform - * @param resultSources the result sources - * @param redirContext the row redirection shared context - * @param control the control object. - * @return the result table - */ - public static Table compute(@NotNull final String description, - @NotNull final QueryTable source, - @NotNull final UpdateByOperator[] ops, - @NotNull final UpdateByWindow[] windows, - @NotNull final ColumnSource[] inputSources, - @NotNull final int[][] operatorInputSourceSlots, - @NotNull final Map> resultSources, - @NotNull final UpdateByRedirectionContext redirContext, - @NotNull final UpdateByControl control, - final boolean applyShifts) { - - final QueryTable result = new QueryTable(source.getRowSet(), resultSources); - final ZeroKeyUpdateBy updateBy = new ZeroKeyUpdateBy(ops, windows, inputSources, operatorInputSourceSlots, - source, redirContext, control, applyShifts); - - updateBy.doInitialAdditions(); - - if (source.isRefreshing()) { - final ZeroKeyUpdateByListener listener = updateBy.newListener(description, result); - source.listenForUpdates(listener); - result.addParentReference(listener); - } - - return result; - } - - protected ZeroKeyUpdateBy(@NotNull final UpdateByOperator[] operators, - @NotNull final UpdateByWindow[] windows, - @NotNull final ColumnSource[] inputSources, - @NotNull final int[][] operatorInputSourceSlots, - @NotNull final QueryTable source, - @NotNull final UpdateByRedirectionContext redirContext, - @NotNull final UpdateByControl control, - final boolean applyShifts) { - super(operators, windows, inputSources, operatorInputSourceSlots, source, redirContext, control); - - // do we need a timestamp SSA? - this.timestampColumnName = Arrays.stream(operators) - .filter(op -> op.getTimestampColumnName() != null) - .map(UpdateByOperator::getTimestampColumnName) - .findFirst().orElse(null); - - if (timestampColumnName != null) { - this.timestampSsa = new LongSegmentedSortedArray(4096); - this.timestampColumnSource = - ReinterpretUtils.maybeConvertToPrimitive(source.getColumnSource(this.timestampColumnName)); - this.timestampColumnSet = source.newModifiedColumnSet(timestampColumnName); - } else { - this.timestampSsa = null; - this.timestampColumnSource = null; - this.timestampColumnSet = null; - } - this.applyShifts = applyShifts; - this.windowOperatorOutputModifiedColumnSets = new ModifiedColumnSet[windows.length][]; - - } - - ZeroKeyUpdateByListener newListener(@NotNull final String description, @NotNull final QueryTable result) { - return new ZeroKeyUpdateByListener(description, source, result); - } - - private void processUpdateForSsa(TableUpdate upstream) { - final boolean stampModified = upstream.modifiedColumnSet().containsAny(timestampColumnSet); - - final RowSet restampRemovals; - final RowSet restampAdditions; - - // modifies are remove + add operations - if (stampModified) { - restampAdditions = upstream.added().union(upstream.modified()); - restampRemovals = upstream.removed().union(upstream.getModifiedPreShift()); - } else { - restampAdditions = upstream.added(); - restampRemovals = upstream.removed(); - } - - // removes - if (restampRemovals.isNonempty()) { - final int size = (int) Math.min(restampRemovals.size(), 4096); - try (final RowSequence.Iterator it = restampRemovals.getRowSequenceIterator(); - final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(size); - final WritableLongChunk ssaValues = WritableLongChunk.makeWritableChunk(size); - final WritableLongChunk ssaKeys = WritableLongChunk.makeWritableChunk(size)) { - - MutableLong lastTimestamp = new MutableLong(NULL_LONG); - while (it.hasMore()) { - RowSequence chunkRs = it.getNextRowSequenceWithLength(4096); - - // get the chunks for values and keys - LongChunk valuesChunk = - timestampColumnSource.getPrevChunk(context, chunkRs).asLongChunk(); - LongChunk keysChunk = chunkRs.asRowKeyChunk(); - - // push only non-null values/keys into the Ssa - fillChunkWithNonNull(keysChunk, valuesChunk, ssaKeys, ssaValues, lastTimestamp); - timestampSsa.remove(ssaValues, ssaKeys); - } - } - } - - // shifts - if (upstream.shifted().nonempty()) { - final int size = Math.max( - upstream.modified().intSize() + Math.max(upstream.added().intSize(), upstream.removed().intSize()), - (int) upstream.shifted().getEffectiveSize()); - try (final RowSet fullPrevRowSet = source.getRowSet().copyPrev(); - final WritableRowSet previousToShift = fullPrevRowSet.minus(restampRemovals); - final ColumnSource.GetContext getContext = timestampColumnSource.makeGetContext(size)) { - - // no need to consider upstream removals - previousToShift.remove(upstream.removed()); - - final RowSetShiftData.Iterator sit = upstream.shifted().applyIterator(); - while (sit.hasNext()) { - sit.next(); - try (final RowSet subRowSet = previousToShift.subSetByKeyRange(sit.beginRange(), sit.endRange())) { - if (subRowSet.isEmpty()) { - continue; - } - - final LongChunk shiftValues = - timestampColumnSource.getPrevChunk(getContext, subRowSet).asLongChunk(); - - timestampSsa.applyShiftReverse(shiftValues, subRowSet.asRowKeyChunk(), sit.shiftDelta()); - } - } - } - } - - // adds - if (restampAdditions.isNonempty()) { - final int size = (int) Math.min(restampAdditions.size(), 4096); - try (final RowSequence.Iterator it = restampAdditions.getRowSequenceIterator(); - final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(size); - final WritableLongChunk ssaValues = WritableLongChunk.makeWritableChunk(size); - final WritableLongChunk ssaKeys = WritableLongChunk.makeWritableChunk(size)) { - MutableLong lastTimestamp = new MutableLong(NULL_LONG); - while (it.hasMore()) { - RowSequence chunkRs = it.getNextRowSequenceWithLength(4096); - - // get the chunks for values and keys - LongChunk valuesChunk = - timestampColumnSource.getChunk(context, chunkRs).asLongChunk(); - LongChunk keysChunk = chunkRs.asRowKeyChunk(); - - // push only non-null values/keys into the Ssa - fillChunkWithNonNull(keysChunk, valuesChunk, ssaKeys, ssaValues, lastTimestamp); - timestampSsa.insert(ssaValues, ssaKeys); - } - } - } - } - - /** - * helper function to fill a LongChunk while skipping values that are NULL_LONG. Used to populate an SSA from a - * source containing null values - */ - private void fillChunkWithNonNull(LongChunk keysChunk, LongChunk valuesChunk, - WritableLongChunk ssaKeys, WritableLongChunk ssaValues, - MutableLong lastTimestamp) { - // reset the insertion chunks - ssaValues.setSize(0); - ssaKeys.setSize(0); - - // add only non-null timestamps to this Ssa - for (int i = 0; i < valuesChunk.size(); i++) { - long ts = valuesChunk.get(i); - if (ts < lastTimestamp.longValue()) { - throw (new IllegalStateException( - "updateBy time-based operators require non-descending timestamp values")); - } - if (ts != NULL_LONG) { - ssaValues.add(ts); - ssaKeys.add(keysChunk.get(i)); - } - } - } - - - void doInitialAdditions() { - final TableUpdateImpl fakeUpdate = new TableUpdateImpl(source.getRowSet(), - RowSetFactory.empty(), - RowSetFactory.empty(), - RowSetShiftData.EMPTY, - ModifiedColumnSet.ALL); - if (redirContext.isRedirected()) { - redirContext.processUpdateForRedirection(fakeUpdate, source.getRowSet()); - } - - // add all the SSA data - if (timestampColumnName != null) { - processUpdateForSsa(fakeUpdate); - } - - // do the processing for this fake update - try (final UpdateContext ctx = new UpdateContext(fakeUpdate, true)) { - ctx.processRows(); - } - } - - /** - * An object to hold the transient state during a single - * {@link InstrumentedTableUpdateListenerAdapter#onUpdate(TableUpdate)} update cycle. - */ - private class UpdateContext implements SafeCloseable { - /** The expected size of chunks to the various update stages */ - int chunkSize; - - /** A {@link SharedContext} to be used while creating other contexts */ - SharedContext sharedContext = SharedContext.makeSharedContext(); - - /** An array of {@link UpdateByWindow.UpdateByWindowContext}s for each input column */ - final UpdateByWindow.UpdateByWindowContext[] windowContexts; - - /** Indicate if any of the operators in this window are affected by the update. */ - boolean[] windowAffected; - - /** A sharedContext to be used with previous values */ - SharedContext prevSharedContext; - - @SuppressWarnings("resource") - UpdateContext(@NotNull final TableUpdate upstream, - final boolean isInitializeStep) { - - this.windowContexts = new UpdateByWindow.UpdateByWindowContext[windows.length]; - this.windowAffected = new boolean[windows.length]; - - for (int winIdx = 0; winIdx < windows.length; winIdx++) { - // create a context for each window - windowContexts[winIdx] = windows[winIdx].makeWindowContext( - source.getRowSet(), - inputSources, - timestampColumnSource, - timestampSsa, - control.chunkCapacityOrDefault(), - isInitializeStep); - - // compute the affected/influenced operators and rowset within this window - windowAffected[winIdx] = windowContexts[winIdx].computeAffectedRowsAndOperators(upstream); - } - } - - public SharedContext getSharedContext() { - return sharedContext; - } - - public boolean anyModified() { - for (int winIdx = 0; winIdx < windows.length; winIdx++) { - if (windowContexts[winIdx].anyModified()) { - return true; - } - } - return false; - } - - @Override - public void close() { - sharedContext.close(); - - if (prevSharedContext != null) { - prevSharedContext.close(); - } - - for (int winIdx = 0; winIdx < windows.length; winIdx++) { - if (windowContexts[winIdx] != null) { - windowContexts[winIdx].close(); - } - } - } - - private void processRows() { - // this might be parallelized if there are multiple windows - for (int winIdx = 0; winIdx < windows.length; winIdx++) { - if (windowAffected[winIdx]) { - // this will internally call initialize() and finish() for each operator - windowContexts[winIdx].processRows(); - } - } - } - } - - /** - * The Listener for apply an upstream {@link InstrumentedTableUpdateListenerAdapter#onUpdate(TableUpdate) update} - */ - class ZeroKeyUpdateByListener extends InstrumentedTableUpdateListenerAdapter { - private final QueryTable result; - private final ModifiedColumnSet.Transformer transformer; - - public ZeroKeyUpdateByListener(@Nullable String description, - @NotNull final QueryTable source, - @NotNull final QueryTable result) { - super(description, source, false); - this.result = result; - - for (int ii = 0; ii < windows.length; ii++) { - windows[ii].startTrackingModifications(source, result); - final UpdateByOperator[] windowOps = windows[ii].getOperators(); - windowOperatorOutputModifiedColumnSets[ii] = new ModifiedColumnSet[windowOps.length]; - - // these must be created here and relate to the local result table - for (int winOpIdx = 0; winOpIdx < windowOps.length; winOpIdx++) { - windowOperatorOutputModifiedColumnSets[ii][winOpIdx] = - result.newModifiedColumnSet(windowOps[winOpIdx].getOutputColumnNames()); - } - } - this.transformer = - source.newModifiedColumnSetTransformer(result, source.getDefinition().getColumnNamesArray()); - } - - @Override - public void onUpdate(TableUpdate upstream) { - // update the Ssa - if (timestampColumnName != null) { - processUpdateForSsa(upstream); - } - - try (final UpdateContext ctx = new UpdateContext(upstream, false)) { - if (applyShifts) { - if (redirContext.isRedirected()) { - redirContext.processUpdateForRedirection(upstream, source.getRowSet()); - } else { - // We will not mess with shifts if we are using a redirection because we'll have applied the - // shift to the redirection index already by now. - if (upstream.shifted().nonempty()) { - try (final RowSet prevIdx = source.getRowSet().copyPrev()) { - upstream.shifted().apply((begin, end, delta) -> { - try (final RowSet subRowSet = prevIdx.subSetByKeyRange(begin, end)) { - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - operators[opIdx].applyOutputShift(subRowSet, delta); - } - } - }); - } - } - } - } - - // Now do the processing - ctx.processRows(); - - final TableUpdateImpl downstream = new TableUpdateImpl(); - // copy these rowSets since TableUpdateImpl#reset will close them with the upstream update - downstream.added = upstream.added().copy(); - downstream.removed = upstream.removed().copy(); - downstream.shifted = upstream.shifted(); - - downstream.modifiedColumnSet = result.getModifiedColumnSetForUpdates(); - downstream.modifiedColumnSet.clear(); - - final boolean windowsModified = ctx.anyModified(); - - if (upstream.modified().isNonempty() || windowsModified) { - WritableRowSet modifiedRowSet = RowSetFactory.empty(); - downstream.modified = modifiedRowSet; - if (upstream.modified().isNonempty()) { - // Transform any untouched modified columns to the output. - transformer.clearAndTransform(upstream.modifiedColumnSet(), downstream.modifiedColumnSet); - modifiedRowSet.insert(upstream.modified()); - } - - // retrieve the modified rowsets from the windows - for (int winIdx = 0; winIdx < windows.length; winIdx++) { - if (ctx.windowAffected[winIdx]) { - if (ctx.windowContexts[winIdx].anyModified()) { - modifiedRowSet.insert(ctx.windowContexts[winIdx].getModifiedRows()); - } - } - } - - if (windowsModified) { - modifiedRowSet.remove(upstream.added()); - } - } else { - downstream.modified = RowSetFactory.empty(); - } - - // set the modified columns if any operators made changes (add/rem/modify) - for (int winIdx = 0; winIdx < windows.length; winIdx++) { - if (ctx.windowAffected[winIdx]) { - ctx.windowContexts[winIdx].updateOutputModifiedColumnSet(downstream.modifiedColumnSet, - windowOperatorOutputModifiedColumnSets[winIdx]); - } - } - - result.notifyListeners(downstream); - } - } - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateByManager.java new file mode 100644 index 00000000000..42209ceedfc --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateByManager.java @@ -0,0 +1,260 @@ +package io.deephaven.engine.table.impl; + +import io.deephaven.api.updateby.UpdateByControl; +import io.deephaven.engine.rowset.*; +import io.deephaven.engine.table.*; +import io.deephaven.engine.table.impl.updateby.UpdateByWindow; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.util.LinkedList; +import java.util.Map; + +public class ZeroKeyUpdateByManager extends UpdateBy { + // this manager has only one bucket, managed by this object + final UpdateByBucketHelper zeroKeyUpdateBy; + + protected ZeroKeyUpdateByManager(@NotNull final String description, + @NotNull QueryTable source, + @NotNull UpdateByOperator[] operators, + @NotNull UpdateByWindow[] windows, + @NotNull ColumnSource[] inputSources, + @NotNull int[][] operatorInputSourceSlots, + @NotNull final Map> resultSources, + @Nullable String timestampColumnName, + @NotNull UpdateByRedirectionContext redirContext, + @NotNull UpdateByControl control) { + super(description, source, operators, windows, inputSources, operatorInputSourceSlots, resultSources, + timestampColumnName, redirContext, control); + + // this table will always have the rowset of the source + result = new QueryTable(source.getRowSet(), resultSources); + + if (source.isRefreshing()) { + // this is a refreshing source, we will need a listener and recorders + recorders = new LinkedList<>(); + listener = newListener(description); + + // create an intermediate table that will listen to source updates and shift output columns + final QueryTable shiftApplyTable = new QueryTable(source.getRowSet(), source.getColumnSourceMap()); + + source.listenForUpdates(new BaseTable.ListenerImpl("", source, shiftApplyTable) { + @Override + public void onUpdate(@NotNull final TableUpdate upstream) { + shiftOutputColumns(upstream); + super.onUpdate(upstream); + } + }); + + // create a recorder instance sourced from the shifting table + ListenerRecorder shiftRecorder = new ListenerRecorder(description, shiftApplyTable, result); + shiftRecorder.setMergedListener(listener); + shiftApplyTable.listenForUpdates(shiftRecorder); + result.addParentReference(listener); + recorders.offerLast(shiftRecorder); + + // create input and output modified column sets + for (UpdateByOperator op : operators) { + op.createInputModifiedColumnSet(shiftApplyTable); + op.createOutputModifiedColumnSet(result); + } + + // create an updateby bucket instance sourced from the shifting table + zeroKeyUpdateBy = new UpdateByBucketHelper(description, shiftApplyTable, operators, windows, inputSources, + operatorInputSourceSlots, resultSources, timestampColumnName, redirContext, control); + buckets.offerLast(zeroKeyUpdateBy); + + // create a recorder instance sourced from the bucket helper + ListenerRecorder recorder = new ListenerRecorder(description, zeroKeyUpdateBy.result, result); + recorder.setMergedListener(listener); + zeroKeyUpdateBy.result.listenForUpdates(recorder); + recorders.offerLast(recorder); + } else { + // no shifting will be needed, can create directly from source + zeroKeyUpdateBy = new UpdateByBucketHelper(description, source, operators, windows, inputSources, + operatorInputSourceSlots, resultSources, timestampColumnName, redirContext, control); + this.result = zeroKeyUpdateBy.result; + buckets.offerLast(zeroKeyUpdateBy); + + // create input modified column sets only + for (UpdateByOperator op : operators) { + op.createInputModifiedColumnSet(source); + } + } + + if (redirContext.isRedirected()) { + // make a dummy update to generate the initial row keys + final TableUpdateImpl fakeUpdate = new TableUpdateImpl(source.getRowSet(), + RowSetFactory.empty(), + RowSetFactory.empty(), + RowSetShiftData.EMPTY, + ModifiedColumnSet.EMPTY); + redirContext.processUpdateForRedirection(fakeUpdate, source.getRowSet()); + } + + // do the actual computations + UpdateByBucketHelper[] dirtyBuckets = new UpdateByBucketHelper[] {zeroKeyUpdateBy}; + processBuckets(dirtyBuckets, true); + finalizeBuckets(dirtyBuckets); + } + + // private ColumnSource getCachedColumn(ColumnSource inputSource, final RowSet inputRowSet) { + // final SparseArrayColumnSource outputSource = SparseArrayColumnSource + // .getSparseMemoryColumnSource(inputSource.getType(), inputSource.getComponentType()); + // + // final int CHUNK_SIZE = 1 << 16; + // + // try (final RowSequence.Iterator rsIt = inputRowSet.getRowSequenceIterator(); + // final ChunkSink.FillFromContext ffc = + // outputSource.makeFillFromContext(CHUNK_SIZE); + // final ChunkSource.GetContext gc = inputSource.makeGetContext(CHUNK_SIZE)) { + // while (rsIt.hasMore()) { + // final RowSequence chunkOk = rsIt.getNextRowSequenceWithLength(CHUNK_SIZE); + // final Chunk values = inputSource.getChunk(gc, chunkOk); + // outputSource.fillFromChunk(ffc, values, chunkOk); + // } + // } + // + // return outputSource; + // } + // + // private int[] getAffectedWindows(UpdateByBucketHelper.UpdateContext context) { + // final TIntArrayList list = new TIntArrayList(windows.length); + // for (int winIdx = 0; winIdx < windows.length; winIdx++) { + // if (windows[winIdx].isAffected(context.windowContexts[winIdx])) { + // list.add(winIdx); + // } + // } + // return list.toArray(); + // } + // + // private void addRowSetToInputSourceCache(int srcIdx, final RowSet rowSet) { + // // create if it doesn't exist + // if (cachedInputSources[srcIdx] == null) { + // cachedInputSources[srcIdx] = new CachedInputSource(); + // } + // cachedInputSources[srcIdx].addRowSet(rowSet); + // } + // + // /** + // * Examine the buckets and identify the input sources that will benefit from caching. Accumulate the bucket + // rowsets + // * for each source independently + // */ + // private void computeCachedColumnContents(UpdateByBucketHelper[] buckets, boolean initialStep) { + // // track for each window what sources we need to cache + // final boolean[] windowSourceCacheNeeded = new boolean[inputSources.length]; + // + // for (int winIdx = 0; winIdx < windows.length; winIdx++) { + // Arrays.fill(windowSourceCacheNeeded, false); + // + // // for each bucket + // for (UpdateByBucketHelper bucket : buckets) { + // UpdateByWindow.UpdateByWindowContext bucketCtx = bucket.windowContexts[winIdx]; + // if (initialStep || (bucket.isDirty && bucketCtx.isDirty)) { + // for (int opIdx : windows[winIdx].getDirtyOperators(bucketCtx)) { + // for (int srcIdx : windows[winIdx].getOperatorSourceSlots(opIdx)) { + // if (inputSourceCacheNeeded[srcIdx]) { + // windowSourceCacheNeeded[srcIdx] = true; + // addRowSetToInputSourceCache(srcIdx, bucketCtx.getInfluencerRows()); + // } + // } + // } + // } + // } + // + // // add one to all the reference counts this windows + // for (int srcIdx = 0; srcIdx < inputSources.length; srcIdx++) { + // if (windowSourceCacheNeeded[srcIdx]) { + // cachedInputSources[srcIdx].addReference(); + // } + // } + // } + // } + // + // private void maybeCacheInputSources(int[] srcArr) { + // + // } + // + // private void maybeReleaseInputSources(int[] srcArr) { + // + // } + // + // /** + // * The Listener for apply an upstream {@link InstrumentedTableUpdateListenerAdapter#onUpdate(TableUpdate) update} + // */ + // class ZeroKeyUpdateByManagerListener extends InstrumentedTableUpdateListenerAdapter { + // public ZeroKeyUpdateByManagerListener(@Nullable String description, + // @NotNull final QueryTable source, + // @NotNull final QueryTable result) { + // super(description, source, false); + // + // } + // + // @Override + // public void onUpdate(TableUpdate upstream) { + // // do the actual computations + // processBuckets(false); + // + // + // + // final boolean windowsModified = ctx.anyModified(); + // + // if (upstream.modified().isNonempty() || windowsModified) { + // WritableRowSet modifiedRowSet = RowSetFactory.empty(); + // downstream.modified = modifiedRowSet; + // if (upstream.modified().isNonempty()) { + // // Transform any untouched modified columns to the output. + // transformer.clearAndTransform(upstream.modifiedColumnSet(), downstream.modifiedColumnSet); + // modifiedRowSet.insert(upstream.modified()); + // } + // + // + // + // if (windowsModified) { + // modifiedRowSet.remove(upstream.added()); + // } + // } else { + // downstream.modified = RowSetFactory.empty(); + // } + // + // // set the modified columns if any operators made changes (add/rem/modify) + // for (int winIdx = 0; winIdx < windows.length; winIdx++) { + // if (ctx.windowAffected[winIdx]) { + // ctx.windowContexts[winIdx].updateOutputModifiedColumnSet(downstream.modifiedColumnSet, + // windowOperatorOutputModifiedColumnSets[winIdx]); + // } + // } + // + // result.notifyListeners(downstream); + // } + // } + + + /** + * Perform an updateBy without any key columns. + * + * @param description the operation description + * @param source the source table + * @param operators the operations to perform + * @param resultSources the result sources + * @param redirContext the row redirection shared context + * @param control the control object. + * @return the result table + */ + public static Table compute(@NotNull final String description, + @NotNull final QueryTable source, + @NotNull final UpdateByOperator[] operators, + @NotNull final UpdateByWindow[] windows, + @NotNull final ColumnSource[] inputSources, + @NotNull final int[][] operatorInputSourceSlots, + @NotNull final Map> resultSources, + @Nullable final String timestampColumnName, + @NotNull final UpdateByRedirectionContext redirContext, + @NotNull final UpdateByControl control) { + + final ZeroKeyUpdateByManager manager = new ZeroKeyUpdateByManager(description, source, operators, windows, + inputSources, operatorInputSourceSlots, resultSources, timestampColumnName, redirContext, control); + return manager.result; + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java index f2331bcc8e3..5d9ec89b56a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java @@ -1,5 +1,7 @@ package io.deephaven.engine.table.impl.updateby; +import gnu.trove.list.array.TIntArrayList; +import gnu.trove.set.hash.TIntHashSet; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; @@ -19,16 +21,21 @@ public abstract class UpdateByWindow { @Nullable protected final String timestampColumnName; + // store the operators for this window protected final UpdateByOperator[] operators; + // store the index in the {@link UpdateBy.inputSources} protected final int[][] operatorInputSourceSlots; - // individual input modifiedColumnSets for the operators - protected final ModifiedColumnSet[] operatorInputModifiedColumnSets; - - protected boolean trackModifications; + /** This context will store the necessary info to process a single window for a single bucket */ public abstract class UpdateByWindowContext implements SafeCloseable { + /** Indicates this bucket window needs to be processed */ + protected boolean isDirty; + + /** Indicates this operator needs to be processed */ + protected final boolean[] operatorIsDirty; + /** store a reference to the source rowset */ protected final TrackingRowSet sourceRowSet; @@ -40,9 +47,6 @@ public abstract class UpdateByWindowContext implements SafeCloseable { @Nullable protected final LongSegmentedSortedArray timestampSsa; - /** An array of boolean denoting which operators are affected by the current update. */ - protected final boolean[] opAffected; - /** An array of context objects for each underlying operator */ protected final UpdateByOperator.UpdateContext[] opContext; @@ -65,10 +69,6 @@ public abstract class UpdateByWindowContext implements SafeCloseable { /** the rows that contain values used to compute affected row values */ protected RowSet influencerRows; - /** keep track of what rows were modified (we'll use a single set for all operators sharing a window) */ - protected RowSetBuilderSequential modifiedBuilder; - protected RowSet newModified; - protected int workingChunkSize; public UpdateByWindowContext(final TrackingRowSet sourceRowSet, final ColumnSource[] inputSources, @@ -79,7 +79,7 @@ public UpdateByWindowContext(final TrackingRowSet sourceRowSet, final ColumnSour this.timestampColumnSource = timestampColumnSource; this.timestampSsa = timestampSsa; - this.opAffected = new boolean[operators.length]; + this.operatorIsDirty = new boolean[operators.length]; this.opContext = new UpdateByOperator.UpdateContext[operators.length]; this.inputSourceGetContexts = new ChunkSource.GetContext[inputSources.length]; this.inputSourceChunkPopulated = new boolean[inputSources.length]; @@ -88,48 +88,32 @@ public UpdateByWindowContext(final TrackingRowSet sourceRowSet, final ColumnSour this.workingChunkSize = chunkSize; this.initialStep = initialStep; + this.isDirty = false; } - public abstract boolean computeAffectedRowsAndOperators(@NotNull final TableUpdate upstream); - public abstract void processRows(); - protected abstract void makeOperatorContexts(); - - public boolean anyModified() { - return newModified != null && newModified.isNonempty(); - } - - public RowSet getModifiedRows() { - return newModified; - } - - public void updateOutputModifiedColumnSet(ModifiedColumnSet outputModifiedColumnSet, - ModifiedColumnSet[] operatorOutputModifiedColumnSets) { - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - outputModifiedColumnSet.setAll(operatorOutputModifiedColumnSets[opIdx]); - } - } - } - - public RowSet getAffectedRows() { - return affectedRows; - } - - public RowSet getInfluencerRows() { - return influencerRows; - } - - protected void prepareValuesChunkForSource(final int srcIdx, final RowSequence rs) { - if (rs.isEmpty()) { - return; - } - if (!inputSourceChunkPopulated[srcIdx]) { - inputSourceChunks[srcIdx] = inputSources[srcIdx].getChunk(inputSourceGetContexts[srcIdx], rs); - inputSourceChunkPopulated[srcIdx] = true; - } - } + // public boolean anyModified() { + // return newModified != null && newModified.isNonempty(); + // } + // + // public void updateOutputModifiedColumnSet(ModifiedColumnSet outputModifiedColumnSet, + // ModifiedColumnSet[] operatorOutputModifiedColumnSets) { + // for (int opIdx = 0; opIdx < operators.length; opIdx++) { + // if (operatorIsDirty[opIdx]) { + // outputModifiedColumnSet.setAll(operatorOutputModifiedColumnSets[opIdx]); + // } + // } + // } + // + // public RowSet getAffectedRows() { + // return affectedRows; + // } + // + // public RowSet getInfluencerRows() { + // return influencerRows; + // } + // @Override public void close() { @@ -138,11 +122,10 @@ public void close() { influencerRows.close(); influencerRows = null; } - try (final RowSet ignoredRs1 = affectedRows; - final RowSet ignoredRs2 = newModified) { + try (final RowSet ignoredRs1 = affectedRows) { } for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { + if (opContext[opIdx] != null) { opContext[opIdx].close(); } } @@ -162,22 +145,11 @@ public abstract UpdateByWindowContext makeWindowContext(final TrackingRowSet sou final int chunkSize, final boolean isInitializeStep); - public void startTrackingModifications(@NotNull final QueryTable source, @NotNull final QueryTable result) { - trackModifications = true; - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - operatorInputModifiedColumnSets[opIdx] = - source.newModifiedColumnSet(operators[opIdx].getAffectingColumnNames()); - } - } - protected UpdateByWindow(UpdateByOperator[] operators, int[][] operatorInputSourceSlots, @Nullable String timestampColumnName) { this.operators = operators; this.operatorInputSourceSlots = operatorInputSourceSlots; this.timestampColumnName = timestampColumnName; - - operatorInputModifiedColumnSets = new ModifiedColumnSet[operators.length]; - trackModifications = false; } public static UpdateByWindow createFromOperatorArray(final UpdateByOperator[] operators, @@ -220,6 +192,43 @@ public UpdateByOperator[] getOperators() { return operators; } + // region context-based functions + + public abstract void computeAffectedRowsAndOperators(final UpdateByWindowContext context, + @NotNull final TableUpdate upstream); + + protected abstract void makeOperatorContexts(final UpdateByWindowContext context); + + protected void prepareValuesChunkForSource(final UpdateByWindowContext context, final int srcIdx, + final RowSequence rs) { + if (rs.isEmpty()) { + return; + } + if (!context.inputSourceChunkPopulated[srcIdx]) { + context.inputSourceChunks[srcIdx] = + context.inputSources[srcIdx].getChunk(context.inputSourceGetContexts[srcIdx], rs); + context.inputSourceChunkPopulated[srcIdx] = true; + } + } + + public abstract void processRows(final UpdateByWindowContext context, + final ColumnSource[] inputSources, + final boolean initialStep); + + public boolean isWindowDirty(final UpdateByWindowContext context) { + return context.isDirty; + } + + public boolean isOperatorDirty(final UpdateByWindowContext context, int winOpIdx) { + return context.operatorIsDirty[winOpIdx]; + } + + public RowSet getModifiedRows(final UpdateByWindowContext context) { + return context.affectedRows; + } + + // endregion + protected static int hashCode(boolean windowed, @NotNull String[] inputColumnNames, @Nullable String timestampColumnName, long prevUnits, long fwdUnits) { @@ -252,10 +261,7 @@ public static int hashCodeFromOperator(final UpdateByOperator op) { } public static boolean isEquivalentWindow(final UpdateByOperator opA, final UpdateByOperator opB) { - final boolean aWindowed = opA instanceof UpdateByWindowedOperator; - final boolean bWindowed = opB instanceof UpdateByWindowedOperator; - - // verify input columns + // verify input columns match String[] opAInput = opA.getInputColumnNames(); String[] opBInput = opB.getInputColumnNames(); @@ -268,6 +274,9 @@ public static boolean isEquivalentWindow(final UpdateByOperator opA, final Updat } } + final boolean aWindowed = opA instanceof UpdateByWindowedOperator; + final boolean bWindowed = opB instanceof UpdateByWindowedOperator; + // equivalent if both are cumulative, not equivalent if only one is cumulative if (!aWindowed && !bWindowed) { return true; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java index 7667c7cecc8..7ec06488c6d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java @@ -21,205 +21,188 @@ // this class is currently too big, should specialize into CumWindow, TickWindow, TimeWindow to simplify implementation public class UpdateByWindowCumulative extends UpdateByWindow { - public class UpdateByWindowCumulativeContext extends UpdateByWindowContext { - public UpdateByWindowCumulativeContext(final TrackingRowSet sourceRowSet, final ColumnSource[] inputSources, - @Nullable final ColumnSource timestampColumnSource, - @Nullable final LongSegmentedSortedArray timestampSsa, final int chunkSize, final boolean initialStep) { - super(sourceRowSet, inputSources, timestampColumnSource, timestampSsa, chunkSize, initialStep); - } - @Override - public void close() { - super.close(); - } + public UpdateByWindowCumulative(UpdateByOperator[] operators, int[][] operatorSourceSlots, + @Nullable String timestampColumnName) { + super(operators, operatorSourceSlots, timestampColumnName); + } - @Override - protected void makeOperatorContexts() { - // use this to make which input sources are initialized - Arrays.fill(inputSourceChunkPopulated, false); - - // working chunk size need not be larger than affectedRows.size() - workingChunkSize = Math.min(workingChunkSize, affectedRows.intSize()); - - // create contexts for the affected operators - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - // create the fill contexts for the input sources - final int[] sourceIndices = operatorInputSourceSlots[opIdx]; - final ColumnSource[] inputSourceArr = new ColumnSource[sourceIndices.length]; - for (int ii = 0; ii < sourceIndices.length; ii++) { - int sourceSlot = sourceIndices[ii]; - if (!inputSourceChunkPopulated[sourceSlot]) { - inputSourceGetContexts[sourceSlot] = - inputSources[sourceSlot].makeGetContext(workingChunkSize); - inputSourceChunkPopulated[sourceSlot] = true; - } - inputSourceArr[ii] = inputSources[sourceSlot]; + @Override + protected void makeOperatorContexts(UpdateByWindowContext context) { + // use this to make which input sources are initialized + Arrays.fill(context.inputSourceChunkPopulated, false); + + // working chunk size need not be larger than affectedRows.size() + context.workingChunkSize = Math.min(context.workingChunkSize, context.affectedRows.intSize()); + + // create contexts for the affected operators + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (context.operatorIsDirty[opIdx]) { + // create the fill contexts for the input sources + final int[] sourceIndices = operatorInputSourceSlots[opIdx]; + final ColumnSource[] inputSourceArr = new ColumnSource[sourceIndices.length]; + for (int ii = 0; ii < sourceIndices.length; ii++) { + int sourceSlot = sourceIndices[ii]; + if (!context.inputSourceChunkPopulated[sourceSlot]) { + context.inputSourceGetContexts[sourceSlot] = + context.inputSources[sourceSlot].makeGetContext(context.workingChunkSize); + context.inputSourceChunkPopulated[sourceSlot] = true; } - opContext[opIdx] = operators[opIdx].makeUpdateContext(workingChunkSize, inputSourceArr); + inputSourceArr[ii] = context.inputSources[sourceSlot]; } + context.opContext[opIdx] = operators[opIdx].makeUpdateContext(context.workingChunkSize, inputSourceArr); } } + } - @Override - public boolean computeAffectedRowsAndOperators(@NotNull final TableUpdate upstream) { - // all rows are affected on the initial step - if (initialStep) { - affectedRows = sourceRowSet.copy(); - influencerRows = affectedRows; + public UpdateByWindowContext makeWindowContext(final TrackingRowSet sourceRowSet, + final ColumnSource[] inputSources, + final ColumnSource timestampColumnSource, + final LongSegmentedSortedArray timestampSsa, + final int chunkSize, + final boolean isInitializeStep) { + return new UpdateByWindowContext(sourceRowSet, inputSources, timestampColumnSource, timestampSsa, + chunkSize, + isInitializeStep) {}; + } - // mark all operators as affected by this update - Arrays.fill(opAffected, true); + @Override + public void computeAffectedRowsAndOperators(UpdateByWindowContext context, @NotNull TableUpdate upstream) { + // all rows are affected on the initial step + if (context.initialStep) { + context.affectedRows = context.sourceRowSet.copy(); + context.influencerRows = context.affectedRows; - makeOperatorContexts(); - return true; - } + // mark all operators as affected by this update + Arrays.fill(context.operatorIsDirty, true); - // determine which operators are affected by this update - boolean anyAffected = false; - boolean allAffected = upstream.added().isNonempty() || - upstream.removed().isNonempty(); - - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - opAffected[opIdx] = allAffected - || (upstream.modifiedColumnSet().nonempty() && (operatorInputModifiedColumnSets[opIdx] == null - || upstream.modifiedColumnSet().containsAny(operatorInputModifiedColumnSets[opIdx]))); - if (opAffected[opIdx]) { - anyAffected = true; - } - } + makeOperatorContexts(context); + context.isDirty = true; + return; + } - if (!anyAffected) { - return false; + // determine which operators are affected by this update + context.isDirty = false; + boolean allAffected = upstream.added().isNonempty() || + upstream.removed().isNonempty(); + + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + context.operatorIsDirty[opIdx] = allAffected + || (upstream.modifiedColumnSet().nonempty() && (operators[opIdx].getInputModifiedColumnSet() == null + || upstream.modifiedColumnSet().containsAny(operators[opIdx].getInputModifiedColumnSet()))); + if (context.operatorIsDirty[opIdx]) { + context.isDirty = true; } + } - long smallestModifiedKey = smallestAffectedKey(upstream.added(), upstream.modified(), upstream.removed(), - upstream.shifted(), sourceRowSet); + if (!context.isDirty) { + return; + } - affectedRows = smallestModifiedKey == Long.MAX_VALUE - ? RowSetFactory.empty() - : sourceRowSet.subSetByKeyRange(smallestModifiedKey, sourceRowSet.lastRowKey()); - influencerRows = affectedRows; + long smallestModifiedKey = smallestAffectedKey(upstream.added(), upstream.modified(), upstream.removed(), + upstream.shifted(), context.sourceRowSet); - makeOperatorContexts(); - return true; - } + context.affectedRows = smallestModifiedKey == Long.MAX_VALUE + ? RowSetFactory.empty() + : context.sourceRowSet.subSetByKeyRange(smallestModifiedKey, context.sourceRowSet.lastRowKey()); + context.influencerRows = context.affectedRows; - @Override - public void processRows() { - if (trackModifications) { - modifiedBuilder = RowSetFactory.builderSequential(); - } + makeOperatorContexts(context); + } - // find the key before the first affected row - final long keyBefore; - try (final RowSet.SearchIterator rIt = sourceRowSet.reverseIterator()) { - rIt.advance(affectedRows.firstRowKey()); - if (rIt.hasNext()) { - keyBefore = rIt.nextLong(); - } else { - keyBefore = NULL_ROW_KEY; - } + + @Override + public void processRows(UpdateByWindowContext context, final ColumnSource[] inputSources, + final boolean initialStep) { + // find the key before the first affected row + final long keyBefore; + try (final RowSet.SearchIterator rIt = context.sourceRowSet.reverseIterator()) { + rIt.advance(context.affectedRows.firstRowKey()); + if (rIt.hasNext()) { + keyBefore = rIt.nextLong(); + } else { + keyBefore = NULL_ROW_KEY; } + } - // and preload that data for these operators - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - UpdateByCumulativeOperator cumOp = (UpdateByCumulativeOperator) operators[opIdx]; - if (cumOp.getTimestampColumnName() == null || keyBefore == NULL_ROW_KEY) { - // this operator doesn't care about timestamps or we know we are at the beginning of the rowset - cumOp.initializeUpdate(opContext[opIdx], keyBefore, NULL_LONG); - } else { - // this operator cares about timestamps, so make sure it is starting from a valid value and - // valid timestamp by moving backward until the conditions are met - UpdateByCumulativeOperator.Context cumOpContext = - (UpdateByCumulativeOperator.Context) opContext[opIdx]; - long potentialResetTimestamp = timestampColumnSource.getLong(keyBefore); - - if (potentialResetTimestamp == NULL_LONG || !cumOpContext.isValueValid(keyBefore)) { - try (final RowSet.SearchIterator rIt = sourceRowSet.reverseIterator()) { - if (rIt.advance(keyBefore)) { - while (rIt.hasNext()) { - final long nextKey = rIt.nextLong(); - potentialResetTimestamp = timestampColumnSource.getLong(nextKey); - if (potentialResetTimestamp != NULL_LONG && - cumOpContext.isValueValid(nextKey)) { - break; - } + // and preload that data for these operators + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (context.operatorIsDirty[opIdx]) { + UpdateByCumulativeOperator cumOp = (UpdateByCumulativeOperator) operators[opIdx]; + if (cumOp.getTimestampColumnName() == null || keyBefore == NULL_ROW_KEY) { + // this operator doesn't care about timestamps or we know we are at the beginning of the rowset + cumOp.initializeUpdate(context.opContext[opIdx], keyBefore, NULL_LONG); + } else { + // this operator cares about timestamps, so make sure it is starting from a valid value and + // valid timestamp by moving backward until the conditions are met + UpdateByCumulativeOperator.Context cumOpContext = + (UpdateByCumulativeOperator.Context) context.opContext[opIdx]; + long potentialResetTimestamp = context.timestampColumnSource.getLong(keyBefore); + + if (potentialResetTimestamp == NULL_LONG || !cumOpContext.isValueValid(keyBefore)) { + try (final RowSet.SearchIterator rIt = context.sourceRowSet.reverseIterator()) { + if (rIt.advance(keyBefore)) { + while (rIt.hasNext()) { + final long nextKey = rIt.nextLong(); + potentialResetTimestamp = context.timestampColumnSource.getLong(nextKey); + if (potentialResetTimestamp != NULL_LONG && + cumOpContext.isValueValid(nextKey)) { + break; } } } } - // call the specialized version of `intializeUpdate()` for these operators - cumOp.initializeUpdate(opContext[opIdx], keyBefore, potentialResetTimestamp); } + // call the specialized version of `intializeUpdate()` for these operators + cumOp.initializeUpdate(context.opContext[opIdx], keyBefore, potentialResetTimestamp); } } + } - try (final RowSequence.Iterator it = affectedRows.getRowSequenceIterator(); - ChunkSource.GetContext tsGetCtx = - timestampColumnSource == null ? null - : timestampColumnSource.makeGetContext(workingChunkSize)) { - while (it.hasMore()) { - final RowSequence rs = it.getNextRowSequenceWithLength(workingChunkSize); - final int size = rs.intSize(); - Arrays.fill(inputSourceChunkPopulated, false); - - // create the timestamp chunk if needed - LongChunk tsChunk = timestampColumnSource == null ? null - : timestampColumnSource.getChunk(tsGetCtx, rs).asLongChunk(); - - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - // prep the chunk array needed by the accumulate call - final int[] srcIndices = operatorInputSourceSlots[opIdx]; - Chunk[] chunkArr = new Chunk[srcIndices.length]; - for (int ii = 0; ii < srcIndices.length; ii++) { - int srcIdx = srcIndices[ii]; - // chunk prep - prepareValuesChunkForSource(srcIdx, rs); - chunkArr[ii] = inputSourceChunks[srcIdx]; - } - - // make the specialized call for cumulative operators - ((UpdateByCumulativeOperator.Context) opContext[opIdx]).accumulate( - rs, - chunkArr, - tsChunk, - size); + try (final RowSequence.Iterator it = context.affectedRows.getRowSequenceIterator(); + ChunkSource.GetContext tsGetCtx = + context.timestampColumnSource == null ? null + : context.timestampColumnSource.makeGetContext(context.workingChunkSize)) { + while (it.hasMore()) { + final RowSequence rs = it.getNextRowSequenceWithLength(context.workingChunkSize); + final int size = rs.intSize(); + Arrays.fill(context.inputSourceChunkPopulated, false); + + // create the timestamp chunk if needed + LongChunk tsChunk = context.timestampColumnSource == null ? null + : context.timestampColumnSource.getChunk(tsGetCtx, rs).asLongChunk(); + + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (context.operatorIsDirty[opIdx]) { + // prep the chunk array needed by the accumulate call + final int[] srcIndices = operatorInputSourceSlots[opIdx]; + Chunk[] chunkArr = new Chunk[srcIndices.length]; + for (int ii = 0; ii < srcIndices.length; ii++) { + int srcIdx = srcIndices[ii]; + // chunk prep + prepareValuesChunkForSource(context, srcIdx, rs); + chunkArr[ii] = context.inputSourceChunks[srcIdx]; } - } - // all these rows were modified - if (modifiedBuilder != null) { - modifiedBuilder.appendRowSequence(rs); + // make the specialized call for cumulative operators + ((UpdateByCumulativeOperator.Context) context.opContext[opIdx]).accumulate( + rs, + chunkArr, + tsChunk, + size); } } } + } - // call `finishUpdate()` function for each operator - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - operators[opIdx].finishUpdate(opContext[opIdx]); - } - } - - if (trackModifications) { - newModified = modifiedBuilder.build(); + // call `finishUpdate()` function for each operator + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (context.operatorIsDirty[opIdx]) { + operators[opIdx].finishUpdate(context.opContext[opIdx]); } } } - public UpdateByWindowContext makeWindowContext(final TrackingRowSet sourceRowSet, - final ColumnSource[] inputSources, - final ColumnSource timestampColumnSource, - final LongSegmentedSortedArray timestampSsa, - final int chunkSize, - final boolean isInitializeStep) { - return new UpdateByWindowCumulativeContext(sourceRowSet, inputSources, timestampColumnSource, timestampSsa, - chunkSize, - isInitializeStep); - } /** * Find the smallest valued key that participated in the upstream {@link TableUpdate}. @@ -294,9 +277,4 @@ private static long smallestAffectedKey(@NotNull final RowSet added, return smallestModifiedKey; } - - public UpdateByWindowCumulative(UpdateByOperator[] operators, int[][] operatorSourceSlots, - @Nullable String timestampColumnName) { - super(operators, operatorSourceSlots, timestampColumnName); - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java index 1ffc1856392..41a708d0809 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java @@ -66,367 +66,352 @@ public void close() { // leveraging try with resources to auto-close } } + } - @Override - protected void makeOperatorContexts() { - // use this to determine which input sources are initialized - Arrays.fill(inputSourceChunkPopulated, false); - - // create contexts for the affected operators - currentGetContextSize = WINDOW_CHUNK_SIZE; + public UpdateByWindowContext makeWindowContext(final TrackingRowSet sourceRowSet, + final ColumnSource[] inputSources, + final ColumnSource timestampColumnSource, + final LongSegmentedSortedArray timestampSsa, + final int chunkSize, + final boolean isInitializeStep) { + return new UpdateByWindowTicksContext(sourceRowSet, inputSources, timestampColumnSource, timestampSsa, + chunkSize, + isInitializeStep); + } - // working chunk size need not be larger than affectedRows.size() - workingChunkSize = Math.min(workingChunkSize, affectedRows.intSize()); + private static WritableRowSet computeAffectedRowsTicks(final RowSet sourceSet, final RowSet subset, + final RowSet invertedSubSet, long revTicks, long fwdTicks) { + // swap fwd/rev to get the influencer windows + return computeInfluencerRowsTicks(sourceSet, subset, invertedSubSet, fwdTicks, revTicks); + } - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - if (opAffected[opIdx]) { - // create the fill contexts for the input sources - final int[] sourceIndices = operatorInputSourceSlots[opIdx]; - final ColumnSource[] inputSourceArr = new ColumnSource[sourceIndices.length]; - for (int ii = 0; ii < sourceIndices.length; ii++) { - int sourceSlot = sourceIndices[ii]; - if (!inputSourceChunkPopulated[sourceSlot]) { - inputSourceGetContexts[sourceSlot] = - inputSources[sourceSlot].makeGetContext(currentGetContextSize); - inputSourceChunkPopulated[sourceSlot] = true; - } - inputSourceArr[ii] = inputSources[sourceSlot]; - } - opContext[opIdx] = operators[opIdx].makeUpdateContext(workingChunkSize, inputSourceArr); - } - } - } + private static WritableRowSet computeInfluencerRowsTicks(final RowSet sourceSet, final RowSet subset, + final RowSet invertedSubSet, long revTicks, long fwdTicks) { + if (sourceSet.size() == subset.size()) { + return sourceSet.copy(); } - protected void ensureGetContextSize(long newSize) { - if (currentGetContextSize < newSize) { - long size = currentGetContextSize; - while (size < newSize) { - size *= 2; - } - currentGetContextSize = LongSizedDataStructure.intSize( - "ensureGetContextSize exceeded Integer.MAX_VALUE", - size); - - // use this to determine which input sources are initialized - Arrays.fill(inputSourceChunkPopulated, false); - - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - final int[] sourceIndices = operatorInputSourceSlots[opIdx]; - for (int sourceSlot : sourceIndices) { - if (!inputSourceChunkPopulated[sourceSlot]) { - // close the existing context - inputSourceGetContexts[sourceSlot].close(); - - // create a new context of the larger size - inputSourceGetContexts[sourceSlot] = - inputSources[sourceSlot].makeGetContext(currentGetContextSize); - inputSourceChunkPopulated[sourceSlot] = true; - } - } - } - } - } - } + long maxPos = sourceSet.size() - 1; - // windowed by time/ticks is more complex to compute: find all the changed rows and the rows that would - // be affected by the changes (includes newly added rows) and need to be recomputed. Then include all - // the rows that are affected by deletions (if any). After the affected rows have been identified, - // determine which rows will be needed to compute new values for the affected rows (influencer rows) - @Override - public boolean computeAffectedRowsAndOperators(@NotNull final TableUpdate upstream) { - // all rows are affected on the initial step - if (initialStep) { - affectedRows = sourceRowSet.copy(); - influencerRows = affectedRows; + final RowSetBuilderSequential builder = RowSetFactory.builderSequential(); + final MutableLong minPos = new MutableLong(0L); - // no need to invert, just create a flat rowset - affectedRowPositions = RowSetFactory.flat(sourceRowSet.size()); - influencerPositions = RowSetFactory.flat(sourceRowSet.size()); + invertedSubSet.forAllRowKeyRanges((s, e) -> { + long sPos = Math.max(s - revTicks, minPos.longValue()); + long ePos = Math.min(e + fwdTicks, maxPos); + builder.appendRange(sPos, ePos); + minPos.setValue(ePos + 1); + }); - // mark all operators as affected by this update - Arrays.fill(opAffected, true); + try (final RowSet positions = builder.build()) { + return sourceSet.subSetForPositions(positions); + } + } - makeOperatorContexts(); - return true; + private void ensureGetContextSize(UpdateByWindowTicksContext ctx, long newSize) { + if (ctx.currentGetContextSize < newSize) { + long size = ctx.currentGetContextSize; + while (size < newSize) { + size *= 2; } + ctx.currentGetContextSize = LongSizedDataStructure.intSize( + "ensureGetContextSize exceeded Integer.MAX_VALUE", + size); - // determine which operators are affected by this update - boolean anyAffected = false; - boolean allAffected = upstream.added().isNonempty() || - upstream.removed().isNonempty(); + // use this to determine which input sources are initialized + Arrays.fill(ctx.inputSourceChunkPopulated, false); for (int opIdx = 0; opIdx < operators.length; opIdx++) { - opAffected[opIdx] = allAffected - || (upstream.modifiedColumnSet().nonempty() && (operatorInputModifiedColumnSets[opIdx] == null - || upstream.modifiedColumnSet().containsAny(operatorInputModifiedColumnSets[opIdx]))); - if (opAffected[opIdx]) { - anyAffected = true; + if (ctx.operatorIsDirty[opIdx]) { + final int[] sourceIndices = operatorInputSourceSlots[opIdx]; + for (int sourceSlot : sourceIndices) { + if (!ctx.inputSourceChunkPopulated[sourceSlot]) { + // close the existing context + ctx.inputSourceGetContexts[sourceSlot].close(); + + // create a new context of the larger size + ctx.inputSourceGetContexts[sourceSlot] = + ctx.inputSources[sourceSlot].makeGetContext(ctx.currentGetContextSize); + ctx.inputSourceChunkPopulated[sourceSlot] = true; + } + } } } + } + } - if (!anyAffected) { - return false; - } + /*** + * This function takes care of loading/preparing the next set of influencer data, in this case we load the next + * chunk of key and position data and reset the index + */ + private void loadNextInfluencerChunks(UpdateByWindowTicksContext ctx) { + if (!ctx.influencerIt.hasMore()) { + ctx.nextInfluencerPos = Integer.MAX_VALUE; + ctx.nextInfluencerKey = Long.MAX_VALUE; + return; + } - // changed rows are all mods+adds - WritableRowSet changed = upstream.added().union(upstream.modified()); + final RowSequence influencerRs = + ctx.influencerIt.getNextRowSequenceWithLength(UpdateByWindowTicksContext.WINDOW_CHUNK_SIZE); + ctx.influencerKeyChunk = influencerRs.asRowKeyChunk(); - // need a writable rowset - WritableRowSet tmpAffected; + final RowSequence influencePosRs = + ctx.influencerPosIt.getNextRowSequenceWithLength(UpdateByWindowTicksContext.WINDOW_CHUNK_SIZE); + ctx.influencerPosChunk = influencePosRs.asRowKeyChunk(); - // compute the rows affected from these changes - try (final WritableRowSet changedInverted = sourceRowSet.invert(changed)) { - tmpAffected = computeAffectedRowsTicks(sourceRowSet, changed, changedInverted, prevUnits, fwdUnits); - } + Assert.eqTrue(influencePosRs.lastRowKey() < Integer.MAX_VALUE, + "updateBy window positions exceeded maximum size"); - // other rows can be affected by removes - if (upstream.removed().isNonempty()) { - try (final RowSet prev = sourceRowSet.copyPrev(); - final RowSet removedPositions = prev.invert(upstream.removed()); - final WritableRowSet affectedByRemoves = - computeAffectedRowsTicks(prev, upstream.removed(), removedPositions, prevUnits, - fwdUnits)) { - // apply shifts to get back to pos-shift space - upstream.shifted().apply(affectedByRemoves); - // retain only the rows that still exist in the sourceRowSet - affectedByRemoves.retain(sourceRowSet); - tmpAffected.insert(affectedByRemoves); - } + ctx.influencerPosChunkSize = ctx.influencerPosChunk.size(); + + ctx.nextInfluencerIndex = 0; + ctx.nextInfluencerPos = LongSizedDataStructure.intSize( + "updateBy window positions exceeded maximum size", + ctx.influencerPosChunk.get(ctx.nextInfluencerIndex)); + ctx.nextInfluencerKey = ctx.influencerKeyChunk.get(ctx.nextInfluencerIndex); + } + + // windowed by time/ticks is more complex to compute: find all the changed rows and the rows that would + // be affected by the changes (includes newly added rows) and need to be recomputed. Then include all + // the rows that are affected by deletions (if any). After the affected rows have been identified, + // determine which rows will be needed to compute new values for the affected rows (influencer rows) + @Override + public void computeAffectedRowsAndOperators(UpdateByWindowContext context, @NotNull TableUpdate upstream) { + UpdateByWindowTicksContext ctx = (UpdateByWindowTicksContext) context; + + // all rows are affected on the initial step + if (ctx.initialStep) { + ctx.affectedRows = ctx.sourceRowSet.copy(); + ctx.influencerRows = ctx.affectedRows; + + // no need to invert, just create a flat rowset + ctx.affectedRowPositions = RowSetFactory.flat(ctx.sourceRowSet.size()); + ctx.influencerPositions = RowSetFactory.flat(ctx.sourceRowSet.size()); + + // mark all operators as affected by this update + Arrays.fill(ctx.operatorIsDirty, true); + + makeOperatorContexts(ctx); + ctx.isDirty = true; + return; + } + + // determine which operators are affected by this update + ctx.isDirty = false; + boolean allAffected = upstream.added().isNonempty() || + upstream.removed().isNonempty(); + + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + ctx.operatorIsDirty[opIdx] = allAffected + || (upstream.modifiedColumnSet().nonempty() && (operators[opIdx].getInputModifiedColumnSet() == null + || upstream.modifiedColumnSet().containsAny(operators[opIdx].getInputModifiedColumnSet()))); + if (ctx.operatorIsDirty[opIdx]) { + ctx.isDirty = true; } + } - affectedRows = tmpAffected; + if (!ctx.isDirty) { + return; + } - // now get influencer rows for the affected rows - // generate position data rowsets for efficiently computed position offsets - affectedRowPositions = sourceRowSet.invert(affectedRows); + // changed rows are all mods+adds + WritableRowSet changed = upstream.added().union(upstream.modified()); - influencerRows = computeInfluencerRowsTicks(sourceRowSet, affectedRows, affectedRowPositions, prevUnits, - fwdUnits); - influencerPositions = sourceRowSet.invert(influencerRows); + // need a writable rowset + WritableRowSet tmpAffected; - makeOperatorContexts(); - return true; + // compute the rows affected from these changes + try (final WritableRowSet changedInverted = ctx.sourceRowSet.invert(changed)) { + tmpAffected = computeAffectedRowsTicks(ctx.sourceRowSet, changed, changedInverted, prevUnits, fwdUnits); } - /*** - * This function takes care of loading/preparing the next set of influencer data, in this case we load the next - * chunk of key and position data and reset the index - */ - private void loadNextInfluencerChunks() { - if (!influencerIt.hasMore()) { - nextInfluencerPos = Integer.MAX_VALUE; - nextInfluencerKey = Long.MAX_VALUE; - return; + // other rows can be affected by removes + if (upstream.removed().isNonempty()) { + try (final RowSet prev = ctx.sourceRowSet.copyPrev(); + final RowSet removedPositions = prev.invert(upstream.removed()); + final WritableRowSet affectedByRemoves = + computeAffectedRowsTicks(prev, upstream.removed(), removedPositions, prevUnits, + fwdUnits)) { + // apply shifts to get back to pos-shift space + upstream.shifted().apply(affectedByRemoves); + // retain only the rows that still exist in the sourceRowSet + affectedByRemoves.retain(ctx.sourceRowSet); + tmpAffected.insert(affectedByRemoves); } + } - final RowSequence influencerRs = influencerIt.getNextRowSequenceWithLength(WINDOW_CHUNK_SIZE); - influencerKeyChunk = influencerRs.asRowKeyChunk(); - - final RowSequence influencePosRs = influencerPosIt.getNextRowSequenceWithLength(WINDOW_CHUNK_SIZE); - influencerPosChunk = influencePosRs.asRowKeyChunk(); + ctx.affectedRows = tmpAffected; - Assert.eqTrue(influencePosRs.lastRowKey() < Integer.MAX_VALUE, - "updateBy window positions exceeded maximum size"); + // now get influencer rows for the affected rows + // generate position data rowsets for efficiently computed position offsets + ctx.affectedRowPositions = ctx.sourceRowSet.invert(ctx.affectedRows); - influencerPosChunkSize = influencerPosChunk.size(); + ctx.influencerRows = computeInfluencerRowsTicks(ctx.sourceRowSet, ctx.affectedRows, ctx.affectedRowPositions, + prevUnits, fwdUnits); + ctx.influencerPositions = ctx.sourceRowSet.invert(ctx.influencerRows); - nextInfluencerIndex = 0; - nextInfluencerPos = LongSizedDataStructure.intSize( - "updateBy window positions exceeded maximum size", - influencerPosChunk.get(nextInfluencerIndex)); - nextInfluencerKey = influencerKeyChunk.get(nextInfluencerIndex); - } + makeOperatorContexts(ctx); - /*** - * This function process the affected rows chunkwise, and will advance the moving window (which is the same for - * all operators in this collection). For each row in the dataset the sliding window will adjust and - * instructions for pushing/popping data will be created for the operators. For each chunk of `affected` rows, - * we will identify exactly which `influencer` rows are needed and will provide those and the push/pop - * instructions to the operators. - * - * Downstream operators should manage local storage in a RingBuffer or other efficient structure since our pop() - * calls do not provide the popped data - */ - public void processRows() { - if (trackModifications) { - modifiedBuilder = RowSetFactory.builderSequential(); - } + } - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - UpdateByWindowedOperator winOp = (UpdateByWindowedOperator) operators[opIdx]; - // call the specialized version of `intializeUpdate()` for these operators - winOp.initializeUpdate(opContext[opIdx]); + @Override + protected void makeOperatorContexts(UpdateByWindowContext context) { + UpdateByWindowTicksContext ctx = (UpdateByWindowTicksContext) context; + + // use this to determine which input sources are initialized + Arrays.fill(ctx.inputSourceChunkPopulated, false); + + // create contexts for the affected operators + ctx.currentGetContextSize = UpdateByWindowTicksContext.WINDOW_CHUNK_SIZE; + + // working chunk size need not be larger than affectedRows.size() + ctx.workingChunkSize = Math.min(ctx.workingChunkSize, ctx.affectedRows.intSize()); + + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (ctx.operatorIsDirty[opIdx]) { + // create the fill contexts for the input sources + final int[] sourceIndices = operatorInputSourceSlots[opIdx]; + final ColumnSource[] inputSourceArr = new ColumnSource[sourceIndices.length]; + for (int ii = 0; ii < sourceIndices.length; ii++) { + int sourceSlot = sourceIndices[ii]; + if (!ctx.inputSourceChunkPopulated[sourceSlot]) { + ctx.inputSourceGetContexts[sourceSlot] = + ctx.inputSources[sourceSlot].makeGetContext(ctx.currentGetContextSize); + ctx.inputSourceChunkPopulated[sourceSlot] = true; + } + inputSourceArr[ii] = ctx.inputSources[sourceSlot]; } + ctx.opContext[opIdx] = operators[opIdx].makeUpdateContext(ctx.workingChunkSize, inputSourceArr); } + } + } - influencerIt = influencerRows.getRowSequenceIterator(); - influencerPosIt = influencerPositions.getRowSequenceIterator(); - - try (final RowSequence.Iterator it = affectedRows.getRowSequenceIterator(); - final RowSequence.Iterator posIt = affectedRowPositions.getRowSequenceIterator(); - final WritableIntChunk pushChunk = - WritableIntChunk.makeWritableChunk(workingChunkSize); - final WritableIntChunk popChunk = - WritableIntChunk.makeWritableChunk(workingChunkSize)) { + @Override + public void processRows(UpdateByWindowContext context, ColumnSource[] inputSources, boolean initialStep) { + UpdateByWindowTicksContext ctx = (UpdateByWindowTicksContext) context; - // load the first chunk of influencer values (fillWindowTicks() will call in future) - loadNextInfluencerChunks(); + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (ctx.operatorIsDirty[opIdx]) { + UpdateByWindowedOperator winOp = (UpdateByWindowedOperator) operators[opIdx]; + // call the specialized version of `intializeUpdate()` for these operators + winOp.initializeUpdate(ctx.opContext[opIdx]); + } + } - final long sourceRowSetSize = sourceRowSet.size(); + ctx.influencerIt = ctx.influencerRows.getRowSequenceIterator(); + ctx.influencerPosIt = ctx.influencerPositions.getRowSequenceIterator(); - while (it.hasMore()) { - final RowSequence chunkRs = it.getNextRowSequenceWithLength(workingChunkSize); - final RowSequence chunkPosRs = posIt.getNextRowSequenceWithLength(workingChunkSize); - final int chunkRsSize = chunkRs.intSize(); + try (final RowSequence.Iterator it = ctx.affectedRows.getRowSequenceIterator(); + final RowSequence.Iterator posIt = ctx.affectedRowPositions.getRowSequenceIterator(); + final WritableIntChunk pushChunk = + WritableIntChunk.makeWritableChunk(ctx.workingChunkSize); + final WritableIntChunk popChunk = + WritableIntChunk.makeWritableChunk(ctx.workingChunkSize)) { - final LongChunk posChunk = chunkPosRs.asRowKeyChunk(); + // load the first chunk of influencer values (fillWindowTicks() will call in future) + loadNextInfluencerChunks(ctx); - // we are going to track all the influencer rows that affect this chunk of data - final RowSetBuilderSequential chunkInfluencerBuilder = RowSetFactory.builderSequential(); + final long sourceRowSetSize = ctx.sourceRowSet.size(); - // chunk processing - for (int ii = 0; ii < chunkRsSize; ii++) { - // read the current position - final long currentPos = posChunk.get(ii); + while (it.hasMore()) { + final RowSequence chunkRs = it.getNextRowSequenceWithLength(ctx.workingChunkSize); + final RowSequence chunkPosRs = posIt.getNextRowSequenceWithLength(ctx.workingChunkSize); + final int chunkRsSize = chunkRs.intSize(); - // compute the head and tail positions (inclusive) - final long head = Math.max(0, currentPos - prevUnits + 1); - final long tail = Math.min(sourceRowSetSize - 1, currentPos + fwdUnits); + final LongChunk posChunk = chunkPosRs.asRowKeyChunk(); - // pop out all values from the current window that are not in the new window - int popCount = 0; - while (!currentWindowPositions.isEmpty() && currentWindowPositions.front() < head) { - currentWindowPositions.remove(); - popCount++; - } + // we are going to track all the influencer rows that affect this chunk of data + final RowSetBuilderSequential chunkInfluencerBuilder = RowSetFactory.builderSequential(); - // skip values until they match the window (this can only happen on initial addition of rows - // to the table, because we short-circuited the precise building of the influencer rows for - // efficiency) - while (nextInfluencerPos < head) { - nextInfluencerIndex++; - - if (nextInfluencerIndex < influencerPosChunkSize) { - nextInfluencerPos = (int) influencerPosChunk.get(nextInfluencerIndex); - nextInfluencerKey = influencerKeyChunk.get(nextInfluencerIndex); - } else { - // try to bring in new data - loadNextInfluencerChunks(); - } - } + // chunk processing + for (int ii = 0; ii < chunkRsSize; ii++) { + // read the current position + final long currentPos = posChunk.get(ii); - // push matching values - int pushCount = 0; - while (nextInfluencerPos <= tail) { - currentWindowPositions.add(nextInfluencerPos); - pushCount++; - // add this key to the needed set for this chunk - chunkInfluencerBuilder.appendKey(nextInfluencerKey); - nextInfluencerIndex++; - - if (nextInfluencerIndex < influencerPosChunkSize) { - nextInfluencerPos = (int) influencerPosChunk.get(nextInfluencerIndex); - nextInfluencerKey = influencerKeyChunk.get(nextInfluencerIndex); - } else { - // try to bring in new data - loadNextInfluencerChunks(); - } - } + // compute the head and tail positions (inclusive) + final long head = Math.max(0, currentPos - prevUnits + 1); + final long tail = Math.min(sourceRowSetSize - 1, currentPos + fwdUnits); - // write the push and pop counts to the chunks - popChunk.set(ii, popCount); - pushChunk.set(ii, pushCount); + // pop out all values from the current window that are not in the new window + int popCount = 0; + while (!ctx.currentWindowPositions.isEmpty() && ctx.currentWindowPositions.front() < head) { + ctx.currentWindowPositions.remove(); + popCount++; } - // execute the operators - try (final RowSet chunkInfluencerRs = chunkInfluencerBuilder.build()) { - ensureGetContextSize(chunkInfluencerRs.size()); - - Arrays.fill(inputSourceChunkPopulated, false); - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - // prep the chunk array needed by the accumulate call - final int[] srcIndices = operatorInputSourceSlots[opIdx]; - Chunk[] chunkArr = new Chunk[srcIndices.length]; - for (int ii = 0; ii < srcIndices.length; ii++) { - int srcIdx = srcIndices[ii]; - // chunk prep - prepareValuesChunkForSource(srcIdx, chunkInfluencerRs); - chunkArr[ii] = inputSourceChunks[srcIdx]; - } - - // make the specialized call for windowed operators - ((UpdateByWindowedOperator.Context) opContext[opIdx]).accumulate( - chunkRs, - chunkArr, - pushChunk, - popChunk, - chunkRsSize); - } + // skip values until they match the window (this can only happen on initial addition of rows + // to the table, because we short-circuited the precise building of the influencer rows for + // efficiency) + while (ctx.nextInfluencerPos < head) { + ctx.nextInfluencerIndex++; + + if (ctx.nextInfluencerIndex < ctx.influencerPosChunkSize) { + ctx.nextInfluencerPos = (int) ctx.influencerPosChunk.get(ctx.nextInfluencerIndex); + ctx.nextInfluencerKey = ctx.influencerKeyChunk.get(ctx.nextInfluencerIndex); + } else { + // try to bring in new data + loadNextInfluencerChunks(ctx); } } - // all these rows were modified - if (modifiedBuilder != null) { - modifiedBuilder.appendRowSequence(chunkRs); + // push matching values + int pushCount = 0; + while (ctx.nextInfluencerPos <= tail) { + ctx.currentWindowPositions.add(ctx.nextInfluencerPos); + pushCount++; + // add this key to the needed set for this chunk + chunkInfluencerBuilder.appendKey(ctx.nextInfluencerKey); + ctx.nextInfluencerIndex++; + + if (ctx.nextInfluencerIndex < ctx.influencerPosChunkSize) { + ctx.nextInfluencerPos = (int) ctx.influencerPosChunk.get(ctx.nextInfluencerIndex); + ctx.nextInfluencerKey = ctx.influencerKeyChunk.get(ctx.nextInfluencerIndex); + } else { + // try to bring in new data + loadNextInfluencerChunks(ctx); + } } - } - } - // call `finishUpdate()` function for each operator - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - operators[opIdx].finishUpdate(opContext[opIdx]); + // write the push and pop counts to the chunks + popChunk.set(ii, popCount); + pushChunk.set(ii, pushCount); } - } - - if (trackModifications) { - newModified = modifiedBuilder.build(); - } - } - } - - public UpdateByWindowContext makeWindowContext(final TrackingRowSet sourceRowSet, - final ColumnSource[] inputSources, - final ColumnSource timestampColumnSource, - final LongSegmentedSortedArray timestampSsa, - final int chunkSize, - final boolean isInitializeStep) { - return new UpdateByWindowTicksContext(sourceRowSet, inputSources, timestampColumnSource, timestampSsa, - chunkSize, - isInitializeStep); - } - private static WritableRowSet computeAffectedRowsTicks(final RowSet sourceSet, final RowSet subset, - final RowSet invertedSubSet, long revTicks, long fwdTicks) { - // swap fwd/rev to get the influencer windows - return computeInfluencerRowsTicks(sourceSet, subset, invertedSubSet, fwdTicks, revTicks); - } + // execute the operators + try (final RowSet chunkInfluencerRs = chunkInfluencerBuilder.build()) { + ensureGetContextSize(ctx, chunkInfluencerRs.size()); + + Arrays.fill(ctx.inputSourceChunkPopulated, false); + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (ctx.operatorIsDirty[opIdx]) { + // prep the chunk array needed by the accumulate call + final int[] srcIndices = operatorInputSourceSlots[opIdx]; + Chunk[] chunkArr = new Chunk[srcIndices.length]; + for (int ii = 0; ii < srcIndices.length; ii++) { + int srcIdx = srcIndices[ii]; + // chunk prep + prepareValuesChunkForSource(ctx, srcIdx, chunkInfluencerRs); + chunkArr[ii] = ctx.inputSourceChunks[srcIdx]; + } - private static WritableRowSet computeInfluencerRowsTicks(final RowSet sourceSet, final RowSet subset, - final RowSet invertedSubSet, long revTicks, long fwdTicks) { - if (sourceSet.size() == subset.size()) { - return sourceSet.copy(); + // make the specialized call for windowed operators + ((UpdateByWindowedOperator.Context) ctx.opContext[opIdx]).accumulate( + chunkRs, + chunkArr, + pushChunk, + popChunk, + chunkRsSize); + } + } + } + } } - long maxPos = sourceSet.size() - 1; - - final RowSetBuilderSequential builder = RowSetFactory.builderSequential(); - final MutableLong minPos = new MutableLong(0L); - - invertedSubSet.forAllRowKeyRanges((s, e) -> { - long sPos = Math.max(s - revTicks, minPos.longValue()); - long ePos = Math.min(e + fwdTicks, maxPos); - builder.appendRange(sPos, ePos); - minPos.setValue(ePos + 1); - }); - - try (final RowSet positions = builder.build()) { - return sourceSet.subSetForPositions(positions); + // call `finishUpdate()` function for each operator + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (ctx.operatorIsDirty[opIdx]) { + operators[opIdx].finishUpdate(ctx.opContext[opIdx]); + } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java index 8591023812b..d2df5dcc437 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java @@ -61,307 +61,6 @@ public void close() { } } - @Override - protected void makeOperatorContexts() { - // use this to make which input sources are initialized - Arrays.fill(inputSourceChunkPopulated, false); - - // create contexts for the affected operators - currentGetContextSize = WINDOW_CHUNK_SIZE; - - // working chunk size need not be larger than affectedRows.size() - workingChunkSize = Math.min(workingChunkSize, affectedRows.intSize()); - - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - // create the fill contexts for the input sources - final int[] sourceIndices = operatorInputSourceSlots[opIdx]; - final ColumnSource[] inputSourceArr = new ColumnSource[sourceIndices.length]; - for (int ii = 0; ii < sourceIndices.length; ii++) { - int sourceSlot = sourceIndices[ii]; - if (!inputSourceChunkPopulated[sourceSlot]) { - inputSourceGetContexts[sourceSlot] = - inputSources[sourceSlot].makeGetContext(currentGetContextSize); - inputSourceChunkPopulated[sourceSlot] = true; - } - inputSourceArr[ii] = inputSources[sourceSlot]; - } - opContext[opIdx] = operators[opIdx].makeUpdateContext(workingChunkSize, inputSourceArr); - } - } - } - - protected void ensureGetContextSize(long newSize) { - if (currentGetContextSize < newSize) { - long size = currentGetContextSize; - while (size < newSize) { - size *= 2; - } - currentGetContextSize = LongSizedDataStructure.intSize( - "ensureGetContextSize exceeded Integer.MAX_VALUE", - size); - - // use this to determine which input sources are initialized - Arrays.fill(inputSourceChunkPopulated, false); - - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - final int[] sourceIndices = operatorInputSourceSlots[opIdx]; - for (int sourceSlot : sourceIndices) { - if (!inputSourceChunkPopulated[sourceSlot]) { - // close the existing context - inputSourceGetContexts[sourceSlot].close(); - - // create a new context of the larger size - inputSourceGetContexts[sourceSlot] = - inputSources[sourceSlot].makeGetContext(currentGetContextSize); - inputSourceChunkPopulated[sourceSlot] = true; - } - } - } - } - } - } - - // windowed by time/ticks is more complex to compute: find all the changed rows and the rows that would - // be affected by the changes (includes newly added rows) and need to be recomputed. Then include all - // the rows that are affected by deletions (if any). After the affected rows have been identified, - // determine which rows will be needed to compute new values for the affected rows (influencer rows) - @Override - public boolean computeAffectedRowsAndOperators(@NotNull final TableUpdate upstream) { - // all rows are affected on the initial step - if (initialStep) { - affectedRows = sourceRowSet.copy(); - influencerRows = affectedRows; - - // mark all operators as affected by this update - Arrays.fill(opAffected, true); - - makeOperatorContexts(); - return true; - } - - // determine which operators are affected by this update - boolean anyAffected = false; - boolean allAffected = upstream.added().isNonempty() || - upstream.removed().isNonempty(); - - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - opAffected[opIdx] = allAffected - || (upstream.modifiedColumnSet().nonempty() && (operatorInputModifiedColumnSets[opIdx] == null - || upstream.modifiedColumnSet().containsAny(operatorInputModifiedColumnSets[opIdx]))); - if (opAffected[opIdx]) { - anyAffected = true; - } - } - - if (!anyAffected) { - return false; - } - - // changed rows are all mods+adds - WritableRowSet changed = upstream.added().union(upstream.modified()); - - // need a writable rowset - WritableRowSet tmpAffected = computeAffectedRowsTime(sourceRowSet, changed, prevUnits, fwdUnits, - timestampColumnSource, timestampSsa, false); - - // other rows can be affected by removes - if (upstream.removed().isNonempty()) { - try (final RowSet prev = sourceRowSet.copyPrev(); - final WritableRowSet affectedByRemoves = - computeAffectedRowsTime(prev, upstream.removed(), prevUnits, fwdUnits, - timestampColumnSource, timestampSsa, true)) { - // we used the SSA (post-shift) to get these keys, no need to shift - // retain only the rows that still exist in the sourceRowSet - affectedByRemoves.retain(sourceRowSet); - tmpAffected.insert(affectedByRemoves); - } - } - - affectedRows = tmpAffected; - - // now get influencer rows for the affected rows - influencerRows = computeInfluencerRowsTime(sourceRowSet, affectedRows, prevUnits, fwdUnits, - timestampColumnSource, timestampSsa, false); - - makeOperatorContexts(); - return true; - } - - /*** - * This function takes care of loading/preparing the next set of influencer data, in this case we load the next - * chunk of key and position data and reset the index - */ - private void loadNextInfluencerChunks() { - if (!influencerIt.hasMore()) { - nextInfluencerTimestamp = Long.MAX_VALUE; - nextInfluencerKey = Long.MAX_VALUE; - return; - } - - final RowSequence influencerRs = influencerIt.getNextRowSequenceWithLength(WINDOW_CHUNK_SIZE); - influencerKeyChunk = influencerRs.asRowKeyChunk(); - influencerTimestampChunk = - timestampColumnSource.getChunk(influencerTimestampContext, influencerRs).asLongChunk(); - - influencerTimestampChunkSize = influencerTimestampChunk.size(); - - nextInfluencerIndex = 0; - nextInfluencerTimestamp = influencerTimestampChunk.get(nextInfluencerIndex); - nextInfluencerKey = influencerKeyChunk.get(nextInfluencerIndex); - } - - /*** - * This function process the affected rows chunkwise, and will advance the moving window (which is the same for - * all operators in this collection). For each row in the dataset the sliding window will adjust and - * instructions for pushing/popping data will be created for the operators. For each chunk of `affected` rows, - * we will identify exactly which `influencer` rows are needed and will provide those and the push/pop - * instructions to the operators. - * - * Downstream operators should manage local storage in a RingBuffer or other efficient structure since our pop() - * calls do not provide the popped data - */ - @Override - public void processRows() { - if (trackModifications) { - modifiedBuilder = RowSetFactory.builderSequential(); - } - - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - UpdateByWindowedOperator winOp = (UpdateByWindowedOperator) operators[opIdx]; - // call the specialized version of `intializeUpdate()` for these operators - winOp.initializeUpdate(opContext[opIdx]); - } - } - - influencerIt = influencerRows.getRowSequenceIterator(); - - try (final RowSequence.Iterator it = affectedRows.getRowSequenceIterator(); - final ChunkSource.GetContext localTimestampContext = - timestampColumnSource.makeGetContext(workingChunkSize); - final WritableIntChunk pushChunk = - WritableIntChunk.makeWritableChunk(workingChunkSize); - final WritableIntChunk popChunk = - WritableIntChunk.makeWritableChunk(workingChunkSize)) { - - // load the first chunk of influencer values (fillWindowTime() will call in future) - loadNextInfluencerChunks(); - - while (it.hasMore()) { - final RowSequence chunkRs = it.getNextRowSequenceWithLength(workingChunkSize); - final int chunkRsSize = chunkRs.intSize(); - - // NOTE: we did not put null values into our SSA and our influencer rowset is built using the - // SSA. there should be no null timestamps considered in the rolling windows - final LongChunk timestampChunk = - timestampColumnSource.getChunk(localTimestampContext, chunkRs).asLongChunk(); - - // we are going to track all the influencer rows that affect this chunk of data - final RowSetBuilderSequential chunkInfluencerBuilder = RowSetFactory.builderSequential(); - - // chunk processing - for (int ii = 0; ii < chunkRsSize; ii++) { - // read the current position - final long currentTimestamp = timestampChunk.get(ii); - - // compute the head and tail positions (inclusive) - final long head = currentTimestamp - prevUnits; - final long tail = currentTimestamp + fwdUnits; - - // pop out all values from the current window that are not in the new window - int popCount = 0; - while (!currentWindowTimestamps.isEmpty() && currentWindowTimestamps.front() < head) { - currentWindowTimestamps.remove(); - popCount++; - } - - - // skip values until they match the window (this can only happen on the initial addition of rows - // to the table, because we short-circuited the precise building of the influencer rows for - // efficiency) - while (nextInfluencerTimestamp < head) { - nextInfluencerIndex++; - - if (nextInfluencerIndex < influencerTimestampChunkSize) { - nextInfluencerTimestamp = influencerTimestampChunk.get(nextInfluencerIndex); - nextInfluencerKey = influencerKeyChunk.get(nextInfluencerIndex); - } else { - // try to bring in new data - loadNextInfluencerChunks(); - } - } - - // push matching values - int pushCount = 0; - while (nextInfluencerTimestamp <= tail) { - currentWindowTimestamps.add(nextInfluencerTimestamp); - pushCount++; - // add this key to the needed set for this chunk - chunkInfluencerBuilder.appendKey(nextInfluencerKey); - nextInfluencerIndex++; - - if (nextInfluencerIndex < influencerTimestampChunkSize) { - nextInfluencerTimestamp = influencerTimestampChunk.get(nextInfluencerIndex); - nextInfluencerKey = influencerKeyChunk.get(nextInfluencerIndex); - } else { - // try to bring in new data - loadNextInfluencerChunks(); - } - } - - // write the push and pop counts to the chunks - popChunk.set(ii, popCount); - pushChunk.set(ii, pushCount); - } - - // execute the operators - try (final RowSet chunkInfluencerRs = chunkInfluencerBuilder.build()) { - ensureGetContextSize(chunkInfluencerRs.size()); - - Arrays.fill(inputSourceChunkPopulated, false); - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - // prep the chunk array needed by the accumulate call - final int[] srcIndices = operatorInputSourceSlots[opIdx]; - Chunk[] chunkArr = new Chunk[srcIndices.length]; - for (int ii = 0; ii < srcIndices.length; ii++) { - int srcIdx = srcIndices[ii]; - // chunk prep - prepareValuesChunkForSource(srcIdx, chunkInfluencerRs); - chunkArr[ii] = inputSourceChunks[srcIdx]; - } - - // make the specialized call for windowed operators - ((UpdateByWindowedOperator.Context) opContext[opIdx]).accumulate( - chunkRs, - chunkArr, - pushChunk, - popChunk, - chunkRsSize); - } - } - } - - // all these rows were modified - if (modifiedBuilder != null) { - modifiedBuilder.appendRowSequence(chunkRs); - } - } - } - - // call `finishUpdate()` function for each operator - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opAffected[opIdx]) { - operators[opIdx].finishUpdate(opContext[opIdx]); - } - } - - if (trackModifications) { - newModified = modifiedBuilder.build(); - } - } } public UpdateByWindowContext makeWindowContext(final TrackingRowSet sourceRowSet, @@ -440,6 +139,302 @@ private static WritableRowSet computeInfluencerRowsTime(final RowSet sourceSet, } } + private void ensureGetContextSize(UpdateByWindowTimeContext ctx, long newSize) { + if (ctx.currentGetContextSize < newSize) { + long size = ctx.currentGetContextSize; + while (size < newSize) { + size *= 2; + } + ctx.currentGetContextSize = LongSizedDataStructure.intSize( + "ensureGetContextSize exceeded Integer.MAX_VALUE", + size); + + // use this to determine which input sources are initialized + Arrays.fill(ctx.inputSourceChunkPopulated, false); + + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (ctx.operatorIsDirty[opIdx]) { + final int[] sourceIndices = operatorInputSourceSlots[opIdx]; + for (int sourceSlot : sourceIndices) { + if (!ctx.inputSourceChunkPopulated[sourceSlot]) { + // close the existing context + ctx.inputSourceGetContexts[sourceSlot].close(); + + // create a new context of the larger size + ctx.inputSourceGetContexts[sourceSlot] = + ctx.inputSources[sourceSlot].makeGetContext(ctx.currentGetContextSize); + ctx.inputSourceChunkPopulated[sourceSlot] = true; + } + } + } + } + } + } + + /*** + * This function takes care of loading/preparing the next set of influencer data, in this case we load the next + * chunk of key and position data and reset the index + */ + private void loadNextInfluencerChunks(UpdateByWindowTimeContext ctx) { + if (!ctx.influencerIt.hasMore()) { + ctx.nextInfluencerTimestamp = Long.MAX_VALUE; + ctx.nextInfluencerKey = Long.MAX_VALUE; + return; + } + + final RowSequence influencerRs = + ctx.influencerIt.getNextRowSequenceWithLength(UpdateByWindowTimeContext.WINDOW_CHUNK_SIZE); + ctx.influencerKeyChunk = influencerRs.asRowKeyChunk(); + ctx.influencerTimestampChunk = + ctx.timestampColumnSource.getChunk(ctx.influencerTimestampContext, influencerRs).asLongChunk(); + + ctx.influencerTimestampChunkSize = ctx.influencerTimestampChunk.size(); + + ctx.nextInfluencerIndex = 0; + ctx.nextInfluencerTimestamp = ctx.influencerTimestampChunk.get(ctx.nextInfluencerIndex); + ctx.nextInfluencerKey = ctx.influencerKeyChunk.get(ctx.nextInfluencerIndex); + } + + // windowed by time/ticks is more complex to compute: find all the changed rows and the rows that would + // be affected by the changes (includes newly added rows) and need to be recomputed. Then include all + // the rows that are affected by deletions (if any). After the affected rows have been identified, + // determine which rows will be needed to compute new values for the affected rows (influencer rows) + @Override + public void computeAffectedRowsAndOperators(UpdateByWindowContext context, @NotNull TableUpdate upstream) { + UpdateByWindowTimeContext ctx = (UpdateByWindowTimeContext) context; + + // all rows are affected on the initial step + if (ctx.initialStep) { + ctx.affectedRows = ctx.sourceRowSet.copy(); + ctx.influencerRows = ctx.affectedRows; + + // mark all operators as affected by this update + Arrays.fill(ctx.operatorIsDirty, true); + + makeOperatorContexts(ctx); + ctx.isDirty = true; + return; + } + + // determine which operators are affected by this update + ctx.isDirty = false; + boolean allAffected = upstream.added().isNonempty() || + upstream.removed().isNonempty(); + + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + ctx.operatorIsDirty[opIdx] = allAffected + || (upstream.modifiedColumnSet().nonempty() && (operators[opIdx].getInputModifiedColumnSet() == null + || upstream.modifiedColumnSet().containsAny(operators[opIdx].getInputModifiedColumnSet()))); + if (ctx.operatorIsDirty[opIdx]) { + ctx.isDirty = true; + } + } + + if (!ctx.isDirty) { + return; + } + + // changed rows are all mods+adds + WritableRowSet changed = upstream.added().union(upstream.modified()); + + // need a writable rowset + WritableRowSet tmpAffected = computeAffectedRowsTime(ctx.sourceRowSet, changed, prevUnits, fwdUnits, + ctx.timestampColumnSource, ctx.timestampSsa, false); + + // other rows can be affected by removes + if (upstream.removed().isNonempty()) { + try (final RowSet prev = ctx.sourceRowSet.copyPrev(); + final WritableRowSet affectedByRemoves = + computeAffectedRowsTime(prev, upstream.removed(), prevUnits, fwdUnits, + ctx.timestampColumnSource, ctx.timestampSsa, true)) { + // we used the SSA (post-shift) to get these keys, no need to shift + // retain only the rows that still exist in the sourceRowSet + affectedByRemoves.retain(ctx.sourceRowSet); + tmpAffected.insert(affectedByRemoves); + } + } + + ctx.affectedRows = tmpAffected; + + // now get influencer rows for the affected rows + ctx.influencerRows = computeInfluencerRowsTime(ctx.sourceRowSet, ctx.affectedRows, prevUnits, fwdUnits, + ctx.timestampColumnSource, ctx.timestampSsa, false); + + makeOperatorContexts(ctx); + } + + @Override + protected void makeOperatorContexts(UpdateByWindowContext context) { + UpdateByWindowTimeContext ctx = (UpdateByWindowTimeContext) context; + + // use this to make which input sources are initialized + Arrays.fill(ctx.inputSourceChunkPopulated, false); + + // create contexts for the affected operators + ctx.currentGetContextSize = UpdateByWindowTimeContext.WINDOW_CHUNK_SIZE; + + // working chunk size need not be larger than affectedRows.size() + ctx.workingChunkSize = Math.min(ctx.workingChunkSize, ctx.affectedRows.intSize()); + + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (ctx.operatorIsDirty[opIdx]) { + // create the fill contexts for the input sources + final int[] sourceIndices = operatorInputSourceSlots[opIdx]; + final ColumnSource[] inputSourceArr = new ColumnSource[sourceIndices.length]; + for (int ii = 0; ii < sourceIndices.length; ii++) { + int sourceSlot = sourceIndices[ii]; + if (!ctx.inputSourceChunkPopulated[sourceSlot]) { + ctx.inputSourceGetContexts[sourceSlot] = + ctx.inputSources[sourceSlot].makeGetContext(ctx.currentGetContextSize); + ctx.inputSourceChunkPopulated[sourceSlot] = true; + } + inputSourceArr[ii] = ctx.inputSources[sourceSlot]; + } + ctx.opContext[opIdx] = operators[opIdx].makeUpdateContext(ctx.workingChunkSize, inputSourceArr); + } + } + } + + + /*** + * This function process the affected rows chunkwise, and will advance the moving window (which is the same for all + * operators in this collection). For each row in the dataset the sliding window will adjust and instructions for + * pushing/popping data will be created for the operators. For each chunk of `affected` rows, we will identify + * exactly which `influencer` rows are needed and will provide those and the push/pop instructions to the operators. + * + * Downstream operators should manage local storage in a RingBuffer or other efficient structure since our pop() + * calls do not provide the popped data + */ + @Override + public void processRows(UpdateByWindowContext context, ColumnSource[] inputSources, boolean initialStep) { + UpdateByWindowTimeContext ctx = (UpdateByWindowTimeContext) context; + + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (ctx.operatorIsDirty[opIdx]) { + UpdateByWindowedOperator winOp = (UpdateByWindowedOperator) operators[opIdx]; + // call the specialized version of `intializeUpdate()` for these operators + winOp.initializeUpdate(ctx.opContext[opIdx]); + } + } + + ctx.influencerIt = ctx.influencerRows.getRowSequenceIterator(); + + try (final RowSequence.Iterator it = ctx.affectedRows.getRowSequenceIterator(); + final ChunkSource.GetContext localTimestampContext = + ctx.timestampColumnSource.makeGetContext(ctx.workingChunkSize); + final WritableIntChunk pushChunk = + WritableIntChunk.makeWritableChunk(ctx.workingChunkSize); + final WritableIntChunk popChunk = + WritableIntChunk.makeWritableChunk(ctx.workingChunkSize)) { + + // load the first chunk of influencer values (fillWindowTime() will call in future) + loadNextInfluencerChunks(ctx); + + while (it.hasMore()) { + final RowSequence chunkRs = it.getNextRowSequenceWithLength(ctx.workingChunkSize); + final int chunkRsSize = chunkRs.intSize(); + + // NOTE: we did not put null values into our SSA and our influencer rowset is built using the + // SSA. there should be no null timestamps considered in the rolling windows + final LongChunk timestampChunk = + ctx.timestampColumnSource.getChunk(localTimestampContext, chunkRs).asLongChunk(); + + // we are going to track all the influencer rows that affect this chunk of data + final RowSetBuilderSequential chunkInfluencerBuilder = RowSetFactory.builderSequential(); + + // chunk processing + for (int ii = 0; ii < chunkRsSize; ii++) { + // read the current position + final long currentTimestamp = timestampChunk.get(ii); + + // compute the head and tail positions (inclusive) + final long head = currentTimestamp - prevUnits; + final long tail = currentTimestamp + fwdUnits; + + // pop out all values from the current window that are not in the new window + int popCount = 0; + while (!ctx.currentWindowTimestamps.isEmpty() && ctx.currentWindowTimestamps.front() < head) { + ctx.currentWindowTimestamps.remove(); + popCount++; + } + + + // skip values until they match the window (this can only happen on the initial addition of rows + // to the table, because we short-circuited the precise building of the influencer rows for + // efficiency) + while (ctx.nextInfluencerTimestamp < head) { + ctx.nextInfluencerIndex++; + + if (ctx.nextInfluencerIndex < ctx.influencerTimestampChunkSize) { + ctx.nextInfluencerTimestamp = ctx.influencerTimestampChunk.get(ctx.nextInfluencerIndex); + ctx.nextInfluencerKey = ctx.influencerKeyChunk.get(ctx.nextInfluencerIndex); + } else { + // try to bring in new data + loadNextInfluencerChunks(ctx); + } + } + + // push matching values + int pushCount = 0; + while (ctx.nextInfluencerTimestamp <= tail) { + ctx.currentWindowTimestamps.add(ctx.nextInfluencerTimestamp); + pushCount++; + // add this key to the needed set for this chunk + chunkInfluencerBuilder.appendKey(ctx.nextInfluencerKey); + ctx.nextInfluencerIndex++; + + if (ctx.nextInfluencerIndex < ctx.influencerTimestampChunkSize) { + ctx.nextInfluencerTimestamp = ctx.influencerTimestampChunk.get(ctx.nextInfluencerIndex); + ctx.nextInfluencerKey = ctx.influencerKeyChunk.get(ctx.nextInfluencerIndex); + } else { + // try to bring in new data + loadNextInfluencerChunks(ctx); + } + } + + // write the push and pop counts to the chunks + popChunk.set(ii, popCount); + pushChunk.set(ii, pushCount); + } + + // execute the operators + try (final RowSet chunkInfluencerRs = chunkInfluencerBuilder.build()) { + ensureGetContextSize(ctx, chunkInfluencerRs.size()); + + Arrays.fill(ctx.inputSourceChunkPopulated, false); + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (ctx.operatorIsDirty[opIdx]) { + // prep the chunk array needed by the accumulate call + final int[] srcIndices = operatorInputSourceSlots[opIdx]; + Chunk[] chunkArr = new Chunk[srcIndices.length]; + for (int ii = 0; ii < srcIndices.length; ii++) { + int srcIdx = srcIndices[ii]; + // chunk prep + prepareValuesChunkForSource(ctx, srcIdx, chunkInfluencerRs); + chunkArr[ii] = ctx.inputSourceChunks[srcIdx]; + } + + // make the specialized call for windowed operators + ((UpdateByWindowedOperator.Context) ctx.opContext[opIdx]).accumulate( + chunkRs, + chunkArr, + pushChunk, + popChunk, + chunkRsSize); + } + } + } + } + } + + // call `finishUpdate()` function for each operator + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + if (ctx.operatorIsDirty[opIdx]) { + operators[opIdx].finishUpdate(ctx.opContext[opIdx]); + } + } + } + UpdateByWindowTime(UpdateByOperator[] operators, int[][] operatorSourceSlots, @Nullable String timestampColumnName, long prevUnits, long fwdUnits) { super(operators, operatorSourceSlots, timestampColumnName); From 907181ee15e0e793bfd8b84da4577f23f62d76ee Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 26 Oct 2022 16:50:11 -0700 Subject: [PATCH 034/123] single threaded caching implemented and working --- .../deephaven/engine/table/impl/UpdateBy.java | 248 +++++++++++++----- .../table/impl/UpdateByBucketHelper.java | 16 +- .../engine/table/impl/UpdateByOperator.java | 3 +- .../table/impl/UpdateByOperatorFactory.java | 16 +- .../table/impl/ZeroKeyUpdateByManager.java | 133 ---------- .../table/impl/updateby/UpdateByWindow.java | 121 +++++---- .../updateby/UpdateByWindowCumulative.java | 152 ++++++----- .../impl/updateby/UpdateByWindowTicks.java | 182 ++++++------- .../impl/updateby/UpdateByWindowTime.java | 144 +++++----- .../updateby/ema/BigDecimalEMAOperator.java | 13 +- .../updateby/ema/BigIntegerEMAOperator.java | 14 +- .../updateby/ema/BigNumberEMAOperator.java | 9 +- .../impl/updateby/ema/ByteEMAOperator.java | 13 +- .../impl/updateby/ema/DoubleEMAOperator.java | 15 +- .../impl/updateby/ema/FloatEMAOperator.java | 15 +- .../impl/updateby/ema/IntEMAOperator.java | 13 +- .../impl/updateby/ema/LongEMAOperator.java | 13 +- .../impl/updateby/ema/ShortEMAOperator.java | 13 +- .../updateby/fill/BooleanFillByOperator.java | 2 +- .../updateby/fill/ByteFillByOperator.java | 2 +- .../updateby/fill/CharFillByOperator.java | 2 +- .../updateby/fill/DoubleFillByOperator.java | 2 +- .../updateby/fill/FloatFillByOperator.java | 2 +- .../impl/updateby/fill/IntFillByOperator.java | 2 +- .../updateby/fill/LongFillByOperator.java | 2 +- .../updateby/fill/ObjectFillByOperator.java | 2 +- .../updateby/fill/ShortFillByOperator.java | 2 +- .../internal/BaseObjectBinaryOperator.java | 2 +- .../minmax/ByteCumMinMaxOperator.java | 2 +- .../minmax/DoubleCumMinMaxOperator.java | 2 +- .../minmax/FloatCumMinMaxOperator.java | 2 +- .../updateby/minmax/IntCumMinMaxOperator.java | 2 +- .../minmax/LongCumMinMaxOperator.java | 2 +- .../minmax/ShortCumMinMaxOperator.java | 2 +- .../prod/BigDecimalCumProdOperator.java | 2 +- .../prod/BigIntegerCumProdOperator.java | 2 +- .../updateby/prod/ByteCumProdOperator.java | 2 +- .../updateby/prod/DoubleCumProdOperator.java | 2 +- .../updateby/prod/FloatCumProdOperator.java | 2 +- .../updateby/prod/IntCumProdOperator.java | 2 +- .../updateby/prod/LongCumProdOperator.java | 2 +- .../updateby/prod/ShortCumProdOperator.java | 2 +- .../BigDecimalRollingSumOperator.java | 2 +- .../BigIntegerRollingSumOperator.java | 2 +- .../rollingsum/ByteRollingSumOperator.java | 2 +- .../rollingsum/DoubleRollingSumOperator.java | 2 +- .../rollingsum/FloatRollingSumOperator.java | 2 +- .../rollingsum/IntRollingSumOperator.java | 2 +- .../rollingsum/LongRollingSumOperator.java | 2 +- .../rollingsum/ShortRollingSumOperator.java | 2 +- .../sum/BigDecimalCumSumOperator.java | 2 +- .../sum/BigIntegerCumSumOperator.java | 2 +- .../impl/updateby/sum/ByteCumSumOperator.java | 2 +- .../updateby/sum/DoubleCumSumOperator.java | 2 +- .../updateby/sum/FloatCumSumOperator.java | 2 +- .../impl/updateby/sum/IntCumSumOperator.java | 2 +- .../impl/updateby/sum/LongCumSumOperator.java | 2 +- .../updateby/sum/ShortCumSumOperator.java | 2 +- 58 files changed, 591 insertions(+), 622 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java index 967c57c3f7b..0aa6140afd4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java @@ -6,6 +6,7 @@ import io.deephaven.api.ColumnName; import io.deephaven.api.updateby.UpdateByOperation; import io.deephaven.api.updateby.UpdateByControl; +import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.exceptions.UncheckedTableException; import io.deephaven.engine.rowset.*; @@ -13,6 +14,7 @@ import io.deephaven.engine.table.impl.sources.FillUnordered; import io.deephaven.engine.table.impl.sources.LongSparseArraySource; import io.deephaven.engine.table.impl.sources.ReinterpretUtils; +import io.deephaven.engine.table.impl.sources.SparseArrayColumnSource; import io.deephaven.engine.table.impl.sources.sparse.SparseConstants; import io.deephaven.engine.table.impl.updateby.UpdateByWindow; import io.deephaven.engine.table.impl.util.InverseRowRedirectionImpl; @@ -47,49 +49,22 @@ public abstract class UpdateBy { // column-caching management - /** Whether caching benefits this UpdateBy operation */ + /** + * Whether caching benefits this UpdateBy operation + */ protected final boolean inputCacheNeeded; - /** Whether caching benefits this input source */ + /** + * Whether caching benefits this input source + */ protected final boolean[] inputSourceCacheNeeded; - /** The cached input sources for this update */ - protected final CachedInputSource[] cachedInputSources; - /** The output table for this UpdateBy operation */ + /** + * The output table for this UpdateBy operation + */ protected QueryTable result; protected LinkedList recorders; protected UpdateByListener listener; - protected static class CachedInputSource implements SafeCloseable { - public boolean isPopulated; - public final AtomicInteger referenceCount; - public WritableRowSet rowSet; - public ColumnSource[] source; - - public CachedInputSource() { - this.referenceCount = new AtomicInteger(1); - isPopulated = false; - } - - public void addReference() { - referenceCount.incrementAndGet(); - } - - public void addRowSet(final RowSet inputRowSet) { - if (rowSet == null) { - rowSet = inputRowSet.copy(); - } else { - // testing shows this union-ing is very efficient - rowSet.insert(inputRowSet); - } - } - - @Override - public void close() { - try (final RowSet ignored = rowSet) { - } - } - } - public static class UpdateByRedirectionContext implements Context { @Nullable protected final WritableRowRedirection rowRedirection; @@ -161,7 +136,7 @@ public void processUpdateForRedirection(@NotNull final TableUpdate upstream, fin } private boolean shiftRedirectedKey(@NotNull final RowSet.SearchIterator iterator, final long delta, - final long key) { + final long key) { final long inner = rowRedirection.remove(key); if (inner != NULL_ROW_KEY) { rowRedirection.put(key + delta, inner); @@ -177,15 +152,15 @@ public void close() { } protected UpdateBy(@NotNull final String description, - @NotNull final QueryTable source, - @NotNull final UpdateByOperator[] operators, - @NotNull final UpdateByWindow[] windows, - @NotNull final ColumnSource[] inputSources, - @NotNull final int[][] operatorInputSourceSlots, - @NotNull final Map> resultSources, - @Nullable String timestampColumnName, - @NotNull final UpdateByRedirectionContext redirContext, - @NotNull final UpdateByControl control) { + @NotNull final QueryTable source, + @NotNull final UpdateByOperator[] operators, + @NotNull final UpdateByWindow[] windows, + @NotNull final ColumnSource[] inputSources, + @NotNull final int[][] operatorInputSourceSlots, + @NotNull final Map> resultSources, + @Nullable String timestampColumnName, + @NotNull final UpdateByRedirectionContext redirContext, + @NotNull final UpdateByControl control) { if (operators.length == 0) { throw new IllegalArgumentException("At least one operator must be specified"); @@ -201,7 +176,6 @@ protected UpdateBy(@NotNull final String description, this.control = control; this.inputSourceCacheNeeded = new boolean[inputSources.length]; - this.cachedInputSources = new CachedInputSource[inputSources.length]; boolean cacheNeeded = false; for (int ii = 0; ii < inputSources.length; ii++) { @@ -213,6 +187,125 @@ protected UpdateBy(@NotNull final String description, buckets = new LinkedList<>(); } + private ColumnSource getCachedColumn(ColumnSource inputSource, final RowSet inputRowSet) { + final SparseArrayColumnSource outputSource = SparseArrayColumnSource + .getSparseMemoryColumnSource(inputSource.getType(), inputSource.getComponentType()); + + final int CHUNK_SIZE = 1 << 16; + + try (final RowSequence.Iterator rsIt = inputRowSet.getRowSequenceIterator(); + final ChunkSink.FillFromContext ffc = + outputSource.makeFillFromContext(CHUNK_SIZE); + final ChunkSource.GetContext gc = inputSource.makeGetContext(CHUNK_SIZE)) { + while (rsIt.hasMore()) { + final RowSequence chunkOk = rsIt.getNextRowSequenceWithLength(CHUNK_SIZE); + final Chunk values = inputSource.getChunk(gc, chunkOk); + outputSource.fillFromChunk(ffc, values, chunkOk); + } + } + + return outputSource; + } + + /** + * Examine the buckets and identify the input sources that will benefit from caching. Accumulate the bucket + * rowsets for each source independently so the caches are as efficient as possible + */ + private void computeCachedColumnContents(UpdateByBucketHelper[] buckets, boolean initialStep, WritableRowSet[] inputSourceRowSets, AtomicInteger[] inputSourceReferenceCounts) { + // on the initial step, everything is dirty and we can optimize + if (initialStep) { + for (int srcIdx = 0; srcIdx < inputSources.length; srcIdx++) { + if (inputSourceCacheNeeded[srcIdx]) { + inputSourceRowSets[srcIdx] = source.getRowSet().copy(); + } + } + + // add reference counts for each window + for (final UpdateByWindow win : windows) { + final int[] uniqueWindowSources = win.getUniqueSourceIndices(); + for (int srcIdx : uniqueWindowSources) { + if (inputSourceCacheNeeded[srcIdx]) { + // increment the reference count for this input source + if (inputSourceReferenceCounts[srcIdx] == null) { + inputSourceReferenceCounts[srcIdx] = new AtomicInteger(1); + } else { + inputSourceReferenceCounts[srcIdx].incrementAndGet(); + } + } + } + } + return; + } + + // on update steps, we can be more precise and cache exactly what is needed by the update + final boolean[] cacheNeeded = new boolean[inputSources.length]; + + for (int winIdx = 0; winIdx < windows.length; winIdx++) { + final UpdateByWindow win = windows[winIdx]; + final int[] uniqueWindowSources = win.getUniqueSourceIndices(); + + Arrays.fill(cacheNeeded, false); + + // for each bucket, need to accumulate the rowset if this window is dirty + for (UpdateByBucketHelper bucket : buckets) { + UpdateByWindow.UpdateByWindowContext winCtx = bucket.windowContexts[winIdx]; + if (win.isWindowDirty(winCtx)) { + // + for (int srcIdx : uniqueWindowSources) { + if (inputSourceCacheNeeded[srcIdx]) { + // record that this window requires this input source + cacheNeeded[srcIdx] = true; + // add this rowset to the running total + if (inputSourceRowSets[srcIdx] == null) { + inputSourceRowSets[srcIdx] = win.getInfluencerRows(winCtx).copy(); + } else { + inputSourceRowSets[srcIdx].insert(win.getInfluencerRows(winCtx)); + } + } + } + } + } + + // add one to all the reference counts this windows + for (int srcIdx : uniqueWindowSources) { + if (cacheNeeded[srcIdx]) { + // increment the reference count for this input source + if (inputSourceReferenceCounts[srcIdx] == null) { + inputSourceReferenceCounts[srcIdx] = new AtomicInteger(1); + } else { + inputSourceReferenceCounts[srcIdx].incrementAndGet(); + } + } + } + } + } + + private void cacheInputSources(int winIdx, ColumnSource[] maybeCachedInputSources, WritableRowSet[] inputSourceRowSets) { + final UpdateByWindow win = windows[winIdx]; + final int[] uniqueWindowSources = win.getUniqueSourceIndices(); + for (int srcIdx : uniqueWindowSources) { + if (maybeCachedInputSources[srcIdx] == null) { + maybeCachedInputSources[srcIdx] = getCachedColumn(inputSources[srcIdx], inputSourceRowSets[srcIdx]); + } + } + } + + private void releaseInputSources(int winIdx, ColumnSource[] maybeCachedInputSources, WritableRowSet[] inputSourceRowSets, AtomicInteger[] inputSourceReferenceCounts) { + final UpdateByWindow win = windows[winIdx]; + final int[] uniqueWindowSources = win.getUniqueSourceIndices(); + for (int srcIdx : uniqueWindowSources) { + if (inputSourceReferenceCounts[srcIdx] != null) { + if (inputSourceReferenceCounts[srcIdx].decrementAndGet() == 0) { + // do the cleanup immediately + inputSourceRowSets[srcIdx].close(); + inputSourceRowSets[srcIdx] = null; + + maybeCachedInputSources[srcIdx] = null; + } + } + } + } + protected void shiftOutputColumns(TableUpdate upstream) { if (redirContext.isRedirected()) { redirContext.processUpdateForRedirection(upstream, source.getRowSet()); @@ -230,17 +323,38 @@ protected void shiftOutputColumns(TableUpdate upstream) { } protected void processBuckets(UpdateByBucketHelper[] dirtyBuckets, final boolean initialStep) { - // if (inputCacheNeeded) { - // computeCachedColumnContents(buckets, initialStep); - // } + if (inputCacheNeeded) { + // this will store the input sources needed for processing + final ColumnSource[] maybeCachedInputSources = new ColumnSource[inputSources.length]; + final WritableRowSet[] inputSourceRowSets = new WritableRowSet[inputSources.length]; + final AtomicInteger[] inputSourceReferenceCounts = new AtomicInteger[inputSources.length]; - // let's start processing windows - for (int winIdx = 0; winIdx < windows.length; winIdx++) { - // maybeCacheInputSources(); - for (UpdateByBucketHelper bucket : dirtyBuckets) { - bucket.processWindow(winIdx, inputSources, initialStep); + computeCachedColumnContents(dirtyBuckets, initialStep, inputSourceRowSets, inputSourceReferenceCounts); + + // `null` marks columns that need to be cached + for (int srcIdx = 0; srcIdx < inputSources.length; srcIdx++) { + maybeCachedInputSources[srcIdx] = inputSourceCacheNeeded[srcIdx] ? null : inputSources[srcIdx]; + } + + for (int winIdx = 0; winIdx < windows.length; winIdx++) { + // cache the sources needed for this window + cacheInputSources(winIdx, maybeCachedInputSources, inputSourceRowSets); + // process the window + for (UpdateByBucketHelper bucket : dirtyBuckets) { + bucket.assignInputSources(winIdx, maybeCachedInputSources); + bucket.processWindow(winIdx, initialStep); + } + // release the cached sources that are no longer needed + releaseInputSources(winIdx, maybeCachedInputSources, inputSourceRowSets, inputSourceReferenceCounts); + } + } else { + // no caching needed, process immediately + for (int winIdx = 0; winIdx < windows.length; winIdx++) { + for (UpdateByBucketHelper bucket : dirtyBuckets) { + bucket.assignInputSources(winIdx, inputSources); + bucket.processWindow(winIdx, initialStep); + } } - // maybeReleaseInputSources() } } @@ -295,14 +409,12 @@ protected void process() { if (win.isWindowDirty(winCtx)) { // add the window modified rows to this set - modifiedRowSet.insert(win.getModifiedRows(winCtx)); + modifiedRowSet.insert(win.getAffectedRows(winCtx)); // add the modified output column sets to the downstream set final UpdateByOperator[] winOps = win.getOperators(); - for (int winOpIdx = 0; winOpIdx < winOps.length; winOpIdx++) { - if (win.isOperatorDirty(winCtx, winOpIdx)) { - // these were created directly from the result output columns so no transformer needed - downstream.modifiedColumnSet.setAll(winOps[winOpIdx].outputModifiedColumnSet); - } + for (int winOpIdx : win.getDirtyOperators(winCtx)) { + // these were created directly from the result output columns so no transformer needed + downstream.modifiedColumnSet.setAll(winOps[winOpIdx].outputModifiedColumnSet); } } } @@ -329,20 +441,20 @@ public UpdateByListener newListener(@NotNull final String description) { } // region UpdateBy implementation + /** * Apply the specified operations to each group of rows in the source table and produce a result table with the same * index as the source with each operator applied. * - * @param source the source to apply to. - * @param clauses the operations to apply. + * @param source the source to apply to. + * @param clauses the operations to apply. * @param byColumns the columns to group by before applying operations - * * @return a new table with the same index as the source with all the operations applied. */ public static Table updateBy(@NotNull final QueryTable source, - @NotNull final Collection clauses, - @NotNull final Collection byColumns, - @NotNull final UpdateByControl control) { + @NotNull final Collection clauses, + @NotNull final Collection byColumns, + @NotNull final UpdateByControl control) { // create the rowRedirection if instructed final WritableRowRedirection rowRedirection; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java index 486f98e62f9..bac8cd9824f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java @@ -1,5 +1,6 @@ package io.deephaven.engine.table.impl; +import gnu.trove.list.array.TIntArrayList; import io.deephaven.api.updateby.UpdateByControl; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.*; @@ -250,7 +251,6 @@ public void prepareForUpdate(final TableUpdate upstream, final boolean initialSt for (int winIdx = 0; winIdx < windows.length; winIdx++) { windowContexts[winIdx] = windows[winIdx].makeWindowContext( sourceRowSet, - inputSources, timestampColumnSource, timestampSsa, control.chunkCapacityOrDefault(), @@ -272,13 +272,15 @@ public boolean isDirty() { return isDirty; } - public void processWindow(final int winIdx, - final ColumnSource[] inputSources, - final boolean initialStep) { - // call the window.process() with the correct context for this bucket - if (windows[winIdx].isWindowDirty(windowContexts[winIdx])) { - windows[winIdx].processRows(windowContexts[winIdx], inputSources, initialStep); + public void assignInputSources(final int winIdx, final ColumnSource[] inputSources) { + windows[winIdx].assignInputSources(windowContexts[winIdx], inputSources); + } + + public void processWindow(final int winIdx, final boolean initialStep) { + if (!windows[winIdx].isWindowDirty(windowContexts[winIdx])) { + return; // no work to do for this bucket window } + windows[winIdx].processRows(windowContexts[winIdx], initialStep); } public void finalizeUpdate() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java index d4b6a93320d..cc1673f5e50 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java @@ -183,11 +183,10 @@ public String[] getOutputColumnNames() { * Make an {@link UpdateContext} suitable for use with updates. * * @param chunkSize The expected size of chunks that will be provided during the update, - * @param inputSourceArr The column sources that contains the input values needed by the operator * @return a new context */ @NotNull - public abstract UpdateContext makeUpdateContext(final int chunkSize, ColumnSource[] inputSourceArr); + public abstract UpdateContext makeUpdateContext(final int chunkSize); /** * Perform and bookkeeping required at the end of a single part of the update. This is always preceded with a call diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java index c2e8dd78e52..e2d327a6aab 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java @@ -228,28 +228,28 @@ private UpdateByOperator makeEmaOperator(@NotNull final MatchPair pair, if (csType == byte.class || csType == Byte.class) { return new ByteEMAOperator(pair, affectingColumns, control, - ema.timeScale().timestampCol(), timeScaleUnits, redirContext); + ema.timeScale().timestampCol(), timeScaleUnits, redirContext, columnSource); } else if (csType == short.class || csType == Short.class) { return new ShortEMAOperator(pair, affectingColumns, control, - ema.timeScale().timestampCol(), timeScaleUnits, redirContext); + ema.timeScale().timestampCol(), timeScaleUnits, redirContext, columnSource); } else if (csType == int.class || csType == Integer.class) { return new IntEMAOperator(pair, affectingColumns, control, - ema.timeScale().timestampCol(), timeScaleUnits, redirContext); + ema.timeScale().timestampCol(), timeScaleUnits, redirContext, columnSource); } else if (csType == long.class || csType == Long.class) { return new LongEMAOperator(pair, affectingColumns, control, - ema.timeScale().timestampCol(), timeScaleUnits, redirContext); + ema.timeScale().timestampCol(), timeScaleUnits, redirContext, columnSource); } else if (csType == float.class || csType == Float.class) { return new FloatEMAOperator(pair, affectingColumns, control, - ema.timeScale().timestampCol(), timeScaleUnits, redirContext); + ema.timeScale().timestampCol(), timeScaleUnits, redirContext, columnSource); } else if (csType == double.class || csType == Double.class) { return new DoubleEMAOperator(pair, affectingColumns, control, - ema.timeScale().timestampCol(), timeScaleUnits, redirContext); + ema.timeScale().timestampCol(), timeScaleUnits, redirContext, columnSource); } else if (csType == BigDecimal.class) { return new BigDecimalEMAOperator(pair, affectingColumns, control, - ema.timeScale().timestampCol(), timeScaleUnits, redirContext); + ema.timeScale().timestampCol(), timeScaleUnits, redirContext, columnSource); } else if (csType == BigInteger.class) { return new BigIntegerEMAOperator(pair, affectingColumns, control, - ema.timeScale().timestampCol(), timeScaleUnits, redirContext); + ema.timeScale().timestampCol(), timeScaleUnits, redirContext, columnSource); } throw new IllegalArgumentException("Can not perform EMA on type " + csType); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateByManager.java index 42209ceedfc..965ca72cce1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateByManager.java @@ -98,139 +98,6 @@ public void onUpdate(@NotNull final TableUpdate upstream) { finalizeBuckets(dirtyBuckets); } - // private ColumnSource getCachedColumn(ColumnSource inputSource, final RowSet inputRowSet) { - // final SparseArrayColumnSource outputSource = SparseArrayColumnSource - // .getSparseMemoryColumnSource(inputSource.getType(), inputSource.getComponentType()); - // - // final int CHUNK_SIZE = 1 << 16; - // - // try (final RowSequence.Iterator rsIt = inputRowSet.getRowSequenceIterator(); - // final ChunkSink.FillFromContext ffc = - // outputSource.makeFillFromContext(CHUNK_SIZE); - // final ChunkSource.GetContext gc = inputSource.makeGetContext(CHUNK_SIZE)) { - // while (rsIt.hasMore()) { - // final RowSequence chunkOk = rsIt.getNextRowSequenceWithLength(CHUNK_SIZE); - // final Chunk values = inputSource.getChunk(gc, chunkOk); - // outputSource.fillFromChunk(ffc, values, chunkOk); - // } - // } - // - // return outputSource; - // } - // - // private int[] getAffectedWindows(UpdateByBucketHelper.UpdateContext context) { - // final TIntArrayList list = new TIntArrayList(windows.length); - // for (int winIdx = 0; winIdx < windows.length; winIdx++) { - // if (windows[winIdx].isAffected(context.windowContexts[winIdx])) { - // list.add(winIdx); - // } - // } - // return list.toArray(); - // } - // - // private void addRowSetToInputSourceCache(int srcIdx, final RowSet rowSet) { - // // create if it doesn't exist - // if (cachedInputSources[srcIdx] == null) { - // cachedInputSources[srcIdx] = new CachedInputSource(); - // } - // cachedInputSources[srcIdx].addRowSet(rowSet); - // } - // - // /** - // * Examine the buckets and identify the input sources that will benefit from caching. Accumulate the bucket - // rowsets - // * for each source independently - // */ - // private void computeCachedColumnContents(UpdateByBucketHelper[] buckets, boolean initialStep) { - // // track for each window what sources we need to cache - // final boolean[] windowSourceCacheNeeded = new boolean[inputSources.length]; - // - // for (int winIdx = 0; winIdx < windows.length; winIdx++) { - // Arrays.fill(windowSourceCacheNeeded, false); - // - // // for each bucket - // for (UpdateByBucketHelper bucket : buckets) { - // UpdateByWindow.UpdateByWindowContext bucketCtx = bucket.windowContexts[winIdx]; - // if (initialStep || (bucket.isDirty && bucketCtx.isDirty)) { - // for (int opIdx : windows[winIdx].getDirtyOperators(bucketCtx)) { - // for (int srcIdx : windows[winIdx].getOperatorSourceSlots(opIdx)) { - // if (inputSourceCacheNeeded[srcIdx]) { - // windowSourceCacheNeeded[srcIdx] = true; - // addRowSetToInputSourceCache(srcIdx, bucketCtx.getInfluencerRows()); - // } - // } - // } - // } - // } - // - // // add one to all the reference counts this windows - // for (int srcIdx = 0; srcIdx < inputSources.length; srcIdx++) { - // if (windowSourceCacheNeeded[srcIdx]) { - // cachedInputSources[srcIdx].addReference(); - // } - // } - // } - // } - // - // private void maybeCacheInputSources(int[] srcArr) { - // - // } - // - // private void maybeReleaseInputSources(int[] srcArr) { - // - // } - // - // /** - // * The Listener for apply an upstream {@link InstrumentedTableUpdateListenerAdapter#onUpdate(TableUpdate) update} - // */ - // class ZeroKeyUpdateByManagerListener extends InstrumentedTableUpdateListenerAdapter { - // public ZeroKeyUpdateByManagerListener(@Nullable String description, - // @NotNull final QueryTable source, - // @NotNull final QueryTable result) { - // super(description, source, false); - // - // } - // - // @Override - // public void onUpdate(TableUpdate upstream) { - // // do the actual computations - // processBuckets(false); - // - // - // - // final boolean windowsModified = ctx.anyModified(); - // - // if (upstream.modified().isNonempty() || windowsModified) { - // WritableRowSet modifiedRowSet = RowSetFactory.empty(); - // downstream.modified = modifiedRowSet; - // if (upstream.modified().isNonempty()) { - // // Transform any untouched modified columns to the output. - // transformer.clearAndTransform(upstream.modifiedColumnSet(), downstream.modifiedColumnSet); - // modifiedRowSet.insert(upstream.modified()); - // } - // - // - // - // if (windowsModified) { - // modifiedRowSet.remove(upstream.added()); - // } - // } else { - // downstream.modified = RowSetFactory.empty(); - // } - // - // // set the modified columns if any operators made changes (add/rem/modify) - // for (int winIdx = 0; winIdx < windows.length; winIdx++) { - // if (ctx.windowAffected[winIdx]) { - // ctx.windowContexts[winIdx].updateOutputModifiedColumnSet(downstream.modifiedColumnSet, - // windowOperatorOutputModifiedColumnSets[winIdx]); - // } - // } - // - // result.notifyListeners(downstream); - // } - // } - - /** * Perform an updateBy without any key columns. * diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java index 5d9ec89b56a..294110de269 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java @@ -1,6 +1,5 @@ package io.deephaven.engine.table.impl.updateby; -import gnu.trove.list.array.TIntArrayList; import gnu.trove.set.hash.TIntHashSet; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.WritableChunk; @@ -8,9 +7,7 @@ import io.deephaven.engine.rowset.*; import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.ModifiedColumnSet; import io.deephaven.engine.table.TableUpdate; -import io.deephaven.engine.table.impl.QueryTable; import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; @@ -28,14 +25,10 @@ public abstract class UpdateByWindow { // store the index in the {@link UpdateBy.inputSources} protected final int[][] operatorInputSourceSlots; + protected int[] uniqueInputSourceIndices; + /** This context will store the necessary info to process a single window for a single bucket */ public abstract class UpdateByWindowContext implements SafeCloseable { - /** Indicates this bucket window needs to be processed */ - protected boolean isDirty; - - /** Indicates this operator needs to be processed */ - protected final boolean[] operatorIsDirty; - /** store a reference to the source rowset */ protected final TrackingRowSet sourceRowSet; @@ -50,19 +43,19 @@ public abstract class UpdateByWindowContext implements SafeCloseable { /** An array of context objects for each underlying operator */ protected final UpdateByOperator.UpdateContext[] opContext; + protected final boolean initialStep; + /** An array of ColumnSources for each underlying operator */ - protected final ColumnSource[] inputSources; + protected ColumnSource[] inputSources; /** An array of {@link ChunkSource.GetContext}s for each input column */ - protected final ChunkSource.GetContext[] inputSourceGetContexts; + protected ChunkSource.GetContext[] inputSourceGetContexts; /** A set of chunks used to store working values */ - protected final Chunk[] inputSourceChunks; + protected Chunk[] inputSourceChunks; /** An indicator of if each slot has been populated with data or not for this phase. */ - protected final boolean[] inputSourceChunkPopulated; - - protected final boolean initialStep; + protected boolean[] inputSourceChunkPopulated; /** the rows affected by this update */ protected RowSet affectedRows; @@ -71,50 +64,29 @@ public abstract class UpdateByWindowContext implements SafeCloseable { protected int workingChunkSize; - public UpdateByWindowContext(final TrackingRowSet sourceRowSet, final ColumnSource[] inputSources, + /** Indicates this bucket window needs to be processed */ + protected boolean isDirty; + + /** Indicates which operators need to be processed */ + protected int[] dirtyOperatorIndices; + + /** Not actually dity, but indicates which sources are need to process this window context */ + protected int[] dirtySourceIndices; + + public UpdateByWindowContext(final TrackingRowSet sourceRowSet, @Nullable final ColumnSource timestampColumnSource, @Nullable final LongSegmentedSortedArray timestampSsa, final int chunkSize, final boolean initialStep) { this.sourceRowSet = sourceRowSet; - this.inputSources = inputSources; this.timestampColumnSource = timestampColumnSource; this.timestampSsa = timestampSsa; - this.operatorIsDirty = new boolean[operators.length]; this.opContext = new UpdateByOperator.UpdateContext[operators.length]; - this.inputSourceGetContexts = new ChunkSource.GetContext[inputSources.length]; - this.inputSourceChunkPopulated = new boolean[inputSources.length]; - // noinspection unchecked - this.inputSourceChunks = new WritableChunk[inputSources.length]; this.workingChunkSize = chunkSize; this.initialStep = initialStep; this.isDirty = false; } - - - // public boolean anyModified() { - // return newModified != null && newModified.isNonempty(); - // } - // - // public void updateOutputModifiedColumnSet(ModifiedColumnSet outputModifiedColumnSet, - // ModifiedColumnSet[] operatorOutputModifiedColumnSets) { - // for (int opIdx = 0; opIdx < operators.length; opIdx++) { - // if (operatorIsDirty[opIdx]) { - // outputModifiedColumnSet.setAll(operatorOutputModifiedColumnSets[opIdx]); - // } - // } - // } - // - // public RowSet getAffectedRows() { - // return affectedRows; - // } - // - // public RowSet getInfluencerRows() { - // return influencerRows; - // } - // - @Override public void close() { // these might be the same object, don't close both! @@ -129,17 +101,18 @@ public void close() { opContext[opIdx].close(); } } - for (int srcIdx = 0; srcIdx < inputSources.length; srcIdx++) { - if (inputSourceGetContexts[srcIdx] != null) { - inputSourceGetContexts[srcIdx].close(); - inputSourceGetContexts[srcIdx] = null; + if (inputSources != null) { + for (int srcIdx = 0; srcIdx < inputSources.length; srcIdx++) { + if (inputSourceGetContexts[srcIdx] != null) { + inputSourceGetContexts[srcIdx].close(); + inputSourceGetContexts[srcIdx] = null; + } } } } } public abstract UpdateByWindowContext makeWindowContext(final TrackingRowSet sourceRowSet, - final ColumnSource[] inputSources, final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa, final int chunkSize, @@ -192,6 +165,21 @@ public UpdateByOperator[] getOperators() { return operators; } + public int[][] getOperatorInputSourceSlots() { + return operatorInputSourceSlots; + } + + public int[] getUniqueSourceIndices() { + if (uniqueInputSourceIndices == null) { + final TIntHashSet set = new TIntHashSet(); + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + set.addAll(operatorInputSourceSlots[opIdx]); + } + uniqueInputSourceIndices = set.toArray(); + } + return uniqueInputSourceIndices; + } + // region context-based functions public abstract void computeAffectedRowsAndOperators(final UpdateByWindowContext context, @@ -199,6 +187,19 @@ public abstract void computeAffectedRowsAndOperators(final UpdateByWindowContext protected abstract void makeOperatorContexts(final UpdateByWindowContext context); + public void assignInputSources(final UpdateByWindowContext context, final ColumnSource[] inputSources) { + context.inputSources = inputSources; + context.inputSourceChunkPopulated = new boolean[inputSources.length]; + context.inputSourceGetContexts = new ChunkSource.GetContext[inputSources.length]; + // noinspection unchecked + context.inputSourceChunks = new WritableChunk[inputSources.length]; + + for (int srcIdx : context.dirtySourceIndices) { + context.inputSourceGetContexts[srcIdx] = + context.inputSources[srcIdx].makeGetContext(context.workingChunkSize); + } + } + protected void prepareValuesChunkForSource(final UpdateByWindowContext context, final int srcIdx, final RowSequence rs) { if (rs.isEmpty()) { @@ -211,22 +212,28 @@ protected void prepareValuesChunkForSource(final UpdateByWindowContext context, } } - public abstract void processRows(final UpdateByWindowContext context, - final ColumnSource[] inputSources, - final boolean initialStep); + public abstract void processRows(final UpdateByWindowContext context, final boolean initialStep); public boolean isWindowDirty(final UpdateByWindowContext context) { return context.isDirty; } - public boolean isOperatorDirty(final UpdateByWindowContext context, int winOpIdx) { - return context.operatorIsDirty[winOpIdx]; + public int[] getDirtyOperators(final UpdateByWindowContext context) { + return context.dirtyOperatorIndices; } - public RowSet getModifiedRows(final UpdateByWindowContext context) { + public int[] getDirtySources(final UpdateByWindowContext context) { + return context.dirtySourceIndices; + } + + public RowSet getAffectedRows(final UpdateByWindowContext context) { return context.affectedRows; } + public RowSet getInfluencerRows(final UpdateByWindowContext context) { + return context.influencerRows; + } + // endregion protected static int hashCode(boolean windowed, @NotNull String[] inputColumnNames, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java index 7ec06488c6d..853d85560ad 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java @@ -1,5 +1,8 @@ package io.deephaven.engine.table.impl.updateby; +import gnu.trove.list.array.TIntArrayList; +import gnu.trove.set.hash.TIntHashSet; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; @@ -7,7 +10,6 @@ import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.TableUpdate; -import io.deephaven.engine.table.impl.QueryTable; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; @@ -15,6 +17,7 @@ import org.jetbrains.annotations.Nullable; import java.util.Arrays; +import java.util.stream.IntStream; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; import static io.deephaven.util.QueryConstants.NULL_LONG; @@ -29,52 +32,36 @@ public UpdateByWindowCumulative(UpdateByOperator[] operators, int[][] operatorSo @Override protected void makeOperatorContexts(UpdateByWindowContext context) { - // use this to make which input sources are initialized - Arrays.fill(context.inputSourceChunkPopulated, false); - // working chunk size need not be larger than affectedRows.size() context.workingChunkSize = Math.min(context.workingChunkSize, context.affectedRows.intSize()); // create contexts for the affected operators - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (context.operatorIsDirty[opIdx]) { - // create the fill contexts for the input sources - final int[] sourceIndices = operatorInputSourceSlots[opIdx]; - final ColumnSource[] inputSourceArr = new ColumnSource[sourceIndices.length]; - for (int ii = 0; ii < sourceIndices.length; ii++) { - int sourceSlot = sourceIndices[ii]; - if (!context.inputSourceChunkPopulated[sourceSlot]) { - context.inputSourceGetContexts[sourceSlot] = - context.inputSources[sourceSlot].makeGetContext(context.workingChunkSize); - context.inputSourceChunkPopulated[sourceSlot] = true; - } - inputSourceArr[ii] = context.inputSources[sourceSlot]; - } - context.opContext[opIdx] = operators[opIdx].makeUpdateContext(context.workingChunkSize, inputSourceArr); - } + for (int opIdx : context.dirtyOperatorIndices) { + context.opContext[opIdx] = operators[opIdx].makeUpdateContext(context.workingChunkSize); } } public UpdateByWindowContext makeWindowContext(final TrackingRowSet sourceRowSet, - final ColumnSource[] inputSources, final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa, final int chunkSize, final boolean isInitializeStep) { - return new UpdateByWindowContext(sourceRowSet, inputSources, timestampColumnSource, timestampSsa, + return new UpdateByWindowContext(sourceRowSet, timestampColumnSource, timestampSsa, chunkSize, isInitializeStep) {}; } @Override public void computeAffectedRowsAndOperators(UpdateByWindowContext context, @NotNull TableUpdate upstream) { + // all rows are affected on the initial step if (context.initialStep) { context.affectedRows = context.sourceRowSet.copy(); context.influencerRows = context.affectedRows; // mark all operators as affected by this update - Arrays.fill(context.operatorIsDirty, true); + context.dirtyOperatorIndices = IntStream.range(0, operators.length).toArray(); + context.dirtySourceIndices = getUniqueSourceIndices(); makeOperatorContexts(context); context.isDirty = true; @@ -86,13 +73,26 @@ public void computeAffectedRowsAndOperators(UpdateByWindowContext context, @NotN boolean allAffected = upstream.added().isNonempty() || upstream.removed().isNonempty(); - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - context.operatorIsDirty[opIdx] = allAffected - || (upstream.modifiedColumnSet().nonempty() && (operators[opIdx].getInputModifiedColumnSet() == null - || upstream.modifiedColumnSet().containsAny(operators[opIdx].getInputModifiedColumnSet()))); - if (context.operatorIsDirty[opIdx]) { - context.isDirty = true; + if (allAffected) { + // mark all operators as affected by this update + context.dirtyOperatorIndices = IntStream.range(0, operators.length).toArray(); + context.dirtySourceIndices = getUniqueSourceIndices(); + context.isDirty = true; + } else { + // determine which operators are affected by this update + TIntArrayList dirtyOperatorList = new TIntArrayList(operators.length); + TIntHashSet inputSourcesSet = new TIntHashSet(getUniqueSourceIndices().length); + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + UpdateByOperator op = operators[opIdx]; + if (upstream.modifiedColumnSet().nonempty() && (op.getInputModifiedColumnSet() == null + || upstream.modifiedColumnSet().containsAny(op.getInputModifiedColumnSet()))) { + dirtyOperatorList.add(opIdx); + inputSourcesSet.addAll(operatorInputSourceSlots[opIdx]); + context.isDirty = true; + } } + context.dirtyOperatorIndices = dirtyOperatorList.toArray(); + context.dirtySourceIndices = inputSourcesSet.toArray(); } if (!context.isDirty) { @@ -110,10 +110,10 @@ public void computeAffectedRowsAndOperators(UpdateByWindowContext context, @NotN makeOperatorContexts(context); } - @Override - public void processRows(UpdateByWindowContext context, final ColumnSource[] inputSources, - final boolean initialStep) { + public void processRows(UpdateByWindowContext context, final boolean initialStep) { + Assert.neqNull(context.inputSources, "assignInputSources() must be called before processRow()"); + // find the key before the first affected row final long keyBefore; try (final RowSet.SearchIterator rIt = context.sourceRowSet.reverseIterator()) { @@ -126,36 +126,34 @@ public void processRows(UpdateByWindowContext context, final ColumnSource[] i } // and preload that data for these operators - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (context.operatorIsDirty[opIdx]) { - UpdateByCumulativeOperator cumOp = (UpdateByCumulativeOperator) operators[opIdx]; - if (cumOp.getTimestampColumnName() == null || keyBefore == NULL_ROW_KEY) { - // this operator doesn't care about timestamps or we know we are at the beginning of the rowset - cumOp.initializeUpdate(context.opContext[opIdx], keyBefore, NULL_LONG); - } else { - // this operator cares about timestamps, so make sure it is starting from a valid value and - // valid timestamp by moving backward until the conditions are met - UpdateByCumulativeOperator.Context cumOpContext = - (UpdateByCumulativeOperator.Context) context.opContext[opIdx]; - long potentialResetTimestamp = context.timestampColumnSource.getLong(keyBefore); - - if (potentialResetTimestamp == NULL_LONG || !cumOpContext.isValueValid(keyBefore)) { - try (final RowSet.SearchIterator rIt = context.sourceRowSet.reverseIterator()) { - if (rIt.advance(keyBefore)) { - while (rIt.hasNext()) { - final long nextKey = rIt.nextLong(); - potentialResetTimestamp = context.timestampColumnSource.getLong(nextKey); - if (potentialResetTimestamp != NULL_LONG && - cumOpContext.isValueValid(nextKey)) { - break; - } + for (int opIdx : context.dirtyOperatorIndices) { + UpdateByCumulativeOperator cumOp = (UpdateByCumulativeOperator) operators[opIdx]; + if (cumOp.getTimestampColumnName() == null || keyBefore == NULL_ROW_KEY) { + // this operator doesn't care about timestamps or we know we are at the beginning of the rowset + cumOp.initializeUpdate(context.opContext[opIdx], keyBefore, NULL_LONG); + } else { + // this operator cares about timestamps, so make sure it is starting from a valid value and + // valid timestamp by moving backward until the conditions are met + UpdateByCumulativeOperator.Context cumOpContext = + (UpdateByCumulativeOperator.Context) context.opContext[opIdx]; + long potentialResetTimestamp = context.timestampColumnSource.getLong(keyBefore); + + if (potentialResetTimestamp == NULL_LONG || !cumOpContext.isValueValid(keyBefore)) { + try (final RowSet.SearchIterator rIt = context.sourceRowSet.reverseIterator()) { + if (rIt.advance(keyBefore)) { + while (rIt.hasNext()) { + final long nextKey = rIt.nextLong(); + potentialResetTimestamp = context.timestampColumnSource.getLong(nextKey); + if (potentialResetTimestamp != NULL_LONG && + cumOpContext.isValueValid(nextKey)) { + break; } } } } - // call the specialized version of `intializeUpdate()` for these operators - cumOp.initializeUpdate(context.opContext[opIdx], keyBefore, potentialResetTimestamp); } + // call the specialized version of `intializeUpdate()` for these operators + cumOp.initializeUpdate(context.opContext[opIdx], keyBefore, potentialResetTimestamp); } } @@ -172,34 +170,30 @@ public void processRows(UpdateByWindowContext context, final ColumnSource[] i LongChunk tsChunk = context.timestampColumnSource == null ? null : context.timestampColumnSource.getChunk(tsGetCtx, rs).asLongChunk(); - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (context.operatorIsDirty[opIdx]) { - // prep the chunk array needed by the accumulate call - final int[] srcIndices = operatorInputSourceSlots[opIdx]; - Chunk[] chunkArr = new Chunk[srcIndices.length]; - for (int ii = 0; ii < srcIndices.length; ii++) { - int srcIdx = srcIndices[ii]; - // chunk prep - prepareValuesChunkForSource(context, srcIdx, rs); - chunkArr[ii] = context.inputSourceChunks[srcIdx]; - } - - // make the specialized call for cumulative operators - ((UpdateByCumulativeOperator.Context) context.opContext[opIdx]).accumulate( - rs, - chunkArr, - tsChunk, - size); + for (int opIdx : context.dirtyOperatorIndices) { + // prep the chunk array needed by the accumulate call + final int[] srcIndices = operatorInputSourceSlots[opIdx]; + Chunk[] chunkArr = new Chunk[srcIndices.length]; + for (int ii = 0; ii < srcIndices.length; ii++) { + int srcIdx = srcIndices[ii]; + // chunk prep + prepareValuesChunkForSource(context, srcIdx, rs); + chunkArr[ii] = context.inputSourceChunks[srcIdx]; } + + // make the specialized call for cumulative operators + ((UpdateByCumulativeOperator.Context) context.opContext[opIdx]).accumulate( + rs, + chunkArr, + tsChunk, + size); } } } // call `finishUpdate()` function for each operator - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (context.operatorIsDirty[opIdx]) { - operators[opIdx].finishUpdate(context.opContext[opIdx]); - } + for (int opIdx : context.dirtyOperatorIndices) { + operators[opIdx].finishUpdate(context.opContext[opIdx]); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java index 41a708d0809..9350b9f41df 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java @@ -1,5 +1,7 @@ package io.deephaven.engine.table.impl.updateby; +import gnu.trove.list.array.TIntArrayList; +import gnu.trove.set.hash.TIntHashSet; import io.deephaven.base.ringbuffer.IntRingBuffer; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; @@ -20,6 +22,7 @@ import org.jetbrains.annotations.Nullable; import java.util.Arrays; +import java.util.stream.IntStream; // this class is currently too big, should specialize into CumWindow, TickWindow, TimeWindow to simplify implementation public class UpdateByWindowTicks extends UpdateByWindow { @@ -45,10 +48,10 @@ public class UpdateByWindowTicksContext extends UpdateByWindow.UpdateByWindowCon protected long influencerPosChunkSize; protected int currentGetContextSize; - public UpdateByWindowTicksContext(final TrackingRowSet sourceRowSet, final ColumnSource[] inputSources, + public UpdateByWindowTicksContext(final TrackingRowSet sourceRowSet, @Nullable final ColumnSource timestampColumnSource, @Nullable final LongSegmentedSortedArray timestampSsa, final int chunkSize, final boolean initialStep) { - super(sourceRowSet, inputSources, null, null, chunkSize, initialStep); + super(sourceRowSet, null, null, chunkSize, initialStep); currentWindowPositions = new IntRingBuffer(512, true); } @@ -68,13 +71,31 @@ public void close() { } } + UpdateByWindowTicks(UpdateByOperator[] operators, int[][] operatorSourceSlots, long prevUnits, long fwdUnits) { + super(operators, operatorSourceSlots, null); + this.prevUnits = prevUnits; + this.fwdUnits = fwdUnits; + } + + @Override + protected void makeOperatorContexts(UpdateByWindowContext context) { + UpdateByWindowTicksContext ctx = (UpdateByWindowTicksContext) context; + + ctx.workingChunkSize = UpdateByWindowTicksContext.WINDOW_CHUNK_SIZE; + ctx.currentGetContextSize = ctx.workingChunkSize; + + // create contexts for the affected operators + for (int opIdx : context.dirtyOperatorIndices) { + context.opContext[opIdx] = operators[opIdx].makeUpdateContext(context.workingChunkSize); + } + } + public UpdateByWindowContext makeWindowContext(final TrackingRowSet sourceRowSet, - final ColumnSource[] inputSources, final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa, final int chunkSize, final boolean isInitializeStep) { - return new UpdateByWindowTicksContext(sourceRowSet, inputSources, timestampColumnSource, timestampSsa, + return new UpdateByWindowTicksContext(sourceRowSet, timestampColumnSource, timestampSsa, chunkSize, isInitializeStep); } @@ -121,19 +142,17 @@ private void ensureGetContextSize(UpdateByWindowTicksContext ctx, long newSize) // use this to determine which input sources are initialized Arrays.fill(ctx.inputSourceChunkPopulated, false); - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (ctx.operatorIsDirty[opIdx]) { - final int[] sourceIndices = operatorInputSourceSlots[opIdx]; - for (int sourceSlot : sourceIndices) { - if (!ctx.inputSourceChunkPopulated[sourceSlot]) { - // close the existing context - ctx.inputSourceGetContexts[sourceSlot].close(); - - // create a new context of the larger size - ctx.inputSourceGetContexts[sourceSlot] = - ctx.inputSources[sourceSlot].makeGetContext(ctx.currentGetContextSize); - ctx.inputSourceChunkPopulated[sourceSlot] = true; - } + for (int opIdx : ctx.dirtyOperatorIndices) { + final int[] sourceIndices = operatorInputSourceSlots[opIdx]; + for (int sourceSlot : sourceIndices) { + if (!ctx.inputSourceChunkPopulated[sourceSlot]) { + // close the existing context + ctx.inputSourceGetContexts[sourceSlot].close(); + + // create a new context of the larger size + ctx.inputSourceGetContexts[sourceSlot] = + ctx.inputSources[sourceSlot].makeGetContext(ctx.currentGetContextSize); + ctx.inputSourceChunkPopulated[sourceSlot] = true; } } } @@ -177,6 +196,7 @@ private void loadNextInfluencerChunks(UpdateByWindowTicksContext ctx) { // determine which rows will be needed to compute new values for the affected rows (influencer rows) @Override public void computeAffectedRowsAndOperators(UpdateByWindowContext context, @NotNull TableUpdate upstream) { + UpdateByWindowTicksContext ctx = (UpdateByWindowTicksContext) context; // all rows are affected on the initial step @@ -189,7 +209,8 @@ public void computeAffectedRowsAndOperators(UpdateByWindowContext context, @NotN ctx.influencerPositions = RowSetFactory.flat(ctx.sourceRowSet.size()); // mark all operators as affected by this update - Arrays.fill(ctx.operatorIsDirty, true); + context.dirtyOperatorIndices = IntStream.range(0, operators.length).toArray(); + context.dirtySourceIndices = getUniqueSourceIndices(); makeOperatorContexts(ctx); ctx.isDirty = true; @@ -201,13 +222,26 @@ public void computeAffectedRowsAndOperators(UpdateByWindowContext context, @NotN boolean allAffected = upstream.added().isNonempty() || upstream.removed().isNonempty(); - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - ctx.operatorIsDirty[opIdx] = allAffected - || (upstream.modifiedColumnSet().nonempty() && (operators[opIdx].getInputModifiedColumnSet() == null - || upstream.modifiedColumnSet().containsAny(operators[opIdx].getInputModifiedColumnSet()))); - if (ctx.operatorIsDirty[opIdx]) { - ctx.isDirty = true; + if (allAffected) { + // mark all operators as affected by this update + context.dirtyOperatorIndices = IntStream.range(0, operators.length).toArray(); + context.dirtySourceIndices = getUniqueSourceIndices(); + context.isDirty = true; + } else { + // determine which operators wer affected by this update + TIntArrayList dirtyOperatorList = new TIntArrayList(operators.length); + TIntHashSet inputSourcesSet = new TIntHashSet(getUniqueSourceIndices().length); + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + UpdateByOperator op = operators[opIdx]; + if (upstream.modifiedColumnSet().nonempty() && (op.getInputModifiedColumnSet() == null + || upstream.modifiedColumnSet().containsAny(op.getInputModifiedColumnSet()))) { + dirtyOperatorList.add(opIdx); + inputSourcesSet.addAll(operatorInputSourceSlots[opIdx]); + context.isDirty = true; + } } + context.dirtyOperatorIndices = dirtyOperatorList.toArray(); + context.dirtySourceIndices = inputSourcesSet.toArray(); } if (!ctx.isDirty) { @@ -228,10 +262,10 @@ public void computeAffectedRowsAndOperators(UpdateByWindowContext context, @NotN // other rows can be affected by removes if (upstream.removed().isNonempty()) { try (final RowSet prev = ctx.sourceRowSet.copyPrev(); - final RowSet removedPositions = prev.invert(upstream.removed()); - final WritableRowSet affectedByRemoves = - computeAffectedRowsTicks(prev, upstream.removed(), removedPositions, prevUnits, - fwdUnits)) { + final RowSet removedPositions = prev.invert(upstream.removed()); + final WritableRowSet affectedByRemoves = + computeAffectedRowsTicks(prev, upstream.removed(), removedPositions, prevUnits, + fwdUnits)) { // apply shifts to get back to pos-shift space upstream.shifted().apply(affectedByRemoves); // retain only the rows that still exist in the sourceRowSet @@ -255,47 +289,15 @@ public void computeAffectedRowsAndOperators(UpdateByWindowContext context, @NotN } @Override - protected void makeOperatorContexts(UpdateByWindowContext context) { + public void processRows(UpdateByWindowContext context, boolean initialStep) { UpdateByWindowTicksContext ctx = (UpdateByWindowTicksContext) context; - // use this to determine which input sources are initialized - Arrays.fill(ctx.inputSourceChunkPopulated, false); - - // create contexts for the affected operators - ctx.currentGetContextSize = UpdateByWindowTicksContext.WINDOW_CHUNK_SIZE; - - // working chunk size need not be larger than affectedRows.size() - ctx.workingChunkSize = Math.min(ctx.workingChunkSize, ctx.affectedRows.intSize()); + Assert.neqNull(context.inputSources, "assignInputSources() must be called before processRow()"); - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (ctx.operatorIsDirty[opIdx]) { - // create the fill contexts for the input sources - final int[] sourceIndices = operatorInputSourceSlots[opIdx]; - final ColumnSource[] inputSourceArr = new ColumnSource[sourceIndices.length]; - for (int ii = 0; ii < sourceIndices.length; ii++) { - int sourceSlot = sourceIndices[ii]; - if (!ctx.inputSourceChunkPopulated[sourceSlot]) { - ctx.inputSourceGetContexts[sourceSlot] = - ctx.inputSources[sourceSlot].makeGetContext(ctx.currentGetContextSize); - ctx.inputSourceChunkPopulated[sourceSlot] = true; - } - inputSourceArr[ii] = ctx.inputSources[sourceSlot]; - } - ctx.opContext[opIdx] = operators[opIdx].makeUpdateContext(ctx.workingChunkSize, inputSourceArr); - } - } - } - - @Override - public void processRows(UpdateByWindowContext context, ColumnSource[] inputSources, boolean initialStep) { - UpdateByWindowTicksContext ctx = (UpdateByWindowTicksContext) context; - - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (ctx.operatorIsDirty[opIdx]) { - UpdateByWindowedOperator winOp = (UpdateByWindowedOperator) operators[opIdx]; - // call the specialized version of `intializeUpdate()` for these operators - winOp.initializeUpdate(ctx.opContext[opIdx]); - } + for (int opIdx : context.dirtyOperatorIndices) { + UpdateByWindowedOperator winOp = (UpdateByWindowedOperator) operators[opIdx]; + // call the specialized version of `intializeUpdate()` for these operators + winOp.initializeUpdate(ctx.opContext[opIdx]); } ctx.influencerIt = ctx.influencerRows.getRowSequenceIterator(); @@ -382,42 +384,32 @@ public void processRows(UpdateByWindowContext context, ColumnSource[] inputSo ensureGetContextSize(ctx, chunkInfluencerRs.size()); Arrays.fill(ctx.inputSourceChunkPopulated, false); - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (ctx.operatorIsDirty[opIdx]) { - // prep the chunk array needed by the accumulate call - final int[] srcIndices = operatorInputSourceSlots[opIdx]; - Chunk[] chunkArr = new Chunk[srcIndices.length]; - for (int ii = 0; ii < srcIndices.length; ii++) { - int srcIdx = srcIndices[ii]; - // chunk prep - prepareValuesChunkForSource(ctx, srcIdx, chunkInfluencerRs); - chunkArr[ii] = ctx.inputSourceChunks[srcIdx]; - } - - // make the specialized call for windowed operators - ((UpdateByWindowedOperator.Context) ctx.opContext[opIdx]).accumulate( - chunkRs, - chunkArr, - pushChunk, - popChunk, - chunkRsSize); + for (int opIdx : context.dirtyOperatorIndices) { + // prep the chunk array needed by the accumulate call + final int[] srcIndices = operatorInputSourceSlots[opIdx]; + Chunk[] chunkArr = new Chunk[srcIndices.length]; + for (int ii = 0; ii < srcIndices.length; ii++) { + int srcIdx = srcIndices[ii]; + // chunk prep + prepareValuesChunkForSource(ctx, srcIdx, chunkInfluencerRs); + chunkArr[ii] = ctx.inputSourceChunks[srcIdx]; } + + // make the specialized call for windowed operators + ((UpdateByWindowedOperator.Context) ctx.opContext[opIdx]).accumulate( + chunkRs, + chunkArr, + pushChunk, + popChunk, + chunkRsSize); } } } } // call `finishUpdate()` function for each operator - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (ctx.operatorIsDirty[opIdx]) { - operators[opIdx].finishUpdate(ctx.opContext[opIdx]); - } + for (int opIdx : context.dirtyOperatorIndices) { + operators[opIdx].finishUpdate(context.opContext[opIdx]); } } - - UpdateByWindowTicks(UpdateByOperator[] operators, int[][] operatorSourceSlots, long prevUnits, long fwdUnits) { - super(operators, operatorSourceSlots, null); - this.prevUnits = prevUnits; - this.fwdUnits = fwdUnits; - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java index d2df5dcc437..af3288a76e4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java @@ -1,5 +1,7 @@ package io.deephaven.engine.table.impl.updateby; +import gnu.trove.list.array.TIntArrayList; +import gnu.trove.set.hash.TIntHashSet; import io.deephaven.base.ringbuffer.LongRingBuffer; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; @@ -19,6 +21,7 @@ import org.jetbrains.annotations.Nullable; import java.util.Arrays; +import java.util.stream.IntStream; import static io.deephaven.util.QueryConstants.NULL_LONG; @@ -43,10 +46,10 @@ public class UpdateByWindowTimeContext extends UpdateByWindowContext { protected long influencerTimestampChunkSize; protected int currentGetContextSize; - public UpdateByWindowTimeContext(final TrackingRowSet sourceRowSet, final ColumnSource[] inputSources, + public UpdateByWindowTimeContext(final TrackingRowSet sourceRowSet, @NotNull final ColumnSource timestampColumnSource, @Nullable final LongSegmentedSortedArray timestampSsa, final int chunkSize, final boolean initialStep) { - super(sourceRowSet, inputSources, timestampColumnSource, timestampSsa, chunkSize, initialStep); + super(sourceRowSet, timestampColumnSource, timestampSsa, chunkSize, initialStep); influencerTimestampContext = timestampColumnSource.makeGetContext(WINDOW_CHUNK_SIZE); currentWindowTimestamps = new LongRingBuffer(512, true); @@ -63,13 +66,32 @@ public void close() { } + UpdateByWindowTime(UpdateByOperator[] operators, int[][] operatorSourceSlots, @Nullable String timestampColumnName, + long prevUnits, long fwdUnits) { + super(operators, operatorSourceSlots, timestampColumnName); + this.prevUnits = prevUnits; + this.fwdUnits = fwdUnits; + } + + @Override + protected void makeOperatorContexts(UpdateByWindowContext context) { + UpdateByWindowTimeContext ctx = (UpdateByWindowTimeContext) context; + + ctx.workingChunkSize = UpdateByWindowTimeContext.WINDOW_CHUNK_SIZE; + ctx.currentGetContextSize = ctx.workingChunkSize; + + // create contexts for the affected operators + for (int opIdx : context.dirtyOperatorIndices) { + context.opContext[opIdx] = operators[opIdx].makeUpdateContext(context.workingChunkSize); + } + } + public UpdateByWindowContext makeWindowContext(final TrackingRowSet sourceRowSet, - final ColumnSource[] inputSources, final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa, final int chunkSize, final boolean isInitializeStep) { - return new UpdateByWindowTimeContext(sourceRowSet, inputSources, timestampColumnSource, timestampSsa, chunkSize, + return new UpdateByWindowTimeContext(sourceRowSet, timestampColumnSource, timestampSsa, chunkSize, isInitializeStep); } @@ -152,19 +174,17 @@ private void ensureGetContextSize(UpdateByWindowTimeContext ctx, long newSize) { // use this to determine which input sources are initialized Arrays.fill(ctx.inputSourceChunkPopulated, false); - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (ctx.operatorIsDirty[opIdx]) { - final int[] sourceIndices = operatorInputSourceSlots[opIdx]; - for (int sourceSlot : sourceIndices) { - if (!ctx.inputSourceChunkPopulated[sourceSlot]) { - // close the existing context - ctx.inputSourceGetContexts[sourceSlot].close(); - - // create a new context of the larger size - ctx.inputSourceGetContexts[sourceSlot] = - ctx.inputSources[sourceSlot].makeGetContext(ctx.currentGetContextSize); - ctx.inputSourceChunkPopulated[sourceSlot] = true; - } + for (int opIdx : ctx.dirtyOperatorIndices) { + final int[] sourceIndices = operatorInputSourceSlots[opIdx]; + for (int sourceSlot : sourceIndices) { + if (!ctx.inputSourceChunkPopulated[sourceSlot]) { + // close the existing context + ctx.inputSourceGetContexts[sourceSlot].close(); + + // create a new context of the larger size + ctx.inputSourceGetContexts[sourceSlot] = + ctx.inputSources[sourceSlot].makeGetContext(ctx.currentGetContextSize); + ctx.inputSourceChunkPopulated[sourceSlot] = true; } } } @@ -209,7 +229,8 @@ public void computeAffectedRowsAndOperators(UpdateByWindowContext context, @NotN ctx.influencerRows = ctx.affectedRows; // mark all operators as affected by this update - Arrays.fill(ctx.operatorIsDirty, true); + context.dirtyOperatorIndices = IntStream.range(0, operators.length).toArray(); + context.dirtySourceIndices = getUniqueSourceIndices(); makeOperatorContexts(ctx); ctx.isDirty = true; @@ -221,13 +242,26 @@ public void computeAffectedRowsAndOperators(UpdateByWindowContext context, @NotN boolean allAffected = upstream.added().isNonempty() || upstream.removed().isNonempty(); - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - ctx.operatorIsDirty[opIdx] = allAffected - || (upstream.modifiedColumnSet().nonempty() && (operators[opIdx].getInputModifiedColumnSet() == null - || upstream.modifiedColumnSet().containsAny(operators[opIdx].getInputModifiedColumnSet()))); - if (ctx.operatorIsDirty[opIdx]) { - ctx.isDirty = true; + if (allAffected) { + // mark all operators as affected by this update + context.dirtyOperatorIndices = IntStream.range(0, operators.length).toArray(); + context.dirtySourceIndices = getUniqueSourceIndices(); + context.isDirty = true; + } else { + // determine which operators are affected by this update + TIntArrayList dirtyOperatorList = new TIntArrayList(operators.length); + TIntHashSet inputSourcesSet = new TIntHashSet(getUniqueSourceIndices().length); + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + UpdateByOperator op = operators[opIdx]; + if (upstream.modifiedColumnSet().nonempty() && (op.getInputModifiedColumnSet() == null + || upstream.modifiedColumnSet().containsAny(op.getInputModifiedColumnSet()))) { + dirtyOperatorList.add(opIdx); + inputSourcesSet.addAll(operatorInputSourceSlots[opIdx]); + context.isDirty = true; + } } + context.dirtyOperatorIndices = dirtyOperatorList.toArray(); + context.dirtySourceIndices = inputSourcesSet.toArray(); } if (!ctx.isDirty) { @@ -263,39 +297,6 @@ public void computeAffectedRowsAndOperators(UpdateByWindowContext context, @NotN makeOperatorContexts(ctx); } - @Override - protected void makeOperatorContexts(UpdateByWindowContext context) { - UpdateByWindowTimeContext ctx = (UpdateByWindowTimeContext) context; - - // use this to make which input sources are initialized - Arrays.fill(ctx.inputSourceChunkPopulated, false); - - // create contexts for the affected operators - ctx.currentGetContextSize = UpdateByWindowTimeContext.WINDOW_CHUNK_SIZE; - - // working chunk size need not be larger than affectedRows.size() - ctx.workingChunkSize = Math.min(ctx.workingChunkSize, ctx.affectedRows.intSize()); - - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (ctx.operatorIsDirty[opIdx]) { - // create the fill contexts for the input sources - final int[] sourceIndices = operatorInputSourceSlots[opIdx]; - final ColumnSource[] inputSourceArr = new ColumnSource[sourceIndices.length]; - for (int ii = 0; ii < sourceIndices.length; ii++) { - int sourceSlot = sourceIndices[ii]; - if (!ctx.inputSourceChunkPopulated[sourceSlot]) { - ctx.inputSourceGetContexts[sourceSlot] = - ctx.inputSources[sourceSlot].makeGetContext(ctx.currentGetContextSize); - ctx.inputSourceChunkPopulated[sourceSlot] = true; - } - inputSourceArr[ii] = ctx.inputSources[sourceSlot]; - } - ctx.opContext[opIdx] = operators[opIdx].makeUpdateContext(ctx.workingChunkSize, inputSourceArr); - } - } - } - - /*** * This function process the affected rows chunkwise, and will advance the moving window (which is the same for all * operators in this collection). For each row in the dataset the sliding window will adjust and instructions for @@ -306,15 +307,13 @@ protected void makeOperatorContexts(UpdateByWindowContext context) { * calls do not provide the popped data */ @Override - public void processRows(UpdateByWindowContext context, ColumnSource[] inputSources, boolean initialStep) { + public void processRows(UpdateByWindowContext context, boolean initialStep) { UpdateByWindowTimeContext ctx = (UpdateByWindowTimeContext) context; - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (ctx.operatorIsDirty[opIdx]) { - UpdateByWindowedOperator winOp = (UpdateByWindowedOperator) operators[opIdx]; - // call the specialized version of `intializeUpdate()` for these operators - winOp.initializeUpdate(ctx.opContext[opIdx]); - } + for (int opIdx : context.dirtyOperatorIndices) { + UpdateByWindowedOperator winOp = (UpdateByWindowedOperator) operators[opIdx]; + // call the specialized version of `intializeUpdate()` for these operators + winOp.initializeUpdate(ctx.opContext[opIdx]); } ctx.influencerIt = ctx.influencerRows.getRowSequenceIterator(); @@ -402,8 +401,7 @@ public void processRows(UpdateByWindowContext context, ColumnSource[] inputSo ensureGetContextSize(ctx, chunkInfluencerRs.size()); Arrays.fill(ctx.inputSourceChunkPopulated, false); - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (ctx.operatorIsDirty[opIdx]) { + for (int opIdx : context.dirtyOperatorIndices) { // prep the chunk array needed by the accumulate call final int[] srcIndices = operatorInputSourceSlots[opIdx]; Chunk[] chunkArr = new Chunk[srcIndices.length]; @@ -424,21 +422,11 @@ public void processRows(UpdateByWindowContext context, ColumnSource[] inputSo } } } - } } // call `finishUpdate()` function for each operator - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (ctx.operatorIsDirty[opIdx]) { - operators[opIdx].finishUpdate(ctx.opContext[opIdx]); - } + for (int opIdx : context.dirtyOperatorIndices) { + operators[opIdx].finishUpdate(context.opContext[opIdx]); } } - - UpdateByWindowTime(UpdateByOperator[] operators, int[][] operatorSourceSlots, @Nullable String timestampColumnName, - long prevUnits, long fwdUnits) { - super(operators, operatorSourceSlots, timestampColumnName); - this.prevUnits = prevUnits; - this.fwdUnits = fwdUnits; - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java index 8549c65a2d8..7df243e9e5c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java @@ -17,8 +17,8 @@ public class BigDecimalEMAOperator extends BigNumberEMAOperator { public class Context extends BigNumberEMAOperator.Context { - protected Context(int chunkSize, ColumnSource[] inputSourceArr) { - super(chunkSize, inputSourceArr); + protected Context(int chunkSize) { + super(chunkSize); } @Override @@ -110,13 +110,14 @@ public BigDecimalEMAOperator(@NotNull final MatchPair pair, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { - super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, redirContext); + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, + final ColumnSource valueSource) { + super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, redirContext, valueSource); } @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, ColumnSource[] inputSourceArr) { - return new Context(chunkSize, inputSourceArr); + public UpdateContext makeUpdateContext(final int chunkSize) { + return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java index cb0d687eab8..d008f0bdfe1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java @@ -18,8 +18,8 @@ public class BigIntegerEMAOperator extends BigNumberEMAOperator { public class Context extends BigNumberEMAOperator.Context { - protected Context(int chunkSize, ColumnSource[] inputSourceArr) { - super(chunkSize, inputSourceArr); + protected Context(int chunkSize) { + super(chunkSize); } @Override @@ -112,14 +112,14 @@ public BigIntegerEMAOperator(@NotNull final MatchPair pair, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext - ) { - super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, redirContext); + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, + final ColumnSource valueSource) { + super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, redirContext, valueSource); } @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, ColumnSource[] inputSourceArr) { - return new Context(chunkSize, inputSourceArr); + public UpdateContext makeUpdateContext(final int chunkSize) { + return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java index ab366c05a85..5e50df22672 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java @@ -20,6 +20,7 @@ import static io.deephaven.util.QueryConstants.NULL_LONG; public abstract class BigNumberEMAOperator extends BaseObjectUpdateByOperator { + protected final ColumnSource valueSource; protected final OperationControl control; protected final double timeScaleUnits; protected final BigDecimal alpha; @@ -27,15 +28,13 @@ public abstract class BigNumberEMAOperator extends BaseObjectUpdateByOperator public abstract class Context extends BaseObjectUpdateByOperator.Context { - protected final ColumnSource valueSource; public LongChunk timestampValueChunk; public ObjectChunk objectValueChunk; long lastStamp = NULL_LONG; - protected Context(int chunkSize, ColumnSource[] inputSourceArr) { + protected Context(int chunkSize) { super(chunkSize); - this.valueSource = inputSourceArr[0]; } @Override @@ -74,12 +73,14 @@ public BigNumberEMAOperator(@NotNull final MatchPair pair, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, + final ColumnSource valueSource) { super(pair, affectingColumns, redirContext, BigDecimal.class); this.control = control; this.timestampColumnName = timestampColumnName; this.timeScaleUnits = (double) timeScaleUnits; + this.valueSource = valueSource; alpha = BigDecimal.valueOf(Math.exp(-1.0 / (double) timeScaleUnits)); oneMinusAlpha = diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java index 5df8c92ee1f..b6920051b81 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java @@ -20,15 +20,14 @@ import static io.deephaven.util.QueryConstants.*; public class ByteEMAOperator extends BasePrimitiveEMAOperator { + public final ColumnSource valueSource; protected class Context extends BasePrimitiveEMAOperator.Context { - public final ColumnSource valueSource; public ByteChunk byteValueChunk; - protected Context(int chunkSize, ColumnSource inputSourceArr[]) { + protected Context(int chunkSize) { super(chunkSize); - this.valueSource = inputSourceArr[0]; } @Override @@ -122,18 +121,20 @@ public ByteEMAOperator(@NotNull final MatchPair pair, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, + final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args ) { super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits,redirContext); + this.valueSource = valueSource; // region constructor // endregion constructor } @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { - return new Context(chunkSize, inputSourceArr); + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java index df5bc51dac0..212c9221adb 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java @@ -22,14 +22,13 @@ import static io.deephaven.util.QueryConstants.*; public class DoubleEMAOperator extends BasePrimitiveEMAOperator { - protected class Context extends BasePrimitiveEMAOperator.Context { - private final ColumnSource valueSource; + private final ColumnSource valueSource; + protected class Context extends BasePrimitiveEMAOperator.Context { public DoubleChunk doubleValueChunk; - protected Context(int chunkSize, ColumnSource[] inputSourceArr) { + protected Context(int chunkSize) { super(chunkSize); - this.valueSource = inputSourceArr[0]; } @Override @@ -139,18 +138,20 @@ public DoubleEMAOperator(@NotNull final MatchPair pair, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, + final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args ) { super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, redirContext); + this.valueSource = valueSource; // region constructor // endregion constructor } @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { - return new Context(chunkSize, inputSourceArr); + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java index 6c45ab51e05..e4b32717f94 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java @@ -17,14 +17,13 @@ import static io.deephaven.util.QueryConstants.*; public class FloatEMAOperator extends BasePrimitiveEMAOperator { - protected class Context extends BasePrimitiveEMAOperator.Context { - private final ColumnSource valueSource; + private final ColumnSource valueSource; + protected class Context extends BasePrimitiveEMAOperator.Context { public FloatChunk floatValueChunk; - protected Context(int chunkSize, ColumnSource[] inputSourceArr) { + protected Context(int chunkSize) { super(chunkSize); - this.valueSource = inputSourceArr[0]; } @Override @@ -134,18 +133,20 @@ public FloatEMAOperator(@NotNull final MatchPair pair, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, + final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args ) { super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, redirContext); + this.valueSource = valueSource; // region constructor // endregion constructor } @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { - return new Context(chunkSize, inputSourceArr); + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java index 439dee4904b..8887dae9029 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java @@ -20,15 +20,14 @@ import static io.deephaven.util.QueryConstants.*; public class IntEMAOperator extends BasePrimitiveEMAOperator { + public final ColumnSource valueSource; protected class Context extends BasePrimitiveEMAOperator.Context { - public final ColumnSource valueSource; public IntChunk intValueChunk; - protected Context(int chunkSize, ColumnSource inputSourceArr[]) { + protected Context(int chunkSize) { super(chunkSize); - this.valueSource = inputSourceArr[0]; } @Override @@ -122,18 +121,20 @@ public IntEMAOperator(@NotNull final MatchPair pair, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, + final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args ) { super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits,redirContext); + this.valueSource = valueSource; // region constructor // endregion constructor } @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { - return new Context(chunkSize, inputSourceArr); + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java index 738d2293edf..17d1fd7407c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java @@ -20,15 +20,14 @@ import static io.deephaven.util.QueryConstants.*; public class LongEMAOperator extends BasePrimitiveEMAOperator { + public final ColumnSource valueSource; protected class Context extends BasePrimitiveEMAOperator.Context { - public final ColumnSource valueSource; public LongChunk longValueChunk; - protected Context(int chunkSize, ColumnSource inputSourceArr[]) { + protected Context(int chunkSize) { super(chunkSize); - this.valueSource = inputSourceArr[0]; } @Override @@ -122,18 +121,20 @@ public LongEMAOperator(@NotNull final MatchPair pair, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, + final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args ) { super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits,redirContext); + this.valueSource = valueSource; // region constructor // endregion constructor } @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { - return new Context(chunkSize, inputSourceArr); + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java index aec0ad682f5..4361f1d94c3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java @@ -15,15 +15,14 @@ import static io.deephaven.util.QueryConstants.*; public class ShortEMAOperator extends BasePrimitiveEMAOperator { + public final ColumnSource valueSource; protected class Context extends BasePrimitiveEMAOperator.Context { - public final ColumnSource valueSource; public ShortChunk shortValueChunk; - protected Context(int chunkSize, ColumnSource inputSourceArr[]) { + protected Context(int chunkSize) { super(chunkSize); - this.valueSource = inputSourceArr[0]; } @Override @@ -117,18 +116,20 @@ public ShortEMAOperator(@NotNull final MatchPair pair, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, + final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args ) { super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits,redirContext); + this.valueSource = valueSource; // region constructor // endregion constructor } @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { - return new Context(chunkSize, inputSourceArr); + public UpdateContext makeUpdateContext(int chunkSize) { + return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java index 503495a4c9b..4c930228f87 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java @@ -65,7 +65,7 @@ public BooleanFillByOperator(@NotNull final MatchPair fillPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(int chunkSize) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java index 15a6d327f5e..ad8a5235b45 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java @@ -58,7 +58,7 @@ public ByteFillByOperator(@NotNull final MatchPair fillPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(int chunkSize) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java index 464635953dc..0dadeed655c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java @@ -53,7 +53,7 @@ public CharFillByOperator(@NotNull final MatchPair fillPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(int chunkSize) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java index 2d5f9f2bbbf..c4fb2f64b2c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java @@ -58,7 +58,7 @@ public DoubleFillByOperator(@NotNull final MatchPair fillPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(int chunkSize) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java index b9997b58a94..b59b07bb777 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java @@ -58,7 +58,7 @@ public FloatFillByOperator(@NotNull final MatchPair fillPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(int chunkSize) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java index 3357e446667..5c7603c2278 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java @@ -58,7 +58,7 @@ public IntFillByOperator(@NotNull final MatchPair fillPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(int chunkSize) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java index 81aef44e612..c155e43c914 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java @@ -68,7 +68,7 @@ public LongFillByOperator(@NotNull final MatchPair fillPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(int chunkSize) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java index f9bc4b13366..4a1033436d8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java @@ -58,7 +58,7 @@ public ObjectFillByOperator(@NotNull final MatchPair fillPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(int chunkSize) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java index 53c2a1d86c1..188a3d2ba6a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java @@ -58,7 +58,7 @@ public ShortFillByOperator(@NotNull final MatchPair fillPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(int chunkSize) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java index b583ca2017f..2319cde7dc4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java @@ -56,7 +56,7 @@ public BaseObjectBinaryOperator(@NotNull final Class type, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(int chunkSize) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java index 9617e72a141..c3f5dc3f053 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java @@ -69,7 +69,7 @@ public ByteCumMinMaxOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(int chunkSize) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java index 0d6094fbd7c..46a67ee716e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java @@ -69,7 +69,7 @@ public DoubleCumMinMaxOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(int chunkSize) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java index 5c734ff4587..96a8028bbfe 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java @@ -64,7 +64,7 @@ public FloatCumMinMaxOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(int chunkSize) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java index 90bb79faa3f..b35635cc2b9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java @@ -69,7 +69,7 @@ public IntCumMinMaxOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(int chunkSize) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java index 238ed7f0729..240f66cb996 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java @@ -79,7 +79,7 @@ public LongCumMinMaxOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(int chunkSize) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java index 702435af3a4..1577805f777 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java @@ -64,7 +64,7 @@ public ShortCumMinMaxOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(int chunkSize) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java index eb68dcda425..f9a6253b218 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java @@ -60,7 +60,7 @@ public BigDecimalCumProdOperator(@NotNull final MatchPair inputPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(int chunkSize) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java index 5fff158db4b..f4fa9a20263 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java @@ -54,7 +54,7 @@ public BigIntegerCumProdOperator(@NotNull final MatchPair inputPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(int chunkSize) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java index e6ed53d76a6..6268151e9c8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java @@ -62,7 +62,7 @@ public ByteCumProdOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(int chunkSize) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java index c7d3c118c41..acacaeba56c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java @@ -63,7 +63,7 @@ public DoubleCumProdOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(int chunkSize) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java index ea9f07220c9..bdb4bc64da6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java @@ -58,7 +58,7 @@ public FloatCumProdOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(int chunkSize) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java index a7e7caefe59..30b08e3d53b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java @@ -62,7 +62,7 @@ public IntCumProdOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(int chunkSize) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java index 9cc48fbe6ef..7bc9d81d3c9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java @@ -62,7 +62,7 @@ public LongCumProdOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(int chunkSize) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java index ce3154f1062..06bbd87f376 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java @@ -57,7 +57,7 @@ public ShortCumProdOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(int chunkSize) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java index 8f75aaeaab8..0a0a25d20cb 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java @@ -81,7 +81,7 @@ public void writeToOutputChunk(int outIdx) { @NotNull @Override - public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java index c754e9c5d97..699dc14bffa 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java @@ -78,7 +78,7 @@ public void writeToOutputChunk(int outIdx) { @NotNull @Override - public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java index d4cb7ceac5e..0e6f65dff1d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java @@ -87,7 +87,7 @@ public void writeToOutputChunk(int outIdx) { @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(final int chunkSize) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java index 058f2d5c354..f61df2b4eac 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java @@ -85,7 +85,7 @@ public void writeToOutputChunk(int outIdx) { @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(final int chunkSize) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java index f7ecfe15e94..02afe940f2e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java @@ -80,7 +80,7 @@ public void writeToOutputChunk(int outIdx) { @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(final int chunkSize) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java index a80aa7004d7..e75b5de169e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java @@ -86,7 +86,7 @@ public void writeToOutputChunk(int outIdx) { @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(final int chunkSize) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java index 9a32bc743e2..b850d56f430 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java @@ -86,7 +86,7 @@ public void writeToOutputChunk(int outIdx) { @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(final int chunkSize) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java index 2a17ca9460b..9f552a0daca 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java @@ -81,7 +81,7 @@ public void writeToOutputChunk(int outIdx) { @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(final int chunkSize) { return new Context(chunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java index ddf5b39a206..ff57c6c31d8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java @@ -61,7 +61,7 @@ public BigDecimalCumSumOperator(@NotNull final MatchPair inputPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(int chunkSize) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java index 35982a5cc13..7790d7dd31b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java @@ -54,7 +54,7 @@ public BigIntegerCumSumOperator(@NotNull final MatchPair inputPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(int chunkSize) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java index e50c9acf804..d9a6ada25f7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java @@ -64,7 +64,7 @@ public ByteCumSumOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(int chunkSize) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java index 965e88f6d88..aef0e9d2736 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java @@ -62,7 +62,7 @@ public DoubleCumSumOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(int chunkSize) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java index 32e41f00e9c..93622801f84 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java @@ -57,7 +57,7 @@ public FloatCumSumOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(int chunkSize) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java index 70527d6d5dd..dfdfbe808c6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java @@ -61,7 +61,7 @@ public IntCumSumOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(int chunkSize) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java index 157d815150c..72c698d64b2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java @@ -61,7 +61,7 @@ public LongCumSumOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(int chunkSize) { return new Context(chunkSize); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java index 8fa409c1b35..c29a01e9475 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java @@ -56,7 +56,7 @@ public ShortCumSumOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize, ColumnSource[] inputSourceArr) { + public UpdateContext makeUpdateContext(int chunkSize) { return new Context(chunkSize); } } From fa568dd7a5e64e13933440c6b630d1a2af3354e7 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 1 Nov 2022 12:19:46 -0700 Subject: [PATCH 035/123] Trying parallel population --- .../BucketedPartitionedUpdateByManager.java | 2 +- .../deephaven/engine/table/impl/UpdateBy.java | 96 ++++++++++++++----- .../engine/table/impl/UpdateByOperator.java | 7 +- .../table/impl/ZeroKeyUpdateByManager.java | 2 +- .../table/impl/updateby/UpdateByWindow.java | 6 ++ .../internal/BaseByteUpdateByOperator.java | 8 ++ .../internal/BaseCharUpdateByOperator.java | 8 ++ .../internal/BaseDoubleUpdateByOperator.java | 8 ++ .../internal/BaseFloatUpdateByOperator.java | 8 ++ .../internal/BaseIntUpdateByOperator.java | 8 ++ .../internal/BaseLongUpdateByOperator.java | 8 ++ .../internal/BaseObjectUpdateByOperator.java | 8 ++ .../internal/BaseShortUpdateByOperator.java | 8 ++ .../BaseWindowedByteUpdateByOperator.java | 8 ++ .../BaseWindowedCharUpdateByOperator.java | 8 ++ .../BaseWindowedDoubleUpdateByOperator.java | 19 ++++ .../BaseWindowedFloatUpdateByOperator.java | 19 ++++ .../BaseWindowedIntUpdateByOperator.java | 8 ++ .../BaseWindowedLongUpdateByOperator.java | 8 ++ .../BaseWindowedObjectUpdateByOperator.java | 8 ++ .../BaseWindowedShortUpdateByOperator.java | 8 ++ .../impl/util/InverseRowRedirectionImpl.java | 2 + .../table/impl/updateby/TestCumSum.java | 2 +- 23 files changed, 238 insertions(+), 29 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java index f8277170efc..f0ccbf91b6f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java @@ -155,7 +155,7 @@ public void onUpdate(@NotNull final TableUpdate upstream) { // do the actual computations UpdateByBucketHelper[] dirtyBuckets = buckets.toArray(UpdateByBucketHelper[]::new); - processBuckets(dirtyBuckets, true); + processBuckets(dirtyBuckets, true, source.getRowSet()); finalizeBuckets(dirtyBuckets); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java index 0aa6140afd4..a98a130c039 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java @@ -6,26 +6,25 @@ import io.deephaven.api.ColumnName; import io.deephaven.api.updateby.UpdateByOperation; import io.deephaven.api.updateby.UpdateByControl; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.ResettableWritableObjectChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.exceptions.UncheckedTableException; import io.deephaven.engine.rowset.*; import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.sources.FillUnordered; -import io.deephaven.engine.table.impl.sources.LongSparseArraySource; -import io.deephaven.engine.table.impl.sources.ReinterpretUtils; -import io.deephaven.engine.table.impl.sources.SparseArrayColumnSource; +import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.sources.sparse.SparseConstants; import io.deephaven.engine.table.impl.updateby.UpdateByWindow; import io.deephaven.engine.table.impl.util.InverseRowRedirectionImpl; import io.deephaven.engine.table.impl.util.LongColumnSourceWritableRowRedirection; import io.deephaven.engine.table.impl.util.WritableRowRedirection; -import io.deephaven.util.SafeCloseable; import org.apache.commons.lang3.mutable.MutableInt; import org.apache.commons.lang3.mutable.MutableLong; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import java.lang.ref.SoftReference; import java.util.*; import java.util.concurrent.atomic.AtomicInteger; @@ -57,6 +56,10 @@ public abstract class UpdateBy { * Whether caching benefits this input source */ protected final boolean[] inputSourceCacheNeeded; + /** + * References to the dense array sources we are using for the cached sources + */ + protected final SoftReference>[] inputSourceCaches; /** * The output table for this UpdateBy operation @@ -183,15 +186,29 @@ protected UpdateBy(@NotNull final String description, cacheNeeded |= inputSourceCacheNeeded[ii]; } this.inputCacheNeeded = cacheNeeded; + // noinspection unchecked + inputSourceCaches = new SoftReference[inputSources.length]; buckets = new LinkedList<>(); } - private ColumnSource getCachedColumn(ColumnSource inputSource, final RowSet inputRowSet) { - final SparseArrayColumnSource outputSource = SparseArrayColumnSource - .getSparseMemoryColumnSource(inputSource.getType(), inputSource.getComponentType()); + private ColumnSource createCachedColumnSource(int srcIdx, final TrackingWritableRowSet inputRowSet) { + final ColumnSource inputSource = inputSources[srcIdx]; - final int CHUNK_SIZE = 1 << 16; + // re-use the dense column cache if it still exists + WritableColumnSource innerSource; + if (inputSourceCaches[srcIdx] == null || (innerSource = inputSourceCaches[srcIdx].get()) == null) { + // create a new dense cache + innerSource = ArrayBackedColumnSource.getMemoryColumnSource(inputSource.getType(), inputSource.getComponentType()); + inputSourceCaches[srcIdx] = new SoftReference<>(innerSource); + } + innerSource.ensureCapacity(inputRowSet.size()); + + final WritableRowRedirection rowRedirection = new InverseRowRedirectionImpl(inputRowSet); + final WritableColumnSource outputSource = + new WritableRedirectedColumnSource(rowRedirection, innerSource, 0); + + final int CHUNK_SIZE = 1 << 16; // copied from SparseSelect try (final RowSequence.Iterator rsIt = inputRowSet.getRowSequenceIterator(); final ChunkSink.FillFromContext ffc = @@ -204,6 +221,7 @@ private ColumnSource getCachedColumn(ColumnSource inputSource, final RowSe } } + // holding this reference will protect `rowDirection` and `innerSource` from GC return outputSource; } @@ -216,7 +234,8 @@ private void computeCachedColumnContents(UpdateByBucketHelper[] buckets, boolean if (initialStep) { for (int srcIdx = 0; srcIdx < inputSources.length; srcIdx++) { if (inputSourceCacheNeeded[srcIdx]) { - inputSourceRowSets[srcIdx] = source.getRowSet().copy(); + // this needs to be a TrackingRowSet to be used by `InverseRowRedirectionImpl` + inputSourceRowSets[srcIdx] = source.getRowSet().copy().toTracking(); } } @@ -257,7 +276,7 @@ private void computeCachedColumnContents(UpdateByBucketHelper[] buckets, boolean cacheNeeded[srcIdx] = true; // add this rowset to the running total if (inputSourceRowSets[srcIdx] == null) { - inputSourceRowSets[srcIdx] = win.getInfluencerRows(winCtx).copy(); + inputSourceRowSets[srcIdx] = win.getInfluencerRows(winCtx).copy().toTracking(); } else { inputSourceRowSets[srcIdx].insert(win.getInfluencerRows(winCtx)); } @@ -280,17 +299,30 @@ private void computeCachedColumnContents(UpdateByBucketHelper[] buckets, boolean } } - private void cacheInputSources(int winIdx, ColumnSource[] maybeCachedInputSources, WritableRowSet[] inputSourceRowSets) { + private void cacheInputSources(int winIdx, ColumnSource[] maybeCachedInputSources, TrackingWritableRowSet[] inputSourceRowSets) { final UpdateByWindow win = windows[winIdx]; final int[] uniqueWindowSources = win.getUniqueSourceIndices(); for (int srcIdx : uniqueWindowSources) { if (maybeCachedInputSources[srcIdx] == null) { - maybeCachedInputSources[srcIdx] = getCachedColumn(inputSources[srcIdx], inputSourceRowSets[srcIdx]); + maybeCachedInputSources[srcIdx] = createCachedColumnSource(srcIdx, inputSourceRowSets[srcIdx]); + } + } + } + + private void fillObjectArraySourceWithNull(ObjectArraySource sourceToNull) { + Assert.neqNull(sourceToNull, "cached column source was null, must have been GC'd"); + try (final ResettableWritableObjectChunk backingChunk = + ResettableWritableObjectChunk.makeResettableChunk()) { + Assert.neqNull(sourceToNull, "cached column source was already GC'd"); + final long targetCapacity = sourceToNull.getCapacity(); + for (long positionToNull = 0; positionToNull < targetCapacity; positionToNull += backingChunk.size()) { + sourceToNull.resetWritableChunkToBackingStore(backingChunk, positionToNull); + backingChunk.fillWithNullValue(0, backingChunk.size()); } } } - private void releaseInputSources(int winIdx, ColumnSource[] maybeCachedInputSources, WritableRowSet[] inputSourceRowSets, AtomicInteger[] inputSourceReferenceCounts) { + private void releaseInputSources(int winIdx, ColumnSource[] maybeCachedInputSources, TrackingWritableRowSet[] inputSourceRowSets, AtomicInteger[] inputSourceReferenceCounts) { final UpdateByWindow win = windows[winIdx]; final int[] uniqueWindowSources = win.getUniqueSourceIndices(); for (int srcIdx : uniqueWindowSources) { @@ -300,6 +332,11 @@ private void releaseInputSources(int winIdx, ColumnSource[] maybeCachedInputS inputSourceRowSets[srcIdx].close(); inputSourceRowSets[srcIdx] = null; + // release any objects we are holding in the cache + if (inputSourceCaches[srcIdx].get() instanceof ObjectArraySource) { + fillObjectArraySourceWithNull((ObjectArraySource) inputSourceCaches[srcIdx].get()); + } + maybeCachedInputSources[srcIdx] = null; } } @@ -322,16 +359,16 @@ protected void shiftOutputColumns(TableUpdate upstream) { } } - protected void processBuckets(UpdateByBucketHelper[] dirtyBuckets, final boolean initialStep) { + protected void processBuckets(UpdateByBucketHelper[] dirtyBuckets, final boolean initialStep, final RowSet added) { if (inputCacheNeeded) { // this will store the input sources needed for processing final ColumnSource[] maybeCachedInputSources = new ColumnSource[inputSources.length]; - final WritableRowSet[] inputSourceRowSets = new WritableRowSet[inputSources.length]; + final TrackingWritableRowSet[] inputSourceRowSets = new TrackingWritableRowSet[inputSources.length]; final AtomicInteger[] inputSourceReferenceCounts = new AtomicInteger[inputSources.length]; computeCachedColumnContents(dirtyBuckets, initialStep, inputSourceRowSets, inputSourceReferenceCounts); - // `null` marks columns that need to be cached + // assign the non-cached input source or leave null for cac for (int srcIdx = 0; srcIdx < inputSources.length; srcIdx++) { maybeCachedInputSources[srcIdx] = inputSourceCacheNeeded[srcIdx] ? null : inputSources[srcIdx]; } @@ -339,6 +376,13 @@ protected void processBuckets(UpdateByBucketHelper[] dirtyBuckets, final boolean for (int winIdx = 0; winIdx < windows.length; winIdx++) { // cache the sources needed for this window cacheInputSources(winIdx, maybeCachedInputSources, inputSourceRowSets); + + if (added.isNonempty()) { + // prepare the array output sources for parallel updates (added records only since we + // handled removes/shifts earlier) + windows[winIdx].prepareForParallelPopulation(added); + } + // process the window for (UpdateByBucketHelper bucket : dirtyBuckets) { bucket.assignInputSources(winIdx, maybeCachedInputSources); @@ -377,13 +421,8 @@ protected void process() { UpdateByBucketHelper[] dirtyBuckets = buckets.stream().filter(b -> b.isDirty()) .toArray(UpdateByBucketHelper[]::new); - // do the actual computations - processBuckets(dirtyBuckets, false); - final TableUpdateImpl downstream = new TableUpdateImpl(); - // get the adds/removes/shifts from the first (source) entry, make a copy since TableUpdateImpl#reset will - // close them with the upstream update ListenerRecorder sourceRecorder = recorders.peekFirst(); downstream.added = sourceRecorder.getAdded().copy(); downstream.removed = sourceRecorder.getRemoved().copy(); @@ -393,6 +432,15 @@ protected void process() { downstream.modifiedColumnSet = result.getModifiedColumnSetForUpdates(); downstream.modifiedColumnSet.clear(); + // do the actual computations + try (final WritableRowSet changedRows = sourceRecorder.getAdded().union(sourceRecorder.getModifiedPreShift())) { + changedRows.insert(sourceRecorder.getRemoved()); + processBuckets(dirtyBuckets, false, changedRows); + } + + // get the adds/removes/shifts from the first (source) entry, make a copy since TableUpdateImpl#reset will + // close them with the upstream update + WritableRowSet modifiedRowSet = RowSetFactory.empty(); downstream.modified = modifiedRowSet; @@ -414,7 +462,7 @@ protected void process() { final UpdateByOperator[] winOps = win.getOperators(); for (int winOpIdx : win.getDirtyOperators(winCtx)) { // these were created directly from the result output columns so no transformer needed - downstream.modifiedColumnSet.setAll(winOps[winOpIdx].outputModifiedColumnSet); + downstream.modifiedColumnSet.setAll(winOps[winOpIdx].getOutputModifiedColumnSet()); } } } @@ -534,8 +582,6 @@ public static Table updateBy(@NotNull final QueryTable source, // the next bit is complicated but the goal is simple. We don't want to have duplicate input column sources, so // we will store each one only once in inputSources and setup some mapping from the opIdx to the input column. - // noinspection unchecked - final ArrayList> inputSourceList = new ArrayList<>(); final int[][] operatorInputSourceSlotArr = new int[opArr.length][]; final TObjectIntHashMap> sourceToSlotMap = new TObjectIntHashMap<>(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java index cc1673f5e50..e8316d81e43 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java @@ -198,10 +198,14 @@ public String[] getOutputColumnNames() { /** * Apply a shift to the operation. - * */ public abstract void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta); + /** + * Prepare this operator output column for parallel updated. + */ + public abstract void prepareForParallelPopulation(final RowSet added); + /** * Create the modified column set for the input columns of this operator. * @@ -233,4 +237,5 @@ public ModifiedColumnSet getInputModifiedColumnSet() { public ModifiedColumnSet getOutputModifiedColumnSet() { return outputModifiedColumnSet; } + } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateByManager.java index 965ca72cce1..8913e357828 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateByManager.java @@ -94,7 +94,7 @@ public void onUpdate(@NotNull final TableUpdate upstream) { // do the actual computations UpdateByBucketHelper[] dirtyBuckets = new UpdateByBucketHelper[] {zeroKeyUpdateBy}; - processBuckets(dirtyBuckets, true); + processBuckets(dirtyBuckets, true, source.getRowSet()); finalizeBuckets(dirtyBuckets); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java index 294110de269..6d05c98990a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java @@ -180,6 +180,12 @@ public int[] getUniqueSourceIndices() { return uniqueInputSourceIndices; } + public void prepareForParallelPopulation(final RowSet added) { + for (UpdateByOperator operator : operators) { + operator.prepareForParallelPopulation(added); + } + } + // region context-based functions public abstract void computeAffectedRowsAndOperators(final UpdateByWindowContext context, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java index 2a175012052..fa7bcd646d2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java @@ -180,6 +180,14 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d } // endregion Shifts + @Override + public void prepareForParallelPopulation(final RowSet added) { + // we don't need to do anything for redirected, that happened earlier + if (!redirContext.isRedirected()) { + ((SparseArrayColumnSource) outputSource).prepareForParallelPopulation(added); + } + } + @NotNull @Override public Map> getOutputColumns() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java index a8160575f78..a3c4166c379 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java @@ -152,6 +152,14 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d } // endregion Shifts + @Override + public void prepareForParallelPopulation(final RowSet added) { + // we don't need to do anything for redirected, that happened earlier + if (!redirContext.isRedirected()) { + ((SparseArrayColumnSource) outputSource).prepareForParallelPopulation(added); + } + } + @NotNull @Override public Map> getOutputColumns() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java index ce8c02175c4..13b3c0b6572 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java @@ -156,6 +156,14 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long } // endregion + @Override + public void prepareForParallelPopulation(final RowSet added) { + // we don't need to do anything for redirected, that happened earlier + if (!redirContext.isRedirected()) { + ((SparseArrayColumnSource) outputSource).prepareForParallelPopulation(added); + } + } + @NotNull @Override public Map> getOutputColumns() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java index d4e60cdad4a..f435e9beb51 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java @@ -151,6 +151,14 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long } // endregion + @Override + public void prepareForParallelPopulation(final RowSet added) { + // we don't need to do anything for redirected, that happened earlier + if (!redirContext.isRedirected()) { + ((SparseArrayColumnSource) outputSource).prepareForParallelPopulation(added); + } + } + @NotNull @Override public Map> getOutputColumns() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java index 0cf0b7f84f7..8cd3f8fbf94 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java @@ -157,6 +157,14 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d } // endregion Shifts + @Override + public void prepareForParallelPopulation(final RowSet added) { + // we don't need to do anything for redirected, that happened earlier + if (!redirContext.isRedirected()) { + ((SparseArrayColumnSource) outputSource).prepareForParallelPopulation(added); + } + } + @NotNull @Override public Map> getOutputColumns() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java index 4f3137b4934..5ddf164b644 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java @@ -157,6 +157,14 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d } // endregion Shifts + @Override + public void prepareForParallelPopulation(final RowSet added) { + // we don't need to do anything for redirected, that happened earlier + if (!redirContext.isRedirected()) { + ((SparseArrayColumnSource) outputSource).prepareForParallelPopulation(added); + } + } + @NotNull @Override public Map> getOutputColumns() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java index eda8d882975..3d9cad434d9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java @@ -160,6 +160,14 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d } // endregion Shifts + @Override + public void prepareForParallelPopulation(final RowSet added) { + // we don't need to do anything for redirected, that happened earlier + if (!redirContext.isRedirected()) { + ((SparseArrayColumnSource) outputSource).prepareForParallelPopulation(added); + } + } + @NotNull @Override public Map> getOutputColumns() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java index c5249cf5996..643dca3433a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java @@ -157,6 +157,14 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d } // endregion Shifts + @Override + public void prepareForParallelPopulation(final RowSet added) { + // we don't need to do anything for redirected, that happened earlier + if (!redirContext.isRedirected()) { + ((SparseArrayColumnSource) outputSource).prepareForParallelPopulation(added); + } + } + @NotNull @Override public Map> getOutputColumns() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java index 0ba15f36b63..07d49158967 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java @@ -188,6 +188,14 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d } // endregion Shifts + @Override + public void prepareForParallelPopulation(final RowSet added) { + // we don't need to do anything for redirected, that happened earlier + if (!redirContext.isRedirected()) { + ((SparseArrayColumnSource) outputSource).prepareForParallelPopulation(added); + } + } + @NotNull @Override public Map> getOutputColumns() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java index f40aca44d72..318418470ac 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java @@ -160,6 +160,14 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d } // endregion Shifts + @Override + public void prepareForParallelPopulation(final RowSet added) { + // we don't need to do anything for redirected, that happened earlier + if (!redirContext.isRedirected()) { + ((SparseArrayColumnSource) outputSource).prepareForParallelPopulation(added); + } + } + @NotNull @Override public Map> getOutputColumns() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java index ddf9d5ac5cc..5368b25288b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java @@ -15,10 +15,14 @@ import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.sources.DoubleArraySource; import io.deephaven.engine.table.impl.sources.DoubleSparseArraySource; +import io.deephaven.engine.table.impl.sources.SparseArrayColumnSource; import io.deephaven.engine.table.impl.sources.WritableRedirectedColumnSource; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import java.util.Collections; +import java.util.Map; + import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; import static io.deephaven.util.QueryConstants.NULL_CHAR; import static io.deephaven.util.QueryConstants.NULL_DOUBLE; @@ -160,4 +164,19 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d ((DoubleSparseArraySource)outputSource).shift(subIndexToShift, delta); } // endregion Shifts + + + @Override + public void prepareForParallelPopulation(final RowSet added) { + // we don't need to do anything for redirected, that happened earlier + if (!redirContext.isRedirected()) { + ((SparseArrayColumnSource) outputSource).prepareForParallelPopulation(added); + } + } + + @NotNull + @Override + public Map> getOutputColumns() { + return Collections.singletonMap(pair.leftColumn, outputSource); + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java index 109f3243aef..5b90856d441 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java @@ -10,10 +10,14 @@ import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.sources.FloatArraySource; import io.deephaven.engine.table.impl.sources.FloatSparseArraySource; +import io.deephaven.engine.table.impl.sources.SparseArrayColumnSource; import io.deephaven.engine.table.impl.sources.WritableRedirectedColumnSource; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import java.util.Collections; +import java.util.Map; + import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; import static io.deephaven.util.QueryConstants.NULL_CHAR; import static io.deephaven.util.QueryConstants.NULL_FLOAT; @@ -155,4 +159,19 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d ((FloatSparseArraySource)outputSource).shift(subIndexToShift, delta); } // endregion Shifts + + + @Override + public void prepareForParallelPopulation(final RowSet added) { + // we don't need to do anything for redirected, that happened earlier + if (!redirContext.isRedirected()) { + ((SparseArrayColumnSource) outputSource).prepareForParallelPopulation(added); + } + } + + @NotNull + @Override + public Map> getOutputColumns() { + return Collections.singletonMap(pair.leftColumn, outputSource); + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java index b5078b1c6e6..b86eeacc494 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java @@ -165,6 +165,14 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d } // endregion Shifts + @Override + public void prepareForParallelPopulation(final RowSet added) { + // we don't need to do anything for redirected, that happened earlier + if (!redirContext.isRedirected()) { + ((SparseArrayColumnSource) outputSource).prepareForParallelPopulation(added); + } + } + @NotNull @Override public Map> getOutputColumns() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java index 2da5ea9625e..cda24603705 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java @@ -165,6 +165,14 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d } // endregion Shifts + @Override + public void prepareForParallelPopulation(final RowSet added) { + // we don't need to do anything for redirected, that happened earlier + if (!redirContext.isRedirected()) { + ((SparseArrayColumnSource) outputSource).prepareForParallelPopulation(added); + } + } + @NotNull @Override public Map> getOutputColumns() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java index f90cedbb6d6..13c2b595b51 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java @@ -168,6 +168,14 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d } // endregion Shifts + @Override + public void prepareForParallelPopulation(final RowSet added) { + // we don't need to do anything for redirected, that happened earlier + if (!redirContext.isRedirected()) { + ((SparseArrayColumnSource) outputSource).prepareForParallelPopulation(added); + } + } + @NotNull @Override public Map> getOutputColumns() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java index 9ec0db50578..5720546867c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java @@ -165,6 +165,14 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d } // endregion Shifts + @Override + public void prepareForParallelPopulation(final RowSet added) { + // we don't need to do anything for redirected, that happened earlier + if (!redirContext.isRedirected()) { + ((SparseArrayColumnSource) outputSource).prepareForParallelPopulation(added); + } + } + @NotNull @Override public Map> getOutputColumns() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/InverseRowRedirectionImpl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/InverseRowRedirectionImpl.java index dde508a362d..23b84ba2e3a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/InverseRowRedirectionImpl.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/InverseRowRedirectionImpl.java @@ -102,6 +102,8 @@ public String toString() { long positionStart = 0; for (final RowSet.RangeIterator rangeIterator = wrappedIndex.rangeIterator(); rangeIterator.hasNext();) { + rangeIterator.next(); + if (positionStart > 0) { builder.append(", "); } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestCumSum.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestCumSum.java index 94c668821e3..47d3dc4e902 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestCumSum.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestCumSum.java @@ -141,7 +141,7 @@ protected Table e() { @Test public void testZeroKeyGeneralTicking() { - final CreateResult result = createTestTable(10000, false, false, true, 0x31313131); + final CreateResult result = createTestTable(100, false, false, true, 0x31313131); final QueryTable t = result.t; final EvalNugget[] nuggets = new EvalNugget[] { From 51a6e7354011a6cc6f9163a6278a439433445e6d Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Thu, 3 Nov 2022 10:40:07 -0700 Subject: [PATCH 036/123] Addressed BooleanSparseArraySource reinterpreted as ByteSource not allowing prepareForParallelPopulation() --- .../BucketedPartitionedUpdateByManager.java | 26 ++-- .../deephaven/engine/table/impl/UpdateBy.java | 147 +++++++++++++----- .../table/impl/ZeroKeyUpdateByManager.java | 4 +- .../sources/BooleanSparseArraySource.java | 4 + .../internal/BaseByteUpdateByOperator.java | 7 +- .../table/impl/updateby/TestCumSum.java | 2 +- 6 files changed, 138 insertions(+), 52 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java index f0ccbf91b6f..708b3225f2a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java @@ -82,7 +82,7 @@ protected BucketedPartitionedUpdateByManager(@NotNull final String description, source.listenForUpdates(new BaseTable.ListenerImpl("", source, shiftApplyTable) { @Override public void onUpdate(@NotNull final TableUpdate upstream) { - shiftOutputColumns(upstream); +// shiftOutputColumns(upstream); super.onUpdate(upstream); } }); @@ -110,16 +110,6 @@ public void onUpdate(@NotNull final TableUpdate upstream) { } } - if (redirContext.isRedirected()) { - // make a dummy update to generate the initial row keys - final TableUpdateImpl fakeUpdate = new TableUpdateImpl(source.getRowSet(), - RowSetFactory.empty(), - RowSetFactory.empty(), - RowSetShiftData.EMPTY, - ModifiedColumnSet.EMPTY); - redirContext.processUpdateForRedirection(fakeUpdate, source.getRowSet()); - } - final PartitionedTable transformed = pt.transform(t -> { UpdateByBucketHelper updateBy = new UpdateByBucketHelper( description, @@ -153,9 +143,19 @@ public void onUpdate(@NotNull final TableUpdate upstream) { result.addParentReference(transformed); - // do the actual computations + if (redirContext.isRedirected()) { + // make a dummy update to generate the initial row keys + final TableUpdateImpl fakeUpdate = new TableUpdateImpl(source.getRowSet(), + RowSetFactory.empty(), + RowSetFactory.empty(), + RowSetShiftData.EMPTY, + ModifiedColumnSet.EMPTY); + redirContext.processUpdateForRedirection(fakeUpdate, source.getRowSet()); + } + UpdateByBucketHelper[] dirtyBuckets = buckets.toArray(UpdateByBucketHelper[]::new); - processBuckets(dirtyBuckets, true, source.getRowSet()); + + processBuckets(dirtyBuckets, true, RowSetShiftData.EMPTY); finalizeBuckets(dirtyBuckets); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java index a98a130c039..5666ef669dd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java @@ -359,47 +359,121 @@ protected void shiftOutputColumns(TableUpdate upstream) { } } - protected void processBuckets(UpdateByBucketHelper[] dirtyBuckets, final boolean initialStep, final RowSet added) { + /*** + * This will handle shifts for the output sources and will also prepare the sources for parallel updates + * @param upstream the {@link TableUpdate} to process + */ + protected void shiftWindowOperators(UpdateByWindow win, RowSetShiftData shift) { + if (!redirContext.isRedirected() && shift.nonempty()) { + try (final RowSet prevIdx = source.getRowSet().copyPrev()) { + shift.apply((begin, end, delta) -> { + try (final RowSet subRowSet = prevIdx.subSetByKeyRange(begin, end)) { + for (UpdateByOperator op : win.getOperators()) { + op.applyOutputShift(subRowSet, delta); + } + } + }); + } + } + } + + protected void processBuckets(UpdateByBucketHelper[] dirtyBuckets, boolean initialStep, RowSetShiftData shifts) { + // we need to ID which rows may be affected by the upstream shifts so we can include them + // in our parallel update preparations + final WritableRowSet shiftedRows; + if (shifts.nonempty()) { + try (final RowSet prev = source.getRowSet().copyPrev(); + final RowSequence.Iterator it = prev.getRowSequenceIterator()) { + + final RowSetBuilderSequential builder = RowSetFactory.builderSequential(); + final int size = shifts.size(); + + // get these in order so we can use a sequential builder + for (int ii = 0; ii < size; ii++) { + final long begin = shifts.getBeginRange(ii); + final long end = shifts.getEndRange(ii); + final long delta = shifts.getShiftDelta(ii); + + it.advance(begin); + final RowSequence rs = it.getNextRowSequenceThrough(end); + builder.appendRowSequenceWithOffset(rs, delta); + } + shiftedRows = builder.build(); + } + } else { + shiftedRows = RowSetFactory.empty(); + } + + final ColumnSource[] maybeCachedInputSources; + final TrackingWritableRowSet[] inputSourceRowSets; + final AtomicInteger[] inputSourceReferenceCounts; if (inputCacheNeeded) { - // this will store the input sources needed for processing - final ColumnSource[] maybeCachedInputSources = new ColumnSource[inputSources.length]; - final TrackingWritableRowSet[] inputSourceRowSets = new TrackingWritableRowSet[inputSources.length]; - final AtomicInteger[] inputSourceReferenceCounts = new AtomicInteger[inputSources.length]; + maybeCachedInputSources = new ColumnSource[inputSources.length]; + inputSourceRowSets = new TrackingWritableRowSet[inputSources.length]; + inputSourceReferenceCounts = new AtomicInteger[inputSources.length]; + // do the hard work of computing what should be cached for each input source computeCachedColumnContents(dirtyBuckets, initialStep, inputSourceRowSets, inputSourceReferenceCounts); - // assign the non-cached input source or leave null for cac + // assign the non-cached input source or leave null for cached sources for (int srcIdx = 0; srcIdx < inputSources.length; srcIdx++) { maybeCachedInputSources[srcIdx] = inputSourceCacheNeeded[srcIdx] ? null : inputSources[srcIdx]; } + } else { + maybeCachedInputSources = inputSources; + inputSourceRowSets = null; + inputSourceReferenceCounts = null; + } + + // process the windows + for (int winIdx = 0; winIdx < windows.length; winIdx++) { + UpdateByWindow win = windows[winIdx]; - for (int winIdx = 0; winIdx < windows.length; winIdx++) { + if (inputCacheNeeded) { // cache the sources needed for this window cacheInputSources(winIdx, maybeCachedInputSources, inputSourceRowSets); + } - if (added.isNonempty()) { - // prepare the array output sources for parallel updates (added records only since we - // handled removes/shifts earlier) - windows[winIdx].prepareForParallelPopulation(added); + if (initialStep) { + // prepare each operator for the parallel updates to come + for (UpdateByOperator op : win.getOperators()) { + op.prepareForParallelPopulation(source.getRowSet()); } - - // process the window - for (UpdateByBucketHelper bucket : dirtyBuckets) { - bucket.assignInputSources(winIdx, maybeCachedInputSources); - bucket.processWindow(winIdx, initialStep); + } else { + try (final WritableRowSet windowRowSet = shiftedRows.copy()) { + // get the total rowset from this window + for (UpdateByBucketHelper bucket : dirtyBuckets) { + // append the dirty rows from this window + if (win.isWindowDirty(bucket.windowContexts[winIdx])) { + windowRowSet.insert(win.getAffectedRows(bucket.windowContexts[winIdx])); + } + } + // prepare each operator for the parallel updates to come + for (UpdateByOperator op : win.getOperators()) { + op.prepareForParallelPopulation(windowRowSet); + } } + } + // now we can shift (after the parallel prep is complete) + shiftWindowOperators(win, shifts); + + // PARALLELIZE THIS INTO + + // process the window + for (UpdateByBucketHelper bucket : dirtyBuckets) { + bucket.assignInputSources(winIdx, maybeCachedInputSources); + bucket.processWindow(winIdx, initialStep); + } + + if (inputCacheNeeded) { // release the cached sources that are no longer needed releaseInputSources(winIdx, maybeCachedInputSources, inputSourceRowSets, inputSourceReferenceCounts); } - } else { - // no caching needed, process immediately - for (int winIdx = 0; winIdx < windows.length; winIdx++) { - for (UpdateByBucketHelper bucket : dirtyBuckets) { - bucket.assignInputSources(winIdx, inputSources); - bucket.processWindow(winIdx, initialStep); - } - } + } + + // we are done with this rowset + shiftedRows.close(); } protected void finalizeBuckets(UpdateByBucketHelper[] dirtyBuckets) { @@ -421,32 +495,35 @@ protected void process() { UpdateByBucketHelper[] dirtyBuckets = buckets.stream().filter(b -> b.isDirty()) .toArray(UpdateByBucketHelper[]::new); + final ListenerRecorder sourceRecorder = recorders.peekFirst(); + final TableUpdate upstream = sourceRecorder.getUpdate(); + final TableUpdateImpl downstream = new TableUpdateImpl(); - ListenerRecorder sourceRecorder = recorders.peekFirst(); - downstream.added = sourceRecorder.getAdded().copy(); - downstream.removed = sourceRecorder.getRemoved().copy(); - downstream.shifted = sourceRecorder.getShifted(); + downstream.added = upstream.added().copy(); + downstream.removed = upstream.removed().copy(); + downstream.shifted = upstream.shifted(); // union the modifies from all the tables (including source) downstream.modifiedColumnSet = result.getModifiedColumnSetForUpdates(); downstream.modifiedColumnSet.clear(); - // do the actual computations - try (final WritableRowSet changedRows = sourceRecorder.getAdded().union(sourceRecorder.getModifiedPreShift())) { - changedRows.insert(sourceRecorder.getRemoved()); - processBuckets(dirtyBuckets, false, changedRows); + if (redirContext.isRedirected()) { + // this does all the work needed for redirected output sources + redirContext.processUpdateForRedirection(upstream, source.getRowSet()); } + processBuckets(dirtyBuckets, false, upstream.shifted()); + // get the adds/removes/shifts from the first (source) entry, make a copy since TableUpdateImpl#reset will // close them with the upstream update WritableRowSet modifiedRowSet = RowSetFactory.empty(); downstream.modified = modifiedRowSet; - if (sourceRecorder.getModified().isNonempty()) { - modifiedRowSet.insert(sourceRecorder.getModified()); - downstream.modifiedColumnSet.setAll(sourceRecorder.getModifiedColumnSet()); + if (upstream.modified().isNonempty()) { + modifiedRowSet.insert(upstream.modified()); + downstream.modifiedColumnSet.setAll(upstream.modifiedColumnSet()); } for (UpdateByBucketHelper bucket : dirtyBuckets) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateByManager.java index 8913e357828..c171066f42f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateByManager.java @@ -41,7 +41,7 @@ protected ZeroKeyUpdateByManager(@NotNull final String description, source.listenForUpdates(new BaseTable.ListenerImpl("", source, shiftApplyTable) { @Override public void onUpdate(@NotNull final TableUpdate upstream) { - shiftOutputColumns(upstream); +// shiftOutputColumns(upstream); super.onUpdate(upstream); } }); @@ -94,7 +94,7 @@ public void onUpdate(@NotNull final TableUpdate upstream) { // do the actual computations UpdateByBucketHelper[] dirtyBuckets = new UpdateByBucketHelper[] {zeroKeyUpdateBy}; - processBuckets(dirtyBuckets, true, source.getRowSet()); + processBuckets(dirtyBuckets, true, RowSetShiftData.EMPTY); finalizeBuckets(dirtyBuckets); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/BooleanSparseArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/BooleanSparseArraySource.java index 5209e62a69f..9d57c7fc2b8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/BooleanSparseArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/BooleanSparseArraySource.java @@ -1096,6 +1096,10 @@ public void fillFromChunk(@NotNull FillFromContext context_unused, @NotNull Chun } } } + + public void prepareForParallelPopulation(RowSet rowSet) { + wrapped.prepareForParallelPopulation(rowSet); + } } // endregion reinterpretation } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java index fa7bcd646d2..8af4e9f461f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java @@ -184,7 +184,12 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d public void prepareForParallelPopulation(final RowSet added) { // we don't need to do anything for redirected, that happened earlier if (!redirContext.isRedirected()) { - ((SparseArrayColumnSource) outputSource).prepareForParallelPopulation(added); + // this might be a Boolean reinterpreted column source + if (outputSource instanceof BooleanSparseArraySource.ReinterpretedAsByte) { + ((BooleanSparseArraySource.ReinterpretedAsByte) outputSource).prepareForParallelPopulation(added); + } else { + ((SparseArrayColumnSource) outputSource).prepareForParallelPopulation(added); + } } } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestCumSum.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestCumSum.java index 47d3dc4e902..766df5c0c89 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestCumSum.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestCumSum.java @@ -163,7 +163,7 @@ protected Table e() { @Test public void testBucketedGeneralTicking() { - final CreateResult result = createTestTable(10000, true, false, true, 0x31313131); + final CreateResult result = createTestTable(100, true, false, true, 0x31313131); final QueryTable t = result.t; final EvalNugget[] nuggets = new EvalNugget[] { From cabf485481c4835ee183be1874afb19194f4f3d9 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Thu, 3 Nov 2022 16:48:45 -0700 Subject: [PATCH 037/123] Implemented UGP parallel/serial task helper --- .../table/impl/updateby/UGPTaskHelper.java | 105 ++++++++++++++++++ 1 file changed, 105 insertions(+) create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UGPTaskHelper.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UGPTaskHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UGPTaskHelper.java new file mode 100644 index 00000000000..fc9aeb8d2a1 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UGPTaskHelper.java @@ -0,0 +1,105 @@ +package io.deephaven.engine.table.impl.updateby; + +import io.deephaven.engine.context.ExecutionContext; +import io.deephaven.engine.updategraph.AbstractNotification; +import io.deephaven.engine.updategraph.UpdateGraphProcessor; + +import java.util.ArrayList; +import java.util.concurrent.atomic.AtomicInteger; + +public class UGPTaskHelper { + + public static abstract class UGPTaskHelperNotification extends AbstractNotification { + protected final ArrayList dataList; + protected final AtomicInteger nextTask; + protected final AtomicInteger tasksCompleted; + protected final WorkerAction workerAction; + protected final Runnable completeAction; + + @FunctionalInterface + public interface WorkerAction { + void run(T data, Runnable complete); + } + + private UGPTaskHelperNotification(ArrayList dataList, AtomicInteger nextTask, AtomicInteger tasksCompleted, WorkerAction workerAction, Runnable completeAction) { + super(true); + this.dataList = dataList; + this.nextTask = nextTask; + this.tasksCompleted = tasksCompleted; + this.workerAction = workerAction; + this.completeAction = completeAction; + } + + @Override + public boolean canExecute(long step) { + return true; + } + + @Override + public ExecutionContext getExecutionContext() { + return null; + } + + @Override + public void run() { + // do the work + int taskIndex = nextTask.getAndIncrement(); + workerAction.run(dataList.get(taskIndex), this::complete); + } + + protected abstract void complete(); + } + + private static class UGPTaskHelperSerialNotification extends UGPTaskHelperNotification { + private UGPTaskHelperSerialNotification(ArrayList dataList, AtomicInteger nextTask, AtomicInteger tasksCompleted, WorkerAction workerAction, Runnable completeAction) { + super(dataList, nextTask, tasksCompleted, workerAction, completeAction); + } + + protected void complete() { + // serially call the next task + int completedCount = tasksCompleted.incrementAndGet(); + + if (completedCount == dataList.size()) { + completeAction.run(); + } else { + // this will recurse N times, is it better to create a new UGP notification and schedule it? + run(); + } + } + } + + private static class UGPTaskHelperParallelNotification extends UGPTaskHelperNotification { + private UGPTaskHelperParallelNotification(ArrayList dataList, AtomicInteger nextTask, AtomicInteger tasksCompleted, WorkerAction workerAction, Runnable completeAction) { + super(dataList, nextTask, tasksCompleted, workerAction, completeAction); + } + + protected void complete() { + // call the complete when all the tasks are done + int completedCount = tasksCompleted.incrementAndGet(); + + if (completedCount == dataList.size()) { + completeAction.run(); + } + } + } + + public static void ExecuteSerial(ArrayList dataObjects, UGPTaskHelperNotification.WorkerAction workerAction, Runnable completedAction) { + final AtomicInteger completed = new AtomicInteger(0); + final AtomicInteger nextTask = new AtomicInteger(0); + + // create and schedule a single notification, will auto serialize + UGPTaskHelperSerialNotification notification = new UGPTaskHelperSerialNotification(dataObjects, nextTask, completed, workerAction, completedAction); + UpdateGraphProcessor.DEFAULT.addNotification(notification); + } + + public static void ExecuteParallel(ArrayList dataObjects, UGPTaskHelperNotification.WorkerAction workerAction, Runnable completedAction) { + final AtomicInteger completed = new AtomicInteger(0); + final AtomicInteger nextTask = new AtomicInteger(0); + + // create and schedule a notification for each object + for (int ii = 0; ii < dataObjects.size(); ii++) { + UGPTaskHelperParallelNotification notification = new UGPTaskHelperParallelNotification(dataObjects, nextTask, completed, workerAction, completedAction); + UpdateGraphProcessor.DEFAULT.addNotification(notification); + } + } +} From 34e884023ddce74efb8845ffdffd69c984f43322 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Thu, 3 Nov 2022 16:55:55 -0700 Subject: [PATCH 038/123] tiny edits --- .../engine/table/impl/updateby/UGPTaskHelper.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UGPTaskHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UGPTaskHelper.java index fc9aeb8d2a1..3f8b594725d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UGPTaskHelper.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UGPTaskHelper.java @@ -50,8 +50,8 @@ public void run() { protected abstract void complete(); } - private static class UGPTaskHelperSerialNotification extends UGPTaskHelperNotification { - private UGPTaskHelperSerialNotification(ArrayList dataList, AtomicInteger nextTask, AtomicInteger tasksCompleted, WorkerAction workerAction, Runnable completeAction) { + private static class UGPTaskHelperSerialNotification extends UGPTaskHelperNotification { + private UGPTaskHelperSerialNotification(ArrayList dataList, AtomicInteger nextTask, AtomicInteger tasksCompleted, WorkerAction workerAction, Runnable completeAction) { super(dataList, nextTask, tasksCompleted, workerAction, completeAction); } @@ -68,8 +68,8 @@ protected void complete() { } } - private static class UGPTaskHelperParallelNotification extends UGPTaskHelperNotification { - private UGPTaskHelperParallelNotification(ArrayList dataList, AtomicInteger nextTask, AtomicInteger tasksCompleted, WorkerAction workerAction, Runnable completeAction) { + private static class UGPTaskHelperParallelNotification extends UGPTaskHelperNotification { + private UGPTaskHelperParallelNotification(ArrayList dataList, AtomicInteger nextTask, AtomicInteger tasksCompleted, WorkerAction workerAction, Runnable completeAction) { super(dataList, nextTask, tasksCompleted, workerAction, completeAction); } @@ -88,7 +88,7 @@ public static void ExecuteSerial(ArrayList dataObjects, UGPTaskHelperNoti final AtomicInteger nextTask = new AtomicInteger(0); // create and schedule a single notification, will auto serialize - UGPTaskHelperSerialNotification notification = new UGPTaskHelperSerialNotification(dataObjects, nextTask, completed, workerAction, completedAction); + UGPTaskHelperSerialNotification notification = new UGPTaskHelperSerialNotification<>(dataObjects, nextTask, completed, workerAction, completedAction); UpdateGraphProcessor.DEFAULT.addNotification(notification); } @@ -98,7 +98,7 @@ public static void ExecuteParallel(ArrayList dataObjects, UGPTaskHelperNot // create and schedule a notification for each object for (int ii = 0; ii < dataObjects.size(); ii++) { - UGPTaskHelperParallelNotification notification = new UGPTaskHelperParallelNotification(dataObjects, nextTask, completed, workerAction, completedAction); + UGPTaskHelperParallelNotification notification = new UGPTaskHelperParallelNotification<>(dataObjects, nextTask, completed, workerAction, completedAction); UpdateGraphProcessor.DEFAULT.addNotification(notification); } } From 75fe1ab4c9175761b58398bf9b457b35776c6039 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 9 Nov 2022 13:04:46 -0800 Subject: [PATCH 039/123] More documentation --- .../BucketedPartitionedUpdateByManager.java | 50 +- .../deephaven/engine/table/impl/UpdateBy.java | 768 +++++++++++------- .../table/impl/UpdateByBucketHelper.java | 20 +- .../table/impl/ZeroKeyUpdateByManager.java | 50 +- .../analyzers/SelectAndViewAnalyzer.java | 167 ++++ .../table/impl/updateby/UGPTaskHelper.java | 105 --- .../table/impl/updateby/UpdateByWindow.java | 13 + .../impl/updateby/UpdateByWindowTicks.java | 8 +- .../impl/updateby/UpdateByWindowTime.java | 36 +- 9 files changed, 727 insertions(+), 490 deletions(-) delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UGPTaskHelper.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java index 708b3225f2a..a98c103ecfc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java @@ -76,30 +76,19 @@ protected BucketedPartitionedUpdateByManager(@NotNull final String description, recorders = new LinkedList<>(); listener = newListener(description); - // create an intermediate table that will listen to source updates and shift output columns - final QueryTable shiftApplyTable = new QueryTable(source.getRowSet(), source.getColumnSourceMap()); - - source.listenForUpdates(new BaseTable.ListenerImpl("", source, shiftApplyTable) { - @Override - public void onUpdate(@NotNull final TableUpdate upstream) { -// shiftOutputColumns(upstream); - super.onUpdate(upstream); - } - }); - - // create a recorder instance sourced from the shifting table - ListenerRecorder shiftRecorder = new ListenerRecorder(description, shiftApplyTable, result); - shiftRecorder.setMergedListener(listener); - shiftApplyTable.listenForUpdates(shiftRecorder); + // create a recorder instance sourced from the source table + ListenerRecorder sourceRecorder = new ListenerRecorder(description, source, result); + sourceRecorder.setMergedListener(listener); + source.listenForUpdates(sourceRecorder); result.addParentReference(listener); - recorders.offerLast(shiftRecorder); + recorders.offerLast(sourceRecorder); // create input and output modified column sets for (UpdateByOperator op : operators) { - op.createInputModifiedColumnSet(shiftApplyTable); + op.createInputModifiedColumnSet(source); op.createOutputModifiedColumnSet(result); } - pt = shiftApplyTable.partitionedAggBy(List.of(), true, null, byColumns); + pt = source.partitionedAggBy(List.of(), true, null, byColumns); } else { // no shifting will be needed, can create directly from source pt = source.partitionedAggBy(List.of(), true, null, byColumns); @@ -110,6 +99,9 @@ public void onUpdate(@NotNull final TableUpdate upstream) { } } + // make the source->result transformer + transformer = source.newModifiedColumnSetTransformer(result, source.getDefinition().getColumnNamesArray()); + final PartitionedTable transformed = pt.transform(t -> { UpdateByBucketHelper updateBy = new UpdateByBucketHelper( description, @@ -143,19 +135,15 @@ public void onUpdate(@NotNull final TableUpdate upstream) { result.addParentReference(transformed); - if (redirContext.isRedirected()) { - // make a dummy update to generate the initial row keys - final TableUpdateImpl fakeUpdate = new TableUpdateImpl(source.getRowSet(), - RowSetFactory.empty(), - RowSetFactory.empty(), - RowSetShiftData.EMPTY, - ModifiedColumnSet.EMPTY); - redirContext.processUpdateForRedirection(fakeUpdate, source.getRowSet()); - } - - UpdateByBucketHelper[] dirtyBuckets = buckets.toArray(UpdateByBucketHelper[]::new); + // make a dummy update to generate the initial row keys + final TableUpdateImpl fakeUpdate = new TableUpdateImpl(source.getRowSet(), + RowSetFactory.empty(), + RowSetFactory.empty(), + RowSetShiftData.EMPTY, + ModifiedColumnSet.EMPTY); - processBuckets(dirtyBuckets, true, RowSetShiftData.EMPTY); - finalizeBuckets(dirtyBuckets); + // do the actual computations + final StateManager sm = new StateManager(fakeUpdate, true); + sm.processUpdate(); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java index 5666ef669dd..91710f6fc49 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java @@ -3,22 +3,29 @@ import gnu.trove.list.array.TIntArrayList; import gnu.trove.map.hash.TIntObjectHashMap; import gnu.trove.map.hash.TObjectIntHashMap; +import io.deephaven.UncheckedDeephavenException; import io.deephaven.api.ColumnName; -import io.deephaven.api.updateby.UpdateByOperation; import io.deephaven.api.updateby.UpdateByControl; +import io.deephaven.api.updateby.UpdateByOperation; +import io.deephaven.base.log.LogOutput; +import io.deephaven.base.log.LogOutputAppendable; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ResettableWritableObjectChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.context.ExecutionContext; +import io.deephaven.engine.exceptions.CancellationException; import io.deephaven.engine.exceptions.UncheckedTableException; import io.deephaven.engine.rowset.*; import io.deephaven.engine.table.*; +import io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer; import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.sources.sparse.SparseConstants; import io.deephaven.engine.table.impl.updateby.UpdateByWindow; import io.deephaven.engine.table.impl.util.InverseRowRedirectionImpl; import io.deephaven.engine.table.impl.util.LongColumnSourceWritableRowRedirection; import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.updategraph.UpdateGraphProcessor; import org.apache.commons.lang3.mutable.MutableInt; import org.apache.commons.lang3.mutable.MutableLong; import org.jetbrains.annotations.NotNull; @@ -26,6 +33,8 @@ import java.lang.ref.SoftReference; import java.util.*; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicInteger; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; @@ -34,38 +43,42 @@ * The core of the {@link Table#updateBy(UpdateByControl, Collection, Collection)} operation. */ public abstract class UpdateBy { + /** Input sources may be reused by mutiple operators, only store and cache unique ones */ protected final ColumnSource[] inputSources; - // some columns will have multiple inputs, such as time-based and Weighted computations + /** Map operators to input sources, note some operators need more than one input, WAvg e.g. */ protected final int[][] operatorInputSourceSlots; + /** All the operators for this UpdateBy manager */ protected final UpdateByOperator[] operators; + /** All the windows for this UpdateBy manager */ protected final UpdateByWindow[] windows; + /** The source table for the UpdateBy operators */ protected final QueryTable source; + /** Helper class for maintaining the RowRedirection when using redirected output sources */ protected final UpdateByRedirectionContext redirContext; + /** User control to specify UpdateBy parameters */ protected final UpdateByControl control; + /** The single timestamp column used by all time-based operators */ protected final String timestampColumnName; - + /** Store every bucket in this list for processing */ protected final LinkedList buckets; - - // column-caching management - - /** - * Whether caching benefits this UpdateBy operation - */ + /** Whether caching benefits this UpdateBy operation */ protected final boolean inputCacheNeeded; - /** - * Whether caching benefits this input source - */ + /** Whether caching benefits each input source */ protected final boolean[] inputSourceCacheNeeded; - /** - * References to the dense array sources we are using for the cached sources - */ + /** References to the dense array sources we are using for the cached sources, it's expected that these will be + * released and need to be created */ protected final SoftReference>[] inputSourceCaches; + /** For easy iteration, create a list of the source indices that need to be cached */ + protected final int[] cacheableSourceIndices; - /** - * The output table for this UpdateBy operation - */ + /** ColumnSet transformer from source to downstream */ + protected ModifiedColumnSet.Transformer transformer; + /** The output table for this UpdateBy operation */ protected QueryTable result; + + /** For refreshing sources, maintain a list of each of the bucket listeners */ protected LinkedList recorders; + /** For refreshing sources, need a merged listener to produce downstream updates */ protected UpdateByListener listener; public static class UpdateByRedirectionContext implements Context { @@ -139,7 +152,7 @@ public void processUpdateForRedirection(@NotNull final TableUpdate upstream, fin } private boolean shiftRedirectedKey(@NotNull final RowSet.SearchIterator iterator, final long delta, - final long key) { + final long key) { final long inner = rowRedirection.remove(key); if (inner != NULL_ROW_KEY) { rowRedirection.put(key + delta, inner); @@ -155,15 +168,15 @@ public void close() { } protected UpdateBy(@NotNull final String description, - @NotNull final QueryTable source, - @NotNull final UpdateByOperator[] operators, - @NotNull final UpdateByWindow[] windows, - @NotNull final ColumnSource[] inputSources, - @NotNull final int[][] operatorInputSourceSlots, - @NotNull final Map> resultSources, - @Nullable String timestampColumnName, - @NotNull final UpdateByRedirectionContext redirContext, - @NotNull final UpdateByControl control) { + @NotNull final QueryTable source, + @NotNull final UpdateByOperator[] operators, + @NotNull final UpdateByWindow[] windows, + @NotNull final ColumnSource[] inputSources, + @NotNull final int[][] operatorInputSourceSlots, + @NotNull final Map> resultSources, + @Nullable String timestampColumnName, + @NotNull final UpdateByRedirectionContext redirContext, + @NotNull final UpdateByControl control) { if (operators.length == 0) { throw new IllegalArgumentException("At least one operator must be specified"); @@ -181,10 +194,16 @@ protected UpdateBy(@NotNull final String description, this.inputSourceCacheNeeded = new boolean[inputSources.length]; boolean cacheNeeded = false; + TIntArrayList cacheableSourceIndicesList = new TIntArrayList(inputSources.length); for (int ii = 0; ii < inputSources.length; ii++) { - inputSourceCacheNeeded[ii] = !FillUnordered.providesFillUnordered(inputSources[ii]); - cacheNeeded |= inputSourceCacheNeeded[ii]; + if (!FillUnordered.providesFillUnordered(inputSources[ii])) { + cacheNeeded = inputSourceCacheNeeded[ii] = true; + cacheableSourceIndicesList.add(ii); + } } + // store this list for fast iteration + cacheableSourceIndices = cacheableSourceIndicesList.toArray(); + this.inputCacheNeeded = cacheNeeded; // noinspection unchecked inputSourceCaches = new SoftReference[inputSources.length]; @@ -192,127 +211,11 @@ protected UpdateBy(@NotNull final String description, buckets = new LinkedList<>(); } - private ColumnSource createCachedColumnSource(int srcIdx, final TrackingWritableRowSet inputRowSet) { - final ColumnSource inputSource = inputSources[srcIdx]; - - // re-use the dense column cache if it still exists - WritableColumnSource innerSource; - if (inputSourceCaches[srcIdx] == null || (innerSource = inputSourceCaches[srcIdx].get()) == null) { - // create a new dense cache - innerSource = ArrayBackedColumnSource.getMemoryColumnSource(inputSource.getType(), inputSource.getComponentType()); - inputSourceCaches[srcIdx] = new SoftReference<>(innerSource); - } - innerSource.ensureCapacity(inputRowSet.size()); - - final WritableRowRedirection rowRedirection = new InverseRowRedirectionImpl(inputRowSet); - final WritableColumnSource outputSource = - new WritableRedirectedColumnSource(rowRedirection, innerSource, 0); - - final int CHUNK_SIZE = 1 << 16; // copied from SparseSelect - - try (final RowSequence.Iterator rsIt = inputRowSet.getRowSequenceIterator(); - final ChunkSink.FillFromContext ffc = - outputSource.makeFillFromContext(CHUNK_SIZE); - final ChunkSource.GetContext gc = inputSource.makeGetContext(CHUNK_SIZE)) { - while (rsIt.hasMore()) { - final RowSequence chunkOk = rsIt.getNextRowSequenceWithLength(CHUNK_SIZE); - final Chunk values = inputSource.getChunk(gc, chunkOk); - outputSource.fillFromChunk(ffc, values, chunkOk); - } - } - - // holding this reference will protect `rowDirection` and `innerSource` from GC - return outputSource; - } - - /** - * Examine the buckets and identify the input sources that will benefit from caching. Accumulate the bucket - * rowsets for each source independently so the caches are as efficient as possible - */ - private void computeCachedColumnContents(UpdateByBucketHelper[] buckets, boolean initialStep, WritableRowSet[] inputSourceRowSets, AtomicInteger[] inputSourceReferenceCounts) { - // on the initial step, everything is dirty and we can optimize - if (initialStep) { - for (int srcIdx = 0; srcIdx < inputSources.length; srcIdx++) { - if (inputSourceCacheNeeded[srcIdx]) { - // this needs to be a TrackingRowSet to be used by `InverseRowRedirectionImpl` - inputSourceRowSets[srcIdx] = source.getRowSet().copy().toTracking(); - } - } - - // add reference counts for each window - for (final UpdateByWindow win : windows) { - final int[] uniqueWindowSources = win.getUniqueSourceIndices(); - for (int srcIdx : uniqueWindowSources) { - if (inputSourceCacheNeeded[srcIdx]) { - // increment the reference count for this input source - if (inputSourceReferenceCounts[srcIdx] == null) { - inputSourceReferenceCounts[srcIdx] = new AtomicInteger(1); - } else { - inputSourceReferenceCounts[srcIdx].incrementAndGet(); - } - } - } - } - return; - } - - // on update steps, we can be more precise and cache exactly what is needed by the update - final boolean[] cacheNeeded = new boolean[inputSources.length]; - - for (int winIdx = 0; winIdx < windows.length; winIdx++) { - final UpdateByWindow win = windows[winIdx]; - final int[] uniqueWindowSources = win.getUniqueSourceIndices(); - - Arrays.fill(cacheNeeded, false); - - // for each bucket, need to accumulate the rowset if this window is dirty - for (UpdateByBucketHelper bucket : buckets) { - UpdateByWindow.UpdateByWindowContext winCtx = bucket.windowContexts[winIdx]; - if (win.isWindowDirty(winCtx)) { - // - for (int srcIdx : uniqueWindowSources) { - if (inputSourceCacheNeeded[srcIdx]) { - // record that this window requires this input source - cacheNeeded[srcIdx] = true; - // add this rowset to the running total - if (inputSourceRowSets[srcIdx] == null) { - inputSourceRowSets[srcIdx] = win.getInfluencerRows(winCtx).copy().toTracking(); - } else { - inputSourceRowSets[srcIdx].insert(win.getInfluencerRows(winCtx)); - } - } - } - } - } - - // add one to all the reference counts this windows - for (int srcIdx : uniqueWindowSources) { - if (cacheNeeded[srcIdx]) { - // increment the reference count for this input source - if (inputSourceReferenceCounts[srcIdx] == null) { - inputSourceReferenceCounts[srcIdx] = new AtomicInteger(1); - } else { - inputSourceReferenceCounts[srcIdx].incrementAndGet(); - } - } - } - } - } - - private void cacheInputSources(int winIdx, ColumnSource[] maybeCachedInputSources, TrackingWritableRowSet[] inputSourceRowSets) { - final UpdateByWindow win = windows[winIdx]; - final int[] uniqueWindowSources = win.getUniqueSourceIndices(); - for (int srcIdx : uniqueWindowSources) { - if (maybeCachedInputSources[srcIdx] == null) { - maybeCachedInputSources[srcIdx] = createCachedColumnSource(srcIdx, inputSourceRowSets[srcIdx]); - } - } - } - + /** Remove all references to Objects for this column source */ private void fillObjectArraySourceWithNull(ObjectArraySource sourceToNull) { Assert.neqNull(sourceToNull, "cached column source was null, must have been GC'd"); try (final ResettableWritableObjectChunk backingChunk = - ResettableWritableObjectChunk.makeResettableChunk()) { + ResettableWritableObjectChunk.makeResettableChunk()) { Assert.neqNull(sourceToNull, "cached column source was already GC'd"); final long targetCapacity = sourceToNull.getCapacity(); for (long positionToNull = 0; positionToNull < targetCapacity; positionToNull += backingChunk.size()) { @@ -322,7 +225,9 @@ private void fillObjectArraySourceWithNull(ObjectArraySource sourceToNull) { } } - private void releaseInputSources(int winIdx, ColumnSource[] maybeCachedInputSources, TrackingWritableRowSet[] inputSourceRowSets, AtomicInteger[] inputSourceReferenceCounts) { + /** Release the input sources that will not be needed for the rest of this update */ + private void releaseInputSources(int winIdx, ColumnSource[] maybeCachedInputSources, + TrackingWritableRowSet[] inputSourceRowSets, AtomicInteger[] inputSourceReferenceCounts) { final UpdateByWindow win = windows[winIdx]; final int[] uniqueWindowSources = win.getUniqueSourceIndices(); for (int srcIdx : uniqueWindowSources) { @@ -343,187 +248,391 @@ private void releaseInputSources(int winIdx, ColumnSource[] maybeCachedInputS } } - protected void shiftOutputColumns(TableUpdate upstream) { - if (redirContext.isRedirected()) { - redirContext.processUpdateForRedirection(upstream, source.getRowSet()); - } else if (upstream.shifted().nonempty()) { - try (final RowSet prevIdx = source.getRowSet().copyPrev()) { - upstream.shifted().apply((begin, end, delta) -> { - try (final RowSet subRowSet = prevIdx.subSetByKeyRange(begin, end)) { - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - operators[opIdx].applyOutputShift(subRowSet, delta); - } - } - }); - } - } - } - - /*** - * This will handle shifts for the output sources and will also prepare the sources for parallel updates - * @param upstream the {@link TableUpdate} to process + /** + * Overview of work performed by {@link StateManager}: + *
                + *
              1. Create `shiftedRows`, the set of rows for the output sources that are affected by shifts
              2. + *
              3. Compute a rowset for each cacheable input source identifying which rows will be needed for processing
              4. + *
              5. Process each window serially + *
                  + *
                1. Cache the input sources that are needed for this window (this can be done in parallel for each column and even in a chunky way)
                2. + *
                3. Compute the modified rowset of output column sources and call `prepareForParallelPopulation()', this could be done in parallel with the caching
                4. + *
                5. When prepareForParallelPopulation() complete, apply upstream shifts to the output sources
                6. + *
                7. When caching and shifts are complete, process the data in this window in parallel by dividing the buckets into sets (N/num_threads) and running a job for each bucket_set 3e) when all buckets processed
                8. + *
                9. When all buckets processed, release the input source caches that will not be re-used later
                10. + *
                + *
              6. + *
              7. When all windows processed, create the downstream update and notify
              8. + *
              9. Release resources
              10. + *
              */ - protected void shiftWindowOperators(UpdateByWindow win, RowSetShiftData shift) { - if (!redirContext.isRedirected() && shift.nonempty()) { - try (final RowSet prevIdx = source.getRowSet().copyPrev()) { - shift.apply((begin, end, delta) -> { - try (final RowSet subRowSet = prevIdx.subSetByKeyRange(begin, end)) { - for (UpdateByOperator op : win.getOperators()) { - op.applyOutputShift(subRowSet, delta); - } - } - }); - } - } - } - protected void processBuckets(UpdateByBucketHelper[] dirtyBuckets, boolean initialStep, RowSetShiftData shifts) { - // we need to ID which rows may be affected by the upstream shifts so we can include them - // in our parallel update preparations - final WritableRowSet shiftedRows; - if (shifts.nonempty()) { - try (final RowSet prev = source.getRowSet().copyPrev(); - final RowSequence.Iterator it = prev.getRowSequenceIterator()) { - - final RowSetBuilderSequential builder = RowSetFactory.builderSequential(); - final int size = shifts.size(); - - // get these in order so we can use a sequential builder - for (int ii = 0; ii < size; ii++) { - final long begin = shifts.getBeginRange(ii); - final long end = shifts.getEndRange(ii); - final long delta = shifts.getShiftDelta(ii); - - it.advance(begin); - final RowSequence rs = it.getNextRowSequenceThrough(end); - builder.appendRowSequenceWithOffset(rs, delta); - } - shiftedRows = builder.build(); - } - } else { - shiftedRows = RowSetFactory.empty(); - } + protected class StateManager implements LogOutputAppendable { + final TableUpdate update; + final boolean initialStep; + final UpdateByBucketHelper[] dirtyBuckets; final ColumnSource[] maybeCachedInputSources; final TrackingWritableRowSet[] inputSourceRowSets; final AtomicInteger[] inputSourceReferenceCounts; - if (inputCacheNeeded) { - maybeCachedInputSources = new ColumnSource[inputSources.length]; - inputSourceRowSets = new TrackingWritableRowSet[inputSources.length]; - inputSourceReferenceCounts = new AtomicInteger[inputSources.length]; - // do the hard work of computing what should be cached for each input source - computeCachedColumnContents(dirtyBuckets, initialStep, inputSourceRowSets, inputSourceReferenceCounts); + final SelectAndViewAnalyzer.JobScheduler jobScheduler; + final CompletableFuture waitForResult; - // assign the non-cached input source or leave null for cached sources - for (int srcIdx = 0; srcIdx < inputSources.length; srcIdx++) { - maybeCachedInputSources[srcIdx] = inputSourceCacheNeeded[srcIdx] ? null : inputSources[srcIdx]; - } - } else { - maybeCachedInputSources = inputSources; - inputSourceRowSets = null; - inputSourceReferenceCounts = null; - } + WritableRowSet shiftedRows; + + public StateManager(TableUpdate update, boolean initialStep) { + this.update = update; + this.initialStep = initialStep; - // process the windows - for (int winIdx = 0; winIdx < windows.length; winIdx++) { - UpdateByWindow win = windows[winIdx]; + // determine which buckets we'll examine during this update + dirtyBuckets = buckets.stream().filter(UpdateByBucketHelper::isDirty).toArray(UpdateByBucketHelper[]::new); if (inputCacheNeeded) { - // cache the sources needed for this window - cacheInputSources(winIdx, maybeCachedInputSources, inputSourceRowSets); + maybeCachedInputSources = new ColumnSource[inputSources.length]; + inputSourceRowSets = new TrackingWritableRowSet[inputSources.length]; + inputSourceReferenceCounts = new AtomicInteger[inputSources.length]; + } else { + maybeCachedInputSources = inputSources; + inputSourceRowSets = null; + inputSourceReferenceCounts = null; } + waitForResult = new CompletableFuture<>(); + if (initialStep) { - // prepare each operator for the parallel updates to come - for (UpdateByOperator op : win.getOperators()) { - op.prepareForParallelPopulation(source.getRowSet()); + if (OperationInitializationThreadPool.NUM_THREADS > 1) { + jobScheduler = new SelectAndViewAnalyzer.OperationInitializationPoolJobScheduler(); + } else { + jobScheduler = SelectAndViewAnalyzer.ImmediateJobScheduler.INSTANCE; } } else { - try (final WritableRowSet windowRowSet = shiftedRows.copy()) { - // get the total rowset from this window - for (UpdateByBucketHelper bucket : dirtyBuckets) { - // append the dirty rows from this window - if (win.isWindowDirty(bucket.windowContexts[winIdx])) { - windowRowSet.insert(win.getAffectedRows(bucket.windowContexts[winIdx])); + if (UpdateGraphProcessor.DEFAULT.getUpdateThreads() > 1) { + jobScheduler = new SelectAndViewAnalyzer.UpdateGraphProcessorJobScheduler(); + } else { + jobScheduler = SelectAndViewAnalyzer.ImmediateJobScheduler.INSTANCE; + } + } + } + + @Override + public LogOutput append(LogOutput logOutput) { + return logOutput.append("UpdateBy.StateManager"); + } + + private void onError(Exception error) { + // Not sure what to do here... + waitForResult.completeExceptionally(error); + System.out.println(error.toString()); + } + + /** + * Accumulate in parallel the dirty bucket rowsets for the cacheable input sources. Calls + * {@code completedAction} when the work is complete + */ + private void computeCachedColumnRowsets(final Runnable completeAction) { + if (!inputCacheNeeded) { + completeAction.run(); + return; + } + + // on the initial step, everything is dirty + if (initialStep) { + for (int srcIdx : cacheableSourceIndices) { + // create a TrackingRowSet to be used by `InverseRowRedirectionImpl` + inputSourceRowSets[srcIdx] = source.getRowSet().copy().toTracking(); + + // how many windows require this input source? + int refCount = 0; + for (UpdateByWindow win : windows) { + if (win.isSourceInUse(srcIdx)) { + refCount++; + } + } + inputSourceReferenceCounts[srcIdx] = new AtomicInteger(refCount); + } + completeAction.run(); + return; + } + + jobScheduler.iterateParallel(ExecutionContext.getContextToRecord(), this, + 0, cacheableSourceIndices.length, + idx -> { + int srcIdx = cacheableSourceIndices[idx]; + int refCount = 0; + for (int winIdx = 0; winIdx < windows.length; winIdx++) { + UpdateByWindow win = windows[winIdx]; + if (win.isSourceInUse(srcIdx)) { + for (UpdateByBucketHelper bucket : dirtyBuckets) { + UpdateByWindow.UpdateByWindowContext winCtx = bucket.windowContexts[winIdx]; + + if (win.isWindowDirty(winCtx)) { + // add this rowset to the running total + if (inputSourceRowSets[srcIdx] == null) { + inputSourceRowSets[srcIdx] = + win.getInfluencerRows(winCtx).copy().toTracking(); + } else { + inputSourceRowSets[srcIdx].insert(win.getInfluencerRows(winCtx)); + } + } + } + refCount++; + } } + inputSourceReferenceCounts[srcIdx] = new AtomicInteger(refCount); + }, + completeAction, + this::onError); + } + + /** + * Create a new input source cache and populate the required rows in parallel. Calls + * {@code completedAction} when the work is complete + */ + private void createCachedColumnSource(int srcIdx, final Runnable completeAction) { + if (maybeCachedInputSources[srcIdx] != null) { + // already cached from another operator (or caching not needed) + completeAction.run(); + return; + } + + final ColumnSource inputSource = inputSources[srcIdx]; + final TrackingWritableRowSet inputRowSet = inputSourceRowSets[srcIdx]; + + // re-use the dense column cache if it still exists + WritableColumnSource innerSource; + if (inputSourceCaches[srcIdx] == null || (innerSource = inputSourceCaches[srcIdx].get()) == null) { + // create a new dense cache + innerSource = ArrayBackedColumnSource.getMemoryColumnSource(inputSource.getType(), + inputSource.getComponentType()); + inputSourceCaches[srcIdx] = new SoftReference<>(innerSource); + } + innerSource.ensureCapacity(inputRowSet.size()); + + final WritableRowRedirection rowRedirection = new InverseRowRedirectionImpl(inputRowSet); + final WritableColumnSource outputSource = + new WritableRedirectedColumnSource(rowRedirection, innerSource, 0); + + // holding this reference will protect `rowDirection` and `innerSource` from GC + maybeCachedInputSources[srcIdx] = outputSource; + + final int PARALLEL_CHUNK_SIZE = 1 << 20; // 1M row chunks (configuration item?) + final int CHUNK_SIZE = 1 << 16; // copied from SparseSelect + + if (inputRowSet.size() >= PARALLEL_CHUNK_SIZE) { + // divide the rowset into reasonable chunks and do the cache population in parallel + final ArrayList populationRowSets = new ArrayList<>(); + try (final RowSequence.Iterator rsIt = inputRowSet.getRowSequenceIterator()) { + while (rsIt.hasMore()) { + final RowSequence chunkOk = rsIt.getNextRowSequenceWithLength(PARALLEL_CHUNK_SIZE); + populationRowSets.add(chunkOk.asRowSet().copy()); } - // prepare each operator for the parallel updates to come - for (UpdateByOperator op : win.getOperators()) { - op.prepareForParallelPopulation(windowRowSet); + } + jobScheduler.iterateParallel(ExecutionContext.getContextToRecord(), this, + 0, populationRowSets.size(), + idx -> { + try (final RowSet chunkRs = populationRowSets.get(idx); + final RowSequence.Iterator rsIt = chunkRs.getRowSequenceIterator(); + final ChunkSink.FillFromContext ffc = + outputSource.makeFillFromContext(CHUNK_SIZE); + final ChunkSource.GetContext gc = inputSource.makeGetContext(CHUNK_SIZE)) { + while (rsIt.hasMore()) { + final RowSequence chunkOk = rsIt.getNextRowSequenceWithLength(CHUNK_SIZE); + final Chunk values = inputSource.getChunk(gc, chunkOk); + outputSource.fillFromChunk(ffc, values, chunkOk); + } + } + }, () -> { + populationRowSets.clear(); + completeAction.run(); + }, + this::onError); + } else { + // run this in serial, not worth parallelization + try (final RowSequence.Iterator rsIt = inputRowSet.getRowSequenceIterator(); + final ChunkSink.FillFromContext ffc = + outputSource.makeFillFromContext(CHUNK_SIZE); + final ChunkSource.GetContext gc = inputSource.makeGetContext(CHUNK_SIZE)) { + while (rsIt.hasMore()) { + final RowSequence chunkOk = rsIt.getNextRowSequenceWithLength(CHUNK_SIZE); + final Chunk values = inputSource.getChunk(gc, chunkOk); + outputSource.fillFromChunk(ffc, values, chunkOk); } } + completeAction.run(); } - // now we can shift (after the parallel prep is complete) - shiftWindowOperators(win, shifts); + } - // PARALLELIZE THIS INTO + /** + * Create cached input sources for all input needed by {@code windows[winIdx]}. Calls + * {@code completedAction} when the work is complete + */ + private void cacheInputSources(final int winIdx, final Runnable completeAction) { + if (inputCacheNeeded) { + final UpdateByWindow win = windows[winIdx]; + final int[] uniqueWindowSources = win.getUniqueSourceIndices(); - // process the window - for (UpdateByBucketHelper bucket : dirtyBuckets) { - bucket.assignInputSources(winIdx, maybeCachedInputSources); - bucket.processWindow(winIdx, initialStep); + jobScheduler.iterateParallel(ExecutionContext.getContextToRecord(), this, 0, uniqueWindowSources.length, + (idx, sourceComplete) -> { + createCachedColumnSource(uniqueWindowSources[idx], sourceComplete); + }, completeAction, this::onError); + } else { + // no work to do, continue + completeAction.run(); } + } - if (inputCacheNeeded) { - // release the cached sources that are no longer needed - releaseInputSources(winIdx, maybeCachedInputSources, inputSourceRowSets, inputSourceReferenceCounts); + /** Shift the operator output columns for this window */ + protected void shiftWindowOperators(UpdateByWindow win, RowSetShiftData shift) { + try (final RowSet prevIdx = source.getRowSet().copyPrev()) { + shift.apply((begin, end, delta) -> { + try (final RowSet subRowSet = prevIdx.subSetByKeyRange(begin, end)) { + for (UpdateByOperator op : win.getOperators()) { + op.applyOutputShift(subRowSet, delta); + } + } + }); } - } - // we are done with this rowset - shiftedRows.close(); - } + /** + * Divide the buckets for {@code windows[winIdx]} into sets and process each set in parallel. Calls + * {@code completedAction} when the work is complete + */ + private void processWindowBuckets(int winIdx, final Runnable completedAction) { + if (jobScheduler.threadCount() > 1) { + // process the buckets in parallel + final int bucketsPerTask = Math.max(1, dirtyBuckets.length / jobScheduler.threadCount()); + TIntArrayList offsetList = new TIntArrayList(); + TIntArrayList countList = new TIntArrayList(); - protected void finalizeBuckets(UpdateByBucketHelper[] dirtyBuckets) { - for (UpdateByBucketHelper bucket : dirtyBuckets) { - bucket.finalizeUpdate(); - } - } + for (int ii = 0; ii < dirtyBuckets.length; ii += bucketsPerTask) { + offsetList.add(ii); + countList.add(Math.min(bucketsPerTask, dirtyBuckets.length - ii)); + } - /** - * The Listener for apply to the constituent table updates - */ - class UpdateByListener extends MergedListener { - public UpdateByListener(@Nullable String description) { - super(UpdateBy.this.recorders, List.of(), description, UpdateBy.this.result); + jobScheduler.iterateParallel(ExecutionContext.getContextToRecord(), this, 0, offsetList.size(), idx -> { + final int bucketOffset = offsetList.get(idx); + final int bucketCount = countList.get(idx); + for (int bucketIdx = bucketOffset; bucketIdx < bucketOffset + bucketCount; bucketIdx++) { + UpdateByBucketHelper bucket = dirtyBuckets[bucketIdx]; + bucket.assignInputSources(winIdx, maybeCachedInputSources); + bucket.processWindow(winIdx, initialStep); + } + }, completedAction, this::onError); + } else { + // minimize overhead when running serially + for (UpdateByBucketHelper bucket : dirtyBuckets) { + bucket.assignInputSources(winIdx, maybeCachedInputSources); + bucket.processWindow(winIdx, initialStep); + } + completedAction.run(); + } + } + + /** + * Process all {@code windows} in a serial manner (to minimize cache memory usage). Will create cached + * input sources, process the buckets, then release the cached columns before starting the next window. Calls + * {@code completedAction} when the work is complete + */ + private void processWindows(final Runnable completeAction) { + jobScheduler.iterateSerial(ExecutionContext.getContextToRecord(), this, 0, windows.length, + (winIdx, windowComplete) -> { + UpdateByWindow win = windows[winIdx]; + + // this is a chain of calls: cache, then shift, then process the dirty buckets for this window + cacheInputSources(winIdx, () -> { + // prepare each operator for the parallel updates to come + if (initialStep) { + for (UpdateByOperator op : win.getOperators()) { + op.prepareForParallelPopulation(source.getRowSet()); + } + } else { + // get the minimal set of rows to be updated for this window + try (final WritableRowSet windowRowSet = shiftedRows.copy()) { + for (UpdateByBucketHelper bucket : dirtyBuckets) { + if (win.isWindowDirty(bucket.windowContexts[winIdx])) { + windowRowSet.insert(win.getAffectedRows(bucket.windowContexts[winIdx])); + } + } + for (UpdateByOperator op : win.getOperators()) { + op.prepareForParallelPopulation(windowRowSet); + } + } + } + + // shift the non-redirected output sources now, after parallelPopulation + if (!redirContext.isRedirected() && update.shifted().nonempty()) { + shiftWindowOperators(win, update.shifted()); + } + + processWindowBuckets(winIdx, () -> { + if (inputCacheNeeded) { + // release the cached sources that are no longer needed + releaseInputSources(winIdx, maybeCachedInputSources, inputSourceRowSets, + inputSourceReferenceCounts); + } + + // signal that the work for this window is complete (will iterate to the next window + // sequentially) + windowComplete.run(); + }); + }); + }, completeAction, this::onError); } - @Override - protected void process() { - UpdateByBucketHelper[] dirtyBuckets = buckets.stream().filter(b -> b.isDirty()) - .toArray(UpdateByBucketHelper[]::new); + /** + * Clean up the resources created during this update and notify downstream if applicable + */ + private void cleanUpAndNotify() { + shiftedRows.close(); - final ListenerRecorder sourceRecorder = recorders.peekFirst(); - final TableUpdate upstream = sourceRecorder.getUpdate(); + // create the downstream before calling finalize() on the buckets (which releases resources) + final TableUpdate downstream; + if (!initialStep) { + downstream = computeDownstreamUpdate(); + } else { + downstream = null; + } + + // allow the helpers to release their resources + for (UpdateByBucketHelper bucket : dirtyBuckets) { + bucket.finalizeUpdate(); + } + + // pass the result downstream + if (downstream != null) { + result.notifyListeners(downstream); + } + // signal to the main task that we have completed our work + waitForResult.complete(null); + } + + /** + * Create the update for downstream listeners. This combines all bucket updates/modifies into a unified + * update + */ + private TableUpdate computeDownstreamUpdate() { final TableUpdateImpl downstream = new TableUpdateImpl(); - downstream.added = upstream.added().copy(); - downstream.removed = upstream.removed().copy(); - downstream.shifted = upstream.shifted(); + downstream.added = update.added().copy(); + downstream.removed = update.removed().copy(); + downstream.shifted = update.shifted(); // union the modifies from all the tables (including source) downstream.modifiedColumnSet = result.getModifiedColumnSetForUpdates(); downstream.modifiedColumnSet.clear(); - if (redirContext.isRedirected()) { - // this does all the work needed for redirected output sources - redirContext.processUpdateForRedirection(upstream, source.getRowSet()); - } - - processBuckets(dirtyBuckets, false, upstream.shifted()); - - // get the adds/removes/shifts from the first (source) entry, make a copy since TableUpdateImpl#reset will + // get the adds/removes/shifts from upstream, make a copy since TableUpdateImpl#reset will // close them with the upstream update WritableRowSet modifiedRowSet = RowSetFactory.empty(); downstream.modified = modifiedRowSet; - if (upstream.modified().isNonempty()) { - modifiedRowSet.insert(upstream.modified()); - downstream.modifiedColumnSet.setAll(upstream.modifiedColumnSet()); + if (update.modified().isNonempty()) { + modifiedRowSet.insert(update.modified()); + transformer.transform(update.modifiedColumnSet(), downstream.modifiedColumnSet); } for (UpdateByBucketHelper bucket : dirtyBuckets) { @@ -548,9 +657,86 @@ protected void process() { // should not include upstream adds as modifies modifiedRowSet.remove(downstream.added); - finalizeBuckets(dirtyBuckets); + return downstream; + } - result.notifyListeners(downstream); + /** + * Process the {@link TableUpdate update} provided in the constructor. This performs much work in parallel and + * leverages {@link io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer.JobScheduler} extensively + */ + public void processUpdate() { + if (redirContext.isRedirected()) { + // this call does all the work needed for redirected output sources, for sparse output sources + // we will process shifts only after a call to `prepareForParallelPopulation()` on each source + redirContext.processUpdateForRedirection(update, source.getRowSet()); + shiftedRows = RowSetFactory.empty(); + } else { + // for our sparse array output sources, we need to identify which rows will be affected by the upstream + // shifts and include them in our parallel update preparations + if (update.shifted().nonempty()) { + try (final RowSet prev = source.getRowSet().copyPrev(); + final RowSequence.Iterator it = prev.getRowSequenceIterator()) { + + final RowSetBuilderSequential builder = RowSetFactory.builderSequential(); + final int size = update.shifted().size(); + + // get these in ascending order and use a sequential builder + for (int ii = 0; ii < size; ii++) { + final long begin = update.shifted().getBeginRange(ii); + final long end = update.shifted().getEndRange(ii); + final long delta = update.shifted().getShiftDelta(ii); + + it.advance(begin); + final RowSequence rs = it.getNextRowSequenceThrough(end); + builder.appendRowSequenceWithOffset(rs, delta); + } + shiftedRows = builder.build(); + } + } else { + shiftedRows = RowSetFactory.empty(); + } + } + + // this is where we leave single-threaded calls and rely on the scheduler to continue the work. Each + // call will chain to another until the sequence is complete + computeCachedColumnRowsets( + () -> processWindows( + () -> cleanUpAndNotify())); + + try { + // need to wait until this future is complete + waitForResult.get(); + } catch (InterruptedException e) { + throw new CancellationException("interrupted while processing updateBy"); + } catch (ExecutionException e) { + if (e.getCause() instanceof RuntimeException) { + throw (RuntimeException) e.getCause(); + } else { + throw new UncheckedDeephavenException("Failure while processing updateBy", + e.getCause()); + } + } + } + } + + /** + * The Listener for apply to the constituent table updates + */ + class UpdateByListener extends MergedListener { + public UpdateByListener(@Nullable String description) { + super(UpdateBy.this.recorders, List.of(), description, UpdateBy.this.result); + } + + @Override + protected void process() { + final ListenerRecorder sourceRecorder = recorders.peekFirst(); + final TableUpdate upstream = sourceRecorder.getUpdate(); + + // we need to keep a reference to TableUpdate during our computation + if (upstream != null) { + final StateManager sm = new StateManager(upstream.acquire(), false); + sm.processUpdate(); + } } @Override @@ -571,15 +757,15 @@ public UpdateByListener newListener(@NotNull final String description) { * Apply the specified operations to each group of rows in the source table and produce a result table with the same * index as the source with each operator applied. * - * @param source the source to apply to. - * @param clauses the operations to apply. + * @param source the source to apply to. + * @param clauses the operations to apply. * @param byColumns the columns to group by before applying operations * @return a new table with the same index as the source with all the operations applied. */ public static Table updateBy(@NotNull final QueryTable source, - @NotNull final Collection clauses, - @NotNull final Collection byColumns, - @NotNull final UpdateByControl control) { + @NotNull final Collection clauses, + @NotNull final Collection byColumns, + @NotNull final UpdateByControl control) { // create the rowRedirection if instructed final WritableRowRedirection rowRedirection; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java index bac8cd9824f..3b5c802cad2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java @@ -57,15 +57,15 @@ class UpdateByBucketHelper { */ protected UpdateByBucketHelper(@NotNull final String description, - @NotNull final QueryTable source, - @NotNull final UpdateByOperator[] operators, - @NotNull final UpdateByWindow[] windows, - @NotNull final ColumnSource[] inputSources, - @NotNull final int[][] operatorInputSourceSlots, - @NotNull final Map> resultSources, - @Nullable String timestampColumnName, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, - @NotNull final UpdateByControl control) { + @NotNull final QueryTable source, + @NotNull final UpdateByOperator[] operators, + @NotNull final UpdateByWindow[] windows, + @NotNull final ColumnSource[] inputSources, + @NotNull final int[][] operatorInputSourceSlots, + @NotNull final Map> resultSources, + @Nullable String timestampColumnName, + @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, + @NotNull final UpdateByControl control) { this.source = source; this.operators = operators; @@ -238,7 +238,7 @@ private void fillChunkWithNonNull(LongChunk keysChunk, LongChunk * @param initialStep Whether this update is part of the initial creation of the bucket */ public void prepareForUpdate(final TableUpdate upstream, final boolean initialStep) { - Assert.eqFalse(isDirty, "UpdateBy bucket was marekd dirty before processing an update"); + Assert.eqFalse(isDirty, "UpdateBy bucket was marked dirty before processing an update"); // add all the SSA data if (timestampColumnName != null) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateByManager.java index c171066f42f..958d1dd11a5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateByManager.java @@ -35,32 +35,21 @@ protected ZeroKeyUpdateByManager(@NotNull final String description, recorders = new LinkedList<>(); listener = newListener(description); - // create an intermediate table that will listen to source updates and shift output columns - final QueryTable shiftApplyTable = new QueryTable(source.getRowSet(), source.getColumnSourceMap()); - - source.listenForUpdates(new BaseTable.ListenerImpl("", source, shiftApplyTable) { - @Override - public void onUpdate(@NotNull final TableUpdate upstream) { -// shiftOutputColumns(upstream); - super.onUpdate(upstream); - } - }); - - // create a recorder instance sourced from the shifting table - ListenerRecorder shiftRecorder = new ListenerRecorder(description, shiftApplyTable, result); - shiftRecorder.setMergedListener(listener); - shiftApplyTable.listenForUpdates(shiftRecorder); + // create a recorder instance sourced from the source table + ListenerRecorder sourceRecorder = new ListenerRecorder(description, source, result); + sourceRecorder.setMergedListener(listener); + source.listenForUpdates(sourceRecorder); result.addParentReference(listener); - recorders.offerLast(shiftRecorder); + recorders.offerLast(sourceRecorder); // create input and output modified column sets for (UpdateByOperator op : operators) { - op.createInputModifiedColumnSet(shiftApplyTable); + op.createInputModifiedColumnSet(source); op.createOutputModifiedColumnSet(result); } - // create an updateby bucket instance sourced from the shifting table - zeroKeyUpdateBy = new UpdateByBucketHelper(description, shiftApplyTable, operators, windows, inputSources, + // create an updateby bucket instance sourced from the source table + zeroKeyUpdateBy = new UpdateByBucketHelper(description, source, operators, windows, inputSources, operatorInputSourceSlots, resultSources, timestampColumnName, redirContext, control); buckets.offerLast(zeroKeyUpdateBy); @@ -82,20 +71,19 @@ public void onUpdate(@NotNull final TableUpdate upstream) { } } - if (redirContext.isRedirected()) { - // make a dummy update to generate the initial row keys - final TableUpdateImpl fakeUpdate = new TableUpdateImpl(source.getRowSet(), - RowSetFactory.empty(), - RowSetFactory.empty(), - RowSetShiftData.EMPTY, - ModifiedColumnSet.EMPTY); - redirContext.processUpdateForRedirection(fakeUpdate, source.getRowSet()); - } + // make the source->result transformer + transformer = source.newModifiedColumnSetTransformer(result, source.getDefinition().getColumnNamesArray()); + + // make a dummy update to generate the initial row keys + final TableUpdateImpl fakeUpdate = new TableUpdateImpl(source.getRowSet(), + RowSetFactory.empty(), + RowSetFactory.empty(), + RowSetShiftData.EMPTY, + ModifiedColumnSet.EMPTY); // do the actual computations - UpdateByBucketHelper[] dirtyBuckets = new UpdateByBucketHelper[] {zeroKeyUpdateBy}; - processBuckets(dirtyBuckets, true, RowSetShiftData.EMPTY); - finalizeBuckets(dirtyBuckets); + final StateManager sm = new StateManager(fakeUpdate, true); + sm.processUpdate(); } /** diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer.java index a249ed69e53..ceb24642fe6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer.java @@ -29,12 +29,14 @@ import io.deephaven.io.log.impl.LogOutputStringImpl; import io.deephaven.util.SafeCloseable; import io.deephaven.util.SafeCloseablePair; +import io.deephaven.util.annotations.FinalDefault; import io.deephaven.util.process.ProcessEnvironment; import io.deephaven.vector.Vector; import org.jetbrains.annotations.Nullable; import java.util.*; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; import java.util.stream.Stream; @@ -532,6 +534,171 @@ void submit( * sub-jobs to split work into. */ int threadCount(); + + /** + * Helper interface for {@code iterateSerial()} and {@code iterateParallel()}. This provides a callable + * interface with {@code index} indicating which iteration to perform. When this returns, the scheduler will + * automatically schedule the next iteration. + */ + @FunctionalInterface + interface IterateAction { + void run(int index); + } + + /** + * Helper interface for {@code iterateSerial()} and {@code iterateParallel()}. This provides a callable + * interface with {@code index} indicating which iteration to perform and {@link Runnable resume} providing a + * mechanism to inform the scheduler that the current task is complete. When {@code resume} is called, the + * scheduler will automatically schedule the next iteration. + * + * NOTE: failing to call {@code resume} will result in the scheduler not scheduling all remaining iterations. + * This will not block the scheduler, but the {@code completeAction} {@link Runnable} will never be called + */ + @FunctionalInterface + interface IterateResumeAction { + void run(int index, Runnable resume); + } + + /** + * Provides a mechanism to iterate over a range of values in parallel using the {@link JobScheduler} + * + * @param executionContext the execution context for this task + * @param description the description for + * @param start the integer value from which to start iterating + * @param count the number of times this task should be called + * @param action the task to perform, the current iteration index is provided as a parameter + * @param completeAction this will be called when all iterations are complete + * @param onError error handler for the scheduler to use while iterating + */ + @FinalDefault + default void iterateParallel(ExecutionContext executionContext, LogOutputAppendable description, int start, + int count, IterateAction action, Runnable completeAction, final Consumer onError) { + final AtomicInteger nextIndex = new AtomicInteger(start); + final AtomicInteger remaining = new AtomicInteger(count); + + final Runnable task = () -> { + // this will run until all tasks have started + while (true) { + int idx = nextIndex.getAndIncrement(); + if (idx < start + count) { + // do the work + action.run(idx); + + // check for completion + if (remaining.decrementAndGet() == 0) { + completeAction.run(); + return; + } + } else { + // no more work to do + return; + } + } + }; + + // create multiple tasks but not more than one per scheduler thread + for (int i = 0; i < Math.min(count, threadCount()); i++) { + submit(executionContext, + task, + description, + onError); + } + } + + /** + * Provides a mechanism to iterate over a range of values in parallel using the {@link JobScheduler} + * + * @param executionContext the execution context for this task + * @param description the description for + * @param start the integer value from which to start iterating + * @param count the number of times this task should be called + * @param action the task to perform, the current iteration index and a resume Runnable are parameters + * @param completeAction this will be called when all iterations are complete + * @param onError error handler for the scheduler to use while iterating + */ + @FinalDefault + default void iterateParallel(ExecutionContext executionContext, LogOutputAppendable description, int start, + int count, IterateResumeAction action, Runnable completeAction, Consumer onError) { + final AtomicInteger nextIndex = new AtomicInteger(start); + final AtomicInteger remaining = new AtomicInteger(count); + + final Runnable resumeAction = () -> { + // check for completion + if (remaining.decrementAndGet() == 0) { + completeAction.run(); + } + }; + + final Runnable task = () -> { + // this will run until all tasks have started + while (true) { + int idx = nextIndex.getAndIncrement(); + if (idx < start + count) { + // do the work + action.run(idx, resumeAction); + } else { + // no more work to do + return; + } + } + }; + + // create multiple tasks but not more than one per scheduler thread + for (int i = 0; i < Math.min(count, threadCount()); i++) { + submit(executionContext, + task, + description, + onError); + } + } + + /** + * Provides a mechanism to iterate over a range of values serially using the {@link JobScheduler}. The + * advantage to using this over a simple iteration is the resumption callable on {@code action} that will + * trigger the next iterable. This allows the next iteration to de delayed until dependendat asynchronous + * serial or parallel scheduler jobs have completed. + * + * @param executionContext the execution context for this task + * @param description the description for + * @param start the integer value from which to start iterating + * @param count the number of times this task should be called + * @param action the task to perform, the current iteration index and a resume Runnable are parameters + * @param completeAction this will be called when all iterations are complete + * @param onError error handler for the scheduler to use while iterating + */ + @FinalDefault + default void iterateSerial(ExecutionContext executionContext, LogOutputAppendable description, int start, + int count, IterateResumeAction action, Runnable completeAction, Consumer onError) { + final AtomicInteger nextIndex = new AtomicInteger(start); + final AtomicInteger remaining = new AtomicInteger(count); + + final Runnable resumeAction = () -> { + // check for completion + if (remaining.decrementAndGet() == 0) { + completeAction.run(); + } + }; + + final Runnable task = () -> { + // this will run until all tasks have started + while (true) { + int idx = nextIndex.getAndIncrement(); + if (idx < start + count) { + // do the work + action.run(idx, resumeAction); + } else { + // no more work to do + return; + } + } + }; + + // create a single task + submit(executionContext, + task, + description, + onError); + } } public static class UpdateGraphProcessorJobScheduler implements SelectAndViewAnalyzer.JobScheduler { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UGPTaskHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UGPTaskHelper.java deleted file mode 100644 index 3f8b594725d..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UGPTaskHelper.java +++ /dev/null @@ -1,105 +0,0 @@ -package io.deephaven.engine.table.impl.updateby; - -import io.deephaven.engine.context.ExecutionContext; -import io.deephaven.engine.updategraph.AbstractNotification; -import io.deephaven.engine.updategraph.UpdateGraphProcessor; - -import java.util.ArrayList; -import java.util.concurrent.atomic.AtomicInteger; - -public class UGPTaskHelper { - - public static abstract class UGPTaskHelperNotification extends AbstractNotification { - protected final ArrayList dataList; - protected final AtomicInteger nextTask; - protected final AtomicInteger tasksCompleted; - protected final WorkerAction workerAction; - protected final Runnable completeAction; - - @FunctionalInterface - public interface WorkerAction { - void run(T data, Runnable complete); - } - - private UGPTaskHelperNotification(ArrayList dataList, AtomicInteger nextTask, AtomicInteger tasksCompleted, WorkerAction workerAction, Runnable completeAction) { - super(true); - this.dataList = dataList; - this.nextTask = nextTask; - this.tasksCompleted = tasksCompleted; - this.workerAction = workerAction; - this.completeAction = completeAction; - } - - @Override - public boolean canExecute(long step) { - return true; - } - - @Override - public ExecutionContext getExecutionContext() { - return null; - } - - @Override - public void run() { - // do the work - int taskIndex = nextTask.getAndIncrement(); - workerAction.run(dataList.get(taskIndex), this::complete); - } - - protected abstract void complete(); - } - - private static class UGPTaskHelperSerialNotification extends UGPTaskHelperNotification { - private UGPTaskHelperSerialNotification(ArrayList dataList, AtomicInteger nextTask, AtomicInteger tasksCompleted, WorkerAction workerAction, Runnable completeAction) { - super(dataList, nextTask, tasksCompleted, workerAction, completeAction); - } - - protected void complete() { - // serially call the next task - int completedCount = tasksCompleted.incrementAndGet(); - - if (completedCount == dataList.size()) { - completeAction.run(); - } else { - // this will recurse N times, is it better to create a new UGP notification and schedule it? - run(); - } - } - } - - private static class UGPTaskHelperParallelNotification extends UGPTaskHelperNotification { - private UGPTaskHelperParallelNotification(ArrayList dataList, AtomicInteger nextTask, AtomicInteger tasksCompleted, WorkerAction workerAction, Runnable completeAction) { - super(dataList, nextTask, tasksCompleted, workerAction, completeAction); - } - - protected void complete() { - // call the complete when all the tasks are done - int completedCount = tasksCompleted.incrementAndGet(); - - if (completedCount == dataList.size()) { - completeAction.run(); - } - } - } - - public static void ExecuteSerial(ArrayList dataObjects, UGPTaskHelperNotification.WorkerAction workerAction, Runnable completedAction) { - final AtomicInteger completed = new AtomicInteger(0); - final AtomicInteger nextTask = new AtomicInteger(0); - - // create and schedule a single notification, will auto serialize - UGPTaskHelperSerialNotification notification = new UGPTaskHelperSerialNotification<>(dataObjects, nextTask, completed, workerAction, completedAction); - UpdateGraphProcessor.DEFAULT.addNotification(notification); - } - - public static void ExecuteParallel(ArrayList dataObjects, UGPTaskHelperNotification.WorkerAction workerAction, Runnable completedAction) { - final AtomicInteger completed = new AtomicInteger(0); - final AtomicInteger nextTask = new AtomicInteger(0); - - // create and schedule a notification for each object - for (int ii = 0; ii < dataObjects.size(); ii++) { - UGPTaskHelperParallelNotification notification = new UGPTaskHelperParallelNotification<>(dataObjects, nextTask, completed, workerAction, completedAction); - UpdateGraphProcessor.DEFAULT.addNotification(notification); - } - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java index 6d05c98990a..482a4cc16f7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java @@ -180,6 +180,19 @@ public int[] getUniqueSourceIndices() { return uniqueInputSourceIndices; } + public boolean isSourceInUse(int srcIdx) { + // this looks worse than it actually is, windows are defined by their input sources so there will be only + // one or two entries in `getUniqueSourceIndices()`. Iterating will be faster than building a lookup table + // or a hashset + for (int winSrcIdx : getUniqueSourceIndices()) { + if (winSrcIdx == srcIdx) { + return true; + } + } + + return false; + } + public void prepareForParallelPopulation(final RowSet added) { for (UpdateByOperator operator : operators) { operator.prepareForParallelPopulation(added); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java index 9350b9f41df..bf1b33da1b0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java @@ -262,10 +262,10 @@ public void computeAffectedRowsAndOperators(UpdateByWindowContext context, @NotN // other rows can be affected by removes if (upstream.removed().isNonempty()) { try (final RowSet prev = ctx.sourceRowSet.copyPrev(); - final RowSet removedPositions = prev.invert(upstream.removed()); - final WritableRowSet affectedByRemoves = - computeAffectedRowsTicks(prev, upstream.removed(), removedPositions, prevUnits, - fwdUnits)) { + final RowSet removedPositions = prev.invert(upstream.removed()); + final WritableRowSet affectedByRemoves = + computeAffectedRowsTicks(prev, upstream.removed(), removedPositions, prevUnits, + fwdUnits)) { // apply shifts to get back to pos-shift space upstream.shifted().apply(affectedByRemoves); // retain only the rows that still exist in the sourceRowSet diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java index af3288a76e4..bb0fb69054b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java @@ -67,7 +67,7 @@ public void close() { } UpdateByWindowTime(UpdateByOperator[] operators, int[][] operatorSourceSlots, @Nullable String timestampColumnName, - long prevUnits, long fwdUnits) { + long prevUnits, long fwdUnits) { super(operators, operatorSourceSlots, timestampColumnName); this.prevUnits = prevUnits; this.fwdUnits = fwdUnits; @@ -402,26 +402,26 @@ public void processRows(UpdateByWindowContext context, boolean initialStep) { Arrays.fill(ctx.inputSourceChunkPopulated, false); for (int opIdx : context.dirtyOperatorIndices) { - // prep the chunk array needed by the accumulate call - final int[] srcIndices = operatorInputSourceSlots[opIdx]; - Chunk[] chunkArr = new Chunk[srcIndices.length]; - for (int ii = 0; ii < srcIndices.length; ii++) { - int srcIdx = srcIndices[ii]; - // chunk prep - prepareValuesChunkForSource(ctx, srcIdx, chunkInfluencerRs); - chunkArr[ii] = ctx.inputSourceChunks[srcIdx]; - } - - // make the specialized call for windowed operators - ((UpdateByWindowedOperator.Context) ctx.opContext[opIdx]).accumulate( - chunkRs, - chunkArr, - pushChunk, - popChunk, - chunkRsSize); + // prep the chunk array needed by the accumulate call + final int[] srcIndices = operatorInputSourceSlots[opIdx]; + Chunk[] chunkArr = new Chunk[srcIndices.length]; + for (int ii = 0; ii < srcIndices.length; ii++) { + int srcIdx = srcIndices[ii]; + // chunk prep + prepareValuesChunkForSource(ctx, srcIdx, chunkInfluencerRs); + chunkArr[ii] = ctx.inputSourceChunks[srcIdx]; } + + // make the specialized call for windowed operators + ((UpdateByWindowedOperator.Context) ctx.opContext[opIdx]).accumulate( + chunkRs, + chunkArr, + pushChunk, + popChunk, + chunkRsSize); } } + } } // call `finishUpdate()` function for each operator From 5d7a2b60793a9c3290bbbfa0cd52dad3b7a08ebc Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 11 Nov 2022 08:26:31 -0800 Subject: [PATCH 040/123] Add'l documentation --- .../deephaven/engine/table/impl/UpdateBy.java | 107 +++++++++------- .../table/impl/UpdateByBucketHelper.java | 32 ++++- .../impl/UpdateByCumulativeOperator.java | 10 ++ .../engine/table/impl/UpdateByOperator.java | 19 ++- .../table/impl/UpdateByWindowedOperator.java | 5 + .../analyzers/SelectAndViewAnalyzer.java | 16 +-- .../table/impl/updateby/UpdateByWindow.java | 118 +++++++++++++++--- .../updateby/UpdateByWindowCumulative.java | 5 +- .../impl/updateby/UpdateByWindowTicks.java | 16 ++- .../impl/updateby/UpdateByWindowTime.java | 12 +- 10 files changed, 246 insertions(+), 94 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java index 91710f6fc49..0b344bb25d2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java @@ -13,6 +13,7 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ResettableWritableObjectChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.configuration.Configuration; import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.exceptions.CancellationException; import io.deephaven.engine.exceptions.UncheckedTableException; @@ -43,6 +44,13 @@ * The core of the {@link Table#updateBy(UpdateByControl, Collection, Collection)} operation. */ public abstract class UpdateBy { + /** When caching a column source, how many rows should we process in each parallel batch? (1M default) */ + public static final int PARALLEL_CACHE_BATCH_SIZE = + Configuration.getInstance().getIntegerWithDefault("UpdateBy.parallelCacheBatchSize", 1 << 20); + /** When caching a column source, what size chunks should be used to move data to the cache? (65K default) */ + public static final int PARALLEL_CACHE_CHUNK_SIZE = + Configuration.getInstance().getIntegerWithDefault("UpdateBy.parallelCacheChunkSize", 1 << 16); + /** Input sources may be reused by mutiple operators, only store and cache unique ones */ protected final ColumnSource[] inputSources; /** Map operators to input sources, note some operators need more than one input, WAvg e.g. */ @@ -65,8 +73,10 @@ public abstract class UpdateBy { protected final boolean inputCacheNeeded; /** Whether caching benefits each input source */ protected final boolean[] inputSourceCacheNeeded; - /** References to the dense array sources we are using for the cached sources, it's expected that these will be - * released and need to be created */ + /** + * References to the dense array sources we are using for the cached sources, it's expected that these will be + * released and need to be created + */ protected final SoftReference>[] inputSourceCaches; /** For easy iteration, create a list of the source indices that need to be cached */ protected final int[] cacheableSourceIndices; @@ -78,7 +88,7 @@ public abstract class UpdateBy { /** For refreshing sources, maintain a list of each of the bucket listeners */ protected LinkedList recorders; - /** For refreshing sources, need a merged listener to produce downstream updates */ + /** For refreshing sources, need a merged listener to produce downstream updates */ protected UpdateByListener listener; public static class UpdateByRedirectionContext implements Context { @@ -211,7 +221,7 @@ protected UpdateBy(@NotNull final String description, buckets = new LinkedList<>(); } - /** Remove all references to Objects for this column source */ + /** Remove all references to Objects for this column source */ private void fillObjectArraySourceWithNull(ObjectArraySource sourceToNull) { Assert.neqNull(sourceToNull, "cached column source was null, must have been GC'd"); try (final ResettableWritableObjectChunk backingChunk = @@ -225,7 +235,7 @@ private void fillObjectArraySourceWithNull(ObjectArraySource sourceToNull) { } } - /** Release the input sources that will not be needed for the rest of this update */ + /** Release the input sources that will not be needed for the rest of this update */ private void releaseInputSources(int winIdx, ColumnSource[] maybeCachedInputSources, TrackingWritableRowSet[] inputSourceRowSets, AtomicInteger[] inputSourceReferenceCounts) { final UpdateByWindow win = windows[winIdx]; @@ -251,19 +261,22 @@ private void releaseInputSources(int winIdx, ColumnSource[] maybeCachedInputS /** * Overview of work performed by {@link StateManager}: *
                - *
              1. Create `shiftedRows`, the set of rows for the output sources that are affected by shifts
              2. - *
              3. Compute a rowset for each cacheable input source identifying which rows will be needed for processing
              4. - *
              5. Process each window serially - *
                  - *
                1. Cache the input sources that are needed for this window (this can be done in parallel for each column and even in a chunky way)
                2. - *
                3. Compute the modified rowset of output column sources and call `prepareForParallelPopulation()', this could be done in parallel with the caching
                4. - *
                5. When prepareForParallelPopulation() complete, apply upstream shifts to the output sources
                6. - *
                7. When caching and shifts are complete, process the data in this window in parallel by dividing the buckets into sets (N/num_threads) and running a job for each bucket_set 3e) when all buckets processed
                8. - *
                9. When all buckets processed, release the input source caches that will not be re-used later
                10. - *
                - *
              6. - *
              7. When all windows processed, create the downstream update and notify
              8. - *
              9. Release resources
              10. + *
              11. Create `shiftedRows`, the set of rows for the output sources that are affected by shifts
              12. + *
              13. Compute a rowset for each cacheable input source identifying which rows will be needed for processing
              14. + *
              15. Process each window serially + *
                  + *
                • Cache the input sources that are needed for this window (this can be done in parallel for each column and + * parallel again for a subset of the rows)
                • + *
                • Compute the modified rowset of output column sources and call `prepareForParallelPopulation()', this could be + * done in parallel with the caching
                • + *
                • When prepareForParallelPopulation() complete, apply upstream shifts to the output sources
                • + *
                • When caching and shifts are complete, process the data in this window in parallel by dividing the buckets + * into sets (N/num_threads) and running a job for each bucket_set 3e) when all buckets processed
                • + *
                • When all buckets processed, release the input source caches that will not be re-used later
                • + *
                + *
              16. + *
              17. When all windows processed, create the downstream update and notify
              18. + *
              19. Release resources
              20. *
              */ @@ -327,7 +340,7 @@ private void onError(Exception error) { } /** - * Accumulate in parallel the dirty bucket rowsets for the cacheable input sources. Calls + * Accumulate in parallel the dirty bucket rowsets for the cacheable input sources. Calls * {@code completedAction} when the work is complete */ private void computeCachedColumnRowsets(final Runnable completeAction) { @@ -336,7 +349,7 @@ private void computeCachedColumnRowsets(final Runnable completeAction) { return; } - // on the initial step, everything is dirty + // initially everything is dirty so cache everything if (initialStep) { for (int srcIdx : cacheableSourceIndices) { // create a TrackingRowSet to be used by `InverseRowRedirectionImpl` @@ -367,7 +380,7 @@ private void computeCachedColumnRowsets(final Runnable completeAction) { UpdateByWindow.UpdateByWindowContext winCtx = bucket.windowContexts[winIdx]; if (win.isWindowDirty(winCtx)) { - // add this rowset to the running total + // add this rowset to the running total for this input source if (inputSourceRowSets[srcIdx] == null) { inputSourceRowSets[srcIdx] = win.getInfluencerRows(winCtx).copy().toTracking(); @@ -386,8 +399,8 @@ private void computeCachedColumnRowsets(final Runnable completeAction) { } /** - * Create a new input source cache and populate the required rows in parallel. Calls - * {@code completedAction} when the work is complete + * Create a new input source cache and populate the required rows in parallel. Calls {@code completedAction} + * when the work is complete */ private void createCachedColumnSource(int srcIdx, final Runnable completeAction) { if (maybeCachedInputSources[srcIdx] != null) { @@ -409,22 +422,20 @@ private void createCachedColumnSource(int srcIdx, final Runnable completeAction) } innerSource.ensureCapacity(inputRowSet.size()); + // there will be no updates to this cached column source, so use a simple redirection final WritableRowRedirection rowRedirection = new InverseRowRedirectionImpl(inputRowSet); final WritableColumnSource outputSource = new WritableRedirectedColumnSource(rowRedirection, innerSource, 0); - // holding this reference will protect `rowDirection` and `innerSource` from GC + // holding this reference should protect `rowDirection` and `innerSource` from GC maybeCachedInputSources[srcIdx] = outputSource; - final int PARALLEL_CHUNK_SIZE = 1 << 20; // 1M row chunks (configuration item?) - final int CHUNK_SIZE = 1 << 16; // copied from SparseSelect - - if (inputRowSet.size() >= PARALLEL_CHUNK_SIZE) { + if (inputRowSet.size() >= PARALLEL_CACHE_BATCH_SIZE) { // divide the rowset into reasonable chunks and do the cache population in parallel final ArrayList populationRowSets = new ArrayList<>(); try (final RowSequence.Iterator rsIt = inputRowSet.getRowSequenceIterator()) { while (rsIt.hasMore()) { - final RowSequence chunkOk = rsIt.getNextRowSequenceWithLength(PARALLEL_CHUNK_SIZE); + final RowSequence chunkOk = rsIt.getNextRowSequenceWithLength(PARALLEL_CACHE_BATCH_SIZE); populationRowSets.add(chunkOk.asRowSet().copy()); } } @@ -434,10 +445,12 @@ private void createCachedColumnSource(int srcIdx, final Runnable completeAction) try (final RowSet chunkRs = populationRowSets.get(idx); final RowSequence.Iterator rsIt = chunkRs.getRowSequenceIterator(); final ChunkSink.FillFromContext ffc = - outputSource.makeFillFromContext(CHUNK_SIZE); - final ChunkSource.GetContext gc = inputSource.makeGetContext(CHUNK_SIZE)) { + outputSource.makeFillFromContext(PARALLEL_CACHE_CHUNK_SIZE); + final ChunkSource.GetContext gc = + inputSource.makeGetContext(PARALLEL_CACHE_CHUNK_SIZE)) { while (rsIt.hasMore()) { - final RowSequence chunkOk = rsIt.getNextRowSequenceWithLength(CHUNK_SIZE); + final RowSequence chunkOk = + rsIt.getNextRowSequenceWithLength(PARALLEL_CACHE_CHUNK_SIZE); final Chunk values = inputSource.getChunk(gc, chunkOk); outputSource.fillFromChunk(ffc, values, chunkOk); } @@ -451,10 +464,10 @@ private void createCachedColumnSource(int srcIdx, final Runnable completeAction) // run this in serial, not worth parallelization try (final RowSequence.Iterator rsIt = inputRowSet.getRowSequenceIterator(); final ChunkSink.FillFromContext ffc = - outputSource.makeFillFromContext(CHUNK_SIZE); - final ChunkSource.GetContext gc = inputSource.makeGetContext(CHUNK_SIZE)) { + outputSource.makeFillFromContext(PARALLEL_CACHE_CHUNK_SIZE); + final ChunkSource.GetContext gc = inputSource.makeGetContext(PARALLEL_CACHE_CHUNK_SIZE)) { while (rsIt.hasMore()) { - final RowSequence chunkOk = rsIt.getNextRowSequenceWithLength(CHUNK_SIZE); + final RowSequence chunkOk = rsIt.getNextRowSequenceWithLength(PARALLEL_CACHE_CHUNK_SIZE); final Chunk values = inputSource.getChunk(gc, chunkOk); outputSource.fillFromChunk(ffc, values, chunkOk); } @@ -464,8 +477,8 @@ private void createCachedColumnSource(int srcIdx, final Runnable completeAction) } /** - * Create cached input sources for all input needed by {@code windows[winIdx]}. Calls - * {@code completedAction} when the work is complete + * Create cached input sources for all input needed by {@code windows[winIdx]}. Calls {@code completedAction} + * when the work is complete */ private void cacheInputSources(final int winIdx, final Runnable completeAction) { if (inputCacheNeeded) { @@ -482,7 +495,9 @@ private void cacheInputSources(final int winIdx, final Runnable completeAction) } } - /** Shift the operator output columns for this window */ + /** + * Shift the operator output columns for this window + */ protected void shiftWindowOperators(UpdateByWindow win, RowSetShiftData shift) { try (final RowSet prevIdx = source.getRowSet().copyPrev()) { shift.apply((begin, end, delta) -> { @@ -496,7 +511,7 @@ protected void shiftWindowOperators(UpdateByWindow win, RowSetShiftData shift) { } /** - * Divide the buckets for {@code windows[winIdx]} into sets and process each set in parallel. Calls + * Divide the buckets for {@code windows[winIdx]} into sets and process each set in parallel. Calls * {@code completedAction} when the work is complete */ private void processWindowBuckets(int winIdx, final Runnable completedAction) { @@ -531,8 +546,8 @@ private void processWindowBuckets(int winIdx, final Runnable completedAction) { } /** - * Process all {@code windows} in a serial manner (to minimize cache memory usage). Will create cached - * input sources, process the buckets, then release the cached columns before starting the next window. Calls + * Process all {@code windows} in a serial manner (to minimize cache memory usage). Will create cached input + * sources, process the buckets, then release the cached columns before starting the next window. Calls * {@code completedAction} when the work is complete */ private void processWindows(final Runnable completeAction) { @@ -610,8 +625,7 @@ private void cleanUpAndNotify() { } /** - * Create the update for downstream listeners. This combines all bucket updates/modifies into a unified - * update + * Create the update for downstream listeners. This combines all bucket updates/modifies into a unified update */ private TableUpdate computeDownstreamUpdate() { final TableUpdateImpl downstream = new TableUpdateImpl(); @@ -661,8 +675,9 @@ private TableUpdate computeDownstreamUpdate() { } /** - * Process the {@link TableUpdate update} provided in the constructor. This performs much work in parallel and - * leverages {@link io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer.JobScheduler} extensively + * Process the {@link TableUpdate update} provided in the constructor. This performs much work in parallel and + * leverages {@link io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer.JobScheduler} + * extensively */ public void processUpdate() { if (redirContext.isRedirected()) { @@ -675,7 +690,7 @@ public void processUpdate() { // shifts and include them in our parallel update preparations if (update.shifted().nonempty()) { try (final RowSet prev = source.getRowSet().copyPrev(); - final RowSequence.Iterator it = prev.getRowSequenceIterator()) { + final RowSequence.Iterator it = prev.getRowSequenceIterator()) { final RowSetBuilderSequential builder = RowSetFactory.builderSequential(); final int size = update.shifted().size(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java index 3b5c802cad2..d6c5488b6e1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java @@ -20,7 +20,8 @@ import static io.deephaven.util.QueryConstants.NULL_LONG; /** - * An implementation of {@link UpdateBy} dedicated to zero key computation. + * An helper class of {@link UpdateBy} dedicated to zero key computation. This will manage the computation of a single + * bucket of rows */ class UpdateByBucketHelper { protected final ColumnSource[] inputSources; @@ -36,24 +37,27 @@ class UpdateByBucketHelper { /** An array of {@link UpdateByWindow.UpdateByWindowContext}s for each window */ final UpdateByWindow.UpdateByWindowContext[] windowContexts; - /** store timestamp data in an Ssa (if needed) */ + /** store timestamp data in an SSA (if needed) */ final String timestampColumnName; final LongSegmentedSortedArray timestampSsa; final ColumnSource timestampColumnSource; final ModifiedColumnSet timestampColumnSet; - /** Indicates this bucket needs to be processed (at least window and operator are dirty) */ + /** Indicates this bucket needs to be processed (at least one window and operator are dirty) */ boolean isDirty; /** * Perform an updateBy without any key columns. * + * @param description descibes this bucket helper * @param source the source table * @param operators, the operations to perform + * @param inputSources the source input sources + * @param operatorInputSourceSlots the mapping from operator index to needed input source indices * @param resultSources the result sources + * @param timestampColumnName the timestamp column used for time-based operations * @param redirContext the row redirection shared context * @param control the control object. - * @return the result table */ protected UpdateByBucketHelper(@NotNull final String description, @@ -268,14 +272,31 @@ public void prepareForUpdate(final TableUpdate upstream, final boolean initialSt } } + /** + * Returns {@code true} if this bucket needs to be processed (at least one operator and window has changes) + */ public boolean isDirty() { return isDirty; } + /** + * Store an array of input sources for the following call to {@code processWindow()}. The function allows for the + * use of cached input sources instead of the original input sources. + * + * @param winIdx the index of the window to modify + * @param inputSources the input sources for the + */ public void assignInputSources(final int winIdx, final ColumnSource[] inputSources) { windows[winIdx].assignInputSources(windowContexts[winIdx], inputSources); } + /** + * Perform all the operator calculations for this bucket using the input sources assigned by the + * {@code assignInputSources()} call. + * + * @param winIdx the index of the window to modify + * @param initialStep the input sources for the + */ public void processWindow(final int winIdx, final boolean initialStep) { if (!windows[winIdx].isWindowDirty(windowContexts[winIdx])) { return; // no work to do for this bucket window @@ -283,6 +304,9 @@ public void processWindow(final int winIdx, final boolean initialStep) { windows[winIdx].processRows(windowContexts[winIdx], initialStep); } + /** + * Close the window contexts and release resources for this bucket + */ public void finalizeUpdate() { for (int winIdx = 0; winIdx < windows.length; winIdx++) { windowContexts[winIdx].close(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java index ed01590b8de..86b253a4d57 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java @@ -42,12 +42,22 @@ public abstract void accumulate(RowSequence inputKeys, int len); } + /** + * An operator that computes a cumulative operation from a column + * + * @param pair the {@link MatchPair} that defines the input/output for this operation + * @param affectingColumns the names of the columns that affect this operation + * @param redirContext the row redirection context to use for the operation + */ public UpdateByCumulativeOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { super(pair, affectingColumns, null, null, 0L, 0L, redirContext); } + /** + * Initialize the bucket context for this cumulative operator + */ abstract public void initializeUpdate(@NotNull final UpdateContext context, final long firstUnmodifiedKey, long firstUnmodifiedTimestamp); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java index e8316d81e43..40f656fc932 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java @@ -2,6 +2,7 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; @@ -21,13 +22,13 @@ * interface, the pattern of calls will be as follows. * *
                - *
              1. Reprocess - *
                  - *
                • {@link #initializeUpdate(UpdateContext, RowSet, UpdateBy.UpdateType)}
                • - *
                • {@link #UpdateByWindow.processRows}
                • + *
                • {@link UpdateByCumulativeOperator#initializeUpdate(UpdateContext, long, long)} for cumulative operators or + * {@link UpdateByWindowedOperator#initializeUpdate(UpdateContext)} for windowed operators
                • + *
                • {@link io.deephaven.engine.table.impl.UpdateByCumulativeOperator.Context#accumulate(RowSequence, Chunk[], LongChunk, int)} + * for cumulative operators or + * {@link UpdateByWindowedOperator.Context#accumulate(RowSequence, Chunk[], IntChunk, IntChunk, int)} for windowed + * operators
                • *
                • {@link #finishUpdate(UpdateContext)}
                • - *
                - *
              2. *
              */ public abstract class UpdateByOperator { @@ -92,7 +93,7 @@ public interface UpdateContext extends SafeCloseable { } - public UpdateByOperator(@NotNull final MatchPair pair, + protected UpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @Nullable final OperationControl control, @Nullable final String timestampColumnName, @@ -208,7 +209,6 @@ public String[] getOutputColumnNames() { /** * Create the modified column set for the input columns of this operator. - * */ public void createInputModifiedColumnSet(@NotNull final QueryTable source) { inputModifiedColumnSet = source.newModifiedColumnSet(getAffectingColumnNames()); @@ -216,7 +216,6 @@ public void createInputModifiedColumnSet(@NotNull final QueryTable source) { /** * Create the modified column set for the output columns from this operator. - * */ public void createOutputModifiedColumnSet(@NotNull final QueryTable result) { outputModifiedColumnSet = result.newModifiedColumnSet(getOutputColumnNames()); @@ -224,7 +223,6 @@ public void createOutputModifiedColumnSet(@NotNull final QueryTable result) { /** * Return the modified column set for the input columns of this operator. - * */ public ModifiedColumnSet getInputModifiedColumnSet() { return inputModifiedColumnSet; @@ -232,7 +230,6 @@ public ModifiedColumnSet getInputModifiedColumnSet() { /** * Return the modified column set for the output columns from this operator. - * */ public ModifiedColumnSet getOutputModifiedColumnSet() { return outputModifiedColumnSet; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java index 1e611633a47..60bb11e9204 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java @@ -31,6 +31,8 @@ public abstract void accumulate(RowSequence inputKeys, * @param affectingColumns the names of the columns that affect this operation * @param control the control parameters for operation * @param timestampColumnName the optional time stamp column for windowing (uses ticks if not provided) + * @param reverseTimeScaleUnits the time (us) or ticks to extend the window backwards + * @param forwardTimeScaleUnits the time (us) or ticks to extend the window forwards * @param redirContext the row redirection context to use for the operation */ public UpdateByWindowedOperator(@NotNull final MatchPair pair, @@ -44,6 +46,9 @@ public UpdateByWindowedOperator(@NotNull final MatchPair pair, redirContext); } + /** + * Initialize the bucket context for this windowed operator + */ public abstract void initializeUpdate(@NotNull final UpdateContext context); @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer.java index ceb24642fe6..a0a02de2d8d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer.java @@ -536,8 +536,8 @@ void submit( int threadCount(); /** - * Helper interface for {@code iterateSerial()} and {@code iterateParallel()}. This provides a callable - * interface with {@code index} indicating which iteration to perform. When this returns, the scheduler will + * Helper interface for {@code iterateSerial()} and {@code iterateParallel()}. This provides a callable + * interface with {@code index} indicating which iteration to perform. When this returns, the scheduler will * automatically schedule the next iteration. */ @FunctionalInterface @@ -546,9 +546,9 @@ interface IterateAction { } /** - * Helper interface for {@code iterateSerial()} and {@code iterateParallel()}. This provides a callable + * Helper interface for {@code iterateSerial()} and {@code iterateParallel()}. This provides a callable * interface with {@code index} indicating which iteration to perform and {@link Runnable resume} providing a - * mechanism to inform the scheduler that the current task is complete. When {@code resume} is called, the + * mechanism to inform the scheduler that the current task is complete. When {@code resume} is called, the * scheduler will automatically schedule the next iteration. * * NOTE: failing to call {@code resume} will result in the scheduler not scheduling all remaining iterations. @@ -653,10 +653,10 @@ default void iterateParallel(ExecutionContext executionContext, LogOutputAppenda } /** - * Provides a mechanism to iterate over a range of values serially using the {@link JobScheduler}. The - * advantage to using this over a simple iteration is the resumption callable on {@code action} that will - * trigger the next iterable. This allows the next iteration to de delayed until dependendat asynchronous - * serial or parallel scheduler jobs have completed. + * Provides a mechanism to iterate over a range of values serially using the {@link JobScheduler}. The advantage + * to using this over a simple iteration is the resumption callable on {@code action} that will trigger the next + * iterable. This allows the next iteration to de delayed until dependendat asynchronous serial or parallel + * scheduler jobs have completed. * * @param executionContext the execution context for this task * @param description the description for diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java index 482a4cc16f7..f611dfe3946 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java @@ -31,46 +31,36 @@ public abstract class UpdateByWindow { public abstract class UpdateByWindowContext implements SafeCloseable { /** store a reference to the source rowset */ protected final TrackingRowSet sourceRowSet; - /** the column source providing the timestamp data for this window */ @Nullable protected final ColumnSource timestampColumnSource; - /** the timestamp SSA providing fast lookup for time windows */ @Nullable protected final LongSegmentedSortedArray timestampSsa; - /** An array of context objects for each underlying operator */ protected final UpdateByOperator.UpdateContext[] opContext; - + /** Whether this is the creation phase of this operator */ protected final boolean initialStep; /** An array of ColumnSources for each underlying operator */ protected ColumnSource[] inputSources; - /** An array of {@link ChunkSource.GetContext}s for each input column */ protected ChunkSource.GetContext[] inputSourceGetContexts; - /** A set of chunks used to store working values */ protected Chunk[] inputSourceChunks; - /** An indicator of if each slot has been populated with data or not for this phase. */ protected boolean[] inputSourceChunkPopulated; - - /** the rows affected by this update */ + /** The rows affected by this update */ protected RowSet affectedRows; - /** the rows that contain values used to compute affected row values */ + /** The rows that will be needed to re-compute `affectedRows` */ protected RowSet influencerRows; - + /** Size to use for chunked operations */ protected int workingChunkSize; - /** Indicates this bucket window needs to be processed */ protected boolean isDirty; - /** Indicates which operators need to be processed */ protected int[] dirtyOperatorIndices; - - /** Not actually dity, but indicates which sources are need to process this window context */ + /** Indicates which sources are needed to process this window context */ protected int[] dirtySourceIndices; public UpdateByWindowContext(final TrackingRowSet sourceRowSet, @@ -125,6 +115,14 @@ protected UpdateByWindow(UpdateByOperator[] operators, int[][] operatorInputSour this.timestampColumnName = timestampColumnName; } + /** + * Create a new window and populate it with the specified operators and input source mapping. + * + * @param operators the array of operators that belong to this window + * @param operatorSourceSlots the mapping of operator indices to required input sources indices + * + * @return a new {@link UpdateByWindow window} from these operators + */ public static UpdateByWindow createFromOperatorArray(final UpdateByOperator[] operators, final int[][] operatorSourceSlots) { // review operators to extract timestamp column (if one exists) @@ -156,15 +154,24 @@ public static UpdateByWindow createFromOperatorArray(final UpdateByOperator[] op } } + /** + * Returns the timestamp column name for this window (or null if no timestamps in use) + */ @Nullable public String getTimestampColumnName() { return timestampColumnName; } + /** + * Returns the operators for this window (a subset of the total operators for this UpdateBy call) + */ public UpdateByOperator[] getOperators() { return operators; } + /** + * Returns the mapping from operator indices to input source indices + */ public int[][] getOperatorInputSourceSlots() { return operatorInputSourceSlots; } @@ -180,6 +187,11 @@ public int[] getUniqueSourceIndices() { return uniqueInputSourceIndices; } + /** + * Returns `true` if the input source is used by this window's operators + * + * @param srcIdx the index of the input source + */ public boolean isSourceInUse(int srcIdx) { // this looks worse than it actually is, windows are defined by their input sources so there will be only // one or two entries in `getUniqueSourceIndices()`. Iterating will be faster than building a lookup table @@ -189,23 +201,46 @@ public boolean isSourceInUse(int srcIdx) { return true; } } - return false; } - public void prepareForParallelPopulation(final RowSet added) { + /** + * Pre-create all the modified/new rows in the output source so they can be updated in parallel tasks + * + * @param changes the rowset indicating which rows will be modifed or added this cycle + */ + public void prepareForParallelPopulation(final RowSet changes) { for (UpdateByOperator operator : operators) { - operator.prepareForParallelPopulation(added); + operator.prepareForParallelPopulation(changes); } } // region context-based functions + /** + * Examine the upstream {@link TableUpdate update} and determine which operators and rows are affected and will need + * to be recomputed. This also sets the {@code isDirty} flags on the window context and operator contexts + * + * @param context the window context that will store the results. + * @param upstream the update that indicates incoming changes to the data. + */ public abstract void computeAffectedRowsAndOperators(final UpdateByWindowContext context, @NotNull final TableUpdate upstream); + /** + * Generate the contexts used by the operators for this bucket. + * + * @param context the window context that will store the results. + */ protected abstract void makeOperatorContexts(final UpdateByWindowContext context); + /** + * Accepts and stores the input source array that will be used for future computation. This call allows use of + * cached input sources instead of potentially slow access to the original input sources + * + * @param context the window context that will store these sources. + * @param inputSources the (potentially cached) input sources to use for processing. + */ public void assignInputSources(final UpdateByWindowContext context, final ColumnSource[] inputSources) { context.inputSources = inputSources; context.inputSourceChunkPopulated = new boolean[inputSources.length]; @@ -219,6 +254,13 @@ public void assignInputSources(final UpdateByWindowContext context, final Column } } + /** + * Prepare a chunk of data from this input source for later computations + * + * @param context the window context that will store the results. + * @param srcIdx the index of the input source. + * @param rs the rows to retrieve. + */ protected void prepareValuesChunkForSource(final UpdateByWindowContext context, final int srcIdx, final RowSequence rs) { if (rs.isEmpty()) { @@ -231,30 +273,64 @@ protected void prepareValuesChunkForSource(final UpdateByWindowContext context, } } + /** + * Perform the computations and store the results in the operator output sources + * + * @param context the window context that will manage the results. + * @param initialStep whether this is the creation step of the table. + */ public abstract void processRows(final UpdateByWindowContext context, final boolean initialStep); + /** + * Returns `true` if the window for this bucket needs to be processed this cycle. + * + * @param context the window context that will manage the results. + */ public boolean isWindowDirty(final UpdateByWindowContext context) { return context.isDirty; } + /** + * Returns `true` if the window for this bucket needs to be processed this cycle. + * + * @param context the window context that will manage the results. + */ public int[] getDirtyOperators(final UpdateByWindowContext context) { return context.dirtyOperatorIndices; } + /** + * Returns the list of input sources that will be needed to process the `dirty` operators for this bucket + * + * @param context the window context that will manage the results. + */ public int[] getDirtySources(final UpdateByWindowContext context) { return context.dirtySourceIndices; } + /** + * Returns the rows that will be recomputed for this bucket this cycle + * + * @param context the window context that will manage the results. + */ public RowSet getAffectedRows(final UpdateByWindowContext context) { return context.affectedRows; } + /** + * Returns the rows that are needed to recompute the `affected` rows (that `influence` the results) + * + * @param context the window context that will manage the results. + */ public RowSet getInfluencerRows(final UpdateByWindowContext context) { return context.influencerRows; } // endregion + /** + * Returns a hash code to help distinguish between windows on the same UpdateBy call + */ protected static int hashCode(boolean windowed, @NotNull String[] inputColumnNames, @Nullable String timestampColumnName, long prevUnits, long fwdUnits) { @@ -278,6 +354,9 @@ protected static int hashCode(boolean windowed, @NotNull String[] inputColumnNam return hash; } + /** + * Returns a hash code given a particular operator + */ public static int hashCodeFromOperator(final UpdateByOperator op) { return hashCode(op instanceof UpdateByWindowedOperator, op.getInputColumnNames(), @@ -286,6 +365,9 @@ public static int hashCodeFromOperator(final UpdateByOperator op) { op.getPrevWindowUnits()); } + /** + * Returns `true` if two operators are compatible and can be executed as part of the same window + */ public static boolean isEquivalentWindow(final UpdateByOperator opA, final UpdateByOperator opB) { // verify input columns match String[] opAInput = opA.getInputColumnNames(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java index 853d85560ad..bf6e8b6b9a2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java @@ -22,7 +22,10 @@ import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; import static io.deephaven.util.QueryConstants.NULL_LONG; -// this class is currently too big, should specialize into CumWindow, TickWindow, TimeWindow to simplify implementation +/** + * This is the specialization of {@link UpdateByWindow} that handles `cumulative` operators. These operators do not + * maintain a window of data and can be computed from the previous value and the current value. + */ public class UpdateByWindowCumulative extends UpdateByWindow { public UpdateByWindowCumulative(UpdateByOperator[] operators, int[][] operatorSourceSlots, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java index bf1b33da1b0..7ef41469be3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java @@ -24,7 +24,11 @@ import java.util.Arrays; import java.util.stream.IntStream; -// this class is currently too big, should specialize into CumWindow, TickWindow, TimeWindow to simplify implementation +/** + * This is the specialization of {@link UpdateByWindow} that handles tick based `windowed` operators. These operators + * maintain a window of data based on row distance rather than timestamps. Window-based operators must maintain a buffer + * of `influencer` values to add to the rolling window as the current row changes. + */ public class UpdateByWindowTicks extends UpdateByWindow { protected final long prevUnits; protected final long fwdUnits; @@ -190,10 +194,12 @@ private void loadNextInfluencerChunks(UpdateByWindowTicksContext ctx) { ctx.nextInfluencerKey = ctx.influencerKeyChunk.get(ctx.nextInfluencerIndex); } - // windowed by time/ticks is more complex to compute: find all the changed rows and the rows that would - // be affected by the changes (includes newly added rows) and need to be recomputed. Then include all - // the rows that are affected by deletions (if any). After the affected rows have been identified, - // determine which rows will be needed to compute new values for the affected rows (influencer rows) + /** + * Finding the `affected` and `influencer` rowsets for a windowed operation is complex. We must identify modified + * rows (including added rows) and deleted rows and determine which rows are `affected` by the change given the + * window parameters. After these rows have been identified, must determine which rows will be needed to recompute + * these values (i.e. that fall within the window and will `influence` this computation). + */ @Override public void computeAffectedRowsAndOperators(UpdateByWindowContext context, @NotNull TableUpdate upstream) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java index bb0fb69054b..aca464a6887 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java @@ -25,7 +25,11 @@ import static io.deephaven.util.QueryConstants.NULL_LONG; -// this class is currently too big, should specialize into CumWindow, TickWindow, TimeWindow to simplify implementation +/** + * This is the specialization of {@link UpdateByWindow} that handles time-based `windowed` operators. These operators + * maintain a window of data based on a timestamp column rather than row distances. Window-based operators must maintain + * a buffer of `influencer` values to add to the rolling window as the current row changes. + */ public class UpdateByWindowTime extends UpdateByWindow { protected final long prevUnits; protected final long fwdUnits; @@ -95,6 +99,12 @@ public UpdateByWindowContext makeWindowContext(final TrackingRowSet sourceRowSet isInitializeStep); } + /** + * Finding the `affected` and `influencer` rowsets for a windowed operation is complex. We must identify modified + * rows (including added rows) and deleted rows and determine which rows are `affected` by the change given the + * window parameters. After these rows have been identified, must determine which rows will be needed to recompute + * these values (i.e. that fall within the window and will `influence` this computation). + */ private static WritableRowSet computeAffectedRowsTime(final RowSet sourceSet, final RowSet subset, long revNanos, long fwdNanos, final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa, boolean usePrev) { From a35b4d4ed2cf716c72a98cd7781e57789099bb32 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 11 Nov 2022 15:22:57 -0800 Subject: [PATCH 041/123] More bug fixes, close of week checkin --- .../deephaven/engine/table/impl/UpdateBy.java | 44 +++++++++----- .../analyzers/SelectAndViewAnalyzer.java | 60 +++++++++++++------ .../updateby/fill/BooleanFillByOperator.java | 5 -- .../updateby/fill/ByteFillByOperator.java | 5 -- .../updateby/fill/CharFillByOperator.java | 5 -- .../updateby/fill/DoubleFillByOperator.java | 5 -- .../updateby/fill/FloatFillByOperator.java | 5 -- .../impl/updateby/fill/IntFillByOperator.java | 5 -- .../updateby/fill/LongFillByOperator.java | 5 -- .../updateby/fill/ObjectFillByOperator.java | 5 -- .../updateby/fill/ShortFillByOperator.java | 5 -- .../internal/BaseByteUpdateByOperator.java | 21 +++---- .../internal/BaseCharUpdateByOperator.java | 14 ++--- .../internal/BaseDoubleUpdateByOperator.java | 14 ++--- .../internal/BaseFloatUpdateByOperator.java | 14 ++--- .../internal/BaseIntUpdateByOperator.java | 14 ++--- .../internal/BaseLongUpdateByOperator.java | 14 ++--- .../internal/BaseObjectUpdateByOperator.java | 14 ++--- .../internal/BaseShortUpdateByOperator.java | 14 ++--- .../BaseWindowedByteUpdateByOperator.java | 12 +--- .../BaseWindowedCharUpdateByOperator.java | 12 +--- .../BaseWindowedDoubleUpdateByOperator.java | 10 +--- .../BaseWindowedFloatUpdateByOperator.java | 10 +--- .../BaseWindowedIntUpdateByOperator.java | 12 +--- .../BaseWindowedLongUpdateByOperator.java | 12 +--- .../BaseWindowedObjectUpdateByOperator.java | 12 +--- .../BaseWindowedShortUpdateByOperator.java | 12 +--- .../minmax/ByteCumMinMaxOperator.java | 5 -- .../minmax/DoubleCumMinMaxOperator.java | 5 -- .../minmax/FloatCumMinMaxOperator.java | 5 -- .../updateby/minmax/IntCumMinMaxOperator.java | 5 -- .../minmax/LongCumMinMaxOperator.java | 5 -- .../minmax/ShortCumMinMaxOperator.java | 5 -- .../updateby/prod/ByteCumProdOperator.java | 5 -- .../updateby/prod/DoubleCumProdOperator.java | 5 -- .../updateby/prod/FloatCumProdOperator.java | 5 -- .../updateby/prod/IntCumProdOperator.java | 5 -- .../updateby/prod/LongCumProdOperator.java | 5 -- .../updateby/prod/ShortCumProdOperator.java | 5 -- .../sum/BigDecimalCumSumOperator.java | 5 -- .../sum/BigIntegerCumSumOperator.java | 5 -- .../impl/updateby/sum/ByteCumSumOperator.java | 5 -- .../updateby/sum/DoubleCumSumOperator.java | 5 -- .../updateby/sum/FloatCumSumOperator.java | 5 -- .../impl/updateby/sum/IntCumSumOperator.java | 5 -- .../impl/updateby/sum/LongCumSumOperator.java | 5 -- .../updateby/sum/ShortCumSumOperator.java | 5 -- 47 files changed, 143 insertions(+), 317 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java index 0b344bb25d2..f67dbc25428 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java @@ -334,9 +334,8 @@ public LogOutput append(LogOutput logOutput) { } private void onError(Exception error) { - // Not sure what to do here... + // signal to the future that an exception has occured waitForResult.completeExceptionally(error); - System.out.println(error.toString()); } /** @@ -518,8 +517,8 @@ private void processWindowBuckets(int winIdx, final Runnable completedAction) { if (jobScheduler.threadCount() > 1) { // process the buckets in parallel final int bucketsPerTask = Math.max(1, dirtyBuckets.length / jobScheduler.threadCount()); - TIntArrayList offsetList = new TIntArrayList(); - TIntArrayList countList = new TIntArrayList(); + final TIntArrayList offsetList = new TIntArrayList(); + final TIntArrayList countList = new TIntArrayList(); for (int ii = 0; ii < dirtyBuckets.length; ii += bucketsPerTask) { offsetList.add(ii); @@ -576,8 +575,19 @@ private void processWindows(final Runnable completeAction) { } } - // shift the non-redirected output sources now, after parallelPopulation - if (!redirContext.isRedirected() && update.shifted().nonempty()) { + if (redirContext.isRedirected()) { + for (UpdateByOperator op : windows[winIdx].getOperators()) { + // If we're redirected we have to make sure we tell the output source its actual + // size, or we're going to have a bad time. This is not necessary for + // non-redirections since the SparseArraySources do not need to do anything with + // capacity. + op.getOutputColumns().forEach((name, outputSource) -> { + // The redirection index does not use the 0th index for some reason. + ((WritableColumnSource)outputSource).ensureCapacity(redirContext.requiredCapacity()); + }) ; + } + } else if (update.shifted().nonempty()) { + // shift the non-redirected output sources now, after parallelPopulation shiftWindowOperators(win, update.shifted()); } @@ -597,9 +607,10 @@ private void processWindows(final Runnable completeAction) { } /** - * Clean up the resources created during this update and notify downstream if applicable + * Clean up the resources created during this update and notify downstream if applicable. Calls + * {@code completedAction} when the work is complete */ - private void cleanUpAndNotify() { + private void cleanUpAndNotify(final Runnable completeAction) { shiftedRows.close(); // create the downstream before calling finalize() on the buckets (which releases resources) @@ -620,8 +631,7 @@ private void cleanUpAndNotify() { result.notifyListeners(downstream); } - // signal to the main task that we have completed our work - waitForResult.complete(null); + completeAction.run(); } /** @@ -716,7 +726,12 @@ public void processUpdate() { // call will chain to another until the sequence is complete computeCachedColumnRowsets( () -> processWindows( - () -> cleanUpAndNotify())); + () -> cleanUpAndNotify( + () -> { + // signal to the main task that we have completed our work + waitForResult.complete(null); + } + ))); try { // need to wait until this future is complete @@ -748,14 +763,13 @@ protected void process() { final TableUpdate upstream = sourceRecorder.getUpdate(); // we need to keep a reference to TableUpdate during our computation - if (upstream != null) { - final StateManager sm = new StateManager(upstream.acquire(), false); - sm.processUpdate(); - } + final StateManager sm = new StateManager(upstream.acquire(), false); + sm.processUpdate(); } @Override protected boolean canExecute(final long step) { + synchronized (recorders) { return recorders.stream().allMatch(lr -> lr.satisfied(step)); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer.java index a0a02de2d8d..3b0504d8514 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer.java @@ -573,6 +573,12 @@ interface IterateResumeAction { @FinalDefault default void iterateParallel(ExecutionContext executionContext, LogOutputAppendable description, int start, int count, IterateAction action, Runnable completeAction, final Consumer onError) { + + if (count == 0) { + // no work to do + completeAction.run(); + } + final AtomicInteger nextIndex = new AtomicInteger(start); final AtomicInteger remaining = new AtomicInteger(count); @@ -619,6 +625,12 @@ default void iterateParallel(ExecutionContext executionContext, LogOutputAppenda @FinalDefault default void iterateParallel(ExecutionContext executionContext, LogOutputAppendable description, int start, int count, IterateResumeAction action, Runnable completeAction, Consumer onError) { + + if (count == 0) { + // no work to do + completeAction.run(); + } + final AtomicInteger nextIndex = new AtomicInteger(start); final AtomicInteger remaining = new AtomicInteger(count); @@ -669,33 +681,47 @@ default void iterateParallel(ExecutionContext executionContext, LogOutputAppenda @FinalDefault default void iterateSerial(ExecutionContext executionContext, LogOutputAppendable description, int start, int count, IterateResumeAction action, Runnable completeAction, Consumer onError) { + + if (count == 0) { + // no work to do + completeAction.run(); + } + final AtomicInteger nextIndex = new AtomicInteger(start); final AtomicInteger remaining = new AtomicInteger(count); - final Runnable resumeAction = () -> { - // check for completion - if (remaining.decrementAndGet() == 0) { - completeAction.run(); - } - }; - - final Runnable task = () -> { - // this will run until all tasks have started - while (true) { - int idx = nextIndex.getAndIncrement(); - if (idx < start + count) { - // do the work - action.run(idx, resumeAction); + // no lambda, need the `this` reference to re-execute + final Runnable resumeAction = new Runnable() { + @Override + public void run() { + // check for completion + if (remaining.decrementAndGet() == 0) { + completeAction.run(); } else { - // no more work to do - return; + // schedule the next task + submit(executionContext, + () -> { + int idx = nextIndex.getAndIncrement(); + if (idx < start + count) { + // do the work + action.run(idx, this); + } + }, + description, + onError); } + } }; // create a single task submit(executionContext, - task, + () -> { + int idx = nextIndex.getAndIncrement(); + if (idx < start + count) { + action.run(idx, resumeAction); + } + }, description, onError); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java index 4c930228f87..ebf23616859 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java @@ -46,11 +46,6 @@ public void push(long key, int pos) { curVal = currentVal; } } - - @Override - public void reset() { - curVal = NULL_BOOLEAN_AS_BYTE; - } } public BooleanFillByOperator(@NotNull final MatchPair fillPair, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java index ad8a5235b45..bc3d2ddeb60 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java @@ -39,11 +39,6 @@ public void push(long key, int pos) { curVal = currentVal; } } - - @Override - public void reset() { - curVal = NULL_BYTE; - } } public ByteFillByOperator(@NotNull final MatchPair fillPair, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java index 0dadeed655c..e6f8dd97797 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java @@ -34,11 +34,6 @@ public void push(long key, int pos) { curVal = currentVal; } } - - @Override - public void reset() { - curVal = NULL_CHAR; - } } public CharFillByOperator(@NotNull final MatchPair fillPair, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java index c4fb2f64b2c..9487194c57d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java @@ -39,11 +39,6 @@ public void push(long key, int pos) { curVal = currentVal; } } - - @Override - public void reset() { - curVal = NULL_DOUBLE; - } } public DoubleFillByOperator(@NotNull final MatchPair fillPair, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java index b59b07bb777..3b9499b4a1c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java @@ -39,11 +39,6 @@ public void push(long key, int pos) { curVal = currentVal; } } - - @Override - public void reset() { - curVal = NULL_FLOAT; - } } public FloatFillByOperator(@NotNull final MatchPair fillPair, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java index 5c7603c2278..290a1914723 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java @@ -39,11 +39,6 @@ public void push(long key, int pos) { curVal = currentVal; } } - - @Override - public void reset() { - curVal = NULL_INT; - } } public IntFillByOperator(@NotNull final MatchPair fillPair, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java index c155e43c914..aa3f9007f10 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java @@ -47,11 +47,6 @@ public void push(long key, int pos) { curVal = currentVal; } } - - @Override - public void reset() { - curVal = NULL_LONG; - } } public LongFillByOperator(@NotNull final MatchPair fillPair, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java index 4a1033436d8..90fc1ba4774 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java @@ -38,11 +38,6 @@ public void push(long key, int pos) { curVal = currentVal; } } - - @Override - public void reset() { - curVal = null; - } } public ObjectFillByOperator(@NotNull final MatchPair fillPair, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java index 188a3d2ba6a..867052661b0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java @@ -39,11 +39,6 @@ public void push(long key, int pos) { curVal = currentVal; } } - - @Override - public void reset() { - curVal = NULL_SHORT; - } } public ShortFillByOperator(@NotNull final MatchPair fillPair, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java index 8af4e9f461f..8d088273209 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java @@ -24,6 +24,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import java.util.Collections; @@ -93,6 +94,11 @@ public void writeToOutputChunk(int outIdx) { public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); } + + @FinalDefault + public void reset() { + curVal = NULL_BYTE; + } } /** @@ -151,14 +157,6 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie } else { ctx.reset(); } - - // If we're redirected we have to make sure we tell the output source it's actual size, or we're going - // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not - // need to do anything with capacity. - if(redirContext.isRedirected()) { - // The redirection index does not use the 0th index for some reason. - outputSource.ensureCapacity(redirContext.requiredCapacity()); - } } @Override @@ -184,12 +182,7 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d public void prepareForParallelPopulation(final RowSet added) { // we don't need to do anything for redirected, that happened earlier if (!redirContext.isRedirected()) { - // this might be a Boolean reinterpreted column source - if (outputSource instanceof BooleanSparseArraySource.ReinterpretedAsByte) { - ((BooleanSparseArraySource.ReinterpretedAsByte) outputSource).prepareForParallelPopulation(added); - } else { - ((SparseArrayColumnSource) outputSource).prepareForParallelPopulation(added); - } + ((SparseArrayColumnSource) outputSource).prepareForParallelPopulation(added); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java index a3c4166c379..ad7e57221ff 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java @@ -14,6 +14,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import java.util.Collections; @@ -82,6 +83,11 @@ public void writeToOutputChunk(int outIdx) { public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); } + + @FinalDefault + public void reset() { + curVal = NULL_CHAR; + } } /** @@ -127,14 +133,6 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie } else { ctx.reset(); } - - // If we're redirected we have to make sure we tell the output source it's actual size, or we're going - // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not - // need to do anything with capacity. - if(redirContext.isRedirected()) { - // The redirection index does not use the 0th index for some reason. - outputSource.ensureCapacity(redirContext.requiredCapacity()); - } } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java index 13b3c0b6572..f0057007d0c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java @@ -17,6 +17,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -87,6 +88,11 @@ public void writeToOutputChunk(int outIdx) { public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); } + + @FinalDefault + public void reset() { + curVal = NULL_DOUBLE; + } } /** @@ -131,14 +137,6 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie } else { ctx.reset(); } - - // If we're redirected we have to make sure we tell the output source it's actual size, or we're going - // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not - // need to do anything with capacity. - if(redirContext.isRedirected()) { - // The redirection index does not use the 0th index for some reason. - outputSource.ensureCapacity(redirContext.requiredCapacity()); - } } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java index f435e9beb51..3d836837494 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java @@ -12,6 +12,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -82,6 +83,11 @@ public void writeToOutputChunk(int outIdx) { public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); } + + @FinalDefault + public void reset() { + curVal = NULL_FLOAT; + } } /** @@ -126,14 +132,6 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie } else { ctx.reset(); } - - // If we're redirected we have to make sure we tell the output source it's actual size, or we're going - // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not - // need to do anything with capacity. - if(redirContext.isRedirected()) { - // The redirection index does not use the 0th index for some reason. - outputSource.ensureCapacity(redirContext.requiredCapacity()); - } } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java index 8cd3f8fbf94..47da06915a3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java @@ -19,6 +19,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import java.util.Collections; @@ -87,6 +88,11 @@ public void writeToOutputChunk(int outIdx) { public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); } + + @FinalDefault + public void reset() { + curVal = NULL_INT; + } } /** @@ -132,14 +138,6 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie } else { ctx.reset(); } - - // If we're redirected we have to make sure we tell the output source it's actual size, or we're going - // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not - // need to do anything with capacity. - if(redirContext.isRedirected()) { - // The redirection index does not use the 0th index for some reason. - outputSource.ensureCapacity(redirContext.requiredCapacity()); - } } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java index 5ddf164b644..a0b062a8aab 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java @@ -19,6 +19,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import java.util.Collections; @@ -87,6 +88,11 @@ public void writeToOutputChunk(int outIdx) { public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); } + + @FinalDefault + public void reset() { + curVal = NULL_LONG; + } } /** @@ -132,14 +138,6 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie } else { ctx.reset(); } - - // If we're redirected we have to make sure we tell the output source it's actual size, or we're going - // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not - // need to do anything with capacity. - if(redirContext.isRedirected()) { - // The redirection index does not use the 0th index for some reason. - outputSource.ensureCapacity(redirContext.requiredCapacity()); - } } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java index 3d9cad434d9..73b6bac35af 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java @@ -19,6 +19,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import java.util.Collections; @@ -88,6 +89,11 @@ public void writeToOutputChunk(int outIdx) { public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); } + + @FinalDefault + public void reset() { + curVal = null; + } } /** @@ -135,14 +141,6 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie } else { ctx.reset(); } - - // If we're redirected we have to make sure we tell the output source it's actual size, or we're going - // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not - // need to do anything with capacity. - if(redirContext.isRedirected()) { - // The redirection index does not use the 0th index for some reason. - outputSource.ensureCapacity(redirContext.requiredCapacity()); - } } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java index 643dca3433a..a10e5b0da0e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java @@ -19,6 +19,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import java.util.Collections; @@ -87,6 +88,11 @@ public void writeToOutputChunk(int outIdx) { public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); } + + @FinalDefault + public void reset() { + curVal = NULL_SHORT; + } } /** @@ -132,14 +138,6 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie } else { ctx.reset(); } - - // If we're redirected we have to make sure we tell the output source it's actual size, or we're going - // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not - // need to do anything with capacity. - if(redirContext.isRedirected()) { - // The redirection index does not use the 0th index for some reason. - outputSource.ensureCapacity(redirContext.requiredCapacity()); - } } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java index 07d49158967..b132623ea50 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java @@ -23,6 +23,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -108,7 +109,7 @@ public void close() { outputFillContext.close(); } - @Override + @FinalDefault public void reset() { curVal = NULL_BYTE; nullCount = 0; @@ -126,7 +127,7 @@ public BaseWindowedByteUpdateByOperator(@NotNull final MatchPair pair, // endregion extra-constructor-args ) { super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); - if(this.redirContext.isRedirected()) { + if (this.redirContext.isRedirected()) { // region create-dense this.maybeInnerSource = makeDenseSource(); // endregion create-dense @@ -160,13 +161,6 @@ protected WritableColumnSource makeDenseSource() { @Override public void initializeUpdate(@NotNull UpdateContext context) { - // If we're redirected we have to make sure we tell the output source it's actual size, or we're going - // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not - // need to do anything with capacity. - if(redirContext.isRedirected()) { - // The redirection index does not use the 0th index for some reason. - outputSource.ensureCapacity(redirContext.requiredCapacity()); - } } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java index 318418470ac..ce3b65b5b7b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java @@ -13,6 +13,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -97,7 +98,7 @@ public void close() { outputFillContext.close(); } - @Override + @FinalDefault public void reset() { curVal = NULL_CHAR; nullCount = 0; @@ -115,7 +116,7 @@ public BaseWindowedCharUpdateByOperator(@NotNull final MatchPair pair, // endregion extra-constructor-args ) { super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); - if(this.redirContext.isRedirected()) { + if (this.redirContext.isRedirected()) { // region create-dense this.maybeInnerSource = new CharacterArraySource(); // endregion create-dense @@ -136,13 +137,6 @@ public BaseWindowedCharUpdateByOperator(@NotNull final MatchPair pair, @Override public void initializeUpdate(@NotNull UpdateContext context) { - // If we're redirected we have to make sure we tell the output source it's actual size, or we're going - // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not - // need to do anything with capacity. - if(redirContext.isRedirected()) { - // The redirection index does not use the 0th index for some reason. - outputSource.ensureCapacity(redirContext.requiredCapacity()); - } } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java index 5368b25288b..41f706146b8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java @@ -17,6 +17,7 @@ import io.deephaven.engine.table.impl.sources.DoubleSparseArraySource; import io.deephaven.engine.table.impl.sources.SparseArrayColumnSource; import io.deephaven.engine.table.impl.sources.WritableRedirectedColumnSource; +import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -102,7 +103,7 @@ public void close() { outputFillContext.close(); } - @Override + @FinalDefault public void reset() { curVal = NULL_DOUBLE; nullCount = 0; @@ -141,13 +142,6 @@ public BaseWindowedDoubleUpdateByOperator(@NotNull final MatchPair pair, @Override public void initializeUpdate(@NotNull UpdateContext context) { - // If we're redirected we have to make sure we tell the output source it's actual size, or we're going - // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not - // need to do anything with capacity. - if(redirContext.isRedirected()) { - // The redirection index does not use the 0th index for some reason. - outputSource.ensureCapacity(redirContext.requiredCapacity()); - } } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java index 5b90856d441..7da492a15db 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java @@ -12,6 +12,7 @@ import io.deephaven.engine.table.impl.sources.FloatSparseArraySource; import io.deephaven.engine.table.impl.sources.SparseArrayColumnSource; import io.deephaven.engine.table.impl.sources.WritableRedirectedColumnSource; +import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -97,7 +98,7 @@ public void close() { outputFillContext.close(); } - @Override + @FinalDefault public void reset() { curVal = NULL_FLOAT; nullCount = 0; @@ -136,13 +137,6 @@ public BaseWindowedFloatUpdateByOperator(@NotNull final MatchPair pair, @Override public void initializeUpdate(@NotNull UpdateContext context) { - // If we're redirected we have to make sure we tell the output source it's actual size, or we're going - // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not - // need to do anything with capacity. - if(redirContext.isRedirected()) { - // The redirection index does not use the 0th index for some reason. - outputSource.ensureCapacity(redirContext.requiredCapacity()); - } } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java index b86eeacc494..9fd339b2675 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java @@ -18,6 +18,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -102,7 +103,7 @@ public void close() { outputFillContext.close(); } - @Override + @FinalDefault public void reset() { curVal = NULL_INT; nullCount = 0; @@ -120,7 +121,7 @@ public BaseWindowedIntUpdateByOperator(@NotNull final MatchPair pair, // endregion extra-constructor-args ) { super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); - if(this.redirContext.isRedirected()) { + if (this.redirContext.isRedirected()) { // region create-dense this.maybeInnerSource = new IntegerArraySource(); // endregion create-dense @@ -141,13 +142,6 @@ public BaseWindowedIntUpdateByOperator(@NotNull final MatchPair pair, @Override public void initializeUpdate(@NotNull UpdateContext context) { - // If we're redirected we have to make sure we tell the output source it's actual size, or we're going - // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not - // need to do anything with capacity. - if(redirContext.isRedirected()) { - // The redirection index does not use the 0th index for some reason. - outputSource.ensureCapacity(redirContext.requiredCapacity()); - } } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java index cda24603705..ac653e743fe 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java @@ -18,6 +18,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -102,7 +103,7 @@ public void close() { outputFillContext.close(); } - @Override + @FinalDefault public void reset() { curVal = NULL_LONG; nullCount = 0; @@ -120,7 +121,7 @@ public BaseWindowedLongUpdateByOperator(@NotNull final MatchPair pair, // endregion extra-constructor-args ) { super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); - if(this.redirContext.isRedirected()) { + if (this.redirContext.isRedirected()) { // region create-dense this.maybeInnerSource = new LongArraySource(); // endregion create-dense @@ -141,13 +142,6 @@ public BaseWindowedLongUpdateByOperator(@NotNull final MatchPair pair, @Override public void initializeUpdate(@NotNull UpdateContext context) { - // If we're redirected we have to make sure we tell the output source it's actual size, or we're going - // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not - // need to do anything with capacity. - if(redirContext.isRedirected()) { - // The redirection index does not use the 0th index for some reason. - outputSource.ensureCapacity(redirContext.requiredCapacity()); - } } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java index 13c2b595b51..7feadf83774 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java @@ -18,6 +18,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -103,7 +104,7 @@ public void close() { outputFillContext.close(); } - @Override + @FinalDefault public void reset() { curVal = null; nullCount = 0; @@ -122,7 +123,7 @@ public BaseWindowedObjectUpdateByOperator(@NotNull final MatchPair pair, // endregion extra-constructor-args ) { super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); - if(this.redirContext.isRedirected()) { + if (this.redirContext.isRedirected()) { // region create-dense this.maybeInnerSource = new ObjectArraySource(colType); // endregion create-dense @@ -144,13 +145,6 @@ public BaseWindowedObjectUpdateByOperator(@NotNull final MatchPair pair, @Override public void initializeUpdate(@NotNull UpdateContext context) { - // If we're redirected we have to make sure we tell the output source it's actual size, or we're going - // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not - // need to do anything with capacity. - if(redirContext.isRedirected()) { - // The redirection index does not use the 0th index for some reason. - outputSource.ensureCapacity(redirContext.requiredCapacity()); - } } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java index 5720546867c..e651c8de26e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java @@ -18,6 +18,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -102,7 +103,7 @@ public void close() { outputFillContext.close(); } - @Override + @FinalDefault public void reset() { curVal = NULL_SHORT; nullCount = 0; @@ -120,7 +121,7 @@ public BaseWindowedShortUpdateByOperator(@NotNull final MatchPair pair, // endregion extra-constructor-args ) { super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); - if(this.redirContext.isRedirected()) { + if (this.redirContext.isRedirected()) { // region create-dense this.maybeInnerSource = new ShortArraySource(); // endregion create-dense @@ -141,13 +142,6 @@ public BaseWindowedShortUpdateByOperator(@NotNull final MatchPair pair, @Override public void initializeUpdate(@NotNull UpdateContext context) { - // If we're redirected we have to make sure we tell the output source it's actual size, or we're going - // to have a bad time. This is not necessary for non-redirections since the SparseArraySources do not - // need to do anything with capacity. - if(redirContext.isRedirected()) { - // The redirection index does not use the 0th index for some reason. - outputSource.ensureCapacity(redirContext.requiredCapacity()); - } } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java index c3f5dc3f053..b9c76e793bf 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java @@ -48,11 +48,6 @@ public void push(long key, int pos) { } } } - - @Override - public void reset() { - curVal = NULL_BYTE; - } } public ByteCumMinMaxOperator(@NotNull final MatchPair pair, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java index 46a67ee716e..0e92207c541 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java @@ -48,11 +48,6 @@ public void push(long key, int pos) { } } } - - @Override - public void reset() { - curVal = NULL_DOUBLE; - } } public DoubleCumMinMaxOperator(@NotNull final MatchPair pair, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java index 96a8028bbfe..91af03ec7c8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java @@ -43,11 +43,6 @@ public void push(long key, int pos) { } } } - - @Override - public void reset() { - curVal = NULL_FLOAT; - } } public FloatCumMinMaxOperator(@NotNull final MatchPair pair, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java index b35635cc2b9..72ab41299c0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java @@ -48,11 +48,6 @@ public void push(long key, int pos) { } } } - - @Override - public void reset() { - curVal = NULL_INT; - } } public IntCumMinMaxOperator(@NotNull final MatchPair pair, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java index 240f66cb996..48754aa1bff 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java @@ -56,11 +56,6 @@ public void push(long key, int pos) { } } } - - @Override - public void reset() { - curVal = NULL_LONG; - } } public LongCumMinMaxOperator(@NotNull final MatchPair pair, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java index 1577805f777..a3af67e5540 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java @@ -43,11 +43,6 @@ public void push(long key, int pos) { } } } - - @Override - public void reset() { - curVal = NULL_SHORT; - } } public ShortCumMinMaxOperator(@NotNull final MatchPair pair, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java index 6268151e9c8..ee708671c89 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java @@ -43,11 +43,6 @@ public void push(long key, int pos) { curVal *= currentVal; } } - - @Override - public void reset() { - curVal = NULL_BYTE; - } } public ByteCumProdOperator(@NotNull final MatchPair pair, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java index acacaeba56c..0950e93b070 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java @@ -44,11 +44,6 @@ public void push(long key, int pos) { curVal *= currentVal; } } - - @Override - public void reset() { - curVal = NULL_DOUBLE; - } } public DoubleCumProdOperator(@NotNull final MatchPair pair, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java index bdb4bc64da6..d300323cd8e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java @@ -39,11 +39,6 @@ public void push(long key, int pos) { curVal *= currentVal; } } - - @Override - public void reset() { - curVal = NULL_FLOAT; - } } public FloatCumProdOperator(@NotNull final MatchPair pair, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java index 30b08e3d53b..64a0be386f3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java @@ -43,11 +43,6 @@ public void push(long key, int pos) { curVal *= currentVal; } } - - @Override - public void reset() { - curVal = NULL_INT; - } } public IntCumProdOperator(@NotNull final MatchPair pair, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java index 7bc9d81d3c9..da13a95591f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java @@ -43,11 +43,6 @@ public void push(long key, int pos) { curVal *= currentVal; } } - - @Override - public void reset() { - curVal = NULL_LONG; - } } public LongCumProdOperator(@NotNull final MatchPair pair, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java index 06bbd87f376..18bb22419ba 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java @@ -38,11 +38,6 @@ public void push(long key, int pos) { curVal *= currentVal; } } - - @Override - public void reset() { - curVal = NULL_SHORT; - } } public ShortCumProdOperator(@NotNull final MatchPair pair, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java index ff57c6c31d8..92fa490ae74 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java @@ -45,11 +45,6 @@ public void push(long key, int pos) { } } } - - @Override - public void reset() { - curVal = null; - } } public BigDecimalCumSumOperator(@NotNull final MatchPair inputPair, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java index 7790d7dd31b..adc9953eafb 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java @@ -40,11 +40,6 @@ public void push(long key, int pos) { } } } - - @Override - public void reset() { - curVal = null; - } } public BigIntegerCumSumOperator(@NotNull final MatchPair inputPair, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java index d9a6ada25f7..f8f3d256f00 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java @@ -43,11 +43,6 @@ public void push(long key, int pos) { curVal += currentVal; } } - - @Override - public void reset() { - curVal = NULL_LONG; - } } public ByteCumSumOperator(@NotNull final MatchPair pair, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java index aef0e9d2736..d1f2d55b454 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java @@ -43,11 +43,6 @@ public void push(long key, int pos) { curVal += currentVal; } } - - @Override - public void reset() { - curVal = NULL_DOUBLE; - } } public DoubleCumSumOperator(@NotNull final MatchPair pair, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java index 93622801f84..d208b59ab5b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java @@ -38,11 +38,6 @@ public void push(long key, int pos) { curVal += currentVal; } } - - @Override - public void reset() { - curVal = NULL_FLOAT; - } } public FloatCumSumOperator(@NotNull final MatchPair pair, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java index dfdfbe808c6..ffed2bfea7e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java @@ -42,11 +42,6 @@ public void push(long key, int pos) { curVal += currentVal; } } - - @Override - public void reset() { - curVal = NULL_LONG; - } } public IntCumSumOperator(@NotNull final MatchPair pair, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java index 72c698d64b2..4dea4b48563 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java @@ -42,11 +42,6 @@ public void push(long key, int pos) { curVal += currentVal; } } - - @Override - public void reset() { - curVal = NULL_LONG; - } } public LongCumSumOperator(@NotNull final MatchPair pair, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java index c29a01e9475..2cf38220460 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java @@ -37,11 +37,6 @@ public void push(long key, int pos) { curVal += currentVal; } } - - @Override - public void reset() { - curVal = NULL_LONG; - } } public ShortCumSumOperator(@NotNull final MatchPair pair, From a7fc819e4b2a4d3110afe934caf417c55c900668 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Thu, 17 Nov 2022 17:44:07 -0800 Subject: [PATCH 042/123] Add prepareForParallelPopulation to dense array sources. --- .../BucketedPartitionedUpdateByManager.java | 10 ++ .../deephaven/engine/table/impl/UpdateBy.java | 123 ++++++++++-------- .../analyzers/SelectAndViewAnalyzer.java | 6 +- .../impl/sources/AbstractLongArraySource.java | 99 +++++++++++--- .../table/impl/sources/ArraySourceHelper.java | 16 ++- .../impl/sources/BooleanArraySource.java | 93 +++++++++++-- .../sources/BooleanSparseArraySource.java | 16 ++- .../table/impl/sources/ByteArraySource.java | 83 ++++++++++-- .../impl/sources/CharacterArraySource.java | 83 ++++++++++-- .../table/impl/sources/DoubleArraySource.java | 83 ++++++++++-- .../table/impl/sources/FloatArraySource.java | 83 ++++++++++-- .../impl/sources/IntegerArraySource.java | 83 ++++++++++-- .../table/impl/sources/ObjectArraySource.java | 81 ++++++++++-- .../table/impl/sources/ShortArraySource.java | 83 ++++++++++-- .../internal/BaseByteUpdateByOperator.java | 13 +- .../internal/BaseCharUpdateByOperator.java | 13 +- .../internal/BaseDoubleUpdateByOperator.java | 7 +- .../internal/BaseFloatUpdateByOperator.java | 7 +- .../internal/BaseIntUpdateByOperator.java | 13 +- .../internal/BaseLongUpdateByOperator.java | 13 +- .../internal/BaseObjectUpdateByOperator.java | 13 +- .../internal/BaseShortUpdateByOperator.java | 13 +- .../BaseWindowedByteUpdateByOperator.java | 7 +- .../BaseWindowedCharUpdateByOperator.java | 7 +- .../BaseWindowedDoubleUpdateByOperator.java | 8 +- .../BaseWindowedFloatUpdateByOperator.java | 8 +- .../BaseWindowedIntUpdateByOperator.java | 7 +- .../BaseWindowedLongUpdateByOperator.java | 7 +- .../BaseWindowedObjectUpdateByOperator.java | 7 +- .../BaseWindowedShortUpdateByOperator.java | 7 +- .../impl/updateby/TestUpdateByGeneral.java | 3 - .../ReplicateSourcesAndChunks.java | 20 ++- 32 files changed, 877 insertions(+), 238 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java index a98c103ecfc..b0b5f333f12 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java @@ -135,6 +135,16 @@ protected BucketedPartitionedUpdateByManager(@NotNull final String description, result.addParentReference(transformed); + if (source.isRefreshing()) { + // create a recorder instance sourced from the transformed table + ListenerRecorder sourceRecorder = new ListenerRecorder(description, transformed.table(), result); + sourceRecorder.setMergedListener(listener); + transformed.table().listenForUpdates(sourceRecorder); + result.addParentReference(listener); + recorders.offerLast(sourceRecorder); + } + + // make a dummy update to generate the initial row keys final TableUpdateImpl fakeUpdate = new TableUpdateImpl(source.getRowSet(), RowSetFactory.empty(), diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java index f67dbc25428..c584d1af999 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java @@ -12,12 +12,14 @@ import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ResettableWritableObjectChunk; +import io.deephaven.chunk.WritableLongChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.configuration.Configuration; import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.exceptions.CancellationException; import io.deephaven.engine.exceptions.UncheckedTableException; import io.deephaven.engine.rowset.*; +import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer; import io.deephaven.engine.table.impl.sources.*; @@ -170,6 +172,24 @@ private boolean shiftRedirectedKey(@NotNull final RowSet.SearchIterator iterator return !iterator.hasNext(); } + public RowSet getInnerKeys(final RowSet outerKeys, final SharedContext sharedContext) { + if (rowRedirection == null) { + return null; + } + RowSetBuilderRandom builder = RowSetFactory.builderRandom(); + final int chunkSize = Math.min(outerKeys.intSize(), 4096); + try (final RowSequence.Iterator it = outerKeys.getRowSequenceIterator(); + ChunkSource.FillContext fillContext = rowRedirection.makeFillContext(chunkSize, sharedContext); + WritableLongChunk chunk = WritableLongChunk.makeWritableChunk(chunkSize)) { + while (it.hasMore()) { + final RowSequence rs = it.getNextRowSequenceWithLength(chunkSize); + rowRedirection.fillChunk(fillContext, chunk, rs); + builder.addRowKeysChunk(chunk); + } + } + return builder.build(); + } + @Override public void close() { try (final WritableRowSet ignored = freeRows) { @@ -292,6 +312,8 @@ protected class StateManager implements LogOutputAppendable { final SelectAndViewAnalyzer.JobScheduler jobScheduler; final CompletableFuture waitForResult; + final SharedContext sharedContext; + WritableRowSet shiftedRows; public StateManager(TableUpdate update, boolean initialStep) { @@ -311,21 +333,23 @@ public StateManager(TableUpdate update, boolean initialStep) { inputSourceReferenceCounts = null; } - waitForResult = new CompletableFuture<>(); - if (initialStep) { if (OperationInitializationThreadPool.NUM_THREADS > 1) { jobScheduler = new SelectAndViewAnalyzer.OperationInitializationPoolJobScheduler(); } else { jobScheduler = SelectAndViewAnalyzer.ImmediateJobScheduler.INSTANCE; } + waitForResult = new CompletableFuture<>(); } else { if (UpdateGraphProcessor.DEFAULT.getUpdateThreads() > 1) { jobScheduler = new SelectAndViewAnalyzer.UpdateGraphProcessorJobScheduler(); } else { jobScheduler = SelectAndViewAnalyzer.ImmediateJobScheduler.INSTANCE; } + waitForResult = null; } + + sharedContext = SharedContext.makeSharedContext(); } @Override @@ -494,27 +518,12 @@ private void cacheInputSources(final int winIdx, final Runnable completeAction) } } - /** - * Shift the operator output columns for this window - */ - protected void shiftWindowOperators(UpdateByWindow win, RowSetShiftData shift) { - try (final RowSet prevIdx = source.getRowSet().copyPrev()) { - shift.apply((begin, end, delta) -> { - try (final RowSet subRowSet = prevIdx.subSetByKeyRange(begin, end)) { - for (UpdateByOperator op : win.getOperators()) { - op.applyOutputShift(subRowSet, delta); - } - } - }); - } - } - /** * Divide the buckets for {@code windows[winIdx]} into sets and process each set in parallel. Calls * {@code completedAction} when the work is complete */ private void processWindowBuckets(int winIdx, final Runnable completedAction) { - if (jobScheduler.threadCount() > 1) { + if (jobScheduler.threadCount() > 1 && dirtyBuckets.length > 1) { // process the buckets in parallel final int bucketsPerTask = Math.max(1, dirtyBuckets.length / jobScheduler.threadCount()); final TIntArrayList offsetList = new TIntArrayList(); @@ -558,8 +567,13 @@ private void processWindows(final Runnable completeAction) { cacheInputSources(winIdx, () -> { // prepare each operator for the parallel updates to come if (initialStep) { - for (UpdateByOperator op : win.getOperators()) { - op.prepareForParallelPopulation(source.getRowSet()); + // prepare the entire set of rows on the initial step + try (final RowSet changedRows = redirContext.isRedirected() + ? RowSetFactory.flat(redirContext.requiredCapacity()) + : source.getRowSet().copy()) { + for (UpdateByOperator op : win.getOperators()) { + op.prepareForParallelPopulation(changedRows); + } } } else { // get the minimal set of rows to be updated for this window @@ -569,26 +583,27 @@ private void processWindows(final Runnable completeAction) { windowRowSet.insert(win.getAffectedRows(bucket.windowContexts[winIdx])); } } - for (UpdateByOperator op : win.getOperators()) { - op.prepareForParallelPopulation(windowRowSet); + try (final RowSet changedRows = redirContext.isRedirected() + ? redirContext.getInnerKeys(windowRowSet, sharedContext) + : windowRowSet.copy()) { + for (UpdateByOperator op : win.getOperators()) { + op.prepareForParallelPopulation(changedRows); + } } } } - if (redirContext.isRedirected()) { - for (UpdateByOperator op : windows[winIdx].getOperators()) { - // If we're redirected we have to make sure we tell the output source its actual - // size, or we're going to have a bad time. This is not necessary for - // non-redirections since the SparseArraySources do not need to do anything with - // capacity. - op.getOutputColumns().forEach((name, outputSource) -> { - // The redirection index does not use the 0th index for some reason. - ((WritableColumnSource)outputSource).ensureCapacity(redirContext.requiredCapacity()); - }) ; - } - } else if (update.shifted().nonempty()) { + if (!redirContext.isRedirected() && update.shifted().nonempty()) { // shift the non-redirected output sources now, after parallelPopulation - shiftWindowOperators(win, update.shifted()); + try (final RowSet prevIdx = source.getRowSet().copyPrev()) { + update.shifted().apply((begin, end, delta) -> { + try (final RowSet subRowSet = prevIdx.subSetByKeyRange(begin, end)) { + for (UpdateByOperator op : win.getOperators()) { + op.applyOutputShift(subRowSet, delta); + } + } + }); + } } processWindowBuckets(winIdx, () -> { @@ -611,7 +626,10 @@ private void processWindows(final Runnable completeAction) { * {@code completedAction} when the work is complete */ private void cleanUpAndNotify(final Runnable completeAction) { - shiftedRows.close(); + try (final RowSet ignoredRs = shiftedRows; + final SharedContext ignoredCtx = sharedContext) { + // auto close these resources + } // create the downstream before calling finalize() on the buckets (which releases resources) final TableUpdate downstream; @@ -729,21 +747,24 @@ public void processUpdate() { () -> cleanUpAndNotify( () -> { // signal to the main task that we have completed our work - waitForResult.complete(null); - } - ))); - - try { - // need to wait until this future is complete - waitForResult.get(); - } catch (InterruptedException e) { - throw new CancellationException("interrupted while processing updateBy"); - } catch (ExecutionException e) { - if (e.getCause() instanceof RuntimeException) { - throw (RuntimeException) e.getCause(); - } else { - throw new UncheckedDeephavenException("Failure while processing updateBy", - e.getCause()); + if (waitForResult != null) { + waitForResult.complete(null); + } + }))); + + if (waitForResult != null) { + try { + // need to wait until this future is complete + waitForResult.get(); + } catch (InterruptedException e) { + throw new CancellationException("interrupted while processing updateBy"); + } catch (ExecutionException e) { + if (e.getCause() instanceof RuntimeException) { + throw (RuntimeException) e.getCause(); + } else { + throw new UncheckedDeephavenException("Failure while processing updateBy", + e.getCause()); + } } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer.java index 3b0504d8514..46ed0a686a7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer.java @@ -563,7 +563,7 @@ interface IterateResumeAction { * Provides a mechanism to iterate over a range of values in parallel using the {@link JobScheduler} * * @param executionContext the execution context for this task - * @param description the description for + * @param description the description to use for logging * @param start the integer value from which to start iterating * @param count the number of times this task should be called * @param action the task to perform, the current iteration index is provided as a parameter @@ -615,7 +615,7 @@ default void iterateParallel(ExecutionContext executionContext, LogOutputAppenda * Provides a mechanism to iterate over a range of values in parallel using the {@link JobScheduler} * * @param executionContext the execution context for this task - * @param description the description for + * @param description the description to use for logging * @param start the integer value from which to start iterating * @param count the number of times this task should be called * @param action the task to perform, the current iteration index and a resume Runnable are parameters @@ -671,7 +671,7 @@ default void iterateParallel(ExecutionContext executionContext, LogOutputAppenda * scheduler jobs have completed. * * @param executionContext the execution context for this task - * @param description the description for + * @param description the description to use for logging * @param start the integer value from which to start iterating * @param count the number of times this task should be called * @param action the task to perform, the current iteration index and a resume Runnable are parameters diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/AbstractLongArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/AbstractLongArraySource.java index c57ec3f2e8f..aff59a32dcf 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/AbstractLongArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/AbstractLongArraySource.java @@ -3,14 +3,17 @@ */ package io.deephaven.engine.table.impl.sources; +import gnu.trove.list.array.TIntArrayList; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.*; +import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeyRanges; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.impl.MutableColumnSourceGetDefaults; +import io.deephaven.engine.updategraph.LogicalClock; import io.deephaven.util.SoftRecycler; import org.jetbrains.annotations.NotNull; @@ -47,6 +50,66 @@ public void ensureCapacity(long capacity, boolean nullFill) { ensureCapacity(capacity, blocks, prevBlocks, nullFill); } + /** + * This version of `prepareForParallelPopulation` will internally call {@link #ensureCapacity(long, boolean)} to + * make sure there is room for the incoming values. + * + * @param changedIndices indices in the dense table + */ + @Override + public void prepareForParallelPopulation(RowSet changedIndices) { + final long currentStep = LogicalClock.DEFAULT.currentStep(); + if (ensurePreviousClockCycle == currentStep) { + throw new IllegalStateException("May not call ensurePrevious twice on one clock cycle!"); + } + ensurePreviousClockCycle = currentStep; + + if (changedIndices.isEmpty()) { + return; + } + + // ensure that this source will have sufficient capacity to store these indices, does not need to be + // null-filled as the values will be immediately written + ensureCapacity(changedIndices.lastRowKey() + 1, false); + + if (prevFlusher != null) { + prevFlusher.maybeActivate(); + } else { + // we are not tracking this source yet so we have nothing to do for the previous values + return; + } + + try (final RowSequence.Iterator it = changedIndices.getRowSequenceIterator()) { + do { + final long firstKey = it.peekNextKey(); + + final int block = (int) (firstKey >> LOG_BLOCK_SIZE); + + final long[] inUse; + if (prevBlocks[block] == null) { + prevBlocks[block] = recycler.borrowItem(); + prevInUse[block] = inUse = inUseRecycler.borrowItem(); + if (prevAllocated == null) { + prevAllocated = new TIntArrayList(); + } + prevAllocated.add(block); + } else { + inUse = prevInUse[block]; + } + + final long maxKeyInCurrentBlock = firstKey | INDEX_MASK; + + it.getNextRowSequenceThrough(maxKeyInCurrentBlock).forAllRowKeys(key -> { + final int nextIndexWithinBlock = (int) (key & INDEX_MASK); + final int nextIndexWithinInUse = nextIndexWithinBlock >> LOG_INUSE_BITSET_SIZE; + final long nextMaskWithinInUse = 1L << (nextIndexWithinBlock & IN_USE_MASK); + prevBlocks[block][nextIndexWithinBlock] = blocks[block][nextIndexWithinBlock]; + inUse[nextIndexWithinInUse] |= nextMaskWithinInUse; + }); + } while (it.hasMore()); + } + } + @Override public final void set(long key, long value) { final int block = (int) (key >> LOG_BLOCK_SIZE); @@ -438,9 +501,9 @@ void fillFromChunkByRanges(@NotNull RowSequence rowSequence, Chunk chunk = src.asLongChunk(); final LongChunk ranges = rowSequence.asRowKeyRangesChunk(); - final boolean hasPrev = prevFlusher != null; + final boolean trackPrevious = prevFlusher != null && ensurePreviousClockCycle != LogicalClock.DEFAULT.currentStep(); - if (hasPrev) { + if (trackPrevious) { prevFlusher.maybeActivate(); } @@ -466,7 +529,7 @@ void fillFromChunkByRanges(@NotNull RowSequence rowSequence, Chunk chunk = src.asLongChunk(); final LongChunk keys = rowSequence.asRowKeyChunk(); - final boolean hasPrev = prevFlusher != null; + final boolean trackPrevious = prevFlusher != null && ensurePreviousClockCycle != LogicalClock.DEFAULT.currentStep(); - if (hasPrev) { + if (trackPrevious) { prevFlusher.maybeActivate(); } @@ -535,7 +598,7 @@ void fillFromChunkByKeys(@NotNull RowSequence rowSequence, Chunk void fillFromChunkByRanges(final @NotNull RowSequence rowSequence, final Chu final ObjectChunk chunk = src.asObjectChunk(); final LongChunk ranges = rowSequence.asRowKeyRangesChunk(); - final boolean hasPrev = prevFlusher != null; + final boolean trackPrevious = prevFlusher != null && ensurePreviousClockCycle != LogicalClock.DEFAULT.currentStep(); - if (hasPrev) { + if (trackPrevious) { prevFlusher.maybeActivate(); } @@ -578,7 +641,7 @@ void fillFromChunkByRanges(final @NotNull RowSequence rowSequence, final Chu knownUnaliasedBlock = inner; // This 'if' with its constant condition should be very friendly to the branch predictor. - if (hasPrev) { + if (trackPrevious) { // this should be vectorized for (int jj = 0; jj < length; ++jj) { if (shouldRecordPrevious(firstKey + jj, prevBlocks, recycler)) { @@ -600,9 +663,9 @@ void fillFromChunkByKeys(final @NotNull RowSequence rowSequence, final Chunk final ObjectChunk chunk = src.asObjectChunk(); final LongChunk keys = rowSequence.asRowKeyChunk(); - final boolean hasPrev = prevFlusher != null; + final boolean trackPrevious = prevFlusher != null && ensurePreviousClockCycle != LogicalClock.DEFAULT.currentStep(); - if (hasPrev) { + if (trackPrevious) { prevFlusher.maybeActivate(); } @@ -624,7 +687,7 @@ void fillFromChunkByKeys(final @NotNull RowSequence rowSequence, final Chunk while (ii <= lastII) { final int indexWithinBlock = (int) (keys.get(ii) & INDEX_MASK); - if (hasPrev) { + if (trackPrevious) { if (shouldRecordPrevious(ii, prevBlocks, recycler)) { prevBlocks[block][indexWithinBlock] = inner[indexWithinBlock]; } @@ -642,9 +705,9 @@ public void fillFromChunkUnordered(@NotNull FillFromContext context, @NotNull Ch } final LongChunk chunk = src.asLongChunk(); - final boolean hasPrev = prevFlusher != null; + final boolean trackPrevious = prevFlusher != null && ensurePreviousClockCycle != LogicalClock.DEFAULT.currentStep(); - if (hasPrev) { + if (trackPrevious) { prevFlusher.maybeActivate(); } @@ -664,7 +727,7 @@ public void fillFromChunkUnordered(@NotNull FillFromContext context, @NotNull Ch do { final int indexWithinBlock = (int) (key & INDEX_MASK); - if (hasPrev) { + if (trackPrevious) { if (shouldRecordPrevious(key, prevBlocks, recycler)) { prevBlocks[block][indexWithinBlock] = inner[indexWithinBlock]; } @@ -681,9 +744,9 @@ void fillFromChunkUnordered(@NotNull Chunk src, @NotNull L } final ObjectChunk chunk = src.asObjectChunk(); - final boolean hasPrev = prevFlusher != null; + final boolean trackPrevious = prevFlusher != null && ensurePreviousClockCycle != LogicalClock.DEFAULT.currentStep(); - if (hasPrev) { + if (trackPrevious) { prevFlusher.maybeActivate(); } @@ -703,7 +766,7 @@ void fillFromChunkUnordered(@NotNull Chunk src, @NotNull L do { final int indexWithinBlock = (int) (key & INDEX_MASK); - if (hasPrev) { + if (trackPrevious) { if (shouldRecordPrevious(key, prevBlocks, recycler)) { prevBlocks[block][indexWithinBlock] = inner[indexWithinBlock]; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ArraySourceHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ArraySourceHelper.java index fc3e8b4b023..4a301ffe20d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ArraySourceHelper.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ArraySourceHelper.java @@ -5,8 +5,11 @@ import io.deephaven.base.verify.Assert; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.SharedContext; import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.WritableSourceWithPrepareForParallelPopulation; +import io.deephaven.engine.updategraph.LogicalClock; import io.deephaven.util.datastructures.LongSizedDataStructure; import io.deephaven.chunk.*; import io.deephaven.engine.rowset.RowSequence; @@ -19,14 +22,23 @@ import java.util.Arrays; -abstract class ArraySourceHelper extends ArrayBackedColumnSource { +import static io.deephaven.engine.table.impl.sources.sparse.SparseConstants.*; +import static io.deephaven.engine.table.impl.sources.sparse.SparseConstants.IN_USE_MASK; + +abstract class ArraySourceHelper extends ArrayBackedColumnSource + implements WritableSourceWithPrepareForParallelPopulation { /** * The presence of a prevFlusher means that this ArraySource wants to track previous values. If prevFlusher is null, * the ArraySource does not want (or does not yet want) to track previous values. Deserialized ArraySources never * track previous values. */ protected transient UpdateCommitter> prevFlusher = null; - private transient TIntArrayList prevAllocated = null; + protected transient TIntArrayList prevAllocated = null; + + /** + * If ensure previous has been called, we need not check previous values when filling. + */ + protected transient long ensurePreviousClockCycle = -1; ArraySourceHelper(Class type) { super(type); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/BooleanArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/BooleanArraySource.java index df2db095991..40fce4d304e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/BooleanArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/BooleanArraySource.java @@ -3,12 +3,16 @@ */ package io.deephaven.engine.table.impl.sources; +import gnu.trove.list.array.TIntArrayList; +import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.WritableColumnSource; +import io.deephaven.engine.table.WritableSourceWithPrepareForParallelPopulation; import io.deephaven.engine.table.impl.AbstractColumnSource; import io.deephaven.engine.table.impl.MutableColumnSourceGetDefaults; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeyRanges; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.updategraph.LogicalClock; import io.deephaven.util.BooleanUtils; import io.deephaven.chunk.*; import io.deephaven.engine.rowset.chunkattributes.RowKeys; @@ -47,6 +51,70 @@ public void ensureCapacity(long capacity, boolean nullFill) { ensureCapacity(capacity, blocks, prevBlocks, nullFill); } + /** + * This version of `prepareForParallelPopulation` will internally call {@link #ensureCapacity(long, boolean)} to + * make sure there is room for the incoming values. + * + * @param changedIndices indices in the dense table + */ + @Override + public void prepareForParallelPopulation(RowSet changedIndices) { + final long currentStep = LogicalClock.DEFAULT.currentStep(); + if (ensurePreviousClockCycle == currentStep) { + throw new IllegalStateException("May not call ensurePrevious twice on one clock cycle!"); + } + ensurePreviousClockCycle = currentStep; + + if (changedIndices.isEmpty()) { + return; + } + + // ensure that this source will have sufficient capacity to store these indices, does not need to be + // null-filled as the values will be immediately written + ensureCapacity(changedIndices.lastRowKey() + 1, false); + + if (prevFlusher != null) { + prevFlusher.maybeActivate(); + } else { + // we are not tracking this source yet so we have nothing to do for the previous values + return; + } + + try (final RowSequence.Iterator it = changedIndices.getRowSequenceIterator()) { + do { + final long firstKey = it.peekNextKey(); + + final int block = (int) (firstKey >> LOG_BLOCK_SIZE); + final int indexWithinBlock = (int) (firstKey & INDEX_MASK); + final int indexWithinInUse = indexWithinBlock >> LOG_INUSE_BITSET_SIZE; + final long maskWithinInUse = 1L << (indexWithinBlock & IN_USE_MASK); + + final long[] inUse; + if (prevBlocks[block] == null) { + prevBlocks[block] = recycler.borrowItem(); + prevInUse[block] = inUse = inUseRecycler.borrowItem(); + if (prevAllocated == null) { + prevAllocated = new TIntArrayList(); + } + prevAllocated.add(block); + } else { + inUse = prevInUse[block]; + } + inUse[indexWithinInUse] |= maskWithinInUse; + + final long maxKeyInCurrentBlock = firstKey | INDEX_MASK; + + it.getNextRowSequenceThrough(maxKeyInCurrentBlock).forAllRowKeys(key -> { + final int nextIndexWithinBlock = (int) (key & INDEX_MASK); + final int nextIndexWithinInUse = nextIndexWithinBlock >> LOG_INUSE_BITSET_SIZE; + final long nextMaskWithinInUse = 1L << (nextIndexWithinBlock & IN_USE_MASK); + prevBlocks[block][nextIndexWithinBlock] = blocks[block][nextIndexWithinBlock]; + inUse[nextIndexWithinInUse] |= nextMaskWithinInUse; + }); + } while (it.hasMore()); + } + } + @Override public void setNull(long key) { set(key, NULL_BOOLEAN_AS_BYTE); @@ -341,9 +409,9 @@ private interface Reader { private void fillFromChunkByRanges(@NotNull RowSequence rowSequence, Chunk src, Reader reader) { final LongChunk ranges = rowSequence.asRowKeyRangesChunk(); - final boolean hasPrev = prevFlusher != null; + final boolean trackPrevious = prevFlusher != null && ensurePreviousClockCycle != LogicalClock.DEFAULT.currentStep(); - if (hasPrev) { + if (trackPrevious) { prevFlusher.maybeActivate(); } @@ -368,7 +436,7 @@ private void fillFromChunkByRanges(@NotNull RowSequence rowSequence, Chunk src, Reader reader) { final LongChunk keys = rowSequence.asRowKeyChunk(); - final boolean hasPrev = prevFlusher != null; + final boolean trackPrevious = prevFlusher != null && ensurePreviousClockCycle != LogicalClock.DEFAULT.currentStep(); - if (hasPrev) { + if (trackPrevious) { prevFlusher.maybeActivate(); } @@ -416,7 +484,7 @@ private void fillFromChunkByKeys(@NotNull RowSequence rowSequence, Chunk src, @NotNu if (keys.size() == 0) { return; } - final boolean hasPrev = prevFlusher != null; + final boolean trackPrevious = prevFlusher != null && ensurePreviousClockCycle != LogicalClock.DEFAULT.currentStep(); - if (hasPrev) { + if (trackPrevious) { prevFlusher.maybeActivate(); } @@ -459,7 +527,7 @@ private void fillFromChunkUnordered(@NotNull Chunk src, @NotNu do { final int indexWithinBlock = (int) (key & INDEX_MASK); - if (hasPrev) { + if (trackPrevious) { if (shouldRecordPrevious(key, prevBlocks, recycler)) { prevBlocks[block][indexWithinBlock] = inner[indexWithinBlock]; } @@ -470,7 +538,7 @@ private void fillFromChunkUnordered(@NotNull Chunk src, @NotNu } } - private class ReinterpretedAsByte extends AbstractColumnSource implements MutableColumnSourceGetDefaults.ForByte, FillUnordered, WritableColumnSource { + private class ReinterpretedAsByte extends AbstractColumnSource implements MutableColumnSourceGetDefaults.ForByte, FillUnordered, WritableColumnSource, WritableSourceWithPrepareForParallelPopulation { private ReinterpretedAsByte() { super(byte.class); } @@ -627,6 +695,11 @@ public void ensureCapacity(long capacity, boolean nullFill) { BooleanArraySource.this.ensureCapacity(capacity, nullFill); } + @Override + public void prepareForParallelPopulation(RowSet rowSet) { + BooleanArraySource.this.prepareForParallelPopulation(rowSet); + } + @Override public FillFromContext makeFillFromContext(int chunkCapacity) { return BooleanArraySource.this.makeFillFromContext(chunkCapacity); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/BooleanSparseArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/BooleanSparseArraySource.java index 9d57c7fc2b8..d8e9809f7d3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/BooleanSparseArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/BooleanSparseArraySource.java @@ -12,6 +12,7 @@ import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.util.BooleanUtils; import static io.deephaven.util.BooleanUtils.NULL_BOOLEAN_AS_BYTE; +import io.deephaven.engine.table.WritableSourceWithPrepareForParallelPopulation; import io.deephaven.engine.table.impl.DefaultGetContext; import io.deephaven.chunk.*; @@ -843,7 +844,7 @@ protected ColumnSource doReinterpret( return (ColumnSource) new BooleanSparseArraySource.ReinterpretedAsByte(this); } - public static class ReinterpretedAsByte extends AbstractColumnSource implements MutableColumnSourceGetDefaults.ForByte, FillUnordered, WritableColumnSource { + public static class ReinterpretedAsByte extends AbstractColumnSource implements MutableColumnSourceGetDefaults.ForByte, FillUnordered, WritableColumnSource, WritableSourceWithPrepareForParallelPopulation { private final BooleanSparseArraySource wrapped; private ReinterpretedAsByte(BooleanSparseArraySource wrapped) { @@ -1049,9 +1050,9 @@ public void fillFromChunk(@NotNull FillFromContext context_unused, @NotNull Chun final ByteChunk chunk = src.asByteChunk(); final LongChunk keys = RowSequence.asRowKeyChunk(); - final boolean hasPrev = wrapped.prevFlusher != null; + final boolean trackPrevious = wrapped.prevFlusher != null && wrapped.ensurePreviousClockCycle != LogicalClock.DEFAULT.currentStep(); - if (hasPrev) { + if (trackPrevious) { wrapped.prevFlusher.maybeActivate(); } @@ -1073,13 +1074,13 @@ public void fillFromChunk(@NotNull FillFromContext context_unused, @NotNull Chun } // This conditional with its constant condition should be very friendly to the branch predictor. - final byte[] prevBlock = hasPrev ? wrapped.ensurePrevBlock(firstRowKey, block0, block1, block2) : null; - final long[] inUse = hasPrev ? wrapped.prevInUse.get(block0).get(block1).get(block2) : null; + final byte[] prevBlock = trackPrevious ? wrapped.ensurePrevBlock(firstRowKey, block0, block1, block2) : null; + final long[] inUse = trackPrevious ? wrapped.prevInUse.get(block0).get(block1).get(block2) : null; while (ii <= lastII) { final int indexWithinBlock = (int) (keys.get(ii) & INDEX_MASK); // This 'if' with its constant condition should be very friendly to the branch predictor. - if (hasPrev) { + if (trackPrevious) { assert inUse != null; assert prevBlock != null; @@ -1097,8 +1098,9 @@ public void fillFromChunk(@NotNull FillFromContext context_unused, @NotNull Chun } } + @Override public void prepareForParallelPopulation(RowSet rowSet) { - wrapped.prepareForParallelPopulation(rowSet); + wrapped.prepareForParallelPopulation(rowSet); } } // endregion reinterpretation diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ByteArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ByteArraySource.java index a0919b438ac..e06a3eec7f3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ByteArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ByteArraySource.java @@ -8,13 +8,16 @@ */ package io.deephaven.engine.table.impl.sources; +import gnu.trove.list.array.TIntArrayList; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.impl.MutableColumnSourceGetDefaults; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeyRanges; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.updategraph.LogicalClock; import io.deephaven.util.compare.ByteComparisons; import io.deephaven.chunk.*; import io.deephaven.chunk.ResettableWritableChunk; @@ -27,6 +30,8 @@ import java.util.Arrays; +import static io.deephaven.engine.table.impl.sources.sparse.SparseConstants.*; +import static io.deephaven.engine.table.impl.sources.sparse.SparseConstants.BLOCK2_MASK; import static io.deephaven.util.QueryConstants.NULL_BYTE; import static io.deephaven.util.type.TypeUtils.box; import static io.deephaven.util.type.TypeUtils.unbox; @@ -63,6 +68,66 @@ public void ensureCapacity(long capacity, boolean nullFill) { ensureCapacity(capacity, blocks, prevBlocks, nullFill); } + /** + * This version of `prepareForParallelPopulation` will internally call {@link #ensureCapacity(long, boolean)} to + * make sure there is room for the incoming values. + * + * @param changedIndices indices in the dense table + */ + @Override + public void prepareForParallelPopulation(RowSet changedIndices) { + final long currentStep = LogicalClock.DEFAULT.currentStep(); + if (ensurePreviousClockCycle == currentStep) { + throw new IllegalStateException("May not call ensurePrevious twice on one clock cycle!"); + } + ensurePreviousClockCycle = currentStep; + + if (changedIndices.isEmpty()) { + return; + } + + // ensure that this source will have sufficient capacity to store these indices, does not need to be + // null-filled as the values will be immediately written + ensureCapacity(changedIndices.lastRowKey() + 1, false); + + if (prevFlusher != null) { + prevFlusher.maybeActivate(); + } else { + // we are not tracking this source yet so we have nothing to do for the previous values + return; + } + + try (final RowSequence.Iterator it = changedIndices.getRowSequenceIterator()) { + do { + final long firstKey = it.peekNextKey(); + + final int block = (int) (firstKey >> LOG_BLOCK_SIZE); + + final long[] inUse; + if (prevBlocks[block] == null) { + prevBlocks[block] = recycler.borrowItem(); + prevInUse[block] = inUse = inUseRecycler.borrowItem(); + if (prevAllocated == null) { + prevAllocated = new TIntArrayList(); + } + prevAllocated.add(block); + } else { + inUse = prevInUse[block]; + } + + final long maxKeyInCurrentBlock = firstKey | INDEX_MASK; + + it.getNextRowSequenceThrough(maxKeyInCurrentBlock).forAllRowKeys(key -> { + final int nextIndexWithinBlock = (int) (key & INDEX_MASK); + final int nextIndexWithinInUse = nextIndexWithinBlock >> LOG_INUSE_BITSET_SIZE; + final long nextMaskWithinInUse = 1L << (nextIndexWithinBlock & IN_USE_MASK); + prevBlocks[block][nextIndexWithinBlock] = blocks[block][nextIndexWithinBlock]; + inUse[nextIndexWithinInUse] |= nextMaskWithinInUse; + }); + } while (it.hasMore()); + } + } + @Override public final void set(long key, Byte value) { set(key, unbox(value)); @@ -348,9 +413,9 @@ void fillFromChunkByRanges(@NotNull RowSequence rowSequence, Chunk chunk = src.asByteChunk(); final LongChunk ranges = rowSequence.asRowKeyRangesChunk(); - final boolean hasPrev = prevFlusher != null; + final boolean trackPrevious = prevFlusher != null && ensurePreviousClockCycle != LogicalClock.DEFAULT.currentStep(); - if (hasPrev) { + if (trackPrevious) { prevFlusher.maybeActivate(); } @@ -376,7 +441,7 @@ void fillFromChunkByRanges(@NotNull RowSequence rowSequence, Chunk chunk = src.asByteChunk(); final LongChunk keys = rowSequence.asRowKeyChunk(); - final boolean hasPrev = prevFlusher != null; + final boolean trackPrevious = prevFlusher != null && ensurePreviousClockCycle != LogicalClock.DEFAULT.currentStep(); - if (hasPrev) { + if (trackPrevious) { prevFlusher.maybeActivate(); } @@ -448,7 +513,7 @@ void fillFromChunkByKeys(@NotNull RowSequence rowSequence, Chunk chunk = src.asByteChunk(); - final boolean hasPrev = prevFlusher != null; + final boolean trackPrevious = prevFlusher != null && ensurePreviousClockCycle != LogicalClock.DEFAULT.currentStep(); - if (hasPrev) { + if (trackPrevious) { prevFlusher.maybeActivate(); } @@ -488,7 +553,7 @@ public void fillFromChunkUnordered(@NotNull FillFromContext context, @NotNull Ch do { final int indexWithinBlock = (int) (key & INDEX_MASK); - if (hasPrev) { + if (trackPrevious) { if (shouldRecordPrevious(key, prevBlocks, recycler)) { prevBlocks[block][indexWithinBlock] = inner[indexWithinBlock]; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/CharacterArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/CharacterArraySource.java index 342250b0e15..fe84494e539 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/CharacterArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/CharacterArraySource.java @@ -3,13 +3,16 @@ */ package io.deephaven.engine.table.impl.sources; +import gnu.trove.list.array.TIntArrayList; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.impl.MutableColumnSourceGetDefaults; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeyRanges; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.updategraph.LogicalClock; import io.deephaven.util.compare.CharComparisons; import io.deephaven.chunk.*; import io.deephaven.chunk.ResettableWritableChunk; @@ -22,6 +25,8 @@ import java.util.Arrays; +import static io.deephaven.engine.table.impl.sources.sparse.SparseConstants.*; +import static io.deephaven.engine.table.impl.sources.sparse.SparseConstants.BLOCK2_MASK; import static io.deephaven.util.QueryConstants.NULL_CHAR; import static io.deephaven.util.type.TypeUtils.box; import static io.deephaven.util.type.TypeUtils.unbox; @@ -58,6 +63,66 @@ public void ensureCapacity(long capacity, boolean nullFill) { ensureCapacity(capacity, blocks, prevBlocks, nullFill); } + /** + * This version of `prepareForParallelPopulation` will internally call {@link #ensureCapacity(long, boolean)} to + * make sure there is room for the incoming values. + * + * @param changedIndices indices in the dense table + */ + @Override + public void prepareForParallelPopulation(RowSet changedIndices) { + final long currentStep = LogicalClock.DEFAULT.currentStep(); + if (ensurePreviousClockCycle == currentStep) { + throw new IllegalStateException("May not call ensurePrevious twice on one clock cycle!"); + } + ensurePreviousClockCycle = currentStep; + + if (changedIndices.isEmpty()) { + return; + } + + // ensure that this source will have sufficient capacity to store these indices, does not need to be + // null-filled as the values will be immediately written + ensureCapacity(changedIndices.lastRowKey() + 1, false); + + if (prevFlusher != null) { + prevFlusher.maybeActivate(); + } else { + // we are not tracking this source yet so we have nothing to do for the previous values + return; + } + + try (final RowSequence.Iterator it = changedIndices.getRowSequenceIterator()) { + do { + final long firstKey = it.peekNextKey(); + + final int block = (int) (firstKey >> LOG_BLOCK_SIZE); + + final long[] inUse; + if (prevBlocks[block] == null) { + prevBlocks[block] = recycler.borrowItem(); + prevInUse[block] = inUse = inUseRecycler.borrowItem(); + if (prevAllocated == null) { + prevAllocated = new TIntArrayList(); + } + prevAllocated.add(block); + } else { + inUse = prevInUse[block]; + } + + final long maxKeyInCurrentBlock = firstKey | INDEX_MASK; + + it.getNextRowSequenceThrough(maxKeyInCurrentBlock).forAllRowKeys(key -> { + final int nextIndexWithinBlock = (int) (key & INDEX_MASK); + final int nextIndexWithinInUse = nextIndexWithinBlock >> LOG_INUSE_BITSET_SIZE; + final long nextMaskWithinInUse = 1L << (nextIndexWithinBlock & IN_USE_MASK); + prevBlocks[block][nextIndexWithinBlock] = blocks[block][nextIndexWithinBlock]; + inUse[nextIndexWithinInUse] |= nextMaskWithinInUse; + }); + } while (it.hasMore()); + } + } + @Override public final void set(long key, Character value) { set(key, unbox(value)); @@ -343,9 +408,9 @@ void fillFromChunkByRanges(@NotNull RowSequence rowSequence, Chunk chunk = src.asCharChunk(); final LongChunk ranges = rowSequence.asRowKeyRangesChunk(); - final boolean hasPrev = prevFlusher != null; + final boolean trackPrevious = prevFlusher != null && ensurePreviousClockCycle != LogicalClock.DEFAULT.currentStep(); - if (hasPrev) { + if (trackPrevious) { prevFlusher.maybeActivate(); } @@ -371,7 +436,7 @@ void fillFromChunkByRanges(@NotNull RowSequence rowSequence, Chunk chunk = src.asCharChunk(); final LongChunk keys = rowSequence.asRowKeyChunk(); - final boolean hasPrev = prevFlusher != null; + final boolean trackPrevious = prevFlusher != null && ensurePreviousClockCycle != LogicalClock.DEFAULT.currentStep(); - if (hasPrev) { + if (trackPrevious) { prevFlusher.maybeActivate(); } @@ -443,7 +508,7 @@ void fillFromChunkByKeys(@NotNull RowSequence rowSequence, Chunk chunk = src.asCharChunk(); - final boolean hasPrev = prevFlusher != null; + final boolean trackPrevious = prevFlusher != null && ensurePreviousClockCycle != LogicalClock.DEFAULT.currentStep(); - if (hasPrev) { + if (trackPrevious) { prevFlusher.maybeActivate(); } @@ -483,7 +548,7 @@ public void fillFromChunkUnordered(@NotNull FillFromContext context, @NotNull Ch do { final int indexWithinBlock = (int) (key & INDEX_MASK); - if (hasPrev) { + if (trackPrevious) { if (shouldRecordPrevious(key, prevBlocks, recycler)) { prevBlocks[block][indexWithinBlock] = inner[indexWithinBlock]; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/DoubleArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/DoubleArraySource.java index 46d4f2e1234..014776ea032 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/DoubleArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/DoubleArraySource.java @@ -8,13 +8,16 @@ */ package io.deephaven.engine.table.impl.sources; +import gnu.trove.list.array.TIntArrayList; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.impl.MutableColumnSourceGetDefaults; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeyRanges; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.updategraph.LogicalClock; import io.deephaven.util.compare.DoubleComparisons; import io.deephaven.chunk.*; import io.deephaven.chunk.ResettableWritableChunk; @@ -27,6 +30,8 @@ import java.util.Arrays; +import static io.deephaven.engine.table.impl.sources.sparse.SparseConstants.*; +import static io.deephaven.engine.table.impl.sources.sparse.SparseConstants.BLOCK2_MASK; import static io.deephaven.util.QueryConstants.NULL_DOUBLE; import static io.deephaven.util.type.TypeUtils.box; import static io.deephaven.util.type.TypeUtils.unbox; @@ -63,6 +68,66 @@ public void ensureCapacity(long capacity, boolean nullFill) { ensureCapacity(capacity, blocks, prevBlocks, nullFill); } + /** + * This version of `prepareForParallelPopulation` will internally call {@link #ensureCapacity(long, boolean)} to + * make sure there is room for the incoming values. + * + * @param changedIndices indices in the dense table + */ + @Override + public void prepareForParallelPopulation(RowSet changedIndices) { + final long currentStep = LogicalClock.DEFAULT.currentStep(); + if (ensurePreviousClockCycle == currentStep) { + throw new IllegalStateException("May not call ensurePrevious twice on one clock cycle!"); + } + ensurePreviousClockCycle = currentStep; + + if (changedIndices.isEmpty()) { + return; + } + + // ensure that this source will have sufficient capacity to store these indices, does not need to be + // null-filled as the values will be immediately written + ensureCapacity(changedIndices.lastRowKey() + 1, false); + + if (prevFlusher != null) { + prevFlusher.maybeActivate(); + } else { + // we are not tracking this source yet so we have nothing to do for the previous values + return; + } + + try (final RowSequence.Iterator it = changedIndices.getRowSequenceIterator()) { + do { + final long firstKey = it.peekNextKey(); + + final int block = (int) (firstKey >> LOG_BLOCK_SIZE); + + final long[] inUse; + if (prevBlocks[block] == null) { + prevBlocks[block] = recycler.borrowItem(); + prevInUse[block] = inUse = inUseRecycler.borrowItem(); + if (prevAllocated == null) { + prevAllocated = new TIntArrayList(); + } + prevAllocated.add(block); + } else { + inUse = prevInUse[block]; + } + + final long maxKeyInCurrentBlock = firstKey | INDEX_MASK; + + it.getNextRowSequenceThrough(maxKeyInCurrentBlock).forAllRowKeys(key -> { + final int nextIndexWithinBlock = (int) (key & INDEX_MASK); + final int nextIndexWithinInUse = nextIndexWithinBlock >> LOG_INUSE_BITSET_SIZE; + final long nextMaskWithinInUse = 1L << (nextIndexWithinBlock & IN_USE_MASK); + prevBlocks[block][nextIndexWithinBlock] = blocks[block][nextIndexWithinBlock]; + inUse[nextIndexWithinInUse] |= nextMaskWithinInUse; + }); + } while (it.hasMore()); + } + } + @Override public final void set(long key, Double value) { set(key, unbox(value)); @@ -348,9 +413,9 @@ void fillFromChunkByRanges(@NotNull RowSequence rowSequence, Chunk chunk = src.asDoubleChunk(); final LongChunk ranges = rowSequence.asRowKeyRangesChunk(); - final boolean hasPrev = prevFlusher != null; + final boolean trackPrevious = prevFlusher != null && ensurePreviousClockCycle != LogicalClock.DEFAULT.currentStep(); - if (hasPrev) { + if (trackPrevious) { prevFlusher.maybeActivate(); } @@ -376,7 +441,7 @@ void fillFromChunkByRanges(@NotNull RowSequence rowSequence, Chunk chunk = src.asDoubleChunk(); final LongChunk keys = rowSequence.asRowKeyChunk(); - final boolean hasPrev = prevFlusher != null; + final boolean trackPrevious = prevFlusher != null && ensurePreviousClockCycle != LogicalClock.DEFAULT.currentStep(); - if (hasPrev) { + if (trackPrevious) { prevFlusher.maybeActivate(); } @@ -448,7 +513,7 @@ void fillFromChunkByKeys(@NotNull RowSequence rowSequence, Chunk chunk = src.asDoubleChunk(); - final boolean hasPrev = prevFlusher != null; + final boolean trackPrevious = prevFlusher != null && ensurePreviousClockCycle != LogicalClock.DEFAULT.currentStep(); - if (hasPrev) { + if (trackPrevious) { prevFlusher.maybeActivate(); } @@ -488,7 +553,7 @@ public void fillFromChunkUnordered(@NotNull FillFromContext context, @NotNull Ch do { final int indexWithinBlock = (int) (key & INDEX_MASK); - if (hasPrev) { + if (trackPrevious) { if (shouldRecordPrevious(key, prevBlocks, recycler)) { prevBlocks[block][indexWithinBlock] = inner[indexWithinBlock]; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/FloatArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/FloatArraySource.java index b1ab617a253..57c1cd1c9b4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/FloatArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/FloatArraySource.java @@ -8,13 +8,16 @@ */ package io.deephaven.engine.table.impl.sources; +import gnu.trove.list.array.TIntArrayList; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.impl.MutableColumnSourceGetDefaults; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeyRanges; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.updategraph.LogicalClock; import io.deephaven.util.compare.FloatComparisons; import io.deephaven.chunk.*; import io.deephaven.chunk.ResettableWritableChunk; @@ -27,6 +30,8 @@ import java.util.Arrays; +import static io.deephaven.engine.table.impl.sources.sparse.SparseConstants.*; +import static io.deephaven.engine.table.impl.sources.sparse.SparseConstants.BLOCK2_MASK; import static io.deephaven.util.QueryConstants.NULL_FLOAT; import static io.deephaven.util.type.TypeUtils.box; import static io.deephaven.util.type.TypeUtils.unbox; @@ -63,6 +68,66 @@ public void ensureCapacity(long capacity, boolean nullFill) { ensureCapacity(capacity, blocks, prevBlocks, nullFill); } + /** + * This version of `prepareForParallelPopulation` will internally call {@link #ensureCapacity(long, boolean)} to + * make sure there is room for the incoming values. + * + * @param changedIndices indices in the dense table + */ + @Override + public void prepareForParallelPopulation(RowSet changedIndices) { + final long currentStep = LogicalClock.DEFAULT.currentStep(); + if (ensurePreviousClockCycle == currentStep) { + throw new IllegalStateException("May not call ensurePrevious twice on one clock cycle!"); + } + ensurePreviousClockCycle = currentStep; + + if (changedIndices.isEmpty()) { + return; + } + + // ensure that this source will have sufficient capacity to store these indices, does not need to be + // null-filled as the values will be immediately written + ensureCapacity(changedIndices.lastRowKey() + 1, false); + + if (prevFlusher != null) { + prevFlusher.maybeActivate(); + } else { + // we are not tracking this source yet so we have nothing to do for the previous values + return; + } + + try (final RowSequence.Iterator it = changedIndices.getRowSequenceIterator()) { + do { + final long firstKey = it.peekNextKey(); + + final int block = (int) (firstKey >> LOG_BLOCK_SIZE); + + final long[] inUse; + if (prevBlocks[block] == null) { + prevBlocks[block] = recycler.borrowItem(); + prevInUse[block] = inUse = inUseRecycler.borrowItem(); + if (prevAllocated == null) { + prevAllocated = new TIntArrayList(); + } + prevAllocated.add(block); + } else { + inUse = prevInUse[block]; + } + + final long maxKeyInCurrentBlock = firstKey | INDEX_MASK; + + it.getNextRowSequenceThrough(maxKeyInCurrentBlock).forAllRowKeys(key -> { + final int nextIndexWithinBlock = (int) (key & INDEX_MASK); + final int nextIndexWithinInUse = nextIndexWithinBlock >> LOG_INUSE_BITSET_SIZE; + final long nextMaskWithinInUse = 1L << (nextIndexWithinBlock & IN_USE_MASK); + prevBlocks[block][nextIndexWithinBlock] = blocks[block][nextIndexWithinBlock]; + inUse[nextIndexWithinInUse] |= nextMaskWithinInUse; + }); + } while (it.hasMore()); + } + } + @Override public final void set(long key, Float value) { set(key, unbox(value)); @@ -348,9 +413,9 @@ void fillFromChunkByRanges(@NotNull RowSequence rowSequence, Chunk chunk = src.asFloatChunk(); final LongChunk ranges = rowSequence.asRowKeyRangesChunk(); - final boolean hasPrev = prevFlusher != null; + final boolean trackPrevious = prevFlusher != null && ensurePreviousClockCycle != LogicalClock.DEFAULT.currentStep(); - if (hasPrev) { + if (trackPrevious) { prevFlusher.maybeActivate(); } @@ -376,7 +441,7 @@ void fillFromChunkByRanges(@NotNull RowSequence rowSequence, Chunk chunk = src.asFloatChunk(); final LongChunk keys = rowSequence.asRowKeyChunk(); - final boolean hasPrev = prevFlusher != null; + final boolean trackPrevious = prevFlusher != null && ensurePreviousClockCycle != LogicalClock.DEFAULT.currentStep(); - if (hasPrev) { + if (trackPrevious) { prevFlusher.maybeActivate(); } @@ -448,7 +513,7 @@ void fillFromChunkByKeys(@NotNull RowSequence rowSequence, Chunk chunk = src.asFloatChunk(); - final boolean hasPrev = prevFlusher != null; + final boolean trackPrevious = prevFlusher != null && ensurePreviousClockCycle != LogicalClock.DEFAULT.currentStep(); - if (hasPrev) { + if (trackPrevious) { prevFlusher.maybeActivate(); } @@ -488,7 +553,7 @@ public void fillFromChunkUnordered(@NotNull FillFromContext context, @NotNull Ch do { final int indexWithinBlock = (int) (key & INDEX_MASK); - if (hasPrev) { + if (trackPrevious) { if (shouldRecordPrevious(key, prevBlocks, recycler)) { prevBlocks[block][indexWithinBlock] = inner[indexWithinBlock]; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/IntegerArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/IntegerArraySource.java index b62bb540e66..f1cae208688 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/IntegerArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/IntegerArraySource.java @@ -8,13 +8,16 @@ */ package io.deephaven.engine.table.impl.sources; +import gnu.trove.list.array.TIntArrayList; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.impl.MutableColumnSourceGetDefaults; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeyRanges; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.updategraph.LogicalClock; import io.deephaven.util.compare.IntComparisons; import io.deephaven.chunk.*; import io.deephaven.chunk.ResettableWritableChunk; @@ -27,6 +30,8 @@ import java.util.Arrays; +import static io.deephaven.engine.table.impl.sources.sparse.SparseConstants.*; +import static io.deephaven.engine.table.impl.sources.sparse.SparseConstants.BLOCK2_MASK; import static io.deephaven.util.QueryConstants.NULL_INT; import static io.deephaven.util.type.TypeUtils.box; import static io.deephaven.util.type.TypeUtils.unbox; @@ -63,6 +68,66 @@ public void ensureCapacity(long capacity, boolean nullFill) { ensureCapacity(capacity, blocks, prevBlocks, nullFill); } + /** + * This version of `prepareForParallelPopulation` will internally call {@link #ensureCapacity(long, boolean)} to + * make sure there is room for the incoming values. + * + * @param changedIndices indices in the dense table + */ + @Override + public void prepareForParallelPopulation(RowSet changedIndices) { + final long currentStep = LogicalClock.DEFAULT.currentStep(); + if (ensurePreviousClockCycle == currentStep) { + throw new IllegalStateException("May not call ensurePrevious twice on one clock cycle!"); + } + ensurePreviousClockCycle = currentStep; + + if (changedIndices.isEmpty()) { + return; + } + + // ensure that this source will have sufficient capacity to store these indices, does not need to be + // null-filled as the values will be immediately written + ensureCapacity(changedIndices.lastRowKey() + 1, false); + + if (prevFlusher != null) { + prevFlusher.maybeActivate(); + } else { + // we are not tracking this source yet so we have nothing to do for the previous values + return; + } + + try (final RowSequence.Iterator it = changedIndices.getRowSequenceIterator()) { + do { + final long firstKey = it.peekNextKey(); + + final int block = (int) (firstKey >> LOG_BLOCK_SIZE); + + final long[] inUse; + if (prevBlocks[block] == null) { + prevBlocks[block] = recycler.borrowItem(); + prevInUse[block] = inUse = inUseRecycler.borrowItem(); + if (prevAllocated == null) { + prevAllocated = new TIntArrayList(); + } + prevAllocated.add(block); + } else { + inUse = prevInUse[block]; + } + + final long maxKeyInCurrentBlock = firstKey | INDEX_MASK; + + it.getNextRowSequenceThrough(maxKeyInCurrentBlock).forAllRowKeys(key -> { + final int nextIndexWithinBlock = (int) (key & INDEX_MASK); + final int nextIndexWithinInUse = nextIndexWithinBlock >> LOG_INUSE_BITSET_SIZE; + final long nextMaskWithinInUse = 1L << (nextIndexWithinBlock & IN_USE_MASK); + prevBlocks[block][nextIndexWithinBlock] = blocks[block][nextIndexWithinBlock]; + inUse[nextIndexWithinInUse] |= nextMaskWithinInUse; + }); + } while (it.hasMore()); + } + } + @Override public final void set(long key, Integer value) { set(key, unbox(value)); @@ -348,9 +413,9 @@ void fillFromChunkByRanges(@NotNull RowSequence rowSequence, Chunk chunk = src.asIntChunk(); final LongChunk ranges = rowSequence.asRowKeyRangesChunk(); - final boolean hasPrev = prevFlusher != null; + final boolean trackPrevious = prevFlusher != null && ensurePreviousClockCycle != LogicalClock.DEFAULT.currentStep(); - if (hasPrev) { + if (trackPrevious) { prevFlusher.maybeActivate(); } @@ -376,7 +441,7 @@ void fillFromChunkByRanges(@NotNull RowSequence rowSequence, Chunk chunk = src.asIntChunk(); final LongChunk keys = rowSequence.asRowKeyChunk(); - final boolean hasPrev = prevFlusher != null; + final boolean trackPrevious = prevFlusher != null && ensurePreviousClockCycle != LogicalClock.DEFAULT.currentStep(); - if (hasPrev) { + if (trackPrevious) { prevFlusher.maybeActivate(); } @@ -448,7 +513,7 @@ void fillFromChunkByKeys(@NotNull RowSequence rowSequence, Chunk chunk = src.asIntChunk(); - final boolean hasPrev = prevFlusher != null; + final boolean trackPrevious = prevFlusher != null && ensurePreviousClockCycle != LogicalClock.DEFAULT.currentStep(); - if (hasPrev) { + if (trackPrevious) { prevFlusher.maybeActivate(); } @@ -488,7 +553,7 @@ public void fillFromChunkUnordered(@NotNull FillFromContext context, @NotNull Ch do { final int indexWithinBlock = (int) (key & INDEX_MASK); - if (hasPrev) { + if (trackPrevious) { if (shouldRecordPrevious(key, prevBlocks, recycler)) { prevBlocks[block][indexWithinBlock] = inner[indexWithinBlock]; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ObjectArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ObjectArraySource.java index 2d9099368d4..3dc01f0faf5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ObjectArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ObjectArraySource.java @@ -3,11 +3,14 @@ */ package io.deephaven.engine.table.impl.sources; +import gnu.trove.list.array.TIntArrayList; import io.deephaven.base.verify.Assert; +import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.impl.MutableColumnSourceGetDefaults; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeyRanges; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.updategraph.LogicalClock; import io.deephaven.vector.Vector; import io.deephaven.chunk.*; import io.deephaven.engine.rowset.chunkattributes.RowKeys; @@ -56,6 +59,66 @@ public void ensureCapacity(long capacity, boolean nullFill) { ensureCapacity(capacity, blocks, prevBlocks, nullFill); } + /** + * This version of `prepareForParallelPopulation` will internally call {@link #ensureCapacity(long, boolean)} to + * make sure there is room for the incoming values. + * + * @param changedIndices indices in the dense table + */ + @Override + public void prepareForParallelPopulation(RowSet changedIndices) { + final long currentStep = LogicalClock.DEFAULT.currentStep(); + if (ensurePreviousClockCycle == currentStep) { + throw new IllegalStateException("May not call ensurePrevious twice on one clock cycle!"); + } + ensurePreviousClockCycle = currentStep; + + if (changedIndices.isEmpty()) { + return; + } + + // ensure that this source will have sufficient capacity to store these indices, does not need to be + // null-filled as the values will be immediately written + ensureCapacity(changedIndices.lastRowKey() + 1, false); + + if (prevFlusher != null) { + prevFlusher.maybeActivate(); + } else { + // we are not tracking this source yet so we have nothing to do for the previous values + return; + } + + try (final RowSequence.Iterator it = changedIndices.getRowSequenceIterator()) { + do { + final long firstKey = it.peekNextKey(); + + final int block = (int) (firstKey >> LOG_BLOCK_SIZE); + + final long[] inUse; + if (prevBlocks[block] == null) { + prevBlocks[block] = (T[]) recycler.borrowItem(); + prevInUse[block] = inUse = inUseRecycler.borrowItem(); + if (prevAllocated == null) { + prevAllocated = new TIntArrayList(); + } + prevAllocated.add(block); + } else { + inUse = prevInUse[block]; + } + + final long maxKeyInCurrentBlock = firstKey | INDEX_MASK; + + it.getNextRowSequenceThrough(maxKeyInCurrentBlock).forAllRowKeys(key -> { + final int nextIndexWithinBlock = (int) (key & INDEX_MASK); + final int nextIndexWithinInUse = nextIndexWithinBlock >> LOG_INUSE_BITSET_SIZE; + final long nextMaskWithinInUse = 1L << (nextIndexWithinBlock & IN_USE_MASK); + prevBlocks[block][nextIndexWithinBlock] = blocks[block][nextIndexWithinBlock]; + inUse[nextIndexWithinInUse] |= nextMaskWithinInUse; + }); + } while (it.hasMore()); + } + } + @Override public void setNull(long key) { set(key, null); @@ -280,9 +343,9 @@ void fillFromChunkByRanges(@NotNull RowSequence rowSequence, Chunk chunk = src.asObjectChunk(); final LongChunk ranges = rowSequence.asRowKeyRangesChunk(); - final boolean hasPrev = prevFlusher != null; + final boolean trackPrevious = prevFlusher != null && ensurePreviousClockCycle != LogicalClock.DEFAULT.currentStep(); - if (hasPrev) { + if (trackPrevious) { prevFlusher.maybeActivate(); } @@ -308,7 +371,7 @@ void fillFromChunkByRanges(@NotNull RowSequence rowSequence, Chunk chunk = src.asObjectChunk(); final LongChunk keys = rowSequence.asRowKeyChunk(); - final boolean hasPrev = prevFlusher != null; + final boolean trackPrevious = prevFlusher != null && ensurePreviousClockCycle != LogicalClock.DEFAULT.currentStep(); - if (hasPrev) { + if (trackPrevious) { prevFlusher.maybeActivate(); } @@ -380,7 +443,7 @@ void fillFromChunkByKeys(@NotNull RowSequence rowSequence, Chunk src, @NotNull LongChunk keys) { final ObjectChunk chunk = src.asObjectChunk(); - final boolean hasPrev = prevFlusher != null; + final boolean trackPrevious = prevFlusher != null && ensurePreviousClockCycle != LogicalClock.DEFAULT.currentStep(); - if (hasPrev) { + if (trackPrevious) { prevFlusher.maybeActivate(); } @@ -417,7 +480,7 @@ public void fillFromChunkUnordered(@NotNull FillFromContext context, @NotNull Ch do { final int indexWithinBlock = (int) (key & INDEX_MASK); - if (hasPrev) { + if (trackPrevious) { if (shouldRecordPrevious(key, prevBlocks, recycler)) { prevBlocks[block][indexWithinBlock] = inner[indexWithinBlock]; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ShortArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ShortArraySource.java index 2af5d2cd3f6..84701305262 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ShortArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ShortArraySource.java @@ -8,13 +8,16 @@ */ package io.deephaven.engine.table.impl.sources; +import gnu.trove.list.array.TIntArrayList; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.impl.MutableColumnSourceGetDefaults; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeyRanges; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.updategraph.LogicalClock; import io.deephaven.util.compare.ShortComparisons; import io.deephaven.chunk.*; import io.deephaven.chunk.ResettableWritableChunk; @@ -27,6 +30,8 @@ import java.util.Arrays; +import static io.deephaven.engine.table.impl.sources.sparse.SparseConstants.*; +import static io.deephaven.engine.table.impl.sources.sparse.SparseConstants.BLOCK2_MASK; import static io.deephaven.util.QueryConstants.NULL_SHORT; import static io.deephaven.util.type.TypeUtils.box; import static io.deephaven.util.type.TypeUtils.unbox; @@ -63,6 +68,66 @@ public void ensureCapacity(long capacity, boolean nullFill) { ensureCapacity(capacity, blocks, prevBlocks, nullFill); } + /** + * This version of `prepareForParallelPopulation` will internally call {@link #ensureCapacity(long, boolean)} to + * make sure there is room for the incoming values. + * + * @param changedIndices indices in the dense table + */ + @Override + public void prepareForParallelPopulation(RowSet changedIndices) { + final long currentStep = LogicalClock.DEFAULT.currentStep(); + if (ensurePreviousClockCycle == currentStep) { + throw new IllegalStateException("May not call ensurePrevious twice on one clock cycle!"); + } + ensurePreviousClockCycle = currentStep; + + if (changedIndices.isEmpty()) { + return; + } + + // ensure that this source will have sufficient capacity to store these indices, does not need to be + // null-filled as the values will be immediately written + ensureCapacity(changedIndices.lastRowKey() + 1, false); + + if (prevFlusher != null) { + prevFlusher.maybeActivate(); + } else { + // we are not tracking this source yet so we have nothing to do for the previous values + return; + } + + try (final RowSequence.Iterator it = changedIndices.getRowSequenceIterator()) { + do { + final long firstKey = it.peekNextKey(); + + final int block = (int) (firstKey >> LOG_BLOCK_SIZE); + + final long[] inUse; + if (prevBlocks[block] == null) { + prevBlocks[block] = recycler.borrowItem(); + prevInUse[block] = inUse = inUseRecycler.borrowItem(); + if (prevAllocated == null) { + prevAllocated = new TIntArrayList(); + } + prevAllocated.add(block); + } else { + inUse = prevInUse[block]; + } + + final long maxKeyInCurrentBlock = firstKey | INDEX_MASK; + + it.getNextRowSequenceThrough(maxKeyInCurrentBlock).forAllRowKeys(key -> { + final int nextIndexWithinBlock = (int) (key & INDEX_MASK); + final int nextIndexWithinInUse = nextIndexWithinBlock >> LOG_INUSE_BITSET_SIZE; + final long nextMaskWithinInUse = 1L << (nextIndexWithinBlock & IN_USE_MASK); + prevBlocks[block][nextIndexWithinBlock] = blocks[block][nextIndexWithinBlock]; + inUse[nextIndexWithinInUse] |= nextMaskWithinInUse; + }); + } while (it.hasMore()); + } + } + @Override public final void set(long key, Short value) { set(key, unbox(value)); @@ -348,9 +413,9 @@ void fillFromChunkByRanges(@NotNull RowSequence rowSequence, Chunk chunk = src.asShortChunk(); final LongChunk ranges = rowSequence.asRowKeyRangesChunk(); - final boolean hasPrev = prevFlusher != null; + final boolean trackPrevious = prevFlusher != null && ensurePreviousClockCycle != LogicalClock.DEFAULT.currentStep(); - if (hasPrev) { + if (trackPrevious) { prevFlusher.maybeActivate(); } @@ -376,7 +441,7 @@ void fillFromChunkByRanges(@NotNull RowSequence rowSequence, Chunk chunk = src.asShortChunk(); final LongChunk keys = rowSequence.asRowKeyChunk(); - final boolean hasPrev = prevFlusher != null; + final boolean trackPrevious = prevFlusher != null && ensurePreviousClockCycle != LogicalClock.DEFAULT.currentStep(); - if (hasPrev) { + if (trackPrevious) { prevFlusher.maybeActivate(); } @@ -448,7 +513,7 @@ void fillFromChunkByKeys(@NotNull RowSequence rowSequence, Chunk chunk = src.asShortChunk(); - final boolean hasPrev = prevFlusher != null; + final boolean trackPrevious = prevFlusher != null && ensurePreviousClockCycle != LogicalClock.DEFAULT.currentStep(); - if (hasPrev) { + if (trackPrevious) { prevFlusher.maybeActivate(); } @@ -488,7 +553,7 @@ public void fillFromChunkUnordered(@NotNull FillFromContext context, @NotNull Ch do { final int indexWithinBlock = (int) (key & INDEX_MASK); - if (hasPrev) { + if (trackPrevious) { if (shouldRecordPrevious(key, prevBlocks, recycler)) { prevBlocks[block][indexWithinBlock] = inner[indexWithinBlock]; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java index 8d088273209..9ce76e2c256 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java @@ -10,6 +10,7 @@ import io.deephaven.engine.table.impl.sources.ByteSparseArraySource; import io.deephaven.engine.table.WritableColumnSource; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableByteChunk; @@ -17,10 +18,7 @@ import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.table.ChunkSink; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.WritableColumnSource; +import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; @@ -180,9 +178,10 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d @Override public void prepareForParallelPopulation(final RowSet added) { - // we don't need to do anything for redirected, that happened earlier - if (!redirContext.isRedirected()) { - ((SparseArrayColumnSource) outputSource).prepareForParallelPopulation(added); + if (redirContext.isRedirected()) { + ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); + } else { + ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java index ad7e57221ff..a32860545d7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java @@ -1,5 +1,6 @@ package io.deephaven.engine.table.impl.updateby.internal; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableCharChunk; @@ -7,10 +8,7 @@ import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.table.ChunkSink; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.WritableColumnSource; +import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; @@ -152,9 +150,10 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d @Override public void prepareForParallelPopulation(final RowSet added) { - // we don't need to do anything for redirected, that happened earlier - if (!redirContext.isRedirected()) { - ((SparseArrayColumnSource) outputSource).prepareForParallelPopulation(added); + if (redirContext.isRedirected()) { + ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); + } else { + ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java index f0057007d0c..84c3f829b7a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java @@ -156,9 +156,10 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long @Override public void prepareForParallelPopulation(final RowSet added) { - // we don't need to do anything for redirected, that happened earlier - if (!redirContext.isRedirected()) { - ((SparseArrayColumnSource) outputSource).prepareForParallelPopulation(added); + if (redirContext.isRedirected()) { + ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); + } else { + ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java index 3d836837494..94ada3c63ea 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java @@ -151,9 +151,10 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long @Override public void prepareForParallelPopulation(final RowSet added) { - // we don't need to do anything for redirected, that happened earlier - if (!redirContext.isRedirected()) { - ((SparseArrayColumnSource) outputSource).prepareForParallelPopulation(added); + if (redirContext.isRedirected()) { + ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); + } else { + ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java index 47da06915a3..9722bcd9180 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java @@ -5,6 +5,7 @@ */ package io.deephaven.engine.table.impl.updateby.internal; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableIntChunk; @@ -12,10 +13,7 @@ import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.table.ChunkSink; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.WritableColumnSource; +import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; @@ -157,9 +155,10 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d @Override public void prepareForParallelPopulation(final RowSet added) { - // we don't need to do anything for redirected, that happened earlier - if (!redirContext.isRedirected()) { - ((SparseArrayColumnSource) outputSource).prepareForParallelPopulation(added); + if (redirContext.isRedirected()) { + ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); + } else { + ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java index a0b062a8aab..9c514fa7e4c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java @@ -5,6 +5,7 @@ */ package io.deephaven.engine.table.impl.updateby.internal; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableLongChunk; @@ -12,10 +13,7 @@ import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.table.ChunkSink; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.WritableColumnSource; +import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; @@ -157,9 +155,10 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d @Override public void prepareForParallelPopulation(final RowSet added) { - // we don't need to do anything for redirected, that happened earlier - if (!redirContext.isRedirected()) { - ((SparseArrayColumnSource) outputSource).prepareForParallelPopulation(added); + if (redirContext.isRedirected()) { + ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); + } else { + ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java index 73b6bac35af..d1ebb6da64d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java @@ -5,6 +5,7 @@ */ package io.deephaven.engine.table.impl.updateby.internal; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableObjectChunk; @@ -12,10 +13,7 @@ import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.table.ChunkSink; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.WritableColumnSource; +import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; @@ -160,9 +158,10 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d @Override public void prepareForParallelPopulation(final RowSet added) { - // we don't need to do anything for redirected, that happened earlier - if (!redirContext.isRedirected()) { - ((SparseArrayColumnSource) outputSource).prepareForParallelPopulation(added); + if (redirContext.isRedirected()) { + ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); + } else { + ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java index a10e5b0da0e..3c9cf19f648 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java @@ -5,6 +5,7 @@ */ package io.deephaven.engine.table.impl.updateby.internal; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableShortChunk; @@ -12,10 +13,7 @@ import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.table.ChunkSink; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.WritableColumnSource; +import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; @@ -157,9 +155,10 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d @Override public void prepareForParallelPopulation(final RowSet added) { - // we don't need to do anything for redirected, that happened earlier - if (!redirContext.isRedirected()) { - ((SparseArrayColumnSource) outputSource).prepareForParallelPopulation(added); + if (redirContext.isRedirected()) { + ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); + } else { + ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java index b132623ea50..6761eb627f6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java @@ -184,9 +184,10 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d @Override public void prepareForParallelPopulation(final RowSet added) { - // we don't need to do anything for redirected, that happened earlier - if (!redirContext.isRedirected()) { - ((SparseArrayColumnSource) outputSource).prepareForParallelPopulation(added); + if (redirContext.isRedirected()) { + ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); + } else { + ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java index ce3b65b5b7b..c23a9505c1b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java @@ -156,9 +156,10 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d @Override public void prepareForParallelPopulation(final RowSet added) { - // we don't need to do anything for redirected, that happened earlier - if (!redirContext.isRedirected()) { - ((SparseArrayColumnSource) outputSource).prepareForParallelPopulation(added); + if (redirContext.isRedirected()) { + ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); + } else { + ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java index 41f706146b8..d306f4eb76e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java @@ -159,12 +159,12 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d } // endregion Shifts - @Override public void prepareForParallelPopulation(final RowSet added) { - // we don't need to do anything for redirected, that happened earlier - if (!redirContext.isRedirected()) { - ((SparseArrayColumnSource) outputSource).prepareForParallelPopulation(added); + if (redirContext.isRedirected()) { + ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); + } else { + ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java index 7da492a15db..60c2ff27d77 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java @@ -154,12 +154,12 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d } // endregion Shifts - @Override public void prepareForParallelPopulation(final RowSet added) { - // we don't need to do anything for redirected, that happened earlier - if (!redirContext.isRedirected()) { - ((SparseArrayColumnSource) outputSource).prepareForParallelPopulation(added); + if (redirContext.isRedirected()) { + ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); + } else { + ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java index 9fd339b2675..850b72ac209 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java @@ -161,9 +161,10 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d @Override public void prepareForParallelPopulation(final RowSet added) { - // we don't need to do anything for redirected, that happened earlier - if (!redirContext.isRedirected()) { - ((SparseArrayColumnSource) outputSource).prepareForParallelPopulation(added); + if (redirContext.isRedirected()) { + ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); + } else { + ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java index ac653e743fe..8a01898cdf4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java @@ -161,9 +161,10 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d @Override public void prepareForParallelPopulation(final RowSet added) { - // we don't need to do anything for redirected, that happened earlier - if (!redirContext.isRedirected()) { - ((SparseArrayColumnSource) outputSource).prepareForParallelPopulation(added); + if (redirContext.isRedirected()) { + ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); + } else { + ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java index 7feadf83774..2ba01f20634 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java @@ -164,9 +164,10 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d @Override public void prepareForParallelPopulation(final RowSet added) { - // we don't need to do anything for redirected, that happened earlier - if (!redirContext.isRedirected()) { - ((SparseArrayColumnSource) outputSource).prepareForParallelPopulation(added); + if (redirContext.isRedirected()) { + ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); + } else { + ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java index e651c8de26e..53cdc374415 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java @@ -161,9 +161,10 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d @Override public void prepareForParallelPopulation(final RowSet added) { - // we don't need to do anything for redirected, that happened earlier - if (!redirContext.isRedirected()) { - ((SparseArrayColumnSource) outputSource).prepareForParallelPopulation(added); + if (redirContext.isRedirected()) { + ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); + } else { + ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); } } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java index 3736d7cf4b9..519a296cbac 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java @@ -46,8 +46,6 @@ public void testMixedAppendOnlyBucketed() { @Test public void testMixedGeneralZeroKey() { - doTestTicking(false, false, false, 20, 10, 12); - for (int size = 10; size <= 10000; size *= 10) { for (int seed = 10; seed < 20; seed++) { doTestTicking(seed > 15, false, false, 20, size, seed); @@ -89,7 +87,6 @@ protected Table e() { } final String[] columnNamesArray = base.getDefinition().getColumnNamesArray(); - // NOTE: I can't include the float/double based Rolling final Collection clauses = List.of( UpdateByOperation.Fill(), UpdateByOperation.RollingSum(100, 0, diff --git a/replication/static/src/main/java/io/deephaven/replicators/ReplicateSourcesAndChunks.java b/replication/static/src/main/java/io/deephaven/replicators/ReplicateSourcesAndChunks.java index f24dc19296d..05685fc5452 100644 --- a/replication/static/src/main/java/io/deephaven/replicators/ReplicateSourcesAndChunks.java +++ b/replication/static/src/main/java/io/deephaven/replicators/ReplicateSourcesAndChunks.java @@ -639,7 +639,8 @@ private static void replicateSparseBooleanSource() throws IOException { "import io.deephaven.engine.table.impl.AbstractColumnSource;", "import io.deephaven.engine.table.WritableColumnSource;", "import io.deephaven.util.BooleanUtils;", - "import static io.deephaven.util.BooleanUtils.NULL_BOOLEAN_AS_BYTE;"); + "import static io.deephaven.util.BooleanUtils.NULL_BOOLEAN_AS_BYTE;", + "import io.deephaven.engine.table.WritableSourceWithPrepareForParallelPopulation;"); lines = globalReplacements(lines, "BooleanOneOrN", "ByteOneOrN"); lines = globalReplacements(lines, "WritableBooleanChunk", "WritableObjectChunk", "asBooleanChunk", "asObjectChunk", @@ -710,7 +711,7 @@ private static void replicateSparseBooleanSource() throws IOException { " return (ColumnSource) new BooleanSparseArraySource.ReinterpretedAsByte(this);", " }", "", - " public static class ReinterpretedAsByte extends AbstractColumnSource implements MutableColumnSourceGetDefaults.ForByte, FillUnordered, WritableColumnSource {", + " public static class ReinterpretedAsByte extends AbstractColumnSource implements MutableColumnSourceGetDefaults.ForByte, FillUnordered, WritableColumnSource, WritableSourceWithPrepareForParallelPopulation {", " private final BooleanSparseArraySource wrapped;", "", " private ReinterpretedAsByte(BooleanSparseArraySource wrapped) {", @@ -916,9 +917,9 @@ private static void replicateSparseBooleanSource() throws IOException { " final ByteChunk chunk = src.asByteChunk();", " final LongChunk keys = RowSequence.asRowKeyChunk();", "", - " final boolean hasPrev = wrapped.prevFlusher != null;", + " final boolean trackPrevious = wrapped.prevFlusher != null && wrapped.ensurePreviousClockCycle != LogicalClock.DEFAULT.currentStep();", "", - " if (hasPrev) {", + " if (trackPrevious) {", " wrapped.prevFlusher.maybeActivate();", " }", "", @@ -940,13 +941,13 @@ private static void replicateSparseBooleanSource() throws IOException { " }", "", " // This conditional with its constant condition should be very friendly to the branch predictor.", - " final byte[] prevBlock = hasPrev ? wrapped.ensurePrevBlock(firstRowKey, block0, block1, block2) : null;", - " final long[] inUse = hasPrev ? wrapped.prevInUse.get(block0).get(block1).get(block2) : null;", + " final byte[] prevBlock = trackPrevious ? wrapped.ensurePrevBlock(firstRowKey, block0, block1, block2) : null;", + " final long[] inUse = trackPrevious ? wrapped.prevInUse.get(block0).get(block1).get(block2) : null;", "", " while (ii <= lastII) {", " final int indexWithinBlock = (int) (keys.get(ii) & INDEX_MASK);", " // This 'if' with its constant condition should be very friendly to the branch predictor.", - " if (hasPrev) {", + " if (trackPrevious) {", " assert inUse != null;", " assert prevBlock != null;", "", @@ -963,6 +964,11 @@ private static void replicateSparseBooleanSource() throws IOException { " }", " }", " }", + "", + " @Override", + " public void prepareForParallelPopulation(RowSet rowSet) {", + " wrapped.prepareForParallelPopulation(rowSet);", + " }", " }")); FileUtils.writeLines(booleanFile, lines); } From 301266d8bd53b07a700b302e6a5bdae9bda0368b Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 18 Nov 2022 09:22:32 -0800 Subject: [PATCH 043/123] Refactored JobScheduler as a top-level interface and the impl's as top-level classes --- .../engine/table/impl/QueryTable.java | 12 +- .../table/impl/SelectOrUpdateListener.java | 15 +- .../deephaven/engine/table/impl/UpdateBy.java | 17 +- .../impl/select/analyzers/BaseLayer.java | 1 + .../select/analyzers/PreserveColumnLayer.java | 1 + .../select/analyzers/RedirectionLayer.java | 1 + .../analyzers/SelectAndViewAnalyzer.java | 365 +----------------- .../select/analyzers/SelectColumnLayer.java | 5 +- .../select/analyzers/StaticFlattenLayer.java | 1 + .../select/analyzers/ViewColumnLayer.java | 1 + .../impl/util/ImmediateJobScheduler.java | 39 ++ .../engine/table/impl/util/JobScheduler.java | 232 +++++++++++ ...erationInitializationPoolJobScheduler.java | 50 +++ .../UpdateGraphProcessorJobScheduler.java | 70 ++++ 14 files changed, 426 insertions(+), 384 deletions(-) create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/util/ImmediateJobScheduler.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/util/OperationInitializationPoolJobScheduler.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/util/UpdateGraphProcessorJobScheduler.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java index 19b34e00092..61746e1849f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java @@ -33,8 +33,11 @@ import io.deephaven.engine.table.impl.perf.QueryPerformanceNugget; import io.deephaven.engine.table.impl.select.MatchPairFactory; import io.deephaven.engine.table.impl.select.SelectColumnFactory; +import io.deephaven.engine.table.impl.util.ImmediateJobScheduler; +import io.deephaven.engine.table.impl.util.JobScheduler; +import io.deephaven.engine.table.impl.util.OperationInitializationPoolJobScheduler; import io.deephaven.engine.updategraph.DynamicNode; -import io.deephaven.engine.util.ColumnFormattingValues; +import io.deephaven.engine.util.*; import io.deephaven.engine.util.systemicmarking.SystemicObject; import io.deephaven.qst.table.AggregateAllByTable; import io.deephaven.vector.Vector; @@ -43,7 +46,6 @@ import io.deephaven.time.DateTime; import io.deephaven.engine.table.impl.perf.QueryPerformanceRecorder; import io.deephaven.engine.util.systemicmarking.SystemicObjectTracker; -import io.deephaven.engine.util.IterableUtils; import io.deephaven.engine.liveness.Liveness; import io.deephaven.engine.liveness.LivenessReferent; import io.deephaven.engine.table.impl.MemoizedOperationKey.SelectUpdateViewOrUpdateView.Flavor; @@ -1263,15 +1265,15 @@ private Table selectOrUpdate(Flavor flavor, final SelectColumn... selectColumns) RowSetShiftData.EMPTY, ModifiedColumnSet.ALL); final CompletableFuture waitForResult = new CompletableFuture<>(); - final SelectAndViewAnalyzer.JobScheduler jobScheduler; + final JobScheduler jobScheduler; if ((QueryTable.FORCE_PARALLEL_SELECT_AND_UPDATE || (QueryTable.ENABLE_PARALLEL_SELECT_AND_UPDATE && OperationInitializationThreadPool.NUM_THREADS > 1)) && !OperationInitializationThreadPool.isInitializationThread() && analyzer.allowCrossColumnParallelization()) { - jobScheduler = new SelectAndViewAnalyzer.OperationInitializationPoolJobScheduler(); + jobScheduler = new OperationInitializationPoolJobScheduler(); } else { - jobScheduler = SelectAndViewAnalyzer.ImmediateJobScheduler.INSTANCE; + jobScheduler = ImmediateJobScheduler.INSTANCE; } final QueryTable resultTable; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/SelectOrUpdateListener.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/SelectOrUpdateListener.java index 6f240850b97..5d75546a07f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/SelectOrUpdateListener.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/SelectOrUpdateListener.java @@ -3,19 +3,18 @@ */ package io.deephaven.engine.table.impl; -import io.deephaven.engine.exceptions.UncheckedTableException; import io.deephaven.engine.rowset.TrackingRowSet; import io.deephaven.engine.rowset.WritableRowSet; import io.deephaven.engine.table.ModifiedColumnSet; import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.impl.perf.BasePerformanceEntry; import io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer; -import io.deephaven.engine.table.impl.util.AsyncClientErrorNotifier; import io.deephaven.engine.updategraph.TerminalNotification; import io.deephaven.engine.updategraph.UpdateGraphProcessor; -import io.deephaven.engine.util.systemicmarking.SystemicObjectTracker; +import io.deephaven.engine.table.impl.util.ImmediateJobScheduler; +import io.deephaven.engine.table.impl.util.JobScheduler; +import io.deephaven.engine.table.impl.util.UpdateGraphProcessorJobScheduler; -import java.io.IOException; import java.util.BitSet; import java.util.Map; @@ -79,12 +78,12 @@ public void onUpdate(final TableUpdate upstream) { final SelectAndViewAnalyzer.UpdateHelper updateHelper = new SelectAndViewAnalyzer.UpdateHelper(resultRowSet, acquiredUpdate); toClear.remove(resultRowSet); - SelectAndViewAnalyzer.JobScheduler jobScheduler; + JobScheduler jobScheduler; if (enableParallelUpdate) { - jobScheduler = new SelectAndViewAnalyzer.UpdateGraphProcessorJobScheduler(); + jobScheduler = new UpdateGraphProcessorJobScheduler(); } else { - jobScheduler = SelectAndViewAnalyzer.ImmediateJobScheduler.INSTANCE; + jobScheduler = ImmediateJobScheduler.INSTANCE; } analyzer.applyUpdate(acquiredUpdate, toClear, updateHelper, jobScheduler, this, @@ -106,7 +105,7 @@ private void handleException(Exception e) { updateInProgress = false; } - private void completionRoutine(TableUpdate upstream, SelectAndViewAnalyzer.JobScheduler jobScheduler, + private void completionRoutine(TableUpdate upstream, JobScheduler jobScheduler, WritableRowSet toClear, SelectAndViewAnalyzer.UpdateHelper updateHelper) { final TableUpdateImpl downstream = new TableUpdateImpl(upstream.added().copy(), upstream.removed().copy(), upstream.modified().copy(), upstream.shifted(), dependent.getModifiedColumnSetForUpdates()); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java index c584d1af999..db8a31e77b2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java @@ -21,7 +21,6 @@ import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer; import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.sources.sparse.SparseConstants; import io.deephaven.engine.table.impl.updateby.UpdateByWindow; @@ -29,6 +28,10 @@ import io.deephaven.engine.table.impl.util.LongColumnSourceWritableRowRedirection; import io.deephaven.engine.table.impl.util.WritableRowRedirection; import io.deephaven.engine.updategraph.UpdateGraphProcessor; +import io.deephaven.engine.table.impl.util.ImmediateJobScheduler; +import io.deephaven.engine.table.impl.util.JobScheduler; +import io.deephaven.engine.table.impl.util.OperationInitializationPoolJobScheduler; +import io.deephaven.engine.table.impl.util.UpdateGraphProcessorJobScheduler; import org.apache.commons.lang3.mutable.MutableInt; import org.apache.commons.lang3.mutable.MutableLong; import org.jetbrains.annotations.NotNull; @@ -309,7 +312,7 @@ protected class StateManager implements LogOutputAppendable { final TrackingWritableRowSet[] inputSourceRowSets; final AtomicInteger[] inputSourceReferenceCounts; - final SelectAndViewAnalyzer.JobScheduler jobScheduler; + final JobScheduler jobScheduler; final CompletableFuture waitForResult; final SharedContext sharedContext; @@ -335,16 +338,16 @@ public StateManager(TableUpdate update, boolean initialStep) { if (initialStep) { if (OperationInitializationThreadPool.NUM_THREADS > 1) { - jobScheduler = new SelectAndViewAnalyzer.OperationInitializationPoolJobScheduler(); + jobScheduler = new OperationInitializationPoolJobScheduler(); } else { - jobScheduler = SelectAndViewAnalyzer.ImmediateJobScheduler.INSTANCE; + jobScheduler = ImmediateJobScheduler.INSTANCE; } waitForResult = new CompletableFuture<>(); } else { if (UpdateGraphProcessor.DEFAULT.getUpdateThreads() > 1) { - jobScheduler = new SelectAndViewAnalyzer.UpdateGraphProcessorJobScheduler(); + jobScheduler = new UpdateGraphProcessorJobScheduler(); } else { - jobScheduler = SelectAndViewAnalyzer.ImmediateJobScheduler.INSTANCE; + jobScheduler = ImmediateJobScheduler.INSTANCE; } waitForResult = null; } @@ -704,7 +707,7 @@ private TableUpdate computeDownstreamUpdate() { /** * Process the {@link TableUpdate update} provided in the constructor. This performs much work in parallel and - * leverages {@link io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer.JobScheduler} + * leverages {@link JobScheduler} * extensively */ public void processUpdate() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/BaseLayer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/BaseLayer.java index 4754d7d9b4d..d9d90942b48 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/BaseLayer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/BaseLayer.java @@ -11,6 +11,7 @@ import io.deephaven.engine.table.ModifiedColumnSet; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.table.impl.util.JobScheduler; import org.jetbrains.annotations.Nullable; import java.util.*; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/PreserveColumnLayer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/PreserveColumnLayer.java index 9f43e5364f3..8fe1841b205 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/PreserveColumnLayer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/PreserveColumnLayer.java @@ -10,6 +10,7 @@ import io.deephaven.engine.table.impl.select.SelectColumn; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.table.impl.util.JobScheduler; import org.jetbrains.annotations.Nullable; import java.util.Arrays; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/RedirectionLayer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/RedirectionLayer.java index 362d8644577..defbae50046 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/RedirectionLayer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/RedirectionLayer.java @@ -13,6 +13,7 @@ import io.deephaven.engine.table.ModifiedColumnSet; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.impl.util.*; +import io.deephaven.engine.table.impl.util.JobScheduler; import org.apache.commons.lang3.mutable.MutableLong; import org.apache.commons.lang3.mutable.MutableObject; import org.jetbrains.annotations.Nullable; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer.java index 46ed0a686a7..5c870dc1d1b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer.java @@ -3,7 +3,6 @@ */ package io.deephaven.engine.table.impl.select.analyzers; -import io.deephaven.base.log.LogOutput; import io.deephaven.base.log.LogOutputAppendable; import io.deephaven.datastructures.util.CollectionUtil; import io.deephaven.engine.rowset.RowSetFactory; @@ -15,29 +14,21 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.TrackingRowSet; import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.OperationInitializationThreadPool; -import io.deephaven.engine.table.impl.perf.BasePerformanceEntry; import io.deephaven.engine.table.impl.select.SelectColumn; import io.deephaven.engine.table.impl.select.SourceColumn; import io.deephaven.engine.table.impl.select.SwitchColumn; import io.deephaven.engine.table.impl.sources.InMemoryColumnSource; import io.deephaven.engine.table.impl.sources.WritableRedirectedColumnSource; import io.deephaven.engine.table.impl.util.WritableRowRedirection; -import io.deephaven.engine.updategraph.AbstractNotification; -import io.deephaven.engine.updategraph.UpdateGraphProcessor; -import io.deephaven.engine.context.ExecutionContext; +import io.deephaven.engine.table.impl.util.JobScheduler; import io.deephaven.io.log.impl.LogOutputStringImpl; import io.deephaven.util.SafeCloseable; import io.deephaven.util.SafeCloseablePair; -import io.deephaven.util.annotations.FinalDefault; -import io.deephaven.util.process.ProcessEnvironment; import io.deephaven.vector.Vector; import org.jetbrains.annotations.Nullable; import java.util.*; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Consumer; import java.util.stream.Stream; public abstract class SelectAndViewAnalyzer implements LogOutputAppendable { @@ -349,8 +340,8 @@ public void close() { * @param onCompletion Called when an inner column is complete. The outer layer should pass the {@code onCompletion} */ public abstract void applyUpdate(TableUpdate upstream, RowSet toClear, UpdateHelper helper, - JobScheduler jobScheduler, @Nullable LivenessNode liveResultOwner, - SelectLayerCompletionHandler onCompletion); + JobScheduler jobScheduler, @Nullable LivenessNode liveResultOwner, + SelectLayerCompletionHandler onCompletion); /** * Our job here is to calculate the effects: a map from incoming column to a list of columns that it effects. We do @@ -504,356 +495,6 @@ protected void onError(Exception error) { protected abstract void onAllRequiredColumnsCompleted(); } - /** - * An interface for submitting jobs to be executed and accumulating their performance of all the tasks performed off - * thread. - */ - public interface JobScheduler { - /** - * Cause runnable to be executed. - * - * @param executionContext the execution context to run it under - * @param runnable the runnable to execute - * @param description a description for logging - * @param onError a routine to call if an exception occurs while running runnable - */ - void submit( - ExecutionContext executionContext, - Runnable runnable, - final LogOutputAppendable description, - final Consumer onError); - - /** - * The performance statistics of all runnables that have been completed off-thread, or null if it was executed - * in the current thread. - */ - BasePerformanceEntry getAccumulatedPerformance(); - - /** - * How many threads exist in the job scheduler? The job submitters can use this value to determine how many - * sub-jobs to split work into. - */ - int threadCount(); - - /** - * Helper interface for {@code iterateSerial()} and {@code iterateParallel()}. This provides a callable - * interface with {@code index} indicating which iteration to perform. When this returns, the scheduler will - * automatically schedule the next iteration. - */ - @FunctionalInterface - interface IterateAction { - void run(int index); - } - - /** - * Helper interface for {@code iterateSerial()} and {@code iterateParallel()}. This provides a callable - * interface with {@code index} indicating which iteration to perform and {@link Runnable resume} providing a - * mechanism to inform the scheduler that the current task is complete. When {@code resume} is called, the - * scheduler will automatically schedule the next iteration. - * - * NOTE: failing to call {@code resume} will result in the scheduler not scheduling all remaining iterations. - * This will not block the scheduler, but the {@code completeAction} {@link Runnable} will never be called - */ - @FunctionalInterface - interface IterateResumeAction { - void run(int index, Runnable resume); - } - - /** - * Provides a mechanism to iterate over a range of values in parallel using the {@link JobScheduler} - * - * @param executionContext the execution context for this task - * @param description the description to use for logging - * @param start the integer value from which to start iterating - * @param count the number of times this task should be called - * @param action the task to perform, the current iteration index is provided as a parameter - * @param completeAction this will be called when all iterations are complete - * @param onError error handler for the scheduler to use while iterating - */ - @FinalDefault - default void iterateParallel(ExecutionContext executionContext, LogOutputAppendable description, int start, - int count, IterateAction action, Runnable completeAction, final Consumer onError) { - - if (count == 0) { - // no work to do - completeAction.run(); - } - - final AtomicInteger nextIndex = new AtomicInteger(start); - final AtomicInteger remaining = new AtomicInteger(count); - - final Runnable task = () -> { - // this will run until all tasks have started - while (true) { - int idx = nextIndex.getAndIncrement(); - if (idx < start + count) { - // do the work - action.run(idx); - - // check for completion - if (remaining.decrementAndGet() == 0) { - completeAction.run(); - return; - } - } else { - // no more work to do - return; - } - } - }; - - // create multiple tasks but not more than one per scheduler thread - for (int i = 0; i < Math.min(count, threadCount()); i++) { - submit(executionContext, - task, - description, - onError); - } - } - - /** - * Provides a mechanism to iterate over a range of values in parallel using the {@link JobScheduler} - * - * @param executionContext the execution context for this task - * @param description the description to use for logging - * @param start the integer value from which to start iterating - * @param count the number of times this task should be called - * @param action the task to perform, the current iteration index and a resume Runnable are parameters - * @param completeAction this will be called when all iterations are complete - * @param onError error handler for the scheduler to use while iterating - */ - @FinalDefault - default void iterateParallel(ExecutionContext executionContext, LogOutputAppendable description, int start, - int count, IterateResumeAction action, Runnable completeAction, Consumer onError) { - - if (count == 0) { - // no work to do - completeAction.run(); - } - - final AtomicInteger nextIndex = new AtomicInteger(start); - final AtomicInteger remaining = new AtomicInteger(count); - - final Runnable resumeAction = () -> { - // check for completion - if (remaining.decrementAndGet() == 0) { - completeAction.run(); - } - }; - - final Runnable task = () -> { - // this will run until all tasks have started - while (true) { - int idx = nextIndex.getAndIncrement(); - if (idx < start + count) { - // do the work - action.run(idx, resumeAction); - } else { - // no more work to do - return; - } - } - }; - - // create multiple tasks but not more than one per scheduler thread - for (int i = 0; i < Math.min(count, threadCount()); i++) { - submit(executionContext, - task, - description, - onError); - } - } - - /** - * Provides a mechanism to iterate over a range of values serially using the {@link JobScheduler}. The advantage - * to using this over a simple iteration is the resumption callable on {@code action} that will trigger the next - * iterable. This allows the next iteration to de delayed until dependendat asynchronous serial or parallel - * scheduler jobs have completed. - * - * @param executionContext the execution context for this task - * @param description the description to use for logging - * @param start the integer value from which to start iterating - * @param count the number of times this task should be called - * @param action the task to perform, the current iteration index and a resume Runnable are parameters - * @param completeAction this will be called when all iterations are complete - * @param onError error handler for the scheduler to use while iterating - */ - @FinalDefault - default void iterateSerial(ExecutionContext executionContext, LogOutputAppendable description, int start, - int count, IterateResumeAction action, Runnable completeAction, Consumer onError) { - - if (count == 0) { - // no work to do - completeAction.run(); - } - - final AtomicInteger nextIndex = new AtomicInteger(start); - final AtomicInteger remaining = new AtomicInteger(count); - - // no lambda, need the `this` reference to re-execute - final Runnable resumeAction = new Runnable() { - @Override - public void run() { - // check for completion - if (remaining.decrementAndGet() == 0) { - completeAction.run(); - } else { - // schedule the next task - submit(executionContext, - () -> { - int idx = nextIndex.getAndIncrement(); - if (idx < start + count) { - // do the work - action.run(idx, this); - } - }, - description, - onError); - } - - } - }; - - // create a single task - submit(executionContext, - () -> { - int idx = nextIndex.getAndIncrement(); - if (idx < start + count) { - action.run(idx, resumeAction); - } - }, - description, - onError); - } - } - - public static class UpdateGraphProcessorJobScheduler implements SelectAndViewAnalyzer.JobScheduler { - final BasePerformanceEntry accumulatedBaseEntry = new BasePerformanceEntry(); - - @Override - public void submit( - final ExecutionContext executionContext, - final Runnable runnable, - final LogOutputAppendable description, - final Consumer onError) { - UpdateGraphProcessor.DEFAULT.addNotification(new AbstractNotification(false) { - @Override - public boolean canExecute(long step) { - return true; - } - - @Override - public void run() { - final BasePerformanceEntry baseEntry = new BasePerformanceEntry(); - baseEntry.onBaseEntryStart(); - try { - runnable.run(); - } catch (Exception e) { - onError.accept(e); - } catch (Error e) { - ProcessEnvironment.getGlobalFatalErrorReporter().report("SelectAndView Error", e); - throw e; - } finally { - baseEntry.onBaseEntryEnd(); - synchronized (accumulatedBaseEntry) { - accumulatedBaseEntry.accumulate(baseEntry); - } - } - } - - @Override - public LogOutput append(LogOutput output) { - return output.append("{Notification(").append(System.identityHashCode(this)).append(" for ") - .append(description).append("}"); - } - - @Override - public ExecutionContext getExecutionContext() { - return executionContext; - } - }); - } - - @Override - public BasePerformanceEntry getAccumulatedPerformance() { - return accumulatedBaseEntry; - } - - @Override - public int threadCount() { - return UpdateGraphProcessor.DEFAULT.getUpdateThreads(); - } - } - - public static class OperationInitializationPoolJobScheduler implements SelectAndViewAnalyzer.JobScheduler { - final BasePerformanceEntry accumulatedBaseEntry = new BasePerformanceEntry(); - - @Override - public void submit( - final ExecutionContext executionContext, - final Runnable runnable, - final LogOutputAppendable description, - final Consumer onError) { - OperationInitializationThreadPool.executorService.submit(() -> { - final BasePerformanceEntry basePerformanceEntry = new BasePerformanceEntry(); - basePerformanceEntry.onBaseEntryStart(); - try (final SafeCloseable ignored = executionContext == null ? null : executionContext.open()) { - runnable.run(); - } catch (Exception e) { - onError.accept(e); - } catch (Error e) { - ProcessEnvironment.getGlobalFatalErrorReporter().report("SelectAndView Error", e); - throw e; - } finally { - basePerformanceEntry.onBaseEntryEnd(); - synchronized (accumulatedBaseEntry) { - accumulatedBaseEntry.accumulate(basePerformanceEntry); - } - } - }); - } - - @Override - public BasePerformanceEntry getAccumulatedPerformance() { - return accumulatedBaseEntry; - } - - @Override - public int threadCount() { - return OperationInitializationThreadPool.NUM_THREADS; - } - } - - public static class ImmediateJobScheduler implements SelectAndViewAnalyzer.JobScheduler { - public static final ImmediateJobScheduler INSTANCE = new ImmediateJobScheduler(); - - @Override - public void submit( - final ExecutionContext executionContext, - final Runnable runnable, - final LogOutputAppendable description, - final Consumer onError) { - try (SafeCloseable ignored = executionContext != null ? executionContext.open() : null) { - runnable.run(); - } catch (Exception e) { - onError.accept(e); - } catch (Error e) { - ProcessEnvironment.getGlobalFatalErrorReporter().report("SelectAndView Error", e); - throw e; - } - } - - @Override - public BasePerformanceEntry getAccumulatedPerformance() { - return null; - } - - @Override - public int threadCount() { - return 1; - } - } - /** * Create a completion handler that signals a future when the update is completed. * diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectColumnLayer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectColumnLayer.java index a9a12b0942f..47766c8af8d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectColumnLayer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectColumnLayer.java @@ -23,6 +23,7 @@ import io.deephaven.engine.updategraph.DynamicNode; import io.deephaven.engine.updategraph.UpdateCommitterEx; import io.deephaven.engine.updategraph.UpdateGraphProcessor; +import io.deephaven.engine.table.impl.util.JobScheduler; import io.deephaven.engine.util.systemicmarking.SystemicObjectTracker; import io.deephaven.time.DateTime; import io.deephaven.util.SafeCloseable; @@ -117,8 +118,8 @@ private ChunkSource getChunkSource() { @Override public void applyUpdate(final TableUpdate upstream, final RowSet toClear, - final UpdateHelper helper, final JobScheduler jobScheduler, @Nullable final LivenessNode liveResultOwner, - final SelectLayerCompletionHandler onCompletion) { + final UpdateHelper helper, final JobScheduler jobScheduler, @Nullable final LivenessNode liveResultOwner, + final SelectLayerCompletionHandler onCompletion) { if (upstream.removed().isNonempty()) { if (isRedirected) { clearObjectsAtThisLevel(upstream.removed()); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/StaticFlattenLayer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/StaticFlattenLayer.java index e9e2f95294b..4e7075d1361 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/StaticFlattenLayer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/StaticFlattenLayer.java @@ -18,6 +18,7 @@ import io.deephaven.engine.table.impl.sources.RedirectedColumnSource; import io.deephaven.engine.table.impl.util.RowRedirection; import io.deephaven.engine.table.impl.util.WrappedRowSetWritableRowRedirection; +import io.deephaven.engine.table.impl.util.JobScheduler; import org.jetbrains.annotations.Nullable; import java.util.BitSet; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/ViewColumnLayer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/ViewColumnLayer.java index 00d3b943a5c..74753600f4d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/ViewColumnLayer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/ViewColumnLayer.java @@ -13,6 +13,7 @@ import io.deephaven.engine.table.ModifiedColumnSet; import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.impl.select.SelectColumn; +import io.deephaven.engine.table.impl.util.JobScheduler; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/ImmediateJobScheduler.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/ImmediateJobScheduler.java new file mode 100644 index 00000000000..c0bb0a31c1d --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/ImmediateJobScheduler.java @@ -0,0 +1,39 @@ +package io.deephaven.engine.table.impl.util; + +import io.deephaven.base.log.LogOutputAppendable; +import io.deephaven.engine.context.ExecutionContext; +import io.deephaven.engine.table.impl.perf.BasePerformanceEntry; +import io.deephaven.util.SafeCloseable; +import io.deephaven.util.process.ProcessEnvironment; + +import java.util.function.Consumer; + +public class ImmediateJobScheduler implements JobScheduler { + public static final ImmediateJobScheduler INSTANCE = new ImmediateJobScheduler(); + + @Override + public void submit( + final ExecutionContext executionContext, + final Runnable runnable, + final LogOutputAppendable description, + final Consumer onError) { + try (SafeCloseable ignored = executionContext != null ? executionContext.open() : null) { + runnable.run(); + } catch (Exception e) { + onError.accept(e); + } catch (Error e) { + ProcessEnvironment.getGlobalFatalErrorReporter().report("SelectAndView Error", e); + throw e; + } + } + + @Override + public BasePerformanceEntry getAccumulatedPerformance() { + return null; + } + + @Override + public int threadCount() { + return 1; + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java new file mode 100644 index 00000000000..90e34be791e --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java @@ -0,0 +1,232 @@ +package io.deephaven.engine.table.impl.util; + +import io.deephaven.base.log.LogOutputAppendable; +import io.deephaven.engine.context.ExecutionContext; +import io.deephaven.engine.table.impl.perf.BasePerformanceEntry; +import io.deephaven.util.annotations.FinalDefault; + +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Consumer; + +/** + * An interface for submitting jobs to be executed and accumulating their performance of all the tasks performed off + * thread. + */ +public interface JobScheduler { + /** + * Cause runnable to be executed. + * + * @param executionContext the execution context to run it under + * @param runnable the runnable to execute + * @param description a description for logging + * @param onError a routine to call if an exception occurs while running runnable + */ + void submit( + ExecutionContext executionContext, + Runnable runnable, + final LogOutputAppendable description, + final Consumer onError); + + /** + * The performance statistics of all runnables that have been completed off-thread, or null if it was executed + * in the current thread. + */ + BasePerformanceEntry getAccumulatedPerformance(); + + /** + * How many threads exist in the job scheduler? The job submitters can use this value to determine how many + * sub-jobs to split work into. + */ + int threadCount(); + + /** + * Helper interface for {@code iterateSerial()} and {@code iterateParallel()}. This provides a callable + * interface with {@code index} indicating which iteration to perform. When this returns, the scheduler will + * automatically schedule the next iteration. + */ + @FunctionalInterface + interface IterateAction { + void run(int index); + } + + /** + * Helper interface for {@code iterateSerial()} and {@code iterateParallel()}. This provides a callable + * interface with {@code index} indicating which iteration to perform and {@link Runnable resume} providing a + * mechanism to inform the scheduler that the current task is complete. When {@code resume} is called, the + * scheduler will automatically schedule the next iteration. + *

              + * NOTE: failing to call {@code resume} will result in the scheduler not scheduling all remaining iterations. + * This will not block the scheduler, but the {@code completeAction} {@link Runnable} will never be called + */ + @FunctionalInterface + interface IterateResumeAction { + void run(int index, Runnable resume); + } + + /** + * Provides a mechanism to iterate over a range of values in parallel using the {@link JobScheduler} + * + * @param executionContext the execution context for this task + * @param description the description to use for logging + * @param start the integer value from which to start iterating + * @param count the number of times this task should be called + * @param action the task to perform, the current iteration index is provided as a parameter + * @param completeAction this will be called when all iterations are complete + * @param onError error handler for the scheduler to use while iterating + */ + @FinalDefault + default void iterateParallel(ExecutionContext executionContext, LogOutputAppendable description, int start, + int count, IterateAction action, Runnable completeAction, final Consumer onError) { + + if (count == 0) { + // no work to do + completeAction.run(); + } + + final AtomicInteger nextIndex = new AtomicInteger(start); + final AtomicInteger remaining = new AtomicInteger(count); + + final Runnable task = () -> { + // this will run until all tasks have started + while (true) { + int idx = nextIndex.getAndIncrement(); + if (idx < start + count) { + // do the work + action.run(idx); + + // check for completion + if (remaining.decrementAndGet() == 0) { + completeAction.run(); + return; + } + } else { + // no more work to do + return; + } + } + }; + + // create multiple tasks but not more than one per scheduler thread + for (int i = 0; i < Math.min(count, threadCount()); i++) { + submit(executionContext, + task, + description, + onError); + } + } + + /** + * Provides a mechanism to iterate over a range of values in parallel using the {@link JobScheduler} + * + * @param executionContext the execution context for this task + * @param description the description to use for logging + * @param start the integer value from which to start iterating + * @param count the number of times this task should be called + * @param action the task to perform, the current iteration index and a resume Runnable are parameters + * @param completeAction this will be called when all iterations are complete + * @param onError error handler for the scheduler to use while iterating + */ + @FinalDefault + default void iterateParallel(ExecutionContext executionContext, LogOutputAppendable description, int start, + int count, IterateResumeAction action, Runnable completeAction, Consumer onError) { + + if (count == 0) { + // no work to do + completeAction.run(); + } + + final AtomicInteger nextIndex = new AtomicInteger(start); + final AtomicInteger remaining = new AtomicInteger(count); + + final Runnable resumeAction = () -> { + // check for completion + if (remaining.decrementAndGet() == 0) { + completeAction.run(); + } + }; + + final Runnable task = () -> { + // this will run until all tasks have started + while (true) { + int idx = nextIndex.getAndIncrement(); + if (idx < start + count) { + // do the work + action.run(idx, resumeAction); + } else { + // no more work to do + return; + } + } + }; + + // create multiple tasks but not more than one per scheduler thread + for (int i = 0; i < Math.min(count, threadCount()); i++) { + submit(executionContext, + task, + description, + onError); + } + } + + /** + * Provides a mechanism to iterate over a range of values serially using the {@link JobScheduler}. The advantage + * to using this over a simple iteration is the resumption callable on {@code action} that will trigger the next + * iterable. This allows the next iteration to de delayed until dependendat asynchronous serial or parallel + * scheduler jobs have completed. + * + * @param executionContext the execution context for this task + * @param description the description to use for logging + * @param start the integer value from which to start iterating + * @param count the number of times this task should be called + * @param action the task to perform, the current iteration index and a resume Runnable are parameters + * @param completeAction this will be called when all iterations are complete + * @param onError error handler for the scheduler to use while iterating + */ + @FinalDefault + default void iterateSerial(ExecutionContext executionContext, LogOutputAppendable description, int start, + int count, IterateResumeAction action, Runnable completeAction, Consumer onError) { + + if (count == 0) { + // no work to do + completeAction.run(); + } + + final AtomicInteger nextIndex = new AtomicInteger(start); + final AtomicInteger remaining = new AtomicInteger(count); + + // no lambda, need the `this` reference to re-execute + final Runnable resumeAction = new Runnable() { + @Override + public void run() { + // check for completion + if (remaining.decrementAndGet() == 0) { + completeAction.run(); + } else { + // schedule the next task + submit(executionContext, + () -> { + int idx = nextIndex.getAndIncrement(); + if (idx < start + count) { + // do the work + action.run(idx, this); + } + }, + description, + onError); + } + + } + }; + + // create a single task + submit(executionContext, + () -> { + int idx = nextIndex.getAndIncrement(); + if (idx < start + count) { + action.run(idx, resumeAction); + } + }, + description, + onError); + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/OperationInitializationPoolJobScheduler.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/OperationInitializationPoolJobScheduler.java new file mode 100644 index 00000000000..865ca07290f --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/OperationInitializationPoolJobScheduler.java @@ -0,0 +1,50 @@ +package io.deephaven.engine.table.impl.util; + +import io.deephaven.base.log.LogOutputAppendable; +import io.deephaven.engine.context.ExecutionContext; +import io.deephaven.engine.table.impl.OperationInitializationThreadPool; +import io.deephaven.engine.table.impl.perf.BasePerformanceEntry; +import io.deephaven.engine.table.impl.util.JobScheduler; +import io.deephaven.util.SafeCloseable; +import io.deephaven.util.process.ProcessEnvironment; + +import java.util.function.Consumer; + +public class OperationInitializationPoolJobScheduler implements JobScheduler { + final BasePerformanceEntry accumulatedBaseEntry = new BasePerformanceEntry(); + + @Override + public void submit( + final ExecutionContext executionContext, + final Runnable runnable, + final LogOutputAppendable description, + final Consumer onError) { + OperationInitializationThreadPool.executorService.submit(() -> { + final BasePerformanceEntry basePerformanceEntry = new BasePerformanceEntry(); + basePerformanceEntry.onBaseEntryStart(); + try (final SafeCloseable ignored = executionContext == null ? null : executionContext.open()) { + runnable.run(); + } catch (Exception e) { + onError.accept(e); + } catch (Error e) { + ProcessEnvironment.getGlobalFatalErrorReporter().report("SelectAndView Error", e); + throw e; + } finally { + basePerformanceEntry.onBaseEntryEnd(); + synchronized (accumulatedBaseEntry) { + accumulatedBaseEntry.accumulate(basePerformanceEntry); + } + } + }); + } + + @Override + public BasePerformanceEntry getAccumulatedPerformance() { + return accumulatedBaseEntry; + } + + @Override + public int threadCount() { + return OperationInitializationThreadPool.NUM_THREADS; + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/UpdateGraphProcessorJobScheduler.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/UpdateGraphProcessorJobScheduler.java new file mode 100644 index 00000000000..ea4d2afaf9e --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/UpdateGraphProcessorJobScheduler.java @@ -0,0 +1,70 @@ +package io.deephaven.engine.table.impl.util; + +import io.deephaven.base.log.LogOutput; +import io.deephaven.base.log.LogOutputAppendable; +import io.deephaven.engine.context.ExecutionContext; +import io.deephaven.engine.table.impl.perf.BasePerformanceEntry; +import io.deephaven.engine.table.impl.util.JobScheduler; +import io.deephaven.engine.updategraph.AbstractNotification; +import io.deephaven.engine.updategraph.UpdateGraphProcessor; +import io.deephaven.util.process.ProcessEnvironment; + +import java.util.function.Consumer; + +public class UpdateGraphProcessorJobScheduler implements JobScheduler { + final BasePerformanceEntry accumulatedBaseEntry = new BasePerformanceEntry(); + + @Override + public void submit( + final ExecutionContext executionContext, + final Runnable runnable, + final LogOutputAppendable description, + final Consumer onError) { + UpdateGraphProcessor.DEFAULT.addNotification(new AbstractNotification(false) { + @Override + public boolean canExecute(long step) { + return true; + } + + @Override + public void run() { + final BasePerformanceEntry baseEntry = new BasePerformanceEntry(); + baseEntry.onBaseEntryStart(); + try { + runnable.run(); + } catch (Exception e) { + onError.accept(e); + } catch (Error e) { + ProcessEnvironment.getGlobalFatalErrorReporter().report("SelectAndView Error", e); + throw e; + } finally { + baseEntry.onBaseEntryEnd(); + synchronized (accumulatedBaseEntry) { + accumulatedBaseEntry.accumulate(baseEntry); + } + } + } + + @Override + public LogOutput append(LogOutput output) { + return output.append("{Notification(").append(System.identityHashCode(this)).append(" for ") + .append(description).append("}"); + } + + @Override + public ExecutionContext getExecutionContext() { + return executionContext; + } + }); + } + + @Override + public BasePerformanceEntry getAccumulatedPerformance() { + return accumulatedBaseEntry; + } + + @Override + public int threadCount() { + return UpdateGraphProcessor.DEFAULT.getUpdateThreads(); + } +} From 0789b1526156d4c1a249ed9091da30097556ddef Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 18 Nov 2022 12:03:18 -0800 Subject: [PATCH 044/123] Added interruption of tasks for JobScheduler on error --- .../deephaven/engine/table/impl/UpdateBy.java | 3 +- .../analyzers/SelectAndViewAnalyzer.java | 6 +- .../select/analyzers/SelectColumnLayer.java | 4 +- .../engine/table/impl/util/JobScheduler.java | 127 +++++++++++------- 4 files changed, 87 insertions(+), 53 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java index db8a31e77b2..c7712d4e2f1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java @@ -707,8 +707,7 @@ private TableUpdate computeDownstreamUpdate() { /** * Process the {@link TableUpdate update} provided in the constructor. This performs much work in parallel and - * leverages {@link JobScheduler} - * extensively + * leverages {@link JobScheduler} extensively */ public void processUpdate() { if (redirContext.isRedirected()) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer.java index 5c870dc1d1b..95a5fd48de6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer.java @@ -1,4 +1,4 @@ -/** + /** * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending */ package io.deephaven.engine.table.impl.select.analyzers; @@ -340,8 +340,8 @@ public void close() { * @param onCompletion Called when an inner column is complete. The outer layer should pass the {@code onCompletion} */ public abstract void applyUpdate(TableUpdate upstream, RowSet toClear, UpdateHelper helper, - JobScheduler jobScheduler, @Nullable LivenessNode liveResultOwner, - SelectLayerCompletionHandler onCompletion); + JobScheduler jobScheduler, @Nullable LivenessNode liveResultOwner, + SelectLayerCompletionHandler onCompletion); /** * Our job here is to calculate the effects: a map from incoming column to a list of columns that it effects. We do diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectColumnLayer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectColumnLayer.java index 47766c8af8d..4b9d8198390 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectColumnLayer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectColumnLayer.java @@ -118,8 +118,8 @@ private ChunkSource getChunkSource() { @Override public void applyUpdate(final TableUpdate upstream, final RowSet toClear, - final UpdateHelper helper, final JobScheduler jobScheduler, @Nullable final LivenessNode liveResultOwner, - final SelectLayerCompletionHandler onCompletion) { + final UpdateHelper helper, final JobScheduler jobScheduler, @Nullable final LivenessNode liveResultOwner, + final SelectLayerCompletionHandler onCompletion) { if (upstream.removed().isNonempty()) { if (isRedirected) { clearObjectsAtThisLevel(upstream.removed()); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java index 90e34be791e..5fd58183cef 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java @@ -5,6 +5,7 @@ import io.deephaven.engine.table.impl.perf.BasePerformanceEntry; import io.deephaven.util.annotations.FinalDefault; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; @@ -17,9 +18,9 @@ public interface JobScheduler { * Cause runnable to be executed. * * @param executionContext the execution context to run it under - * @param runnable the runnable to execute - * @param description a description for logging - * @param onError a routine to call if an exception occurs while running runnable + * @param runnable the runnable to execute + * @param description a description for logging + * @param onError a routine to call if an exception occurs while running runnable */ void submit( ExecutionContext executionContext, @@ -28,21 +29,21 @@ void submit( final Consumer onError); /** - * The performance statistics of all runnables that have been completed off-thread, or null if it was executed - * in the current thread. + * The performance statistics of all runnables that have been completed off-thread, or null if it was executed in + * the current thread. */ BasePerformanceEntry getAccumulatedPerformance(); /** - * How many threads exist in the job scheduler? The job submitters can use this value to determine how many - * sub-jobs to split work into. + * How many threads exist in the job scheduler? The job submitters can use this value to determine how many sub-jobs + * to split work into. */ int threadCount(); /** - * Helper interface for {@code iterateSerial()} and {@code iterateParallel()}. This provides a callable - * interface with {@code index} indicating which iteration to perform. When this returns, the scheduler will - * automatically schedule the next iteration. + * Helper interface for {@code iterateSerial()} and {@code iterateParallel()}. This provides a callable interface + * with {@code index} indicating which iteration to perform. When this returns, the scheduler will automatically + * schedule the next iteration. */ @FunctionalInterface interface IterateAction { @@ -50,13 +51,13 @@ interface IterateAction { } /** - * Helper interface for {@code iterateSerial()} and {@code iterateParallel()}. This provides a callable - * interface with {@code index} indicating which iteration to perform and {@link Runnable resume} providing a - * mechanism to inform the scheduler that the current task is complete. When {@code resume} is called, the - * scheduler will automatically schedule the next iteration. + * Helper interface for {@code iterateSerial()} and {@code iterateParallel()}. This provides a callable interface + * with {@code index} indicating which iteration to perform and {@link Runnable resume} providing a mechanism to + * inform the scheduler that the current task is complete. When {@code resume} is called, the scheduler will + * automatically schedule the next iteration. *

              - * NOTE: failing to call {@code resume} will result in the scheduler not scheduling all remaining iterations. - * This will not block the scheduler, but the {@code completeAction} {@link Runnable} will never be called + * NOTE: failing to call {@code resume} will result in the scheduler not scheduling all remaining iterations. This + * will not block the scheduler, but the {@code completeAction} {@link Runnable} will never be called */ @FunctionalInterface interface IterateResumeAction { @@ -67,16 +68,16 @@ interface IterateResumeAction { * Provides a mechanism to iterate over a range of values in parallel using the {@link JobScheduler} * * @param executionContext the execution context for this task - * @param description the description to use for logging - * @param start the integer value from which to start iterating - * @param count the number of times this task should be called - * @param action the task to perform, the current iteration index is provided as a parameter - * @param completeAction this will be called when all iterations are complete - * @param onError error handler for the scheduler to use while iterating + * @param description the description to use for logging + * @param start the integer value from which to start iterating + * @param count the number of times this task should be called + * @param action the task to perform, the current iteration index is provided as a parameter + * @param completeAction this will be called when all iterations are complete + * @param onError error handler for the scheduler to use while iterating */ @FinalDefault default void iterateParallel(ExecutionContext executionContext, LogOutputAppendable description, int start, - int count, IterateAction action, Runnable completeAction, final Consumer onError) { + int count, IterateAction action, Runnable completeAction, final Consumer onError) { if (count == 0) { // no work to do @@ -86,9 +87,19 @@ default void iterateParallel(ExecutionContext executionContext, LogOutputAppenda final AtomicInteger nextIndex = new AtomicInteger(start); final AtomicInteger remaining = new AtomicInteger(count); + final AtomicBoolean cancelRemainingExecution = new AtomicBoolean(false); + + final Consumer localError = exception -> { + cancelRemainingExecution.set(true); + onError.accept(exception); + }; + final Runnable task = () -> { // this will run until all tasks have started while (true) { + if (cancelRemainingExecution.get()) { + return; + } int idx = nextIndex.getAndIncrement(); if (idx < start + count) { // do the work @@ -111,24 +122,27 @@ default void iterateParallel(ExecutionContext executionContext, LogOutputAppenda submit(executionContext, task, description, - onError); + localError); } } /** - * Provides a mechanism to iterate over a range of values in parallel using the {@link JobScheduler} + * Provides a mechanism to iterate over a range of values in parallel using the {@link JobScheduler}. The advantage + * to using this over the other method is the resumption callable on {@code action} that will trigger the next + * execution. This allows the next iteration and the completion runnable to be delayed until dependent asynchronous + * serial or parallel scheduler jobs have completed. * * @param executionContext the execution context for this task - * @param description the description to use for logging - * @param start the integer value from which to start iterating - * @param count the number of times this task should be called - * @param action the task to perform, the current iteration index and a resume Runnable are parameters - * @param completeAction this will be called when all iterations are complete - * @param onError error handler for the scheduler to use while iterating + * @param description the description to use for logging + * @param start the integer value from which to start iterating + * @param count the number of times this task should be called + * @param action the task to perform, the current iteration index and a resume Runnable are parameters + * @param completeAction this will be called when all iterations are complete + * @param onError error handler for the scheduler to use while iterating */ @FinalDefault default void iterateParallel(ExecutionContext executionContext, LogOutputAppendable description, int start, - int count, IterateResumeAction action, Runnable completeAction, Consumer onError) { + int count, IterateResumeAction action, Runnable completeAction, Consumer onError) { if (count == 0) { // no work to do @@ -138,6 +152,13 @@ default void iterateParallel(ExecutionContext executionContext, LogOutputAppenda final AtomicInteger nextIndex = new AtomicInteger(start); final AtomicInteger remaining = new AtomicInteger(count); + final AtomicBoolean cancelRemainingExecution = new AtomicBoolean(false); + + final Consumer localError = exception -> { + cancelRemainingExecution.set(true); + onError.accept(exception); + }; + final Runnable resumeAction = () -> { // check for completion if (remaining.decrementAndGet() == 0) { @@ -148,6 +169,9 @@ default void iterateParallel(ExecutionContext executionContext, LogOutputAppenda final Runnable task = () -> { // this will run until all tasks have started while (true) { + if (cancelRemainingExecution.get()) { + return; + } int idx = nextIndex.getAndIncrement(); if (idx < start + count) { // do the work @@ -164,27 +188,27 @@ default void iterateParallel(ExecutionContext executionContext, LogOutputAppenda submit(executionContext, task, description, - onError); + localError); } } /** - * Provides a mechanism to iterate over a range of values serially using the {@link JobScheduler}. The advantage - * to using this over a simple iteration is the resumption callable on {@code action} that will trigger the next - * iterable. This allows the next iteration to de delayed until dependendat asynchronous serial or parallel - * scheduler jobs have completed. + * Provides a mechanism to iterate over a range of values serially using the {@link JobScheduler}. The advantage to + * using this over a simple iteration is the resumption callable on {@code action} that will trigger the next + * execution. This allows the next iteration and the completion runnable to be delayed until dependent asynchronous + * serial or parallel scheduler jobs have completed. * * @param executionContext the execution context for this task - * @param description the description to use for logging - * @param start the integer value from which to start iterating - * @param count the number of times this task should be called - * @param action the task to perform, the current iteration index and a resume Runnable are parameters - * @param completeAction this will be called when all iterations are complete - * @param onError error handler for the scheduler to use while iterating + * @param description the description to use for logging + * @param start the integer value from which to start iterating + * @param count the number of times this task should be called + * @param action the task to perform, the current iteration index and a resume Runnable are parameters + * @param completeAction this will be called when all iterations are complete + * @param onError error handler for the scheduler to use while iterating */ @FinalDefault default void iterateSerial(ExecutionContext executionContext, LogOutputAppendable description, int start, - int count, IterateResumeAction action, Runnable completeAction, Consumer onError) { + int count, IterateResumeAction action, Runnable completeAction, Consumer onError) { if (count == 0) { // no work to do @@ -194,6 +218,13 @@ default void iterateSerial(ExecutionContext executionContext, LogOutputAppendabl final AtomicInteger nextIndex = new AtomicInteger(start); final AtomicInteger remaining = new AtomicInteger(count); + final AtomicBoolean cancelRemainingExecution = new AtomicBoolean(false); + + final Consumer localError = exception -> { + cancelRemainingExecution.set(true); + onError.accept(exception); + }; + // no lambda, need the `this` reference to re-execute final Runnable resumeAction = new Runnable() { @Override @@ -202,6 +233,10 @@ public void run() { if (remaining.decrementAndGet() == 0) { completeAction.run(); } else { + if (cancelRemainingExecution.get()) { + return; + } + // schedule the next task submit(executionContext, () -> { @@ -212,7 +247,7 @@ public void run() { } }, description, - onError); + localError); } } @@ -227,6 +262,6 @@ public void run() { } }, description, - onError); + localError); } } From 3c6ee653158bd7f932fb4d30c44f99604ef6f1e1 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 22 Nov 2022 12:24:42 -0800 Subject: [PATCH 045/123] fixed addUpdateListener() calls --- .../table/impl/BucketedPartitionedUpdateByManager.java | 6 +++--- .../deephaven/engine/table/impl/UpdateByBucketHelper.java | 2 +- .../deephaven/engine/table/impl/ZeroKeyUpdateByManager.java | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java index b0b5f333f12..9b051e8b4e2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java @@ -79,7 +79,7 @@ protected BucketedPartitionedUpdateByManager(@NotNull final String description, // create a recorder instance sourced from the source table ListenerRecorder sourceRecorder = new ListenerRecorder(description, source, result); sourceRecorder.setMergedListener(listener); - source.listenForUpdates(sourceRecorder); + source.addUpdateListener(sourceRecorder); result.addParentReference(listener); recorders.offerLast(sourceRecorder); @@ -118,7 +118,7 @@ protected BucketedPartitionedUpdateByManager(@NotNull final String description, if (listener != null) { ListenerRecorder recorder = new ListenerRecorder(description, updateBy.result, result); recorder.setMergedListener(listener); - updateBy.result.listenForUpdates(recorder); + updateBy.result.addUpdateListener(recorder); // add the listener only while synchronized synchronized (recorders) { @@ -139,7 +139,7 @@ protected BucketedPartitionedUpdateByManager(@NotNull final String description, // create a recorder instance sourced from the transformed table ListenerRecorder sourceRecorder = new ListenerRecorder(description, transformed.table(), result); sourceRecorder.setMergedListener(listener); - transformed.table().listenForUpdates(sourceRecorder); + transformed.table().addUpdateListener(sourceRecorder); result.addParentReference(listener); recorders.offerLast(sourceRecorder); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java index d6c5488b6e1..23f9ce8f39f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java @@ -107,7 +107,7 @@ protected UpdateByBucketHelper(@NotNull final String description, if (source.isRefreshing()) { final ZeroKeyUpdateByListener listener = newListener(description, result); - source.listenForUpdates(listener); + source.addUpdateListener(listener); result.addParentReference(listener); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateByManager.java index 958d1dd11a5..4819f28eb4e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateByManager.java @@ -38,7 +38,7 @@ protected ZeroKeyUpdateByManager(@NotNull final String description, // create a recorder instance sourced from the source table ListenerRecorder sourceRecorder = new ListenerRecorder(description, source, result); sourceRecorder.setMergedListener(listener); - source.listenForUpdates(sourceRecorder); + source.addUpdateListener(sourceRecorder); result.addParentReference(listener); recorders.offerLast(sourceRecorder); @@ -56,7 +56,7 @@ protected ZeroKeyUpdateByManager(@NotNull final String description, // create a recorder instance sourced from the bucket helper ListenerRecorder recorder = new ListenerRecorder(description, zeroKeyUpdateBy.result, result); recorder.setMergedListener(listener); - zeroKeyUpdateBy.result.listenForUpdates(recorder); + zeroKeyUpdateBy.result.addUpdateListener(recorder); recorders.offerLast(recorder); } else { // no shifting will be needed, can create directly from source From a12b2184999a05ced4cc4da921167745a5fbbdd8 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 23 Nov 2022 12:50:50 -0800 Subject: [PATCH 046/123] Addressed (some) PR comments --- .../java/io/deephaven/engine/table/Table.java | 6 +- .../BucketedPartitionedUpdateByManager.java | 124 +++----- .../engine/table/impl/UncoalescedTable.java | 13 - .../deephaven/engine/table/impl/UpdateBy.java | 274 +++++++++--------- .../table/impl/UpdateByBucketHelper.java | 24 +- .../impl/UpdateByCumulativeOperator.java | 11 +- .../engine/table/impl/UpdateByOperator.java | 6 +- .../table/impl/UpdateByOperatorFactory.java | 106 +++---- .../table/impl/UpdateByWindowedOperator.java | 6 +- .../table/impl/ZeroKeyUpdateByManager.java | 109 +++---- .../select/analyzers/RedirectionLayer.java | 1 - .../analyzers/SelectAndViewAnalyzer.java | 2 +- .../ema/BasePrimitiveEMAOperator.java | 10 +- .../updateby/ema/BigDecimalEMAOperator.java | 6 +- .../updateby/ema/BigIntegerEMAOperator.java | 6 +- .../updateby/ema/BigNumberEMAOperator.java | 4 +- .../impl/updateby/ema/ByteEMAOperator.java | 4 +- .../impl/updateby/ema/DoubleEMAOperator.java | 5 +- .../impl/updateby/ema/FloatEMAOperator.java | 5 +- .../impl/updateby/ema/IntEMAOperator.java | 4 +- .../impl/updateby/ema/LongEMAOperator.java | 4 +- .../impl/updateby/ema/ShortEMAOperator.java | 4 +- .../updateby/fill/BooleanFillByOperator.java | 5 +- .../updateby/fill/ByteFillByOperator.java | 5 +- .../updateby/fill/CharFillByOperator.java | 5 +- .../updateby/fill/DoubleFillByOperator.java | 5 +- .../updateby/fill/FloatFillByOperator.java | 5 +- .../impl/updateby/fill/IntFillByOperator.java | 5 +- .../updateby/fill/LongFillByOperator.java | 5 +- .../updateby/fill/ObjectFillByOperator.java | 5 +- .../updateby/fill/ShortFillByOperator.java | 5 +- .../internal/BaseByteUpdateByOperator.java | 16 +- .../internal/BaseCharUpdateByOperator.java | 16 +- .../internal/BaseDoubleUpdateByOperator.java | 19 +- .../internal/BaseFloatUpdateByOperator.java | 19 +- .../internal/BaseIntUpdateByOperator.java | 16 +- .../internal/BaseLongUpdateByOperator.java | 16 +- .../internal/BaseObjectBinaryOperator.java | 12 +- .../internal/BaseObjectUpdateByOperator.java | 16 +- .../internal/BaseShortUpdateByOperator.java | 16 +- .../BaseWindowedByteUpdateByOperator.java | 14 +- .../BaseWindowedCharUpdateByOperator.java | 14 +- .../BaseWindowedDoubleUpdateByOperator.java | 15 +- .../BaseWindowedFloatUpdateByOperator.java | 15 +- .../BaseWindowedIntUpdateByOperator.java | 14 +- .../BaseWindowedLongUpdateByOperator.java | 14 +- .../BaseWindowedObjectUpdateByOperator.java | 14 +- .../BaseWindowedShortUpdateByOperator.java | 14 +- .../minmax/ByteCumMinMaxOperator.java | 5 +- .../minmax/ComparableCumMinMaxOperator.java | 6 +- .../minmax/DoubleCumMinMaxOperator.java | 5 +- .../minmax/FloatCumMinMaxOperator.java | 5 +- .../updateby/minmax/IntCumMinMaxOperator.java | 5 +- .../minmax/LongCumMinMaxOperator.java | 5 +- .../minmax/ShortCumMinMaxOperator.java | 5 +- .../prod/BigDecimalCumProdOperator.java | 7 +- .../prod/BigIntegerCumProdOperator.java | 7 +- .../updateby/prod/ByteCumProdOperator.java | 5 +- .../updateby/prod/DoubleCumProdOperator.java | 6 +- .../updateby/prod/FloatCumProdOperator.java | 6 +- .../updateby/prod/IntCumProdOperator.java | 5 +- .../updateby/prod/LongCumProdOperator.java | 5 +- .../updateby/prod/ShortCumProdOperator.java | 5 +- .../BigDecimalRollingSumOperator.java | 5 +- .../BigIntegerRollingSumOperator.java | 6 +- .../rollingsum/ByteRollingSumOperator.java | 5 +- .../rollingsum/DoubleRollingSumOperator.java | 4 +- .../rollingsum/FloatRollingSumOperator.java | 4 +- .../rollingsum/IntRollingSumOperator.java | 5 +- .../rollingsum/LongRollingSumOperator.java | 5 +- .../rollingsum/ShortRollingSumOperator.java | 5 +- .../sum/BigDecimalCumSumOperator.java | 7 +- .../sum/BigIntegerCumSumOperator.java | 7 +- .../impl/updateby/sum/ByteCumSumOperator.java | 5 +- .../updateby/sum/DoubleCumSumOperator.java | 5 +- .../updateby/sum/FloatCumSumOperator.java | 5 +- .../impl/updateby/sum/IntCumSumOperator.java | 5 +- .../impl/updateby/sum/LongCumSumOperator.java | 5 +- .../updateby/sum/ShortCumSumOperator.java | 5 +- .../impl/util/InverseRowRedirectionImpl.java | 27 +- .../barrage/table/BarrageRedirectedTable.java | 8 +- .../replicators/ReplicateUpdateBy.java | 4 +- 82 files changed, 536 insertions(+), 707 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 c9466d69de6..e0cb72ccc90 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 @@ -863,7 +863,7 @@ public static AsOfMatchRule of(ReverseAsOfJoinRule rule) { * result's constituent tables. * * @param dropKeys Whether to drop key columns in the output constituent tables - * @param keyColumnNames The name of the key columns to partition by + * @param keyColumnNames The names of the key columns to partition by * @return A {@link PartitionedTable} keyed by {@code keyColumnNames} */ @ConcurrentMethod @@ -876,7 +876,7 @@ public static AsOfMatchRule of(ReverseAsOfJoinRule rule) { * result's constituent tables. * * @param dropKeys Whether to drop key columns in the output constituent tables - * @param keyColumnNames The name of the key columns to partition by + * @param keyColumnNames The names of the key columns to partition by * @return A {@link PartitionedTable} keyed by {@code keyColumnNames} */ @ConcurrentMethod @@ -891,7 +891,7 @@ public static AsOfMatchRule of(ReverseAsOfJoinRule rule) { * The underlying partitioned table backing the result contains each row in {@code this} table in exactly one of the * result's constituent tables. * - * @param keyColumnNames The name of the key columns to partition by + * @param keyColumnNames The names of the key columns to partition by * @return A {@link PartitionedTable} keyed by {@code keyColumnNames} */ @ConcurrentMethod diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java index 9b051e8b4e2..2ca4e598587 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java @@ -2,6 +2,7 @@ import io.deephaven.api.ColumnName; import io.deephaven.api.updateby.UpdateByControl; +import io.deephaven.base.Pair; import io.deephaven.engine.rowset.*; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.updateby.UpdateByWindow; @@ -14,46 +15,29 @@ * An implementation of {@link UpdateBy} dedicated to bucketed computation. */ class BucketedPartitionedUpdateByManager extends UpdateBy { + /** The output table for this UpdateBy operation */ + final QueryTable result; + + /** The partitioned table used for identifying buckets */ + final Table transformedTable; + /** * Perform a bucketed updateBy using {@code byColumns} as the keys * * @param description the operation description + * @param operators the operations to perform + * @param windows the unique windows for this UpdateBy + * @param inputSources the primitive input sources + * @param operatorInputSourceSlots maps the operators to source indices * @param source the source table - * @param ops the operations to perform * @param resultSources the result sources * @param byColumns the columns to use for the bucket keys - * @param redirContext the row redirection shared context + * @param timestampColumnName the column to use for all time-aware operators + * @param redirHelper the row redirection helper for dense output sources * @param control the control object. - * @return the result table */ - public static Table compute(@NotNull final String description, - @NotNull final QueryTable source, - @NotNull final UpdateByOperator[] ops, - @NotNull final UpdateByWindow[] windows, - @NotNull final ColumnSource[] inputSources, - @NotNull final int[][] operatorInputSourceSlots, - @NotNull final Map> resultSources, - @NotNull final Collection byColumns, - @Nullable final String timestampColumnName, - @NotNull final UpdateByRedirectionContext redirContext, - @NotNull final UpdateByControl control) { - - final BucketedPartitionedUpdateByManager updateBy = new BucketedPartitionedUpdateByManager(description, - ops, - windows, - inputSources, - operatorInputSourceSlots, - source, - resultSources, - byColumns, - timestampColumnName, - redirContext, - control); - - return updateBy.result; - } - - protected BucketedPartitionedUpdateByManager(@NotNull final String description, + protected BucketedPartitionedUpdateByManager( + @NotNull final String description, @NotNull final UpdateByOperator[] operators, @NotNull final UpdateByWindow[] windows, @NotNull final ColumnSource[] inputSources, @@ -62,26 +46,21 @@ protected BucketedPartitionedUpdateByManager(@NotNull final String description, @NotNull final Map> resultSources, @NotNull final Collection byColumns, @Nullable final String timestampColumnName, - @NotNull final UpdateByRedirectionContext redirContext, + @NotNull final UpdateByRedirectionHelper redirHelper, @NotNull final UpdateByControl control) { - super(description, source, operators, windows, inputSources, operatorInputSourceSlots, resultSources, - timestampColumnName, redirContext, control); + super(source, operators, windows, inputSources, operatorInputSourceSlots, timestampColumnName, redirHelper, + control); // this table will always have the rowset of the source result = new QueryTable(source.getRowSet(), resultSources); final PartitionedTable pt; if (source.isRefreshing()) { - // this is a refreshing source, we will need a listener and recorders - recorders = new LinkedList<>(); - listener = newListener(description); - - // create a recorder instance sourced from the source table - ListenerRecorder sourceRecorder = new ListenerRecorder(description, source, result); - sourceRecorder.setMergedListener(listener); - source.addUpdateListener(sourceRecorder); + // this is a refreshing source, we will need a listener + listener = newUpdateByListener(description); + source.addUpdateListener(listener); + // result will depend on listener result.addParentReference(listener); - recorders.offerLast(sourceRecorder); // create input and output modified column sets for (UpdateByOperator op : operators) { @@ -89,21 +68,15 @@ protected BucketedPartitionedUpdateByManager(@NotNull final String description, op.createOutputModifiedColumnSet(result); } pt = source.partitionedAggBy(List.of(), true, null, byColumns); + + // make the source->result transformer + transformer = source.newModifiedColumnSetTransformer(result, source.getDefinition().getColumnNamesArray()); } else { - // no shifting will be needed, can create directly from source pt = source.partitionedAggBy(List.of(), true, null, byColumns); - - // create input modified column sets only - for (UpdateByOperator op : operators) { - op.createInputModifiedColumnSet(source); - } } - // make the source->result transformer - transformer = source.newModifiedColumnSetTransformer(result, source.getDefinition().getColumnNamesArray()); - final PartitionedTable transformed = pt.transform(t -> { - UpdateByBucketHelper updateBy = new UpdateByBucketHelper( + UpdateByBucketHelper bucket = new UpdateByBucketHelper( description, (QueryTable) t, operators, @@ -112,38 +85,25 @@ protected BucketedPartitionedUpdateByManager(@NotNull final String description, operatorInputSourceSlots, resultSources, timestampColumnName, - redirContext, + redirHelper, control); - if (listener != null) { - ListenerRecorder recorder = new ListenerRecorder(description, updateBy.result, result); - recorder.setMergedListener(listener); - updateBy.result.addUpdateListener(recorder); - - // add the listener only while synchronized - synchronized (recorders) { - recorders.offerLast(recorder); - } - } // add this to the bucket list synchronized (buckets) { - buckets.offerLast(updateBy); + buckets.offer(bucket); } // return the table - return updateBy.result; + return bucket.result; }); - result.addParentReference(transformed); - if (source.isRefreshing()) { - // create a recorder instance sourced from the transformed table - ListenerRecorder sourceRecorder = new ListenerRecorder(description, transformed.table(), result); - sourceRecorder.setMergedListener(listener); - transformed.table().addUpdateListener(sourceRecorder); - result.addParentReference(listener); - recorders.offerLast(sourceRecorder); - } + transformedTable = transformed.table(); + // result also depends on the transformedTable + result.addParentReference(transformedTable); + } else { + transformedTable = null; + } // make a dummy update to generate the initial row keys final TableUpdateImpl fakeUpdate = new TableUpdateImpl(source.getRowSet(), @@ -153,7 +113,19 @@ protected BucketedPartitionedUpdateByManager(@NotNull final String description, ModifiedColumnSet.EMPTY); // do the actual computations - final StateManager sm = new StateManager(fakeUpdate, true); + final PhasedUpdateProcessor sm = new PhasedUpdateProcessor(fakeUpdate, true); sm.processUpdate(); } + + @Override + protected QueryTable result() { + return result; + } + + @Override + protected boolean upstreamSatisfied(final long step) { + // For bucketed, need to verify the source and the transformed table is satisfied. + return source.satisfied(step) && transformedTable.satisfied(step); + } + } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UncoalescedTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UncoalescedTable.java index 6a8358904f4..e85c2e3fb19 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UncoalescedTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UncoalescedTable.java @@ -371,25 +371,12 @@ public Table ungroup(boolean nullFill, Collection columnsT return coalesce().ungroup(nullFill, columnsToUngroup); } - @Override - @ConcurrentMethod - public PartitionedTable partitionBy(boolean dropKeys, String... keyColumnNames) { - return coalesce().partitionBy(dropKeys, keyColumnNames); - } - @Override @ConcurrentMethod public PartitionedTable partitionBy(boolean dropKeys, @NotNull Collection columns) { return coalesce().partitionBy(dropKeys, columns); } - @Override - @ConcurrentMethod - public PartitionedTable partitionedAggBy(Collection aggregations, boolean preserveEmpty, - Table initialGroups, String... keyColumnNames) { - return coalesce().partitionedAggBy(aggregations, preserveEmpty, initialGroups, keyColumnNames); - } - @Override @ConcurrentMethod public PartitionedTable partitionedAggBy(Collection aggregations, boolean preserveEmpty, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java index c7712d4e2f1..ae4975abb94 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java @@ -9,7 +9,6 @@ import io.deephaven.api.updateby.UpdateByOperation; import io.deephaven.base.log.LogOutput; import io.deephaven.base.log.LogOutputAppendable; -import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ResettableWritableObjectChunk; import io.deephaven.chunk.WritableLongChunk; @@ -32,6 +31,8 @@ import io.deephaven.engine.table.impl.util.JobScheduler; import io.deephaven.engine.table.impl.util.OperationInitializationPoolJobScheduler; import io.deephaven.engine.table.impl.util.UpdateGraphProcessorJobScheduler; +import io.deephaven.util.datastructures.linked.IntrusiveDoublyLinkedNode; +import io.deephaven.util.datastructures.linked.IntrusiveDoublyLinkedQueue; import org.apache.commons.lang3.mutable.MutableInt; import org.apache.commons.lang3.mutable.MutableLong; import org.jetbrains.annotations.NotNull; @@ -41,7 +42,7 @@ import java.util.*; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; -import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.IntStream; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; @@ -52,11 +53,11 @@ public abstract class UpdateBy { /** When caching a column source, how many rows should we process in each parallel batch? (1M default) */ public static final int PARALLEL_CACHE_BATCH_SIZE = Configuration.getInstance().getIntegerWithDefault("UpdateBy.parallelCacheBatchSize", 1 << 20); - /** When caching a column source, what size chunks should be used to move data to the cache? (65K default) */ + /** When caching a column source, what size chunks should be used to move data to the cache? (64K default) */ public static final int PARALLEL_CACHE_CHUNK_SIZE = Configuration.getInstance().getIntegerWithDefault("UpdateBy.parallelCacheChunkSize", 1 << 16); - /** Input sources may be reused by mutiple operators, only store and cache unique ones */ + /** Input sources may be reused by multiple operators, only store and cache unique ones (post-reinterpret) */ protected final ColumnSource[] inputSources; /** Map operators to input sources, note some operators need more than one input, WAvg e.g. */ protected final int[][] operatorInputSourceSlots; @@ -67,20 +68,18 @@ public abstract class UpdateBy { /** The source table for the UpdateBy operators */ protected final QueryTable source; /** Helper class for maintaining the RowRedirection when using redirected output sources */ - protected final UpdateByRedirectionContext redirContext; + protected final UpdateByRedirectionHelper redirHelper; /** User control to specify UpdateBy parameters */ protected final UpdateByControl control; /** The single timestamp column used by all time-based operators */ protected final String timestampColumnName; - /** Store every bucket in this list for processing */ - protected final LinkedList buckets; /** Whether caching benefits this UpdateBy operation */ protected final boolean inputCacheNeeded; /** Whether caching benefits each input source */ protected final boolean[] inputSourceCacheNeeded; /** - * References to the dense array sources we are using for the cached sources, it's expected that these will be - * released and need to be created + * References to the dense array sources we are using for the cached sources. It's expected that these will be + * released and need to be created. */ protected final SoftReference>[] inputSourceCaches; /** For easy iteration, create a list of the source indices that need to be cached */ @@ -88,24 +87,23 @@ public abstract class UpdateBy { /** ColumnSet transformer from source to downstream */ protected ModifiedColumnSet.Transformer transformer; - /** The output table for this UpdateBy operation */ - protected QueryTable result; - /** For refreshing sources, maintain a list of each of the bucket listeners */ - protected LinkedList recorders; - /** For refreshing sources, need a merged listener to produce downstream updates */ + /** For refreshing sources, need a listener to react to upstream updates */ protected UpdateByListener listener; - public static class UpdateByRedirectionContext implements Context { + /** Store every bucket in this list for processing */ + protected final IntrusiveDoublyLinkedQueue buckets; + + public static class UpdateByRedirectionHelper { @Nullable protected final WritableRowRedirection rowRedirection; protected final WritableRowSet freeRows; - protected long maxInnerIndex; + protected long maxInnerRowKey; - public UpdateByRedirectionContext(@Nullable final WritableRowRedirection rowRedirection) { + public UpdateByRedirectionHelper(@Nullable final WritableRowRedirection rowRedirection) { this.rowRedirection = rowRedirection; this.freeRows = rowRedirection == null ? null : RowSetFactory.empty(); - this.maxInnerIndex = 0; + this.maxInnerRowKey = 0; } public boolean isRedirected() { @@ -113,7 +111,7 @@ public boolean isRedirected() { } public long requiredCapacity() { - return maxInnerIndex + 1; + return maxInnerRowKey + 1; } @Nullable @@ -121,7 +119,9 @@ public WritableRowRedirection getRowRedirection() { return rowRedirection; } - public void processUpdateForRedirection(@NotNull final TableUpdate upstream, final TrackingRowSet prevRowSet) { + public void processUpdateForRedirection(@NotNull final TableUpdate upstream, + final TrackingRowSet sourceRowSet) { + assert rowRedirection != null; if (upstream.removed().isNonempty()) { final RowSetBuilderRandom freeBuilder = RowSetFactory.builderRandom(); upstream.removed().forAllRowKeys(key -> freeBuilder.addKey(rowRedirection.remove(key))); @@ -129,23 +129,25 @@ public void processUpdateForRedirection(@NotNull final TableUpdate upstream, fin } if (upstream.shifted().nonempty()) { - try (final WritableRowSet prevIndexLessRemoves = prevRowSet.copyPrev()) { - prevIndexLessRemoves.remove(upstream.removed()); - final RowSet.SearchIterator fwdIt = prevIndexLessRemoves.searchIterator(); + try (final WritableRowSet prevRowSetLessRemoves = sourceRowSet.copyPrev()) { + prevRowSetLessRemoves.remove(upstream.removed()); + final RowSet.SearchIterator fwdIt = prevRowSetLessRemoves.searchIterator(); upstream.shifted().apply((start, end, delta) -> { if (delta < 0 && fwdIt.advance(start)) { for (long key = fwdIt.currentValue(); fwdIt.currentValue() <= end; key = fwdIt.nextLong()) { - if (shiftRedirectedKey(fwdIt, delta, key)) + if (shiftRedirectedKey(fwdIt, delta, key)) { break; + } } } else { - try (final RowSet.SearchIterator revIt = prevIndexLessRemoves.reverseIterator()) { + try (final RowSet.SearchIterator revIt = prevRowSetLessRemoves.reverseIterator()) { if (revIt.advance(end)) { for (long key = revIt.currentValue(); revIt.currentValue() >= start; key = revIt.nextLong()) { - if (shiftRedirectedKey(revIt, delta, key)) + if (shiftRedirectedKey(revIt, delta, key)) { break; + } } } } @@ -158,7 +160,7 @@ public void processUpdateForRedirection(@NotNull final TableUpdate upstream, fin final MutableLong lastAllocated = new MutableLong(0); final WritableRowSet.Iterator freeIt = freeRows.iterator(); upstream.added().forAllRowKeys(outerKey -> { - final long innerKey = freeIt.hasNext() ? freeIt.nextLong() : ++maxInnerIndex; + final long innerKey = freeIt.hasNext() ? freeIt.nextLong() : ++maxInnerRowKey; lastAllocated.setValue(innerKey); rowRedirection.put(outerKey, innerKey); }); @@ -168,6 +170,7 @@ public void processUpdateForRedirection(@NotNull final TableUpdate upstream, fin private boolean shiftRedirectedKey(@NotNull final RowSet.SearchIterator iterator, final long delta, final long key) { + assert rowRedirection != null; final long inner = rowRedirection.remove(key); if (inner != NULL_ROW_KEY) { rowRedirection.put(key + delta, inner); @@ -175,14 +178,14 @@ private boolean shiftRedirectedKey(@NotNull final RowSet.SearchIterator iterator return !iterator.hasNext(); } - public RowSet getInnerKeys(final RowSet outerKeys, final SharedContext sharedContext) { + public RowSet getInnerKeys(final RowSet outerKeys) { if (rowRedirection == null) { return null; } RowSetBuilderRandom builder = RowSetFactory.builderRandom(); final int chunkSize = Math.min(outerKeys.intSize(), 4096); try (final RowSequence.Iterator it = outerKeys.getRowSequenceIterator(); - ChunkSource.FillContext fillContext = rowRedirection.makeFillContext(chunkSize, sharedContext); + ChunkSource.FillContext fillContext = rowRedirection.makeFillContext(chunkSize, null); WritableLongChunk chunk = WritableLongChunk.makeWritableChunk(chunkSize)) { while (it.hasMore()) { final RowSequence rs = it.getNextRowSequenceWithLength(chunkSize); @@ -192,23 +195,16 @@ public RowSet getInnerKeys(final RowSet outerKeys, final SharedContext sharedCon } return builder.build(); } - - @Override - public void close() { - try (final WritableRowSet ignored = freeRows) { - } - } } - protected UpdateBy(@NotNull final String description, + protected UpdateBy( @NotNull final QueryTable source, @NotNull final UpdateByOperator[] operators, @NotNull final UpdateByWindow[] windows, @NotNull final ColumnSource[] inputSources, @NotNull final int[][] operatorInputSourceSlots, - @NotNull final Map> resultSources, @Nullable String timestampColumnName, - @NotNull final UpdateByRedirectionContext redirContext, + @NotNull final UpdateByRedirectionHelper redirHelper, @NotNull final UpdateByControl control) { if (operators.length == 0) { @@ -221,68 +217,58 @@ protected UpdateBy(@NotNull final String description, this.inputSources = inputSources; this.operatorInputSourceSlots = operatorInputSourceSlots; this.timestampColumnName = timestampColumnName; - this.redirContext = redirContext; + this.redirHelper = redirHelper; this.control = control; this.inputSourceCacheNeeded = new boolean[inputSources.length]; + cacheableSourceIndices = IntStream.range(0, inputSources.length) + .filter(ii -> !FillUnordered.providesFillUnordered(inputSources[ii])) + .peek(ii -> inputSourceCacheNeeded[ii] = true) + .toArray(); + inputCacheNeeded = cacheableSourceIndices.length > 0; - boolean cacheNeeded = false; - TIntArrayList cacheableSourceIndicesList = new TIntArrayList(inputSources.length); - for (int ii = 0; ii < inputSources.length; ii++) { - if (!FillUnordered.providesFillUnordered(inputSources[ii])) { - cacheNeeded = inputSourceCacheNeeded[ii] = true; - cacheableSourceIndicesList.add(ii); - } - } - // store this list for fast iteration - cacheableSourceIndices = cacheableSourceIndicesList.toArray(); - - this.inputCacheNeeded = cacheNeeded; - // noinspection unchecked inputSourceCaches = new SoftReference[inputSources.length]; - buckets = new LinkedList<>(); + buckets = + new IntrusiveDoublyLinkedQueue<>(IntrusiveDoublyLinkedNode.Adapter.getInstance()); } - /** Remove all references to Objects for this column source */ - private void fillObjectArraySourceWithNull(ObjectArraySource sourceToNull) { - Assert.neqNull(sourceToNull, "cached column source was null, must have been GC'd"); - try (final ResettableWritableObjectChunk backingChunk = - ResettableWritableObjectChunk.makeResettableChunk()) { - Assert.neqNull(sourceToNull, "cached column source was already GC'd"); - final long targetCapacity = sourceToNull.getCapacity(); - for (long positionToNull = 0; positionToNull < targetCapacity; positionToNull += backingChunk.size()) { - sourceToNull.resetWritableChunkToBackingStore(backingChunk, positionToNull); - backingChunk.fillWithNullValue(0, backingChunk.size()); - } - } - } /** Release the input sources that will not be needed for the rest of this update */ private void releaseInputSources(int winIdx, ColumnSource[] maybeCachedInputSources, - TrackingWritableRowSet[] inputSourceRowSets, AtomicInteger[] inputSourceReferenceCounts) { + WritableRowSet[] inputSourceRowSets, int[] inputSourceReferenceCounts) { final UpdateByWindow win = windows[winIdx]; final int[] uniqueWindowSources = win.getUniqueSourceIndices(); - for (int srcIdx : uniqueWindowSources) { - if (inputSourceReferenceCounts[srcIdx] != null) { - if (inputSourceReferenceCounts[srcIdx].decrementAndGet() == 0) { - // do the cleanup immediately - inputSourceRowSets[srcIdx].close(); - inputSourceRowSets[srcIdx] = null; - - // release any objects we are holding in the cache - if (inputSourceCaches[srcIdx].get() instanceof ObjectArraySource) { - fillObjectArraySourceWithNull((ObjectArraySource) inputSourceCaches[srcIdx].get()); - } - maybeCachedInputSources[srcIdx] = null; + try (final ResettableWritableObjectChunk backingChunk = + ResettableWritableObjectChunk.makeResettableChunk()) { + for (int srcIdx : uniqueWindowSources) { + if (inputSourceCacheNeeded[srcIdx]) { + if (--inputSourceReferenceCounts[srcIdx] == 0) { + // do the cleanup immediately + inputSourceRowSets[srcIdx].close(); + inputSourceRowSets[srcIdx] = null; + + // release any objects we are holding in the cache + if (maybeCachedInputSources[srcIdx] instanceof ObjectArraySource) { + final long targetCapacity = inputSourceRowSets[srcIdx].size(); + for (long positionToNull = 0; positionToNull < targetCapacity; positionToNull += + backingChunk.size()) { + ((ObjectArraySource) maybeCachedInputSources[srcIdx]) + .resetWritableChunkToBackingStore(backingChunk, positionToNull); + backingChunk.fillWithNullValue(0, backingChunk.size()); + } + } + + maybeCachedInputSources[srcIdx] = null; + } } } } } /** - * Overview of work performed by {@link StateManager}: + * Overview of work performed by {@link PhasedUpdateProcessor}: *

                *
              1. Create `shiftedRows`, the set of rows for the output sources that are affected by shifts
              2. *
              3. Compute a rowset for each cacheable input source identifying which rows will be needed for processing
              4. @@ -294,7 +280,7 @@ private void releaseInputSources(int winIdx, ColumnSource[] maybeCachedInputS * done in parallel with the caching *
              5. When prepareForParallelPopulation() complete, apply upstream shifts to the output sources
              6. *
              7. When caching and shifts are complete, process the data in this window in parallel by dividing the buckets - * into sets (N/num_threads) and running a job for each bucket_set 3e) when all buckets processed
              8. + * into sets (N/num_threads) and running a job for each bucket_set *
              9. When all buckets processed, release the input source caches that will not be re-used later
              10. * * @@ -303,23 +289,24 @@ private void releaseInputSources(int winIdx, ColumnSource[] maybeCachedInputS *
              */ - protected class StateManager implements LogOutputAppendable { + class PhasedUpdateProcessor implements LogOutputAppendable { final TableUpdate update; final boolean initialStep; final UpdateByBucketHelper[] dirtyBuckets; + /** The active set of sources to use for processing, each source may be cached or original */ final ColumnSource[] maybeCachedInputSources; - final TrackingWritableRowSet[] inputSourceRowSets; - final AtomicInteger[] inputSourceReferenceCounts; + /** For cacheable sources, the minimal rowset to cache (union of bucket influencer rows) */ + final WritableRowSet[] inputSourceRowSets; + /** For cacheable sources, track how many windows require this source */ + final int[] inputSourceReferenceCounts; final JobScheduler jobScheduler; final CompletableFuture waitForResult; - final SharedContext sharedContext; - WritableRowSet shiftedRows; - public StateManager(TableUpdate update, boolean initialStep) { + PhasedUpdateProcessor(TableUpdate update, boolean initialStep) { this.update = update; this.initialStep = initialStep; @@ -328,8 +315,8 @@ public StateManager(TableUpdate update, boolean initialStep) { if (inputCacheNeeded) { maybeCachedInputSources = new ColumnSource[inputSources.length]; - inputSourceRowSets = new TrackingWritableRowSet[inputSources.length]; - inputSourceReferenceCounts = new AtomicInteger[inputSources.length]; + inputSourceRowSets = new WritableRowSet[inputSources.length]; + inputSourceReferenceCounts = new int[inputSources.length]; } else { maybeCachedInputSources = inputSources; inputSourceRowSets = null; @@ -351,18 +338,21 @@ public StateManager(TableUpdate update, boolean initialStep) { } waitForResult = null; } - - sharedContext = SharedContext.makeSharedContext(); } @Override public LogOutput append(LogOutput logOutput) { - return logOutput.append("UpdateBy.StateManager"); + return logOutput.append("UpdateBy.PhasedUpdateProcessor"); } private void onError(Exception error) { - // signal to the future that an exception has occured - waitForResult.completeExceptionally(error); + if (waitForResult != null) { + // signal to the future that an exception has occurred + waitForResult.completeExceptionally(error); + } else { + // this is part of an update, need to notify downstream + result().notifyListenersOnError(error, null); + } } /** @@ -378,17 +368,14 @@ private void computeCachedColumnRowsets(final Runnable completeAction) { // initially everything is dirty so cache everything if (initialStep) { for (int srcIdx : cacheableSourceIndices) { - // create a TrackingRowSet to be used by `InverseRowRedirectionImpl` - inputSourceRowSets[srcIdx] = source.getRowSet().copy().toTracking(); - - // how many windows require this input source? - int refCount = 0; - for (UpdateByWindow win : windows) { - if (win.isSourceInUse(srcIdx)) { - refCount++; - } + if (inputSourceCacheNeeded[srcIdx]) { + // create a RowSet to be used by `InverseRowRedirectionImpl` + inputSourceRowSets[srcIdx] = source.getRowSet().copy(); + + // record how many windows require this input source + inputSourceReferenceCounts[srcIdx] = + (int) Arrays.stream(windows).filter(win -> win.isSourceInUse(srcIdx)).count(); } - inputSourceReferenceCounts[srcIdx] = new AtomicInteger(refCount); } completeAction.run(); return; @@ -398,7 +385,6 @@ private void computeCachedColumnRowsets(final Runnable completeAction) { 0, cacheableSourceIndices.length, idx -> { int srcIdx = cacheableSourceIndices[idx]; - int refCount = 0; for (int winIdx = 0; winIdx < windows.length; winIdx++) { UpdateByWindow win = windows[winIdx]; if (win.isSourceInUse(srcIdx)) { @@ -415,10 +401,9 @@ private void computeCachedColumnRowsets(final Runnable completeAction) { } } } - refCount++; + inputSourceReferenceCounts[srcIdx]++; } } - inputSourceReferenceCounts[srcIdx] = new AtomicInteger(refCount); }, completeAction, this::onError); @@ -436,7 +421,7 @@ private void createCachedColumnSource(int srcIdx, final Runnable completeAction) } final ColumnSource inputSource = inputSources[srcIdx]; - final TrackingWritableRowSet inputRowSet = inputSourceRowSets[srcIdx]; + final WritableRowSet inputRowSet = inputSourceRowSets[srcIdx]; // re-use the dense column cache if it still exists WritableColumnSource innerSource; @@ -451,7 +436,7 @@ private void createCachedColumnSource(int srcIdx, final Runnable completeAction) // there will be no updates to this cached column source, so use a simple redirection final WritableRowRedirection rowRedirection = new InverseRowRedirectionImpl(inputRowSet); final WritableColumnSource outputSource = - new WritableRedirectedColumnSource(rowRedirection, innerSource, 0); + new WritableRedirectedColumnSource<>(rowRedirection, innerSource, 0); // holding this reference should protect `rowDirection` and `innerSource` from GC maybeCachedInputSources[srcIdx] = outputSource; @@ -571,8 +556,8 @@ private void processWindows(final Runnable completeAction) { // prepare each operator for the parallel updates to come if (initialStep) { // prepare the entire set of rows on the initial step - try (final RowSet changedRows = redirContext.isRedirected() - ? RowSetFactory.flat(redirContext.requiredCapacity()) + try (final RowSet changedRows = redirHelper.isRedirected() + ? RowSetFactory.flat(redirHelper.requiredCapacity()) : source.getRowSet().copy()) { for (UpdateByOperator op : win.getOperators()) { op.prepareForParallelPopulation(changedRows); @@ -586,8 +571,8 @@ private void processWindows(final Runnable completeAction) { windowRowSet.insert(win.getAffectedRows(bucket.windowContexts[winIdx])); } } - try (final RowSet changedRows = redirContext.isRedirected() - ? redirContext.getInnerKeys(windowRowSet, sharedContext) + try (final RowSet changedRows = redirHelper.isRedirected() + ? redirHelper.getInnerKeys(windowRowSet) : windowRowSet.copy()) { for (UpdateByOperator op : win.getOperators()) { op.prepareForParallelPopulation(changedRows); @@ -596,7 +581,7 @@ private void processWindows(final Runnable completeAction) { } } - if (!redirContext.isRedirected() && update.shifted().nonempty()) { + if (!redirHelper.isRedirected() && update.shifted().nonempty()) { // shift the non-redirected output sources now, after parallelPopulation try (final RowSet prevIdx = source.getRowSet().copyPrev()) { update.shifted().apply((begin, end, delta) -> { @@ -629,8 +614,7 @@ private void processWindows(final Runnable completeAction) { * {@code completedAction} when the work is complete */ private void cleanUpAndNotify(final Runnable completeAction) { - try (final RowSet ignoredRs = shiftedRows; - final SharedContext ignoredCtx = sharedContext) { + try (final RowSet ignoredRs = shiftedRows) { // auto close these resources } @@ -649,7 +633,7 @@ private void cleanUpAndNotify(final Runnable completeAction) { // pass the result downstream if (downstream != null) { - result.notifyListeners(downstream); + result().notifyListeners(downstream); } completeAction.run(); @@ -666,7 +650,7 @@ private TableUpdate computeDownstreamUpdate() { downstream.shifted = update.shifted(); // union the modifies from all the tables (including source) - downstream.modifiedColumnSet = result.getModifiedColumnSetForUpdates(); + downstream.modifiedColumnSet = result().getModifiedColumnSetForUpdates(); downstream.modifiedColumnSet.clear(); // get the adds/removes/shifts from upstream, make a copy since TableUpdateImpl#reset will @@ -710,10 +694,10 @@ private TableUpdate computeDownstreamUpdate() { * leverages {@link JobScheduler} extensively */ public void processUpdate() { - if (redirContext.isRedirected()) { + if (redirHelper.isRedirected()) { // this call does all the work needed for redirected output sources, for sparse output sources // we will process shifts only after a call to `prepareForParallelPopulation()` on each source - redirContext.processUpdateForRedirection(update, source.getRowSet()); + redirHelper.processUpdateForRedirection(update, source.getRowSet()); shiftedRows = RowSetFactory.empty(); } else { // for our sparse array output sources, we need to identify which rows will be affected by the upstream @@ -775,34 +759,36 @@ public void processUpdate() { /** * The Listener for apply to the constituent table updates */ - class UpdateByListener extends MergedListener { + class UpdateByListener extends InstrumentedTableUpdateListenerAdapter { public UpdateByListener(@Nullable String description) { - super(UpdateBy.this.recorders, List.of(), description, UpdateBy.this.result); + super(description, UpdateBy.this.source, false); } @Override - protected void process() { - final ListenerRecorder sourceRecorder = recorders.peekFirst(); - final TableUpdate upstream = sourceRecorder.getUpdate(); - - // we need to keep a reference to TableUpdate during our computation - final StateManager sm = new StateManager(upstream.acquire(), false); + public void onUpdate(final TableUpdate upstream) { + final PhasedUpdateProcessor sm = new PhasedUpdateProcessor(upstream.acquire(), false); sm.processUpdate(); } @Override - protected boolean canExecute(final long step) { - - synchronized (recorders) { - return recorders.stream().allMatch(lr -> lr.satisfied(step)); + public boolean canExecute(final long step) { + if (!upstreamSatisfied(step)) { + return false; + } + synchronized (buckets) { + return buckets.stream().allMatch(b -> b.result.satisfied(step)); } } } - public UpdateByListener newListener(@NotNull final String description) { + public UpdateByListener newUpdateByListener(@NotNull final String description) { return new UpdateByListener(description); } + protected abstract QueryTable result(); + + protected abstract boolean upstreamSatisfied(final long step); + // region UpdateBy implementation /** @@ -847,15 +833,15 @@ public static Table updateBy(@NotNull final QueryTable source, rowRedirection = null; } - // create an UpdateByRedirectionContext for use by the UpdateByBucketHelper objects - UpdateByRedirectionContext redirContext = new UpdateByRedirectionContext(rowRedirection); + // create an UpdateByRedirectionHelper for use by the UpdateByBucketHelper objects + UpdateByRedirectionHelper redirHelper = new UpdateByRedirectionHelper(rowRedirection); // TODO(deephaven-core#2693): Improve UpdateByBucketHelper implementation for ColumnName // generate a MatchPair array for use by the existing algorithm MatchPair[] pairs = MatchPair.fromPairs(byColumns); final UpdateByOperatorFactory updateByOperatorFactory = - new UpdateByOperatorFactory(source, pairs, redirContext, control); + new UpdateByOperatorFactory(source, pairs, redirHelper, control); final Collection ops = updateByOperatorFactory.getOperators(clauses); final UpdateByOperator[] opArr = ops.toArray(UpdateByOperator.ZERO_LENGTH_OP_ARRAY); @@ -972,16 +958,16 @@ public static Table updateBy(@NotNull final QueryTable source, if (pairs.length == 0) { descriptionBuilder.append(")"); - Table ret = ZeroKeyUpdateByManager.compute( + final ZeroKeyUpdateByManager zkm = new ZeroKeyUpdateByManager( descriptionBuilder.toString(), - source, opArr, windowArr, inputSourceArr, operatorInputSourceSlotArr, + source, resultSources, timestampColumnName, - redirContext, + redirHelper, control); if (source.isRefreshing()) { @@ -991,7 +977,7 @@ public static Table updateBy(@NotNull final QueryTable source, } ops.forEach(UpdateByOperator::startTrackingPrev); } - return ret; + return zkm.result; } descriptionBuilder.append(", pairs={").append(MatchPair.matchString(pairs)).append("})"); @@ -1008,17 +994,17 @@ public static Table updateBy(@NotNull final QueryTable source, String.join(", ", problems) + "}"); } - Table ret = BucketedPartitionedUpdateByManager.compute( + final BucketedPartitionedUpdateByManager bm = new BucketedPartitionedUpdateByManager( descriptionBuilder.toString(), - source, opArr, windowArr, inputSourceArr, operatorInputSourceSlotArr, + source, resultSources, byColumns, timestampColumnName, - redirContext, + redirHelper, control); if (source.isRefreshing()) { @@ -1028,7 +1014,7 @@ public static Table updateBy(@NotNull final QueryTable source, } ops.forEach(UpdateByOperator::startTrackingPrev); } - return ret; + return bm.result; } // endregion } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java index 23f9ce8f39f..9d09b3185d3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java @@ -1,6 +1,5 @@ package io.deephaven.engine.table.impl; -import gnu.trove.list.array.TIntArrayList; import io.deephaven.api.updateby.UpdateByControl; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.*; @@ -11,6 +10,7 @@ import io.deephaven.engine.table.impl.sources.ReinterpretUtils; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.updateby.UpdateByWindow; +import io.deephaven.util.datastructures.linked.IntrusiveDoublyLinkedNode; import org.apache.commons.lang3.mutable.MutableLong; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -21,16 +21,16 @@ /** * An helper class of {@link UpdateBy} dedicated to zero key computation. This will manage the computation of a single - * bucket of rows + * bucket of rows. */ -class UpdateByBucketHelper { +class UpdateByBucketHelper extends IntrusiveDoublyLinkedNode.Impl { protected final ColumnSource[] inputSources; // some columns will have multiple inputs, such as time-based and Weighted computations final int[][] operatorInputSourceSlots; final UpdateByOperator[] operators; final UpdateByWindow[] windows; final QueryTable source; - final UpdateBy.UpdateByRedirectionContext redirContext; + final UpdateBy.UpdateByRedirectionHelper redirHelper; final UpdateByControl control; final QueryTable result; @@ -56,7 +56,7 @@ class UpdateByBucketHelper { * @param operatorInputSourceSlots the mapping from operator index to needed input source indices * @param resultSources the result sources * @param timestampColumnName the timestamp column used for time-based operations - * @param redirContext the row redirection shared context + * @param redirHelper the row redirection shared context * @param control the control object. */ @@ -68,7 +68,7 @@ protected UpdateByBucketHelper(@NotNull final String description, @NotNull final int[][] operatorInputSourceSlots, @NotNull final Map> resultSources, @Nullable String timestampColumnName, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, @NotNull final UpdateByControl control) { this.source = source; @@ -76,7 +76,7 @@ protected UpdateByBucketHelper(@NotNull final String description, this.windows = windows; this.inputSources = inputSources; this.operatorInputSourceSlots = operatorInputSourceSlots; - this.redirContext = redirContext; + this.redirHelper = redirHelper; this.control = control; result = new QueryTable(source.getRowSet(), resultSources); @@ -106,14 +106,14 @@ protected UpdateByBucketHelper(@NotNull final String description, prepareForUpdate(initialUpdate, true); if (source.isRefreshing()) { - final ZeroKeyUpdateByListener listener = newListener(description, result); + final UpdateByBucketHelperListener listener = newListener(description); source.addUpdateListener(listener); result.addParentReference(listener); } } - ZeroKeyUpdateByListener newListener(@NotNull final String description, @NotNull final QueryTable result) { - return new ZeroKeyUpdateByListener(description, source); + UpdateByBucketHelperListener newListener(@NotNull final String description) { + return new UpdateByBucketHelperListener(description, source); } private void processUpdateForSsa(TableUpdate upstream) { @@ -318,8 +318,8 @@ public void finalizeUpdate() { /** * The Listener for apply an upstream {@link InstrumentedTableUpdateListenerAdapter#onUpdate(TableUpdate) update} */ - class ZeroKeyUpdateByListener extends InstrumentedTableUpdateListenerAdapter { - public ZeroKeyUpdateByListener(@Nullable String description, + class UpdateByBucketHelperListener extends InstrumentedTableUpdateListenerAdapter { + public UpdateByBucketHelperListener(@Nullable String description, @NotNull final QueryTable source) { super(description, source, false); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java index 86b253a4d57..82e5fd9b3b1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java @@ -1,17 +1,12 @@ package io.deephaven.engine.table.impl; -import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.sized.SizedLongChunk; import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.table.ChunkSink; import io.deephaven.engine.table.MatchPair; import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.NULL_LONG; @@ -47,12 +42,12 @@ public abstract void accumulate(RowSequence inputKeys, * * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this operation - * @param redirContext the row redirection context to use for the operation + * @param redirHelper the row redirection context to use for the operation */ public UpdateByCumulativeOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { - super(pair, affectingColumns, null, null, 0L, 0L, redirContext); + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper) { + super(pair, affectingColumns, null, null, 0L, 0L, redirHelper); } /** diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java index 40f656fc932..9f1fa44dc08 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java @@ -36,7 +36,7 @@ public abstract class UpdateByOperator { protected final MatchPair pair; protected final String[] affectingColumns; - protected final UpdateBy.UpdateByRedirectionContext redirContext; + protected final UpdateBy.UpdateByRedirectionHelper redirHelper; // these will be used by the timestamp-aware operators (EMA for example) @@ -99,10 +99,10 @@ protected UpdateByOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper) { this.pair = pair; this.affectingColumns = affectingColumns; - this.redirContext = redirContext; + this.redirHelper = redirHelper; this.timestampColumnName = timestampColumnName; this.control = control; this.reverseTimeScaleUnits = reverseTimeScaleUnits; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java index e2d327a6aab..cc77bfd0a40 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java @@ -34,17 +34,17 @@ public class UpdateByOperatorFactory { private final TableDefaults source; private final MatchPair[] groupByColumns; @NotNull - private final UpdateBy.UpdateByRedirectionContext redirContext; + private final UpdateBy.UpdateByRedirectionHelper redirHelper; @NotNull private final UpdateByControl control; public UpdateByOperatorFactory(@NotNull final TableDefaults source, @NotNull final MatchPair[] groupByColumns, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, @NotNull final UpdateByControl control) { this.source = source; this.groupByColumns = groupByColumns; - this.redirContext = redirContext; + this.redirHelper = redirHelper; this.control = control; } @@ -228,28 +228,28 @@ private UpdateByOperator makeEmaOperator(@NotNull final MatchPair pair, if (csType == byte.class || csType == Byte.class) { return new ByteEMAOperator(pair, affectingColumns, control, - ema.timeScale().timestampCol(), timeScaleUnits, redirContext, columnSource); + ema.timeScale().timestampCol(), timeScaleUnits, redirHelper, columnSource); } else if (csType == short.class || csType == Short.class) { return new ShortEMAOperator(pair, affectingColumns, control, - ema.timeScale().timestampCol(), timeScaleUnits, redirContext, columnSource); + ema.timeScale().timestampCol(), timeScaleUnits, redirHelper, columnSource); } else if (csType == int.class || csType == Integer.class) { return new IntEMAOperator(pair, affectingColumns, control, - ema.timeScale().timestampCol(), timeScaleUnits, redirContext, columnSource); + ema.timeScale().timestampCol(), timeScaleUnits, redirHelper, columnSource); } else if (csType == long.class || csType == Long.class) { return new LongEMAOperator(pair, affectingColumns, control, - ema.timeScale().timestampCol(), timeScaleUnits, redirContext, columnSource); + ema.timeScale().timestampCol(), timeScaleUnits, redirHelper, columnSource); } else if (csType == float.class || csType == Float.class) { return new FloatEMAOperator(pair, affectingColumns, control, - ema.timeScale().timestampCol(), timeScaleUnits, redirContext, columnSource); + ema.timeScale().timestampCol(), timeScaleUnits, redirHelper, columnSource); } else if (csType == double.class || csType == Double.class) { return new DoubleEMAOperator(pair, affectingColumns, control, - ema.timeScale().timestampCol(), timeScaleUnits, redirContext, columnSource); + ema.timeScale().timestampCol(), timeScaleUnits, redirHelper, columnSource); } else if (csType == BigDecimal.class) { return new BigDecimalEMAOperator(pair, affectingColumns, control, - ema.timeScale().timestampCol(), timeScaleUnits, redirContext, columnSource); + ema.timeScale().timestampCol(), timeScaleUnits, redirHelper, columnSource); } else if (csType == BigInteger.class) { return new BigIntegerEMAOperator(pair, affectingColumns, control, - ema.timeScale().timestampCol(), timeScaleUnits, redirContext, columnSource); + ema.timeScale().timestampCol(), timeScaleUnits, redirHelper, columnSource); } throw new IllegalArgumentException("Can not perform EMA on type " + csType); @@ -258,21 +258,21 @@ private UpdateByOperator makeEmaOperator(@NotNull final MatchPair pair, private UpdateByOperator makeCumProdOperator(MatchPair fc, TableDefaults source) { final Class csType = source.getColumnSource(fc.rightColumn).getType(); if (csType == byte.class || csType == Byte.class) { - return new ByteCumProdOperator(fc, redirContext); + return new ByteCumProdOperator(fc, redirHelper); } else if (csType == short.class || csType == Short.class) { - return new ShortCumProdOperator(fc, redirContext); + return new ShortCumProdOperator(fc, redirHelper); } else if (csType == int.class || csType == Integer.class) { - return new IntCumProdOperator(fc, redirContext); + return new IntCumProdOperator(fc, redirHelper); } else if (csType == long.class || csType == Long.class) { - return new LongCumProdOperator(fc, redirContext); + return new LongCumProdOperator(fc, redirHelper); } else if (csType == float.class || csType == Float.class) { - return new FloatCumProdOperator(fc, redirContext); + return new FloatCumProdOperator(fc, redirHelper); } else if (csType == double.class || csType == Double.class) { - return new DoubleCumProdOperator(fc, redirContext); + return new DoubleCumProdOperator(fc, redirHelper); } else if (csType == BigDecimal.class) { - return new BigDecimalCumProdOperator(fc, redirContext, control.mathContextOrDefault()); + return new BigDecimalCumProdOperator(fc, redirHelper, control.mathContextOrDefault()); } else if (csType == BigInteger.class) { - return new BigIntegerCumProdOperator(fc, redirContext); + return new BigIntegerCumProdOperator(fc, redirHelper); } throw new IllegalArgumentException("Can not perform Cumulative Min/Max on type " + csType); @@ -282,20 +282,20 @@ private UpdateByOperator makeCumMinMaxOperator(MatchPair fc, TableDefaults sourc final ColumnSource columnSource = source.getColumnSource(fc.rightColumn); final Class csType = columnSource.getType(); if (csType == byte.class || csType == Byte.class) { - return new ByteCumMinMaxOperator(fc, isMax, redirContext); + return new ByteCumMinMaxOperator(fc, isMax, redirHelper); } else if (csType == short.class || csType == Short.class) { - return new ShortCumMinMaxOperator(fc, isMax, redirContext); + return new ShortCumMinMaxOperator(fc, isMax, redirHelper); } else if (csType == int.class || csType == Integer.class) { - return new IntCumMinMaxOperator(fc, isMax, redirContext); + return new IntCumMinMaxOperator(fc, isMax, redirHelper); } else if (csType == long.class || csType == Long.class || isTimeType(csType)) { - return new LongCumMinMaxOperator(fc, isMax, redirContext, csType); + return new LongCumMinMaxOperator(fc, isMax, redirHelper, csType); } else if (csType == float.class || csType == Float.class) { - return new FloatCumMinMaxOperator(fc, isMax, redirContext); + return new FloatCumMinMaxOperator(fc, isMax, redirHelper); } else if (csType == double.class || csType == Double.class) { - return new DoubleCumMinMaxOperator(fc, isMax, redirContext); + return new DoubleCumMinMaxOperator(fc, isMax, redirHelper); } else if (Comparable.class.isAssignableFrom(csType)) { // noinspection unchecked,rawtypes - return new ComparableCumMinMaxOperator(csType, fc, isMax, redirContext); + return new ComparableCumMinMaxOperator(csType, fc, isMax, redirHelper); } throw new IllegalArgumentException("Can not perform Cumulative Min/Max on type " + csType); @@ -304,23 +304,23 @@ private UpdateByOperator makeCumMinMaxOperator(MatchPair fc, TableDefaults sourc private UpdateByOperator makeCumSumOperator(MatchPair fc, TableDefaults source) { final Class csType = source.getColumnSource(fc.rightColumn).getType(); if (csType == Boolean.class || csType == boolean.class) { - return new ByteCumSumOperator(fc, redirContext, NULL_BOOLEAN_AS_BYTE); + return new ByteCumSumOperator(fc, redirHelper, NULL_BOOLEAN_AS_BYTE); } else if (csType == byte.class || csType == Byte.class) { - return new ByteCumSumOperator(fc, redirContext, NULL_BYTE); + return new ByteCumSumOperator(fc, redirHelper, NULL_BYTE); } else if (csType == short.class || csType == Short.class) { - return new ShortCumSumOperator(fc, redirContext); + return new ShortCumSumOperator(fc, redirHelper); } else if (csType == int.class || csType == Integer.class) { - return new IntCumSumOperator(fc, redirContext); + return new IntCumSumOperator(fc, redirHelper); } else if (csType == long.class || csType == Long.class) { - return new LongCumSumOperator(fc, redirContext); + return new LongCumSumOperator(fc, redirHelper); } else if (csType == float.class || csType == Float.class) { - return new FloatCumSumOperator(fc, redirContext); + return new FloatCumSumOperator(fc, redirHelper); } else if (csType == double.class || csType == Double.class) { - return new DoubleCumSumOperator(fc, redirContext); + return new DoubleCumSumOperator(fc, redirHelper); } else if (csType == BigDecimal.class) { - return new BigDecimalCumSumOperator(fc, redirContext, control.mathContextOrDefault()); + return new BigDecimalCumSumOperator(fc, redirHelper, control.mathContextOrDefault()); } else if (csType == BigInteger.class) { - return new BigIntegerCumSumOperator(fc, redirContext); + return new BigIntegerCumSumOperator(fc, redirHelper); } throw new IllegalArgumentException("Can not perform Cumulative Sum on type " + csType); @@ -330,23 +330,23 @@ private UpdateByOperator makeForwardFillOperator(MatchPair fc, TableDefaults sou final ColumnSource columnSource = source.getColumnSource(fc.rightColumn); final Class csType = columnSource.getType(); if (csType == char.class || csType == Character.class) { - return new CharFillByOperator(fc, redirContext); + return new CharFillByOperator(fc, redirHelper); } else if (csType == byte.class || csType == Byte.class) { - return new ByteFillByOperator(fc, redirContext); + return new ByteFillByOperator(fc, redirHelper); } else if (csType == short.class || csType == Short.class) { - return new ShortFillByOperator(fc, redirContext); + return new ShortFillByOperator(fc, redirHelper); } else if (csType == int.class || csType == Integer.class) { - return new IntFillByOperator(fc, redirContext); + return new IntFillByOperator(fc, redirHelper); } else if (csType == long.class || csType == Long.class || isTimeType(csType)) { - return new LongFillByOperator(fc, redirContext, csType); + return new LongFillByOperator(fc, redirHelper, csType); } else if (csType == float.class || csType == Float.class) { - return new FloatFillByOperator(fc, redirContext); + return new FloatFillByOperator(fc, redirHelper); } else if (csType == double.class || csType == Double.class) { - return new DoubleFillByOperator(fc, redirContext); + return new DoubleFillByOperator(fc, redirHelper); } else if (csType == boolean.class || csType == Boolean.class) { - return new BooleanFillByOperator(fc, redirContext); + return new BooleanFillByOperator(fc, redirHelper); } else { - return new ObjectFillByOperator<>(fc, redirContext, csType); + return new ObjectFillByOperator<>(fc, redirHelper, csType); } } @@ -370,40 +370,40 @@ private UpdateByOperator makeRollingSumOperator(@NotNull final MatchPair pair, if (csType == Boolean.class || csType == boolean.class) { return new ByteRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, - redirContext, NULL_BOOLEAN_AS_BYTE); + redirHelper, NULL_BOOLEAN_AS_BYTE); } else if (csType == byte.class || csType == Byte.class) { return new ByteRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, - redirContext, NULL_BYTE); + redirHelper, NULL_BYTE); } else if (csType == short.class || csType == Short.class) { return new ShortRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, - redirContext); + redirHelper); } else if (csType == int.class || csType == Integer.class) { return new IntRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, - redirContext); + redirHelper); } else if (csType == long.class || csType == Long.class) { return new LongRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, - redirContext); + redirHelper); } else if (csType == float.class || csType == Float.class) { return new FloatRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, - redirContext); + redirHelper); } else if (csType == double.class || csType == Double.class) { return new DoubleRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), - prevTimeScaleUnits, fwdTimeScaleUnits, redirContext); + prevTimeScaleUnits, fwdTimeScaleUnits, redirHelper); } else if (csType == BigDecimal.class) { return new BigDecimalRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), - prevTimeScaleUnits, fwdTimeScaleUnits, redirContext, + prevTimeScaleUnits, fwdTimeScaleUnits, redirHelper, control.mathContextOrDefault()); } else if (csType == BigInteger.class) { return new BigIntegerRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), - prevTimeScaleUnits, fwdTimeScaleUnits, redirContext); + prevTimeScaleUnits, fwdTimeScaleUnits, redirHelper); } throw new IllegalArgumentException("Can not perform RollingSum on type " + csType); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java index 60bb11e9204..0ee64ab93ca 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java @@ -33,7 +33,7 @@ public abstract void accumulate(RowSequence inputKeys, * @param timestampColumnName the optional time stamp column for windowing (uses ticks if not provided) * @param reverseTimeScaleUnits the time (us) or ticks to extend the window backwards * @param forwardTimeScaleUnits the time (us) or ticks to extend the window forwards - * @param redirContext the row redirection context to use for the operation + * @param redirHelper the row redirection context to use for the operation */ public UpdateByWindowedOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @@ -41,9 +41,9 @@ public UpdateByWindowedOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper) { super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, - redirContext); + redirHelper); } /** diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateByManager.java index 4819f28eb4e..653e291b44e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateByManager.java @@ -11,36 +11,48 @@ import java.util.Map; public class ZeroKeyUpdateByManager extends UpdateBy { + /** The output table for this UpdateBy operation */ + final QueryTable result; + // this manager has only one bucket, managed by this object final UpdateByBucketHelper zeroKeyUpdateBy; - protected ZeroKeyUpdateByManager(@NotNull final String description, - @NotNull QueryTable source, + /** + * Perform an updateBy without any key columns. + * + * @param description the operation description + * @param operators the operations to perform + * @param windows the unique windows for this UpdateBy + * @param inputSources the primitive input sources + * @param operatorInputSourceSlots maps the operators to source indices + * @param source the source table + * @param resultSources the result sources + * @param timestampColumnName the column to use for all time-aware operators + * @param redirHelper the row redirection helper for dense output sources + * @param control the control object. + */ + protected ZeroKeyUpdateByManager( + @NotNull final String description, @NotNull UpdateByOperator[] operators, @NotNull UpdateByWindow[] windows, @NotNull ColumnSource[] inputSources, @NotNull int[][] operatorInputSourceSlots, + @NotNull QueryTable source, @NotNull final Map> resultSources, @Nullable String timestampColumnName, - @NotNull UpdateByRedirectionContext redirContext, + @NotNull UpdateByRedirectionHelper redirHelper, @NotNull UpdateByControl control) { - super(description, source, operators, windows, inputSources, operatorInputSourceSlots, resultSources, - timestampColumnName, redirContext, control); - - // this table will always have the rowset of the source - result = new QueryTable(source.getRowSet(), resultSources); + super(source, operators, windows, inputSources, operatorInputSourceSlots, timestampColumnName, redirHelper, + control); if (source.isRefreshing()) { - // this is a refreshing source, we will need a listener and recorders - recorders = new LinkedList<>(); - listener = newListener(description); + result = new QueryTable(source.getRowSet(), resultSources); - // create a recorder instance sourced from the source table - ListenerRecorder sourceRecorder = new ListenerRecorder(description, source, result); - sourceRecorder.setMergedListener(listener); - source.addUpdateListener(sourceRecorder); + // this is a refreshing source, we will need a listener + listener = newUpdateByListener(description); + source.addUpdateListener(listener); + // result will depend on listener result.addParentReference(listener); - recorders.offerLast(sourceRecorder); // create input and output modified column sets for (UpdateByOperator op : operators) { @@ -48,32 +60,23 @@ protected ZeroKeyUpdateByManager(@NotNull final String description, op.createOutputModifiedColumnSet(result); } - // create an updateby bucket instance sourced from the source table + // create an updateby bucket instance directly from the source table zeroKeyUpdateBy = new UpdateByBucketHelper(description, source, operators, windows, inputSources, - operatorInputSourceSlots, resultSources, timestampColumnName, redirContext, control); - buckets.offerLast(zeroKeyUpdateBy); + operatorInputSourceSlots, resultSources, timestampColumnName, redirHelper, control); + buckets.offer(zeroKeyUpdateBy); + + // make the source->result transformer + transformer = source.newModifiedColumnSetTransformer(result, source.getDefinition().getColumnNamesArray()); - // create a recorder instance sourced from the bucket helper - ListenerRecorder recorder = new ListenerRecorder(description, zeroKeyUpdateBy.result, result); - recorder.setMergedListener(listener); - zeroKeyUpdateBy.result.addUpdateListener(recorder); - recorders.offerLast(recorder); + // result will depend on zeroKeyUpdateBy + result.addParentReference(zeroKeyUpdateBy); } else { - // no shifting will be needed, can create directly from source zeroKeyUpdateBy = new UpdateByBucketHelper(description, source, operators, windows, inputSources, - operatorInputSourceSlots, resultSources, timestampColumnName, redirContext, control); - this.result = zeroKeyUpdateBy.result; - buckets.offerLast(zeroKeyUpdateBy); - - // create input modified column sets only - for (UpdateByOperator op : operators) { - op.createInputModifiedColumnSet(source); - } + operatorInputSourceSlots, resultSources, timestampColumnName, redirHelper, control); + result = zeroKeyUpdateBy.result; + buckets.offer(zeroKeyUpdateBy); } - // make the source->result transformer - transformer = source.newModifiedColumnSetTransformer(result, source.getDefinition().getColumnNamesArray()); - // make a dummy update to generate the initial row keys final TableUpdateImpl fakeUpdate = new TableUpdateImpl(source.getRowSet(), RowSetFactory.empty(), @@ -82,34 +85,18 @@ protected ZeroKeyUpdateByManager(@NotNull final String description, ModifiedColumnSet.EMPTY); // do the actual computations - final StateManager sm = new StateManager(fakeUpdate, true); + final PhasedUpdateProcessor sm = new PhasedUpdateProcessor(fakeUpdate, true); sm.processUpdate(); } - /** - * Perform an updateBy without any key columns. - * - * @param description the operation description - * @param source the source table - * @param operators the operations to perform - * @param resultSources the result sources - * @param redirContext the row redirection shared context - * @param control the control object. - * @return the result table - */ - public static Table compute(@NotNull final String description, - @NotNull final QueryTable source, - @NotNull final UpdateByOperator[] operators, - @NotNull final UpdateByWindow[] windows, - @NotNull final ColumnSource[] inputSources, - @NotNull final int[][] operatorInputSourceSlots, - @NotNull final Map> resultSources, - @Nullable final String timestampColumnName, - @NotNull final UpdateByRedirectionContext redirContext, - @NotNull final UpdateByControl control) { + @Override + protected QueryTable result() { + return result; + } - final ZeroKeyUpdateByManager manager = new ZeroKeyUpdateByManager(description, source, operators, windows, - inputSources, operatorInputSourceSlots, resultSources, timestampColumnName, redirContext, control); - return manager.result; + @Override + protected boolean upstreamSatisfied(final long step) { + // for Zero-Key, only need to verify the source is satisfied + return source.satisfied(step); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/RedirectionLayer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/RedirectionLayer.java index defbae50046..362d8644577 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/RedirectionLayer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/RedirectionLayer.java @@ -13,7 +13,6 @@ import io.deephaven.engine.table.ModifiedColumnSet; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.impl.util.*; -import io.deephaven.engine.table.impl.util.JobScheduler; import org.apache.commons.lang3.mutable.MutableLong; import org.apache.commons.lang3.mutable.MutableObject; import org.jetbrains.annotations.Nullable; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer.java index 95a5fd48de6..dcde5f2a120 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer.java @@ -1,4 +1,4 @@ - /** +/** * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending */ package io.deephaven.engine.table.impl.select.analyzers; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java index 00e79efae63..c3907b4c621 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java @@ -2,12 +2,8 @@ import io.deephaven.api.updateby.BadDataBehavior; import io.deephaven.api.updateby.OperationControl; -import io.deephaven.base.verify.Assert; -import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.locations.TableDataException; @@ -56,15 +52,15 @@ public void reset() { * integer ticks. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is * measured in ticks, otherwise it is measured in nanoseconds. - * @param redirContext the row redirection context to use for the EMA + * @param redirHelper the row redirection context to use for the EMA */ public BasePrimitiveEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { - super(pair, affectingColumns, redirContext); + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper) { + super(pair, affectingColumns, redirHelper); this.control = control; this.timestampColumnName = timestampColumnName; this.timeScaleUnits = (double) timeScaleUnits; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java index 7df243e9e5c..85f710200c0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java @@ -103,16 +103,16 @@ public void push(long key, int pos) { * integer ticks. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timeRecorder} is provided, this is measured * in ticks, otherwise it is measured in nanoseconds - * @param redirContext the row redirection context to use for the EMA + * @param redirHelper the row redirection context to use for the EMA */ public BigDecimalEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, final ColumnSource valueSource) { - super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, redirContext, valueSource); + super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, redirHelper, valueSource); } @NotNull diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java index d008f0bdfe1..e241a17864b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java @@ -105,16 +105,16 @@ public void push(long key, int pos) { * integer ticks. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timeRecorder} is provided, this is measured * in ticks, otherwise it is measured in nanoseconds - * @param redirContext the row redirection context to use for the EMA + * @param redirHelper the row redirection context to use for the EMA */ public BigIntegerEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, final ColumnSource valueSource) { - super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, redirContext, valueSource); + super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, redirHelper, valueSource); } @NotNull diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java index 5e50df22672..a1bb12c402b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java @@ -73,9 +73,9 @@ public BigNumberEMAOperator(@NotNull final MatchPair pair, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, final ColumnSource valueSource) { - super(pair, affectingColumns, redirContext, BigDecimal.class); + super(pair, affectingColumns, redirHelper, BigDecimal.class); this.control = control; this.timestampColumnName = timestampColumnName; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java index b6920051b81..99565c5479c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java @@ -121,12 +121,12 @@ public ByteEMAOperator(@NotNull final MatchPair pair, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits,redirContext); + super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits,redirHelper); this.valueSource = valueSource; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java index 212c9221adb..35b54f23ff4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java @@ -10,7 +10,6 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.DoubleChunk; import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.ColumnSource; @@ -138,12 +137,12 @@ public DoubleEMAOperator(@NotNull final MatchPair pair, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, redirContext); + super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, redirHelper); this.valueSource = valueSource; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java index e4b32717f94..79c9c27f3b2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java @@ -5,7 +5,6 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.FloatChunk; import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.ColumnSource; @@ -133,12 +132,12 @@ public FloatEMAOperator(@NotNull final MatchPair pair, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, redirContext); + super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, redirHelper); this.valueSource = valueSource; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java index 8887dae9029..4807d44f878 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java @@ -121,12 +121,12 @@ public IntEMAOperator(@NotNull final MatchPair pair, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits,redirContext); + super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits,redirHelper); this.valueSource = valueSource; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java index 17d1fd7407c..ba56d6152c0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java @@ -121,12 +121,12 @@ public LongEMAOperator(@NotNull final MatchPair pair, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits,redirContext); + super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits,redirHelper); this.valueSource = valueSource; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java index 4361f1d94c3..d209b07d1c5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java @@ -116,12 +116,12 @@ public ShortEMAOperator(@NotNull final MatchPair pair, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits,redirContext); + super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits,redirHelper); this.valueSource = valueSource; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java index ebf23616859..d61d76d1759 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java @@ -15,7 +15,6 @@ import io.deephaven.chunk.ByteChunk; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseByteUpdateByOperator; @@ -49,11 +48,11 @@ public void push(long key, int pos) { } public BooleanFillByOperator(@NotNull final MatchPair fillPair, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, redirContext); + super(fillPair, new String[] { fillPair.rightColumn }, redirHelper); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java index bc3d2ddeb60..c1271d06ebd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java @@ -8,7 +8,6 @@ import io.deephaven.chunk.ByteChunk; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseByteUpdateByOperator; @@ -42,11 +41,11 @@ public void push(long key, int pos) { } public ByteFillByOperator(@NotNull final MatchPair fillPair, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, redirContext); + super(fillPair, new String[] { fillPair.rightColumn }, redirHelper); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java index e6f8dd97797..b4326f8fc90 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java @@ -3,7 +3,6 @@ import io.deephaven.chunk.CharChunk; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseCharUpdateByOperator; @@ -37,11 +36,11 @@ public void push(long key, int pos) { } public CharFillByOperator(@NotNull final MatchPair fillPair, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, redirContext); + super(fillPair, new String[] { fillPair.rightColumn }, redirHelper); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java index 9487194c57d..7245bc3a440 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java @@ -8,7 +8,6 @@ import io.deephaven.chunk.DoubleChunk; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseDoubleUpdateByOperator; @@ -42,11 +41,11 @@ public void push(long key, int pos) { } public DoubleFillByOperator(@NotNull final MatchPair fillPair, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, redirContext); + super(fillPair, new String[] { fillPair.rightColumn }, redirHelper); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java index 3b9499b4a1c..db23fcec3f7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java @@ -8,7 +8,6 @@ import io.deephaven.chunk.FloatChunk; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseFloatUpdateByOperator; @@ -42,11 +41,11 @@ public void push(long key, int pos) { } public FloatFillByOperator(@NotNull final MatchPair fillPair, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, redirContext); + super(fillPair, new String[] { fillPair.rightColumn }, redirHelper); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java index 290a1914723..fd477ddcb34 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java @@ -8,7 +8,6 @@ import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseIntUpdateByOperator; @@ -42,11 +41,11 @@ public void push(long key, int pos) { } public IntFillByOperator(@NotNull final MatchPair fillPair, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, redirContext); + super(fillPair, new String[] { fillPair.rightColumn }, redirHelper); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java index aa3f9007f10..a436838ebcc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java @@ -15,7 +15,6 @@ import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; @@ -50,12 +49,12 @@ public void push(long key, int pos) { } public LongFillByOperator(@NotNull final MatchPair fillPair, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args ,@NotNull final Class type // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, redirContext); + super(fillPair, new String[] { fillPair.rightColumn }, redirHelper); // region constructor this.type = type; // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java index 90fc1ba4774..033fd88117c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java @@ -8,7 +8,6 @@ import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; @@ -41,12 +40,12 @@ public void push(long key, int pos) { } public ObjectFillByOperator(@NotNull final MatchPair fillPair, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args , final Class colType // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, redirContext, colType); + super(fillPair, new String[] { fillPair.rightColumn }, redirHelper, colType); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java index 867052661b0..bfe6f5b54f5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java @@ -8,7 +8,6 @@ import io.deephaven.chunk.ShortChunk; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseShortUpdateByOperator; @@ -42,11 +41,11 @@ public void push(long key, int pos) { } public ShortFillByOperator(@NotNull final MatchPair fillPair, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, redirContext); + super(fillPair, new String[] { fillPair.rightColumn }, redirHelper); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java index 9ce76e2c256..79e948cb68c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java @@ -10,11 +10,9 @@ import io.deephaven.engine.table.impl.sources.ByteSparseArraySource; import io.deephaven.engine.table.WritableColumnSource; -import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableByteChunk; -import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; @@ -105,20 +103,20 @@ public void reset() { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param redirContext the {@link UpdateBy.UpdateByRedirectionContext} for the overall update + * @param redirHelper the {@link UpdateBy.UpdateByRedirectionHelper} for the overall update */ public BaseByteUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, redirContext); - if(this.redirContext.isRedirected()) { + super(pair, affectingColumns, redirHelper); + if(this.redirHelper.isRedirected()) { // region create-dense this.maybeInnerSource = makeDenseSource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(this.redirContext.getRowRedirection(), maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(this.redirHelper.getRowRedirection(), maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -160,7 +158,7 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (redirContext.isRedirected()) { + if (redirHelper.isRedirected()) { maybeInnerSource.startTrackingPrevValues(); } } @@ -178,7 +176,7 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d @Override public void prepareForParallelPopulation(final RowSet added) { - if (redirContext.isRedirected()) { + if (redirHelper.isRedirected()) { ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java index a32860545d7..b4f0881e7a6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java @@ -1,10 +1,8 @@ package io.deephaven.engine.table.impl.updateby.internal; -import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableCharChunk; -import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; @@ -94,20 +92,20 @@ public void reset() { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param redirContext the {@link UpdateBy.UpdateByRedirectionContext} for the overall update + * @param redirHelper the {@link UpdateBy.UpdateByRedirectionHelper} for the overall update */ public BaseCharUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, redirContext); - if(this.redirContext.isRedirected()) { + super(pair, affectingColumns, redirHelper); + if(this.redirHelper.isRedirected()) { // region create-dense this.maybeInnerSource = new CharacterArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(this.redirContext.getRowRedirection(), maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(this.redirHelper.getRowRedirection(), maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -136,7 +134,7 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (redirContext.isRedirected()) { + if (redirHelper.isRedirected()) { maybeInnerSource.startTrackingPrevValues(); } } @@ -150,7 +148,7 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d @Override public void prepareForParallelPopulation(final RowSet added) { - if (redirContext.isRedirected()) { + if (redirHelper.isRedirected()) { ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java index 84c3f829b7a..8893a7dea26 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java @@ -5,27 +5,22 @@ */ package io.deephaven.engine.table.impl.updateby.internal; -import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.WritableDoubleChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import java.util.Collections; import java.util.Map; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; -import static io.deephaven.util.QueryConstants.NULL_CHAR; import static io.deephaven.util.QueryConstants.NULL_DOUBLE; public abstract class BaseDoubleUpdateByOperator extends UpdateByCumulativeOperator { @@ -101,20 +96,20 @@ public void reset() { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param redirContext the {@link UpdateBy.UpdateByRedirectionContext} for the overall update + * @param redirHelper the {@link UpdateBy.UpdateByRedirectionHelper} for the overall update */ public BaseDoubleUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, redirContext); - if(this.redirContext.isRedirected()) { + super(pair, affectingColumns, redirHelper); + if(this.redirHelper.isRedirected()) { // region create-dense this.maybeInnerSource = new DoubleArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(this.redirContext.getRowRedirection(), maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(this.redirHelper.getRowRedirection(), maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -142,7 +137,7 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (redirContext.isRedirected()) { + if (redirHelper.isRedirected()) { maybeInnerSource.startTrackingPrevValues(); } } @@ -156,7 +151,7 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long @Override public void prepareForParallelPopulation(final RowSet added) { - if (redirContext.isRedirected()) { + if (redirHelper.isRedirected()) { ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java index 94ada3c63ea..c3bc55d010b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java @@ -1,26 +1,21 @@ package io.deephaven.engine.table.impl.updateby.internal; -import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.WritableFloatChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import java.util.Collections; import java.util.Map; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; -import static io.deephaven.util.QueryConstants.NULL_CHAR; import static io.deephaven.util.QueryConstants.NULL_FLOAT; public abstract class BaseFloatUpdateByOperator extends UpdateByCumulativeOperator { @@ -96,20 +91,20 @@ public void reset() { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param redirContext the {@link UpdateBy.UpdateByRedirectionContext} for the overall update + * @param redirHelper the {@link UpdateBy.UpdateByRedirectionHelper} for the overall update */ public BaseFloatUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, redirContext); - if(this.redirContext.isRedirected()) { + super(pair, affectingColumns, redirHelper); + if(this.redirHelper.isRedirected()) { // region create-dense this.maybeInnerSource = new FloatArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(this.redirContext.getRowRedirection(), maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(this.redirHelper.getRowRedirection(), maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -137,7 +132,7 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (redirContext.isRedirected()) { + if (redirHelper.isRedirected()) { maybeInnerSource.startTrackingPrevValues(); } } @@ -151,7 +146,7 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long @Override public void prepareForParallelPopulation(final RowSet added) { - if (redirContext.isRedirected()) { + if (redirHelper.isRedirected()) { ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java index 9722bcd9180..9f4a0f6a415 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java @@ -5,11 +5,9 @@ */ package io.deephaven.engine.table.impl.updateby.internal; -import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableIntChunk; -import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; @@ -99,20 +97,20 @@ public void reset() { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param redirContext the {@link UpdateBy.UpdateByRedirectionContext} for the overall update + * @param redirHelper the {@link UpdateBy.UpdateByRedirectionHelper} for the overall update */ public BaseIntUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, redirContext); - if(this.redirContext.isRedirected()) { + super(pair, affectingColumns, redirHelper); + if(this.redirHelper.isRedirected()) { // region create-dense this.maybeInnerSource = new IntegerArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(this.redirContext.getRowRedirection(), maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(this.redirHelper.getRowRedirection(), maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -141,7 +139,7 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (redirContext.isRedirected()) { + if (redirHelper.isRedirected()) { maybeInnerSource.startTrackingPrevValues(); } } @@ -155,7 +153,7 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d @Override public void prepareForParallelPopulation(final RowSet added) { - if (redirContext.isRedirected()) { + if (redirHelper.isRedirected()) { ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java index 9c514fa7e4c..8fadb78bbda 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java @@ -5,11 +5,9 @@ */ package io.deephaven.engine.table.impl.updateby.internal; -import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableLongChunk; -import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; @@ -99,20 +97,20 @@ public void reset() { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param redirContext the {@link UpdateBy.UpdateByRedirectionContext} for the overall update + * @param redirHelper the {@link UpdateBy.UpdateByRedirectionHelper} for the overall update */ public BaseLongUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, redirContext); - if(this.redirContext.isRedirected()) { + super(pair, affectingColumns, redirHelper); + if(this.redirHelper.isRedirected()) { // region create-dense this.maybeInnerSource = new LongArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(this.redirContext.getRowRedirection(), maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(this.redirHelper.getRowRedirection(), maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -141,7 +139,7 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (redirContext.isRedirected()) { + if (redirHelper.isRedirected()) { maybeInnerSource.startTrackingPrevValues(); } } @@ -155,7 +153,7 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d @Override public void prepareForParallelPopulation(final RowSet added) { - if (redirContext.isRedirected()) { + if (redirHelper.isRedirected()) { ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java index 2319cde7dc4..1499781c842 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java @@ -1,19 +1,11 @@ package io.deephaven.engine.table.impl.updateby.internal; -import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; -import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; - -import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; public abstract class BaseObjectBinaryOperator extends BaseObjectUpdateByOperator { protected class Context extends BaseObjectUpdateByOperator.Context { @@ -48,8 +40,8 @@ public void reset() { public BaseObjectBinaryOperator(@NotNull final Class type, @NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { - super(pair, affectingColumns, redirContext, type); + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper) { + super(pair, affectingColumns, redirHelper, type); } protected abstract T doOperation(T bucketCurVal, T chunkCurVal); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java index d1ebb6da64d..9ba3225de85 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java @@ -5,11 +5,9 @@ */ package io.deephaven.engine.table.impl.updateby.internal; -import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableObjectChunk; -import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; @@ -100,21 +98,21 @@ public void reset() { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param redirContext the {@link UpdateBy.UpdateByRedirectionContext} for the overall update + * @param redirHelper the {@link UpdateBy.UpdateByRedirectionHelper} for the overall update */ public BaseObjectUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args , final Class colType // endregion extra-constructor-args ) { - super(pair, affectingColumns, redirContext); - if(this.redirContext.isRedirected()) { + super(pair, affectingColumns, redirHelper); + if(this.redirHelper.isRedirected()) { // region create-dense this.maybeInnerSource = new ObjectArraySource(colType); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(this.redirContext.getRowRedirection(), maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(this.redirHelper.getRowRedirection(), maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -144,7 +142,7 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (redirContext.isRedirected()) { + if (redirHelper.isRedirected()) { maybeInnerSource.startTrackingPrevValues(); } } @@ -158,7 +156,7 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d @Override public void prepareForParallelPopulation(final RowSet added) { - if (redirContext.isRedirected()) { + if (redirHelper.isRedirected()) { ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java index 3c9cf19f648..4fdaed13c66 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java @@ -5,11 +5,9 @@ */ package io.deephaven.engine.table.impl.updateby.internal; -import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableShortChunk; -import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; @@ -99,20 +97,20 @@ public void reset() { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param redirContext the {@link UpdateBy.UpdateByRedirectionContext} for the overall update + * @param redirHelper the {@link UpdateBy.UpdateByRedirectionHelper} for the overall update */ public BaseShortUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, redirContext); - if(this.redirContext.isRedirected()) { + super(pair, affectingColumns, redirHelper); + if(this.redirHelper.isRedirected()) { // region create-dense this.maybeInnerSource = new ShortArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(this.redirContext.getRowRedirection(), maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(this.redirHelper.getRowRedirection(), maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -141,7 +139,7 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (redirContext.isRedirected()) { + if (redirHelper.isRedirected()) { maybeInnerSource.startTrackingPrevValues(); } } @@ -155,7 +153,7 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d @Override public void prepareForParallelPopulation(final RowSet added) { - if (redirContext.isRedirected()) { + if (redirHelper.isRedirected()) { ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java index 6761eb627f6..970cb0b9427 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java @@ -16,9 +16,7 @@ import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableByteChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.sized.SizedByteChunk; import io.deephaven.engine.rowset.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; @@ -122,16 +120,16 @@ public BaseWindowedByteUpdateByOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); - if (this.redirContext.isRedirected()) { + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirHelper); + if (this.redirHelper.isRedirected()) { // region create-dense this.maybeInnerSource = makeDenseSource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(this.redirContext.getRowRedirection(), maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(this.redirHelper.getRowRedirection(), maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -166,7 +164,7 @@ public void initializeUpdate(@NotNull UpdateContext context) { @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (redirContext.isRedirected()) { + if (redirHelper.isRedirected()) { maybeInnerSource.startTrackingPrevValues(); } } @@ -184,7 +182,7 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d @Override public void prepareForParallelPopulation(final RowSet added) { - if (redirContext.isRedirected()) { + if (redirHelper.isRedirected()) { ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java index c23a9505c1b..312bf8bed30 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java @@ -6,9 +6,7 @@ import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableCharChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.sized.SizedCharChunk; import io.deephaven.engine.rowset.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; @@ -111,16 +109,16 @@ public BaseWindowedCharUpdateByOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); - if (this.redirContext.isRedirected()) { + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirHelper); + if (this.redirHelper.isRedirected()) { // region create-dense this.maybeInnerSource = new CharacterArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(this.redirContext.getRowRedirection(), maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(this.redirHelper.getRowRedirection(), maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -142,7 +140,7 @@ public void initializeUpdate(@NotNull UpdateContext context) { @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (redirContext.isRedirected()) { + if (redirHelper.isRedirected()) { maybeInnerSource.startTrackingPrevValues(); } } @@ -156,7 +154,7 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d @Override public void prepareForParallelPopulation(final RowSet added) { - if (redirContext.isRedirected()) { + if (redirHelper.isRedirected()) { ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java index d306f4eb76e..216148e25cb 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java @@ -9,13 +9,11 @@ import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.sources.DoubleArraySource; import io.deephaven.engine.table.impl.sources.DoubleSparseArraySource; -import io.deephaven.engine.table.impl.sources.SparseArrayColumnSource; import io.deephaven.engine.table.impl.sources.WritableRedirectedColumnSource; import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; @@ -25,7 +23,6 @@ import java.util.Map; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; -import static io.deephaven.util.QueryConstants.NULL_CHAR; import static io.deephaven.util.QueryConstants.NULL_DOUBLE; public abstract class BaseWindowedDoubleUpdateByOperator extends UpdateByWindowedOperator { @@ -116,16 +113,16 @@ public BaseWindowedDoubleUpdateByOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); - if(this.redirContext.isRedirected()) { + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirHelper); + if(this.redirHelper.isRedirected()) { // region create-dense this.maybeInnerSource = new DoubleArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(this.redirContext.getRowRedirection(), maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(this.redirHelper.getRowRedirection(), maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -147,7 +144,7 @@ public void initializeUpdate(@NotNull UpdateContext context) { @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (redirContext.isRedirected()) { + if (redirHelper.isRedirected()) { maybeInnerSource.startTrackingPrevValues(); } } @@ -161,7 +158,7 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d @Override public void prepareForParallelPopulation(final RowSet added) { - if (redirContext.isRedirected()) { + if (redirHelper.isRedirected()) { ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java index 60c2ff27d77..f5d4692d083 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java @@ -4,13 +4,11 @@ import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.sources.FloatArraySource; import io.deephaven.engine.table.impl.sources.FloatSparseArraySource; -import io.deephaven.engine.table.impl.sources.SparseArrayColumnSource; import io.deephaven.engine.table.impl.sources.WritableRedirectedColumnSource; import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; @@ -20,7 +18,6 @@ import java.util.Map; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; -import static io.deephaven.util.QueryConstants.NULL_CHAR; import static io.deephaven.util.QueryConstants.NULL_FLOAT; public abstract class BaseWindowedFloatUpdateByOperator extends UpdateByWindowedOperator { @@ -111,16 +108,16 @@ public BaseWindowedFloatUpdateByOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); - if(this.redirContext.isRedirected()) { + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirHelper); + if(this.redirHelper.isRedirected()) { // region create-dense this.maybeInnerSource = new FloatArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(this.redirContext.getRowRedirection(), maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(this.redirHelper.getRowRedirection(), maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -142,7 +139,7 @@ public void initializeUpdate(@NotNull UpdateContext context) { @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (redirContext.isRedirected()) { + if (redirHelper.isRedirected()) { maybeInnerSource.startTrackingPrevValues(); } } @@ -156,7 +153,7 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d @Override public void prepareForParallelPopulation(final RowSet added) { - if (redirContext.isRedirected()) { + if (redirHelper.isRedirected()) { ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java index 850b72ac209..680c8198d69 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java @@ -11,9 +11,7 @@ import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.sized.SizedIntChunk; import io.deephaven.engine.rowset.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; @@ -116,16 +114,16 @@ public BaseWindowedIntUpdateByOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); - if (this.redirContext.isRedirected()) { + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirHelper); + if (this.redirHelper.isRedirected()) { // region create-dense this.maybeInnerSource = new IntegerArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(this.redirContext.getRowRedirection(), maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(this.redirHelper.getRowRedirection(), maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -147,7 +145,7 @@ public void initializeUpdate(@NotNull UpdateContext context) { @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (redirContext.isRedirected()) { + if (redirHelper.isRedirected()) { maybeInnerSource.startTrackingPrevValues(); } } @@ -161,7 +159,7 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d @Override public void prepareForParallelPopulation(final RowSet added) { - if (redirContext.isRedirected()) { + if (redirHelper.isRedirected()) { ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java index 8a01898cdf4..c41ad2d401b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java @@ -11,9 +11,7 @@ import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableLongChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.sized.SizedLongChunk; import io.deephaven.engine.rowset.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; @@ -116,16 +114,16 @@ public BaseWindowedLongUpdateByOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); - if (this.redirContext.isRedirected()) { + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirHelper); + if (this.redirHelper.isRedirected()) { // region create-dense this.maybeInnerSource = new LongArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(this.redirContext.getRowRedirection(), maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(this.redirHelper.getRowRedirection(), maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -147,7 +145,7 @@ public void initializeUpdate(@NotNull UpdateContext context) { @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (redirContext.isRedirected()) { + if (redirHelper.isRedirected()) { maybeInnerSource.startTrackingPrevValues(); } } @@ -161,7 +159,7 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d @Override public void prepareForParallelPopulation(final RowSet added) { - if (redirContext.isRedirected()) { + if (redirHelper.isRedirected()) { ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java index 2ba01f20634..05e69164f6f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java @@ -11,9 +11,7 @@ import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.sized.SizedObjectChunk; import io.deephaven.engine.rowset.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; @@ -117,17 +115,17 @@ public BaseWindowedObjectUpdateByOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args , final Class colType // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); - if (this.redirContext.isRedirected()) { + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirHelper); + if (this.redirHelper.isRedirected()) { // region create-dense this.maybeInnerSource = new ObjectArraySource(colType); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(this.redirContext.getRowRedirection(), maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(this.redirHelper.getRowRedirection(), maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -150,7 +148,7 @@ public void initializeUpdate(@NotNull UpdateContext context) { @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (redirContext.isRedirected()) { + if (redirHelper.isRedirected()) { maybeInnerSource.startTrackingPrevValues(); } } @@ -164,7 +162,7 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d @Override public void prepareForParallelPopulation(final RowSet added) { - if (redirContext.isRedirected()) { + if (redirHelper.isRedirected()) { ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java index 53cdc374415..fc2b44253f4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java @@ -11,9 +11,7 @@ import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableShortChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.sized.SizedShortChunk; import io.deephaven.engine.rowset.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; @@ -116,16 +114,16 @@ public BaseWindowedShortUpdateByOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); - if (this.redirContext.isRedirected()) { + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirHelper); + if (this.redirHelper.isRedirected()) { // region create-dense this.maybeInnerSource = new ShortArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(this.redirContext.getRowRedirection(), maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(this.redirHelper.getRowRedirection(), maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -147,7 +145,7 @@ public void initializeUpdate(@NotNull UpdateContext context) { @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (redirContext.isRedirected()) { + if (redirHelper.isRedirected()) { maybeInnerSource.startTrackingPrevValues(); } } @@ -161,7 +159,7 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d @Override public void prepareForParallelPopulation(final RowSet added) { - if (redirContext.isRedirected()) { + if (redirHelper.isRedirected()) { ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java index b9c76e793bf..1713bf7ee28 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java @@ -8,7 +8,6 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ByteChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseByteUpdateByOperator; @@ -52,11 +51,11 @@ public void push(long key, int pos) { public ByteCumMinMaxOperator(@NotNull final MatchPair pair, final boolean isMax, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, redirContext); + super(pair, new String[] { pair.rightColumn }, redirHelper); this.isMax = isMax; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ComparableCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ComparableCumMinMaxOperator.java index d4ad7371e40..d4117ce378a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ComparableCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ComparableCumMinMaxOperator.java @@ -3,9 +3,7 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectBinaryOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; public final class ComparableCumMinMaxOperator> extends BaseObjectBinaryOperator { private final boolean isMax; @@ -13,8 +11,8 @@ public final class ComparableCumMinMaxOperator> extends public ComparableCumMinMaxOperator(final Class colType, @NotNull final MatchPair inputPair, final boolean isMax, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { - super(colType, inputPair, new String[] {inputPair.rightColumn}, redirContext); + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper) { + super(colType, inputPair, new String[] {inputPair.rightColumn}, redirHelper); this.isMax = isMax; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java index 0e92207c541..3d36b84f9b1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java @@ -8,7 +8,6 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.DoubleChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseDoubleUpdateByOperator; @@ -52,11 +51,11 @@ public void push(long key, int pos) { public DoubleCumMinMaxOperator(@NotNull final MatchPair pair, final boolean isMax, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, redirContext); + super(pair, new String[] { pair.rightColumn }, redirHelper); this.isMax = isMax; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java index 91af03ec7c8..d43cda243b9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java @@ -3,7 +3,6 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.FloatChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseFloatUpdateByOperator; @@ -47,11 +46,11 @@ public void push(long key, int pos) { public FloatCumMinMaxOperator(@NotNull final MatchPair pair, final boolean isMax, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, redirContext); + super(pair, new String[] { pair.rightColumn }, redirHelper); this.isMax = isMax; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java index 72ab41299c0..6d07c45d278 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java @@ -8,7 +8,6 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseIntUpdateByOperator; @@ -52,11 +51,11 @@ public void push(long key, int pos) { public IntCumMinMaxOperator(@NotNull final MatchPair pair, final boolean isMax, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, redirContext); + super(pair, new String[] { pair.rightColumn }, redirHelper); this.isMax = isMax; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java index 48754aa1bff..4a9496aabff 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java @@ -15,7 +15,6 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; @@ -60,12 +59,12 @@ public void push(long key, int pos) { public LongCumMinMaxOperator(@NotNull final MatchPair pair, final boolean isMax, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args ,@NotNull final Class type // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, redirContext); + super(pair, new String[] { pair.rightColumn }, redirHelper); this.isMax = isMax; // region constructor this.type = type; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java index a3af67e5540..c450e15ff62 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java @@ -3,7 +3,6 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ShortChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseShortUpdateByOperator; @@ -47,11 +46,11 @@ public void push(long key, int pos) { public ShortCumMinMaxOperator(@NotNull final MatchPair pair, final boolean isMax, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, redirContext); + super(pair, new String[] { pair.rightColumn }, redirHelper); this.isMax = isMax; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java index f9a6253b218..f41d6211cc0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java @@ -3,7 +3,6 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; @@ -12,8 +11,6 @@ import java.math.BigDecimal; import java.math.MathContext; -import static io.deephaven.util.QueryConstants.NULL_FLOAT; - public final class BigDecimalCumProdOperator extends BaseObjectUpdateByOperator { @NotNull private final MathContext mathContext; @@ -52,9 +49,9 @@ public void reset() { } public BigDecimalCumProdOperator(@NotNull final MatchPair inputPair, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, @NotNull final MathContext mathContext) { - super(inputPair, new String[] {inputPair.rightColumn}, redirContext, BigDecimal.class); + super(inputPair, new String[] {inputPair.rightColumn}, redirHelper, BigDecimal.class); this.mathContext = mathContext; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java index f4fa9a20263..eb8ef6f62ec 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java @@ -3,7 +3,6 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; @@ -11,8 +10,6 @@ import java.math.BigInteger; -import static io.deephaven.util.QueryConstants.NULL_FLOAT; - public final class BigIntegerCumProdOperator extends BaseObjectUpdateByOperator { protected class Context extends BaseObjectUpdateByOperator.Context { public ObjectChunk objectValueChunk; @@ -48,8 +45,8 @@ public void reset() { } public BigIntegerCumProdOperator(@NotNull final MatchPair inputPair, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { - super(inputPair, new String[] {inputPair.rightColumn}, redirContext, BigInteger.class); + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper) { + super(inputPair, new String[] {inputPair.rightColumn}, redirHelper, BigInteger.class); } @NotNull diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java index ee708671c89..7f71f13f9b2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java @@ -8,7 +8,6 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ByteChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; @@ -46,11 +45,11 @@ public void push(long key, int pos) { } public ByteCumProdOperator(@NotNull final MatchPair pair, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, redirContext); + super(pair, new String[] { pair.rightColumn }, redirHelper); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java index 0950e93b070..3b7e1ad5983 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java @@ -8,14 +8,12 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.DoubleChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseDoubleUpdateByOperator; import org.jetbrains.annotations.NotNull; import static io.deephaven.util.QueryConstants.NULL_DOUBLE; -import static io.deephaven.util.QueryConstants.NULL_SHORT; public class DoubleCumProdOperator extends BaseDoubleUpdateByOperator { // region extra-fields @@ -47,11 +45,11 @@ public void push(long key, int pos) { } public DoubleCumProdOperator(@NotNull final MatchPair pair, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, redirContext); + super(pair, new String[] { pair.rightColumn }, redirHelper); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java index d300323cd8e..a625da06b7e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java @@ -3,14 +3,12 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.FloatChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseFloatUpdateByOperator; import org.jetbrains.annotations.NotNull; import static io.deephaven.util.QueryConstants.NULL_FLOAT; -import static io.deephaven.util.QueryConstants.NULL_SHORT; public class FloatCumProdOperator extends BaseFloatUpdateByOperator { // region extra-fields @@ -42,11 +40,11 @@ public void push(long key, int pos) { } public FloatCumProdOperator(@NotNull final MatchPair pair, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, redirContext); + super(pair, new String[] { pair.rightColumn }, redirHelper); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java index 64a0be386f3..c0497983d23 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java @@ -8,7 +8,6 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; @@ -46,11 +45,11 @@ public void push(long key, int pos) { } public IntCumProdOperator(@NotNull final MatchPair pair, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, redirContext); + super(pair, new String[] { pair.rightColumn }, redirHelper); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java index da13a95591f..47213098646 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java @@ -8,7 +8,6 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; @@ -46,11 +45,11 @@ public void push(long key, int pos) { } public LongCumProdOperator(@NotNull final MatchPair pair, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, redirContext); + super(pair, new String[] { pair.rightColumn }, redirHelper); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java index 18bb22419ba..dcbfbb462db 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java @@ -3,7 +3,6 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ShortChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; @@ -41,11 +40,11 @@ public void push(long key, int pos) { } public ShortCumProdOperator(@NotNull final MatchPair pair, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, redirContext); + super(pair, new String[] { pair.rightColumn }, redirHelper); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java index 0a0a25d20cb..979019adff6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java @@ -12,7 +12,6 @@ import org.jetbrains.annotations.Nullable; import java.math.BigDecimal; -import java.math.BigInteger; import java.math.MathContext; public final class BigDecimalRollingSumOperator extends BaseWindowedObjectUpdateByOperator { @@ -91,10 +90,10 @@ public BigDecimalRollingSumOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, @NotNull final MathContext mathContext) { super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, - redirContext, BigDecimal.class); + redirHelper, BigDecimal.class); this.mathContext = mathContext; } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java index 699dc14bffa..c11c6f7cefa 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java @@ -13,8 +13,6 @@ import java.math.BigInteger; -import static io.deephaven.util.QueryConstants.NULL_DOUBLE; - public final class BigIntegerRollingSumOperator extends BaseWindowedObjectUpdateByOperator { protected class Context extends BaseWindowedObjectUpdateByOperator.Context { protected ObjectChunk objectInfluencerValuesChunk; @@ -88,11 +86,11 @@ public BigIntegerRollingSumOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext, BigInteger.class); + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirHelper, BigInteger.class); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java index 0e6f65dff1d..ca21babcc67 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java @@ -10,7 +10,6 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ByteChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedLongUpdateByOperator; @@ -97,12 +96,12 @@ public ByteRollingSumOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args ,final byte nullValue // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirHelper); // region constructor this.nullValue = nullValue; // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java index f61df2b4eac..6f413e2e751 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java @@ -95,11 +95,11 @@ public DoubleRollingSumOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirHelper); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java index 02afe940f2e..ca61d385909 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java @@ -90,11 +90,11 @@ public FloatRollingSumOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirHelper); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java index e75b5de169e..783a575bed7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java @@ -10,7 +10,6 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedLongUpdateByOperator; @@ -96,11 +95,11 @@ public IntRollingSumOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirHelper); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java index b850d56f430..56a8e608db7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java @@ -10,7 +10,6 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedLongUpdateByOperator; @@ -96,11 +95,11 @@ public LongRollingSumOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirHelper); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java index 9f552a0daca..5267e5f3546 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java @@ -5,7 +5,6 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ShortChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedLongUpdateByOperator; @@ -91,11 +90,11 @@ public ShortRollingSumOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirContext); + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirHelper); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java index 92fa490ae74..d498f600dcc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java @@ -3,7 +3,6 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; @@ -12,8 +11,6 @@ import java.math.BigDecimal; import java.math.MathContext; -import static io.deephaven.util.QueryConstants.NULL_CHAR; - public final class BigDecimalCumSumOperator extends BaseObjectUpdateByOperator { @NotNull private final MathContext mathContext; @@ -48,9 +45,9 @@ public void push(long key, int pos) { } public BigDecimalCumSumOperator(@NotNull final MatchPair inputPair, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext, + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, @NotNull final MathContext mathContext) { - super(inputPair, new String[] {inputPair.rightColumn}, redirContext, BigDecimal.class); + super(inputPair, new String[] {inputPair.rightColumn}, redirHelper, BigDecimal.class); this.mathContext = mathContext; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java index adc9953eafb..9d6667941ff 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java @@ -3,7 +3,6 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; @@ -11,8 +10,6 @@ import java.math.BigInteger; -import static io.deephaven.util.QueryConstants.NULL_CHAR; - public final class BigIntegerCumSumOperator extends BaseObjectUpdateByOperator { protected class Context extends BaseObjectUpdateByOperator.Context { public ObjectChunk objectValueChunk; @@ -43,8 +40,8 @@ public void push(long key, int pos) { } public BigIntegerCumSumOperator(@NotNull final MatchPair inputPair, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext) { - super(inputPair, new String[] {inputPair.rightColumn}, redirContext, BigInteger.class); + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper) { + super(inputPair, new String[] {inputPair.rightColumn}, redirHelper, BigInteger.class); } @NotNull diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java index f8f3d256f00..f503861abcd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java @@ -7,7 +7,6 @@ import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; @@ -46,12 +45,12 @@ public void push(long key, int pos) { } public ByteCumSumOperator(@NotNull final MatchPair pair, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args ,final byte nullValue // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, redirContext); + super(pair, new String[] { pair.rightColumn }, redirHelper); // region constructor this.nullValue = nullValue; // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java index d1f2d55b454..c73729aeaf5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java @@ -8,7 +8,6 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.DoubleChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseDoubleUpdateByOperator; @@ -46,11 +45,11 @@ public void push(long key, int pos) { } public DoubleCumSumOperator(@NotNull final MatchPair pair, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, redirContext); + super(pair, new String[] { pair.rightColumn }, redirHelper); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java index d208b59ab5b..199538d402d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java @@ -3,7 +3,6 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.FloatChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseFloatUpdateByOperator; @@ -41,11 +40,11 @@ public void push(long key, int pos) { } public FloatCumSumOperator(@NotNull final MatchPair pair, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, redirContext); + super(pair, new String[] { pair.rightColumn }, redirHelper); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java index ffed2bfea7e..0c23f60b076 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java @@ -7,7 +7,6 @@ import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; @@ -45,11 +44,11 @@ public void push(long key, int pos) { } public IntCumSumOperator(@NotNull final MatchPair pair, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, redirContext); + super(pair, new String[] { pair.rightColumn }, redirHelper); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java index 4dea4b48563..de9aabb1273 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java @@ -7,7 +7,6 @@ import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; @@ -45,11 +44,11 @@ public void push(long key, int pos) { } public LongCumSumOperator(@NotNull final MatchPair pair, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, redirContext); + super(pair, new String[] { pair.rightColumn }, redirHelper); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java index 2cf38220460..bc400f434e3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java @@ -2,7 +2,6 @@ import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; @@ -40,11 +39,11 @@ public void push(long key, int pos) { } public ShortCumSumOperator(@NotNull final MatchPair pair, - @NotNull final UpdateBy.UpdateByRedirectionContext redirContext + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, redirContext); + super(pair, new String[] { pair.rightColumn }, redirHelper); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/InverseRowRedirectionImpl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/InverseRowRedirectionImpl.java index 23b84ba2e3a..ec4d9c2d123 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/InverseRowRedirectionImpl.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/InverseRowRedirectionImpl.java @@ -18,10 +18,17 @@ public class InverseRowRedirectionImpl implements WritableRowRedirection { /** * {@link RowSet} used to map from outer key (position in the index) to inner key. */ - private final TrackingRowSet wrappedIndex; + private final RowSet wrappedRowSet; - public InverseRowRedirectionImpl(final TrackingRowSet wrappedIndex) { - this.wrappedIndex = wrappedIndex; + /** + * This class accepts a {@link RowSet} and attempts to cast to a {@link TrackingRowSet} if {@link #getPrev(long)} or + * {@link #fillPrevChunk(ChunkSource.FillContext, WritableLongChunk, RowSequence)} is called. Calling these + * functions on a non-tracking RowSet will result in a {@link ClassCastException}. + * + * @param wrappedRowSet the RowSet (or TrackingRowSet) to use as the redirection source + */ + public InverseRowRedirectionImpl(final RowSet wrappedRowSet) { + this.wrappedRowSet = wrappedRowSet; } @Override @@ -36,25 +43,25 @@ public synchronized long put(long key, long index) { @Override public synchronized long get(long key) { - if (key < 0 || key > wrappedIndex.lastRowKey()) { + if (key < 0 || key > wrappedRowSet.lastRowKey()) { return RowSet.NULL_ROW_KEY; } - return wrappedIndex.find(key); + return wrappedRowSet.find(key); } @Override public synchronized long getPrev(long key) { - if (key < 0 || key > wrappedIndex.lastRowKeyPrev()) { + if (key < 0 || key > wrappedRowSet.trackingCast().lastRowKeyPrev()) { return RowSet.NULL_ROW_KEY; } - return wrappedIndex.findPrev(key); + return wrappedRowSet.trackingCast().findPrev(key); } @Override public void fillChunk(@NotNull final ChunkSource.FillContext fillContext, @NotNull final WritableLongChunk mappedKeysOut, @NotNull final RowSequence keysToMap) { - try (final RowSequence.Iterator okit = wrappedIndex.getRowSequenceIterator()) { + try (final RowSequence.Iterator okit = wrappedRowSet.getRowSequenceIterator()) { doMapping(mappedKeysOut, keysToMap, okit); } } @@ -63,7 +70,7 @@ public void fillChunk(@NotNull final ChunkSource.FillContext fillContext, public void fillPrevChunk(@NotNull final ChunkSource.FillContext fillContext, @NotNull final WritableLongChunk mappedKeysOut, @NotNull final RowSequence keysToMap) { - try (final RowSet prevWrappedIndex = wrappedIndex.copyPrev(); + try (final RowSet prevWrappedIndex = wrappedRowSet.trackingCast().copyPrev(); final RowSequence.Iterator prevOkIt = prevWrappedIndex.getRowSequenceIterator()) { doMapping(mappedKeysOut, keysToMap, prevOkIt); } @@ -101,7 +108,7 @@ public String toString() { long positionStart = 0; - for (final RowSet.RangeIterator rangeIterator = wrappedIndex.rangeIterator(); rangeIterator.hasNext();) { + for (final RowSet.RangeIterator rangeIterator = wrappedRowSet.rangeIterator(); rangeIterator.hasNext();) { rangeIterator.next(); if (positionStart > 0) { diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/table/BarrageRedirectedTable.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/table/BarrageRedirectedTable.java index c68faeeb3ce..bf4b0e3684d 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/table/BarrageRedirectedTable.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/table/BarrageRedirectedTable.java @@ -147,14 +147,14 @@ private UpdateCoalescer processUpdate(final BarrageMessage update, final UpdateC // update the table with the rowsIncluded set (in manageable batch sizes) try (final RowSequence.Iterator rowsIncludedIterator = update.rowsIncluded.getRowSequenceIterator(); - final ChunkSink.FillFromContext redirContext = + final ChunkSink.FillFromContext redirHelper = rowRedirection.makeFillFromContext(addBatchSize)) { while (rowsIncludedIterator.hasMore()) { final RowSequence rowsToRedirect = rowsIncludedIterator.getNextRowSequenceWithLength(addBatchSize); try (final RowSet newRows = getFreeRows(rowsToRedirect.intSize())) { // Update redirection mapping: - rowRedirection.fillFromChunk(redirContext, newRows.asRowKeyChunk(), rowsToRedirect); + rowRedirection.fillFromChunk(redirHelper, newRows.asRowKeyChunk(), rowsToRedirect); // add these rows to the final destination set destinationRowSet.insert(newRows); } @@ -200,7 +200,7 @@ private UpdateCoalescer processUpdate(final BarrageMessage update, final UpdateC final int modBatchSize = (int) Math.min(column.rowsModified.size(), BATCH_SIZE); modifiedColumnSet.setColumnWithIndex(ii); - try (final ChunkSource.FillContext redirContext = rowRedirection.makeFillContext(modBatchSize, null); + try (final ChunkSource.FillContext redirHelper = rowRedirection.makeFillContext(modBatchSize, null); final ChunkSink.FillFromContext fillContext = destSources[ii].makeFillFromContext(modBatchSize); final WritableLongChunk keys = WritableLongChunk.makeWritableChunk(modBatchSize); final RowSequence.Iterator destIterator = column.rowsModified.getRowSequenceIterator()) { @@ -214,7 +214,7 @@ private UpdateCoalescer processUpdate(final BarrageMessage update, final UpdateC int effectiveBatchSize = Math.min(modBatchSize, chunk.size() - chunkOffset); final RowSequence chunkKeys = destIterator.getNextRowSequenceWithLength(effectiveBatchSize); // fill the key chunk with the keys from this rowset - rowRedirection.fillChunk(redirContext, keys, chunkKeys); + rowRedirection.fillChunk(redirHelper, keys, chunkKeys); Chunk slicedChunk = chunk.slice(chunkOffset, effectiveBatchSize); destSources[ii].fillFromChunkUnordered(fillContext, slicedChunk, keys); diff --git a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java index 47bf51a3261..2a68ba21de4 100644 --- a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java +++ b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java @@ -34,8 +34,8 @@ public static void main(String[] args) throws IOException { String objectResult = ReplicatePrimitiveCode.charToObject( "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java"); fixupStandardObject(objectResult, "ObjectFillByOperator", false, - "super\\(fillPair, new String\\[\\] \\{ fillPair.rightColumn \\}, redirContext\\);", - "super(fillPair, new String[] { fillPair.rightColumn }, redirContext, colType);", + "super\\(fillPair, new String\\[\\] \\{ fillPair.rightColumn \\}, redirHelper\\);", + "super(fillPair, new String[] { fillPair.rightColumn }, redirHelper, colType);", " BaseObjectUpdateByOperator", " BaseObjectUpdateByOperator", "public ObjectChunk Date: Tue, 29 Nov 2022 08:23:30 -0800 Subject: [PATCH 047/123] Cleanup after error inside PhasedUpdateProcessor --- .../deephaven/engine/table/impl/UpdateBy.java | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java index ae4975abb94..0783485e834 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java @@ -350,6 +350,7 @@ private void onError(Exception error) { // signal to the future that an exception has occurred waitForResult.completeExceptionally(error); } else { + cleanUpAfterError(); // this is part of an update, need to notify downstream result().notifyListenersOnError(error, null); } @@ -639,6 +640,20 @@ private void cleanUpAndNotify(final Runnable completeAction) { completeAction.run(); } + /** + * Clean up the resources created during this update. + */ + private void cleanUpAfterError() { + try (final RowSet ignoredRs = shiftedRows) { + // auto close these resources + } + + // allow the helpers to release their resources + for (UpdateByBucketHelper bucket : dirtyBuckets) { + bucket.finalizeUpdate(); + } + } + /** * Create the update for downstream listeners. This combines all bucket updates/modifies into a unified update */ @@ -743,11 +758,14 @@ public void processUpdate() { // need to wait until this future is complete waitForResult.get(); } catch (InterruptedException e) { + cleanUpAfterError(); throw new CancellationException("interrupted while processing updateBy"); } catch (ExecutionException e) { + cleanUpAfterError(); if (e.getCause() instanceof RuntimeException) { throw (RuntimeException) e.getCause(); } else { + // rethrow the error throw new UncheckedDeephavenException("Failure while processing updateBy", e.getCause()); } From da57bddca47410efc06fd06b4c60402c75f78bd8 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 29 Nov 2022 14:13:43 -0800 Subject: [PATCH 048/123] Minor change --- .../engine/table/impl/BucketedPartitionedUpdateByManager.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java index 2ca4e598587..8fce8c6df15 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java @@ -125,6 +125,9 @@ protected QueryTable result() { @Override protected boolean upstreamSatisfied(final long step) { // For bucketed, need to verify the source and the transformed table is satisfied. + if (transformedTable == null) { + return false; + } return source.satisfied(step) && transformedTable.satisfied(step); } From 20559921b60fbee6376faa316d5585067167b38d Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 2 Dec 2022 09:42:42 -0800 Subject: [PATCH 049/123] Removed @ConcurrentMethod decorators and changed python wrapper to auto_lock and python unit tests to hold UGP lock --- .../io/deephaven/engine/table/impl/UncoalescedTable.java | 1 - py/server/deephaven/table.py | 3 ++- .../src/main/java/io/deephaven/api/TableOperations.java | 7 ------- .../java/io/deephaven/api/TableOperationsDefaults.java | 6 ------ 4 files changed, 2 insertions(+), 15 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UncoalescedTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UncoalescedTable.java index e85c2e3fb19..ddfbb8a5c8e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UncoalescedTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UncoalescedTable.java @@ -398,7 +398,6 @@ public Table treeTable(String idColumn, String parentColumn) { } @Override - @ConcurrentMethod public Table updateBy(@NotNull final UpdateByControl control, @NotNull final Collection ops, @NotNull final Collection byColumns) { diff --git a/py/server/deephaven/table.py b/py/server/deephaven/table.py index 56cacd5434c..de869af00e3 100644 --- a/py/server/deephaven/table.py +++ b/py/server/deephaven/table.py @@ -1621,7 +1621,8 @@ def update_by(self, ops: Union[UpdateByOperation, List[UpdateByOperation]], try: ops = to_sequence(ops) by = to_sequence(by) - return Table(j_table=self.j_table.updateBy(j_array_list(ops), *by)) + with auto_locking_ctx(self): + return Table(j_table=self.j_table.updateBy(j_array_list(ops), *by)) except Exception as e: raise DHError(e, "table update-by operation failed.") from e diff --git a/table-api/src/main/java/io/deephaven/api/TableOperations.java b/table-api/src/main/java/io/deephaven/api/TableOperations.java index 321901f6476..6e165214361 100644 --- a/table-api/src/main/java/io/deephaven/api/TableOperations.java +++ b/table-api/src/main/java/io/deephaven/api/TableOperations.java @@ -678,7 +678,6 @@ TOPS aggBy(Collection aggregations, boolean preserveEmpty * @param operation the operation to apply to the table. * @return a table with the same rowset, with the specified operation applied to the entire table */ - @ConcurrentMethod TOPS updateBy(UpdateByOperation operation); /** @@ -690,7 +689,6 @@ TOPS aggBy(Collection aggregations, boolean preserveEmpty * @param operations the operations to apply to the table. * @return a table with the same rowset, with the specified operations applied to the entire table. */ - @ConcurrentMethod TOPS updateBy(Collection operations); /** @@ -703,7 +701,6 @@ TOPS aggBy(Collection aggregations, boolean preserveEmpty * @param operations the operations to apply to the table. * @return a table with the same rowset, with the specified operations applied to the entire table */ - @ConcurrentMethod TOPS updateBy(UpdateByControl control, Collection operations); /** @@ -717,7 +714,6 @@ TOPS aggBy(Collection aggregations, boolean preserveEmpty * @return a table with the same rowSet, with the specified operation applied to each group defined by the * {@code byColumns} */ - @ConcurrentMethod TOPS updateBy(UpdateByOperation operation, final String... byColumns); /** @@ -731,7 +727,6 @@ TOPS aggBy(Collection aggregations, boolean preserveEmpty * @return a table with the same rowSet, with the specified operations applied to each group defined by the * {@code byColumns} */ - @ConcurrentMethod TOPS updateBy(Collection operations, final String... byColumns); /** @@ -745,7 +740,6 @@ TOPS aggBy(Collection aggregations, boolean preserveEmpty * @return a table with the same rowSet, with the specified operations applied to each group defined by the * {@code byColumns} */ - @ConcurrentMethod TOPS updateBy(Collection operations, Collection byColumns); /** @@ -760,7 +754,6 @@ TOPS aggBy(Collection aggregations, boolean preserveEmpty * @return a table with the same rowSet, with the specified operations applied to each group defined by the * {@code byColumns} */ - @ConcurrentMethod TOPS updateBy(UpdateByControl control, Collection operations, Collection byColumns); diff --git a/table-api/src/main/java/io/deephaven/api/TableOperationsDefaults.java b/table-api/src/main/java/io/deephaven/api/TableOperationsDefaults.java index 6471a64fe5b..b58a0852c00 100644 --- a/table-api/src/main/java/io/deephaven/api/TableOperationsDefaults.java +++ b/table-api/src/main/java/io/deephaven/api/TableOperationsDefaults.java @@ -319,39 +319,33 @@ default TOPS aggBy(Collection aggregations, // ------------------------------------------------------------------------------------------- @Override - @ConcurrentMethod default TOPS updateBy(final UpdateByOperation operation) { return updateBy(UpdateByControl.defaultInstance(), Collections.singletonList(operation), Collections.emptyList()); } @Override - @ConcurrentMethod default TOPS updateBy(final Collection operations) { return updateBy(UpdateByControl.defaultInstance(), operations, Collections.emptyList()); } @Override - @ConcurrentMethod default TOPS updateBy(final UpdateByControl control, final Collection operations) { return updateBy(control, operations, Collections.emptyList()); } @Override - @ConcurrentMethod default TOPS updateBy(final UpdateByOperation operation, final String... byColumns) { return updateBy(UpdateByControl.defaultInstance(), Collections.singletonList(operation), ColumnName.from(byColumns)); } @Override - @ConcurrentMethod default TOPS updateBy(final Collection operations, final String... byColumns) { return updateBy(UpdateByControl.defaultInstance(), operations, ColumnName.from(byColumns)); } @Override - @ConcurrentMethod default TOPS updateBy(Collection operations, Collection byColumns) { return updateBy(UpdateByControl.defaultInstance(), operations, byColumns); From 9935d853d63e703b7011212ffdccaa7081992869 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 2 Dec 2022 09:43:25 -0800 Subject: [PATCH 050/123] Removed @ConcurrentMethod decorators and changed python wrapper to auto_lock and python unit tests to hold UGP lock --- py/server/tests/test_updateby.py | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/py/server/tests/test_updateby.py b/py/server/tests/test_updateby.py index 98ae9f43e61..95c0f2670b6 100644 --- a/py/server/tests/test_updateby.py +++ b/py/server/tests/test_updateby.py @@ -41,11 +41,12 @@ def test_ema(self): for ema_op in ema_ops: with self.subTest(ema_op): for t in (self.static_table, self.ticking_table): - ema_table = t.update_by(ops=ema_op, by="b") - self.assertTrue(ema_table.is_refreshing is t.is_refreshing) - self.assertEqual(len(ema_table.columns), 1 + len(t.columns)) with ugp.exclusive_lock(): - self.assertEqual(ema_table.size, t.size) + ema_table = t.update_by(ops=ema_op, by="b") + self.assertTrue(ema_table.is_refreshing is t.is_refreshing) + self.assertEqual(len(ema_table.columns), 1 + len(t.columns)) + with ugp.exclusive_lock(): + self.assertEqual(ema_table.size, t.size) def test_simple_ops(self): op_builders = [cum_sum, cum_prod, cum_min, cum_max, forward_fill] @@ -54,10 +55,10 @@ def test_simple_ops(self): for op_builder in op_builders: with self.subTest(op_builder): for t in (self.static_table, self.ticking_table): - updateby_table = t.update_by(ops=op_builder(pairs), by="e") - self.assertTrue(updateby_table.is_refreshing is t.is_refreshing) - self.assertEqual(len(updateby_table.columns), 2 + len(t.columns)) with ugp.exclusive_lock(): + updateby_table = t.update_by(ops=op_builder(pairs), by="e") + self.assertTrue(updateby_table.is_refreshing is t.is_refreshing) + self.assertEqual(len(updateby_table.columns), 2 + len(t.columns)) self.assertEqual(updateby_table.size, t.size) def test_simple_ops_proxy(self): From 1d88c0aa134c0a477d0b86fcc169490fe8aabc9e Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Mon, 5 Dec 2022 16:44:21 -0800 Subject: [PATCH 051/123] WIP --- .../impl/BucketedPartitionedUpdateByManager.java | 4 ---- .../java/io/deephaven/engine/table/impl/UpdateBy.java | 11 +++-------- .../table/impl/updateby/UpdateByWindowCumulative.java | 2 +- .../table/impl/updateby/UpdateByWindowTicks.java | 2 +- .../table/impl/updateby/UpdateByWindowTime.java | 2 +- 5 files changed, 6 insertions(+), 15 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java index 8fce8c6df15..92a56a265b9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java @@ -125,10 +125,6 @@ protected QueryTable result() { @Override protected boolean upstreamSatisfied(final long step) { // For bucketed, need to verify the source and the transformed table is satisfied. - if (transformedTable == null) { - return false; - } return source.satisfied(step) && transformedTable.satisfied(step); } - } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java index 0783485e834..4eb2d952cb9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java @@ -23,14 +23,8 @@ import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.sources.sparse.SparseConstants; import io.deephaven.engine.table.impl.updateby.UpdateByWindow; -import io.deephaven.engine.table.impl.util.InverseRowRedirectionImpl; -import io.deephaven.engine.table.impl.util.LongColumnSourceWritableRowRedirection; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.*; import io.deephaven.engine.updategraph.UpdateGraphProcessor; -import io.deephaven.engine.table.impl.util.ImmediateJobScheduler; -import io.deephaven.engine.table.impl.util.JobScheduler; -import io.deephaven.engine.table.impl.util.OperationInitializationPoolJobScheduler; -import io.deephaven.engine.table.impl.util.UpdateGraphProcessorJobScheduler; import io.deephaven.util.datastructures.linked.IntrusiveDoublyLinkedNode; import io.deephaven.util.datastructures.linked.IntrusiveDoublyLinkedQueue; import org.apache.commons.lang3.mutable.MutableInt; @@ -324,7 +318,8 @@ class PhasedUpdateProcessor implements LogOutputAppendable { } if (initialStep) { - if (OperationInitializationThreadPool.NUM_THREADS > 1) { + if (OperationInitializationThreadPool.NUM_THREADS > 1 + && !OperationInitializationThreadPool.isInitializationThread()) { jobScheduler = new OperationInitializationPoolJobScheduler(); } else { jobScheduler = ImmediateJobScheduler.INSTANCE; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java index bf6e8b6b9a2..995f793f60f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java @@ -67,7 +67,7 @@ public void computeAffectedRowsAndOperators(UpdateByWindowContext context, @NotN context.dirtySourceIndices = getUniqueSourceIndices(); makeOperatorContexts(context); - context.isDirty = true; + context.isDirty = !upstream.empty(); return; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java index 7ef41469be3..42ad04eebd6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java @@ -219,7 +219,7 @@ public void computeAffectedRowsAndOperators(UpdateByWindowContext context, @NotN context.dirtySourceIndices = getUniqueSourceIndices(); makeOperatorContexts(ctx); - ctx.isDirty = true; + ctx.isDirty = !upstream.empty(); return; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java index aca464a6887..65d925f3d05 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java @@ -243,7 +243,7 @@ public void computeAffectedRowsAndOperators(UpdateByWindowContext context, @NotN context.dirtySourceIndices = getUniqueSourceIndices(); makeOperatorContexts(ctx); - ctx.isDirty = true; + ctx.isDirty = !upstream.empty();; return; } From 52f1f884a7ff6ddef553d795cbaf40fc87f217f3 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Mon, 5 Dec 2022 16:48:26 -0800 Subject: [PATCH 052/123] spotless --- .../src/main/java/io/deephaven/engine/table/impl/UpdateBy.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java index 4eb2d952cb9..9bf48555b48 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java @@ -319,7 +319,7 @@ class PhasedUpdateProcessor implements LogOutputAppendable { if (initialStep) { if (OperationInitializationThreadPool.NUM_THREADS > 1 - && !OperationInitializationThreadPool.isInitializationThread()) { + && !OperationInitializationThreadPool.isInitializationThread()) { jobScheduler = new OperationInitializationPoolJobScheduler(); } else { jobScheduler = ImmediateJobScheduler.INSTANCE; From 738a18c9c6c56edb4f11f7562c892599d3ba11d4 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 6 Dec 2022 08:29:55 -0800 Subject: [PATCH 053/123] FinalDefault misapplication corrected --- .../table/impl/updateby/internal/BaseByteUpdateByOperator.java | 2 +- .../table/impl/updateby/internal/BaseCharUpdateByOperator.java | 2 +- .../impl/updateby/internal/BaseDoubleUpdateByOperator.java | 2 +- .../table/impl/updateby/internal/BaseFloatUpdateByOperator.java | 2 +- .../table/impl/updateby/internal/BaseIntUpdateByOperator.java | 2 +- .../table/impl/updateby/internal/BaseLongUpdateByOperator.java | 2 +- .../impl/updateby/internal/BaseObjectUpdateByOperator.java | 2 +- .../table/impl/updateby/internal/BaseShortUpdateByOperator.java | 2 +- .../updateby/internal/BaseWindowedByteUpdateByOperator.java | 2 +- .../updateby/internal/BaseWindowedCharUpdateByOperator.java | 2 +- .../updateby/internal/BaseWindowedDoubleUpdateByOperator.java | 2 +- .../updateby/internal/BaseWindowedFloatUpdateByOperator.java | 2 +- .../impl/updateby/internal/BaseWindowedIntUpdateByOperator.java | 2 +- .../updateby/internal/BaseWindowedLongUpdateByOperator.java | 2 +- .../updateby/internal/BaseWindowedObjectUpdateByOperator.java | 2 +- .../updateby/internal/BaseWindowedShortUpdateByOperator.java | 2 +- 16 files changed, 16 insertions(+), 16 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java index 79e948cb68c..5545e74bdef 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java @@ -91,7 +91,7 @@ public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); } - @FinalDefault + @Override public void reset() { curVal = NULL_BYTE; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java index b4f0881e7a6..ae35df55ab3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java @@ -80,7 +80,7 @@ public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); } - @FinalDefault + @Override public void reset() { curVal = NULL_CHAR; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java index 8893a7dea26..166a1c87722 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java @@ -84,7 +84,7 @@ public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); } - @FinalDefault + @Override public void reset() { curVal = NULL_DOUBLE; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java index c3bc55d010b..66c1910df3b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java @@ -79,7 +79,7 @@ public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); } - @FinalDefault + @Override public void reset() { curVal = NULL_FLOAT; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java index 9f4a0f6a415..98499651302 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java @@ -85,7 +85,7 @@ public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); } - @FinalDefault + @Override public void reset() { curVal = NULL_INT; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java index 8fadb78bbda..b5fe7b19d7e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java @@ -85,7 +85,7 @@ public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); } - @FinalDefault + @Override public void reset() { curVal = NULL_LONG; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java index 9ba3225de85..9e7ee33cf4c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java @@ -86,7 +86,7 @@ public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); } - @FinalDefault + @Override public void reset() { curVal = null; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java index 4fdaed13c66..9588043982c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java @@ -85,7 +85,7 @@ public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); } - @FinalDefault + @Override public void reset() { curVal = NULL_SHORT; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java index 970cb0b9427..d7e68988bf4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java @@ -107,7 +107,7 @@ public void close() { outputFillContext.close(); } - @FinalDefault + @Override public void reset() { curVal = NULL_BYTE; nullCount = 0; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java index 312bf8bed30..52bc1978c59 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java @@ -96,7 +96,7 @@ public void close() { outputFillContext.close(); } - @FinalDefault + @Override public void reset() { curVal = NULL_CHAR; nullCount = 0; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java index 216148e25cb..d2f9340735b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java @@ -100,7 +100,7 @@ public void close() { outputFillContext.close(); } - @FinalDefault + @Override public void reset() { curVal = NULL_DOUBLE; nullCount = 0; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java index f5d4692d083..08a538dfa39 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java @@ -95,7 +95,7 @@ public void close() { outputFillContext.close(); } - @FinalDefault + @Override public void reset() { curVal = NULL_FLOAT; nullCount = 0; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java index 680c8198d69..cd1d51d0c64 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java @@ -101,7 +101,7 @@ public void close() { outputFillContext.close(); } - @FinalDefault + @Override public void reset() { curVal = NULL_INT; nullCount = 0; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java index c41ad2d401b..b4cbbceed53 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java @@ -101,7 +101,7 @@ public void close() { outputFillContext.close(); } - @FinalDefault + @Override public void reset() { curVal = NULL_LONG; nullCount = 0; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java index 05e69164f6f..1d0d81d82ad 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java @@ -102,7 +102,7 @@ public void close() { outputFillContext.close(); } - @FinalDefault + @Override public void reset() { curVal = null; nullCount = 0; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java index fc2b44253f4..a5ff7212f3f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java @@ -101,7 +101,7 @@ public void close() { outputFillContext.close(); } - @FinalDefault + @Override public void reset() { curVal = NULL_SHORT; nullCount = 0; From 01668294d36ef827e30b217f800014569959293e Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 6 Dec 2022 08:54:09 -0800 Subject: [PATCH 054/123] Adjusted imports --- .../deephaven/engine/table/impl/updateby/TestRollingSum.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java index b8de83b6630..8ba3b2f529a 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java @@ -8,7 +8,6 @@ import io.deephaven.engine.table.PartitionedTable; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.impl.*; -import io.deephaven.engine.table.impl.sources.ReinterpretUtils; import io.deephaven.engine.updategraph.UpdateGraphProcessor; import io.deephaven.test.types.OutOfBandTest; import io.deephaven.time.DateTime; @@ -26,11 +25,9 @@ import static io.deephaven.engine.table.impl.GenerateTableUpdates.generateAppends; import static io.deephaven.engine.table.impl.RefreshingTableTestCase.simulateShiftAwareStep; -import static io.deephaven.engine.table.impl.TstUtils.assertTableEquals; -import static io.deephaven.engine.table.impl.TstUtils.testTable; +import static io.deephaven.engine.table.impl.TstUtils.*; import static io.deephaven.engine.util.TableTools.*; import static io.deephaven.function.Basic.isNull; -import static io.deephaven.time.DateTimeUtils.MINUTE; import static io.deephaven.time.DateTimeUtils.convertDateTime; import static io.deephaven.util.QueryConstants.*; import static org.junit.Assert.assertArrayEquals; From ea5c63efed17cef3f4515139e1ac403b28095cec Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 6 Dec 2022 11:38:40 -0800 Subject: [PATCH 055/123] Corrected caching bug --- .../deephaven/engine/table/impl/UpdateBy.java | 18 ++++++++++++++---- 1 file changed, 14 insertions(+), 4 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java index 9bf48555b48..1a386b4e34f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java @@ -311,6 +311,11 @@ class PhasedUpdateProcessor implements LogOutputAppendable { maybeCachedInputSources = new ColumnSource[inputSources.length]; inputSourceRowSets = new WritableRowSet[inputSources.length]; inputSourceReferenceCounts = new int[inputSources.length]; + + // set the uncacheable columns into the array + for (int ii = 0; ii < inputSources.length; ii++) { + maybeCachedInputSources[ii] = inputSourceCacheNeeded[ii] ? inputSources[ii] : null; + } } else { maybeCachedInputSources = inputSources; inputSourceRowSets = null; @@ -380,10 +385,11 @@ private void computeCachedColumnRowsets(final Runnable completeAction) { jobScheduler.iterateParallel(ExecutionContext.getContextToRecord(), this, 0, cacheableSourceIndices.length, idx -> { - int srcIdx = cacheableSourceIndices[idx]; + final int srcIdx = cacheableSourceIndices[idx]; for (int winIdx = 0; winIdx < windows.length; winIdx++) { UpdateByWindow win = windows[winIdx]; if (win.isSourceInUse(srcIdx)) { + boolean srcNeeded = false; for (UpdateByBucketHelper bucket : dirtyBuckets) { UpdateByWindow.UpdateByWindowContext winCtx = bucket.windowContexts[winIdx]; @@ -391,13 +397,17 @@ private void computeCachedColumnRowsets(final Runnable completeAction) { // add this rowset to the running total for this input source if (inputSourceRowSets[srcIdx] == null) { inputSourceRowSets[srcIdx] = - win.getInfluencerRows(winCtx).copy().toTracking(); + win.getInfluencerRows(winCtx).copy(); } else { inputSourceRowSets[srcIdx].insert(win.getInfluencerRows(winCtx)); } + // at least one dirty bucket will need this source + srcNeeded = true; } } - inputSourceReferenceCounts[srcIdx]++; + if (srcNeeded) { + inputSourceReferenceCounts[srcIdx]++; + } } } }, @@ -410,7 +420,7 @@ private void computeCachedColumnRowsets(final Runnable completeAction) { * when the work is complete */ private void createCachedColumnSource(int srcIdx, final Runnable completeAction) { - if (maybeCachedInputSources[srcIdx] != null) { + if (maybeCachedInputSources[srcIdx] != null || inputSourceRowSets[srcIdx] == null) { // already cached from another operator (or caching not needed) completeAction.run(); return; From e48d1d8d1bea48b18a8a78cae71286122e997b3d Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 14 Dec 2022 14:14:31 -0800 Subject: [PATCH 056/123] Cherry picked changes from rcaudy --- .../select/analyzers/SelectColumnLayer.java | 120 ++++++++++-------- 1 file changed, 66 insertions(+), 54 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectColumnLayer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectColumnLayer.java index 4b9d8198390..5882465d65d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectColumnLayer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectColumnLayer.java @@ -11,6 +11,7 @@ import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.liveness.LivenessNode; import io.deephaven.engine.liveness.LivenessReferent; +import io.deephaven.engine.liveness.LivenessScopeStack; import io.deephaven.engine.rowset.*; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.QueryTable; @@ -20,10 +21,10 @@ import io.deephaven.engine.table.impl.sources.ChunkedBackingStoreExposedWritableSource; import io.deephaven.engine.table.impl.sources.ReinterpretUtils; import io.deephaven.engine.table.impl.util.ChunkUtils; +import io.deephaven.engine.table.impl.util.JobScheduler; import io.deephaven.engine.updategraph.DynamicNode; import io.deephaven.engine.updategraph.UpdateCommitterEx; import io.deephaven.engine.updategraph.UpdateGraphProcessor; -import io.deephaven.engine.table.impl.util.JobScheduler; import io.deephaven.engine.util.systemicmarking.SystemicObjectTracker; import io.deephaven.time.DateTime; import io.deephaven.util.SafeCloseable; @@ -42,7 +43,7 @@ final public class SelectColumnLayer extends SelectOrViewColumnLayer { /** * The same reference as super.columnSource, but as a WritableColumnSource and maybe reinterpretted */ - private final WritableColumnSource writableSource; + private final WritableColumnSource writableSource; /** * The execution context the select column layer was constructed in @@ -71,12 +72,12 @@ final public class SelectColumnLayer extends SelectOrViewColumnLayer { private ChunkSource.WithPrev chunkSource; SelectColumnLayer(RowSet parentRowSet, SelectAndViewAnalyzer inner, String name, SelectColumn sc, - WritableColumnSource ws, WritableColumnSource underlying, - String[] deps, ModifiedColumnSet mcsBuilder, boolean isRedirected, - boolean flattenedResult, boolean alreadyFlattenedSources) { + WritableColumnSource ws, WritableColumnSource underlying, + String[] deps, ModifiedColumnSet mcsBuilder, boolean isRedirected, + boolean flattenedResult, boolean alreadyFlattenedSources) { super(inner, name, sc, ws, underlying, deps, mcsBuilder); this.parentRowSet = parentRowSet; - this.writableSource = (WritableColumnSource) ReinterpretUtils.maybeConvertToPrimitive(ws); + this.writableSource = (WritableColumnSource) ReinterpretUtils.maybeConvertToPrimitive(ws); this.isRedirected = isRedirected; this.executionContext = ExecutionContext.getContextToRecord(); @@ -118,8 +119,8 @@ private ChunkSource getChunkSource() { @Override public void applyUpdate(final TableUpdate upstream, final RowSet toClear, - final UpdateHelper helper, final JobScheduler jobScheduler, @Nullable final LivenessNode liveResultOwner, - final SelectLayerCompletionHandler onCompletion) { + final UpdateHelper helper, final JobScheduler jobScheduler, @Nullable final LivenessNode liveResultOwner, + final SelectLayerCompletionHandler onCompletion) { if (upstream.removed().isNonempty()) { if (isRedirected) { clearObjectsAtThisLevel(upstream.removed()); @@ -140,16 +141,21 @@ public void onAllRequiredColumnsCompleted() { // If we have shifts, that makes everything nasty; so we do not want to deal with it final boolean hasShifts = upstream.shifted().nonempty(); + final boolean checkTableOperations = + UpdateGraphProcessor.DEFAULT.getCheckTableOperations() + && !UpdateGraphProcessor.DEFAULT.sharedLock().isHeldByCurrentThread() + && !UpdateGraphProcessor.DEFAULT.exclusiveLock().isHeldByCurrentThread(); + if (canParallelizeThisColumn && jobScheduler.threadCount() > 1 && !hasShifts && ((resultTypeIsTable && totalSize > 0) || totalSize > QueryTable.MINIMUM_PARALLEL_SELECT_ROWS)) { final long divisionSize = resultTypeIsTable ? 1 : Math.max(QueryTable.MINIMUM_PARALLEL_SELECT_ROWS, - (totalSize + jobScheduler.threadCount() - 1) / jobScheduler.threadCount()); + (totalSize + jobScheduler.threadCount() - 1) / jobScheduler.threadCount()); final List updates = new ArrayList<>(); // divide up the additions and modifications try (final RowSequence.Iterator rsAddIt = upstream.added().getRowSequenceIterator(); - final RowSequence.Iterator rsModIt = upstream.modified().getRowSequenceIterator()) { + final RowSequence.Iterator rsModIt = upstream.modified().getRowSequenceIterator()) { while (rsAddIt.hasMore() || rsModIt.hasMore()) { final TableUpdateImpl update = new TableUpdateImpl(); update.modifiedColumnSet = upstream.modifiedColumnSet(); @@ -181,12 +187,14 @@ public void onAllRequiredColumnsCompleted() { jobScheduler.submit( executionContext, () -> prepareParallelUpdate(jobScheduler, upstream, toClear, helper, - liveResultOwner, onCompletion, this::onError, updates), + liveResultOwner, onCompletion, this::onError, updates, + checkTableOperations), SelectColumnLayer.this, this::onError); } else { jobScheduler.submit( executionContext, - () -> doSerialApplyUpdate(upstream, toClear, helper, liveResultOwner, onCompletion), + () -> doSerialApplyUpdate(upstream, toClear, helper, liveResultOwner, onCompletion, + checkTableOperations), SelectColumnLayer.this, this::onError); } } @@ -194,19 +202,15 @@ public void onAllRequiredColumnsCompleted() { } private void prepareParallelUpdate(final JobScheduler jobScheduler, final TableUpdate upstream, - final RowSet toClear, final UpdateHelper helper, @Nullable final LivenessNode liveResultOwner, - final SelectLayerCompletionHandler onCompletion, final Consumer onError, - final List splitUpdates) { + final RowSet toClear, final UpdateHelper helper, @Nullable final LivenessNode liveResultOwner, + final SelectLayerCompletionHandler onCompletion, final Consumer onError, + final List splitUpdates, final boolean checkTableOperations) { // we have to do removal and previous initialization before we can do any of the actual filling in multiple // threads to avoid concurrency problems with our destination column sources doEnsureCapacity(); prepareSourcesForParallelPopulation(upstream); - final boolean checkTableOperations = - UpdateGraphProcessor.DEFAULT.getCheckTableOperations() - && !UpdateGraphProcessor.DEFAULT.sharedLock().isHeldByCurrentThread() - && !UpdateGraphProcessor.DEFAULT.exclusiveLock().isHeldByCurrentThread(); final AtomicInteger divisions = new AtomicInteger(splitUpdates.size()); long destinationOffset = 0; @@ -227,11 +231,16 @@ private void prepareParallelUpdate(final JobScheduler jobScheduler, final TableU } private void doSerialApplyUpdate(final TableUpdate upstream, final RowSet toClear, final UpdateHelper helper, - @Nullable final LivenessNode liveResultOwner, final SelectLayerCompletionHandler onCompletion) { + @Nullable final LivenessNode liveResultOwner, final SelectLayerCompletionHandler onCompletion, + final boolean checkTableOperations) { doEnsureCapacity(); - SystemicObjectTracker.executeSystemically(isSystemic, - () -> doApplyUpdate(upstream, helper, liveResultOwner, 0)); - + final boolean oldCheck = UpdateGraphProcessor.DEFAULT.setCheckTableOperations(checkTableOperations); + try { + SystemicObjectTracker.executeSystemically(isSystemic, + () -> doApplyUpdate(upstream, helper, liveResultOwner, 0)); + } finally { + UpdateGraphProcessor.DEFAULT.setCheckTableOperations(oldCheck); + } if (!isRedirected) { clearObjectsAtThisLevel(toClear); } @@ -239,8 +248,8 @@ private void doSerialApplyUpdate(final TableUpdate upstream, final RowSet toClea } private void doParallelApplyUpdate(final TableUpdate upstream, final RowSet toClear, final UpdateHelper helper, - @Nullable final LivenessNode liveResultOwner, final SelectLayerCompletionHandler onCompletion, - final boolean checkTableOperations, final AtomicInteger divisions, final long startOffset) { + @Nullable final LivenessNode liveResultOwner, final SelectLayerCompletionHandler onCompletion, + final boolean checkTableOperations, final AtomicInteger divisions, final long startOffset) { final boolean oldCheck = UpdateGraphProcessor.DEFAULT.setCheckTableOperations(checkTableOperations); try { SystemicObjectTracker.executeSystemically(isSystemic, @@ -259,7 +268,7 @@ private void doParallelApplyUpdate(final TableUpdate upstream, final RowSet toCl } private Boolean doApplyUpdate(final TableUpdate upstream, final UpdateHelper helper, - @Nullable final LivenessNode liveResultOwner, final long startOffset) { + @Nullable final LivenessNode liveResultOwner, final long startOffset) { final int PAGE_SIZE = 4096; final LongToIntFunction contextSize = (long size) -> size > PAGE_SIZE ? PAGE_SIZE : (int) size; @@ -283,13 +292,16 @@ private Boolean doApplyUpdate(final TableUpdate upstream, final UpdateHelper hel final boolean isBackingChunkExposed = ChunkedBackingStoreExposedWritableSource.exposesChunkedBackingStore(writableSource); - try (final ChunkSink.FillFromContext destContext = - needDestContext ? writableSource.makeFillFromContext(destContextSize) : null; - final ChunkSource.GetContext chunkSourceContext = - needGetContext ? chunkSource.makeGetContext(chunkSourceContextSize) : null; - final ChunkSource.FillContext chunkSourceFillContext = - needGetContext && isBackingChunkExposed ? chunkSource.makeFillContext(chunkSourceContextSize) - : null) { + try (final SafeCloseable ignored = LivenessScopeStack.open(); + final ChunkSink.FillFromContext destContext = needDestContext + ? writableSource.makeFillFromContext(destContextSize) + : null; + final ChunkSource.GetContext chunkSourceContext = needGetContext + ? chunkSource.makeGetContext(chunkSourceContextSize) + : null; + final ChunkSource.FillContext chunkSourceFillContext = needGetContext && isBackingChunkExposed + ? chunkSource.makeFillContext(chunkSourceContextSize) + : null) { // apply shifts! if (!isRedirected && preMoveKeys.isNonempty()) { @@ -298,10 +310,10 @@ private Boolean doApplyUpdate(final TableUpdate upstream, final UpdateHelper hel // note: we cannot use a get context here as destination is identical to source final int shiftContextSize = contextSize.applyAsInt(preMoveKeys.size()); try (final ChunkSource.FillContext srcContext = writableSource.makeFillContext(shiftContextSize); - final WritableChunk chunk = - writableSource.getChunkType().makeWritableChunk(shiftContextSize); - final RowSequence.Iterator srcIter = preMoveKeys.getRowSequenceIterator(); - final RowSequence.Iterator destIter = postMoveKeys.getRowSequenceIterator()) { + final WritableChunk chunk = + writableSource.getChunkType().makeWritableChunk(shiftContextSize); + final RowSequence.Iterator srcIter = preMoveKeys.getRowSequenceIterator(); + final RowSequence.Iterator destIter = postMoveKeys.getRowSequenceIterator()) { while (srcIter.hasMore()) { final RowSequence srcKeys = srcIter.getNextRowSequenceWithLength(PAGE_SIZE); @@ -325,8 +337,8 @@ private Boolean doApplyUpdate(final TableUpdate upstream, final UpdateHelper hel // fill as much as possible as quickly as possible long destinationOffset = startOffset; try (final RowSequence.Iterator keyIter = upstream.added().getRowSequenceIterator(); - final ResettableWritableChunk backingChunk = - writableSource.getChunkType().makeResettableWritableChunk()) { + final ResettableWritableChunk backingChunk = + writableSource.getChunkType().makeResettableWritableChunk()) { while (keyIter.hasMore()) { final long destCapacity = exposedWritableSource .resetWritableChunkToBackingStoreSlice(backingChunk, destinationOffset); @@ -339,13 +351,13 @@ private Boolean doApplyUpdate(final TableUpdate upstream, final UpdateHelper hel } } else { try (final RowSequence.Iterator keyIter = upstream.added().getRowSequenceIterator(); - final RowSequence.Iterator destIter = flattenedResult - ? RowSequenceFactory - .forRange(startOffset, startOffset + upstream.added().size() - 1) - .getRowSequenceIterator() - : null; - final ResettableWritableChunk backingChunk = - writableSource.getChunkType().makeResettableWritableChunk()) { + final RowSequence.Iterator destIter = flattenedResult + ? RowSequenceFactory + .forRange(startOffset, startOffset + upstream.added().size() - 1) + .getRowSequenceIterator() + : null; + final ResettableWritableChunk backingChunk = + writableSource.getChunkType().makeResettableWritableChunk()) { while (keyIter.hasMore()) { final RowSequence keys = keyIter.getNextRowSequenceWithLength(PAGE_SIZE); final RowSequence destKeys; @@ -409,12 +421,12 @@ private Boolean doApplyUpdate(final TableUpdate upstream, final UpdateHelper hel assert chunkSourceContext != null; final boolean needToUnmanagePrevValues = resultTypeIsLivenessReferent && liveResultOwner != null; try (final RowSequence.Iterator keyIter = upstream.modified().getRowSequenceIterator(); - final RowSequence.Iterator prevKeyIter = needToUnmanagePrevValues - ? upstream.getModifiedPreShift().getRowSequenceIterator() - : null; - final ChunkSource.FillContext fillContext = needToUnmanagePrevValues - ? columnSource.makeFillContext(PAGE_SIZE) - : null) { + final RowSequence.Iterator prevKeyIter = needToUnmanagePrevValues + ? upstream.getModifiedPreShift().getRowSequenceIterator() + : null; + final ChunkSource.FillContext fillContext = needToUnmanagePrevValues + ? columnSource.makeFillContext(PAGE_SIZE) + : null) { while (keyIter.hasMore()) { final RowSequence keys = keyIter.getNextRowSequenceWithLength(PAGE_SIZE); final Chunk modifiedResults = chunkSource.getChunk(chunkSourceContext, keys); @@ -455,8 +467,8 @@ private void addRemovesToPrevUnmanager( @NotNull final RowSequence removedKeys, @NotNull final LivenessNode liveResultOwner) { try (final RowSequence.Iterator removedKeysIterator = removedKeys.getRowSequenceIterator(); - final ChunkSource.FillContext fillContext = columnSource.makeFillContext( - (int) Math.min(removedKeys.size(), LARGEST_POOLED_CHUNK_CAPACITY))) { + final ChunkSource.FillContext fillContext = columnSource.makeFillContext( + (int) Math.min(removedKeys.size(), LARGEST_POOLED_CHUNK_CAPACITY))) { while (removedKeysIterator.hasMore()) { final RowSequence chunkRemovedKeys = removedKeysIterator.getNextRowSequenceWithLength(LARGEST_POOLED_CHUNK_CAPACITY); From 5ed31ff8d5ca4856f3d572c1b36ca7658cf62f65 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Mon, 19 Dec 2022 11:28:05 -0800 Subject: [PATCH 057/123] Corrected build errors and removed replication of ObjectRingBuffer and tests --- .../base/ringbuffer/ObjectRingBuffer.java | 195 --------- .../base/ringbuffer/ObjectRingBufferTest.java | 399 ------------------ .../select/analyzers/SelectColumnLayer.java | 1 + .../table/impl/updateby/TestRollingSum.java | 22 +- .../replicators/ReplicateRingBuffers.java | 8 +- 5 files changed, 16 insertions(+), 609 deletions(-) delete mode 100644 Base/src/main/java/io/deephaven/base/ringbuffer/ObjectRingBuffer.java delete mode 100644 Base/src/test/java/io/deephaven/base/ringbuffer/ObjectRingBufferTest.java diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/ObjectRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/ObjectRingBuffer.java deleted file mode 100644 index 9c690c31666..00000000000 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/ObjectRingBuffer.java +++ /dev/null @@ -1,195 +0,0 @@ -/** - * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit CharRingBuffer and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -package io.deephaven.base.ringbuffer; - -import java.io.Serializable; -import java.util.NoSuchElementException; - -/** - * A trivial circular buffer for primitive values, like java.util.concurrent.ArrayBlockingQueue but without all the - * synchronization and collection cruft. Storage is between head (incl.) and tail (excl.) wrapping around the end of the - * array. If the buffer is *not* growable, it will make room for a new element by dropping off the oldest element in the - * buffer instead. - */ -public class ObjectRingBuffer implements Serializable { - protected final boolean growable; - protected Object[] storage; - protected int head, tail; - - private void grow() { - if (growable) { - Object[] newStorage = new Object[storage.length * 2]; - if (tail > head) { - System.arraycopy(storage, head, newStorage, 0, tail - head); - tail = tail - head; - } else { - System.arraycopy(storage, head, newStorage, 0, storage.length - head); - System.arraycopy(storage, 0, newStorage, storage.length - head, tail); - tail += storage.length - head; - } - head = 0; - storage = newStorage; - } else { - head = (head + 1) % storage.length; - } - } - - public boolean isFull() { - return (tail + 1) % storage.length == head; - } - - public ObjectRingBuffer(int capacity) { - this(capacity, true); - } - - public ObjectRingBuffer(int capacity, boolean growable) { - this.growable = growable; - this.storage = new Object[capacity + 1]; - this.tail = this.head = 0; - } - - public boolean isEmpty() { - return tail == head; - } - - public int size() { - return tail >= head ? (tail - head) : (tail + (storage.length - head)); - } - - public int capacity() { - return storage.length - 1; - } - - public int remaining() { - return capacity() - size(); - } - - public void clear() { - tail = head = 0; - } - - public boolean add(Object e) { - if (isFull()) { - grow(); - } - storage[tail] = e; - tail = (tail + 1) % storage.length; - return true; - } - - public Object addOverwrite(Object e, Object notFullResult) { - Object result = notFullResult; - if (isFull()) { - result = remove(); - } - storage[tail] = e; - tail = (tail + 1) % storage.length; - return result; - } - - public boolean offer(Object e) { - if (isFull()) { - return false; - } - storage[tail] = e; - tail = (tail + 1) % storage.length; - return true; - } - - public Object remove() { - if (isEmpty()) { - throw new NoSuchElementException(); - } - Object e = storage[head]; - head = (head + 1) % storage.length; - return e; - } - - public Object poll(Object onEmpty) { - if (isEmpty()) { - return onEmpty; - } - Object e = storage[head]; - head = (head + 1) % storage.length; - return e; - } - - public Object element() { - if (isEmpty()) { - throw new NoSuchElementException(); - } - return storage[head]; - } - - public Object peek(Object onEmpty) { - if (isEmpty()) { - return onEmpty; - } - return storage[head]; - } - - public Object front() { - return front(0); - } - - public Object front(int offset) { - if (offset >= size()) { - throw new NoSuchElementException(); - } - return storage[(head + offset) % storage.length]; - } - - public Object back() { - if (isEmpty()) { - throw new NoSuchElementException(); - } - return tail == 0 ? storage[storage.length - 1] : storage[tail - 1]; - } - - public Object peekBack(Object onEmpty) { - if (isEmpty()) { - return onEmpty; - } - return tail == 0 ? storage[storage.length - 1] : storage[tail - 1]; - } - - public Iterator iterator() { - return new Iterator(); - } - - public class Iterator { - int count = -1; - - public boolean hasNext() { - return count + 1 < size(); - } - - public Object next() { - count++; - return storage[(head + count) % storage.length]; - } - - public void remove() { - throw new UnsupportedOperationException(); - } - } - - public Object[] getAll() { - Object[] result = new Object[size()]; - if (result.length > 0) { - if (tail > head) { - System.arraycopy(storage, head, result, 0, tail - head); - } else { - System.arraycopy(storage, head, result, 0, storage.length - head); - System.arraycopy(storage, 0, result, storage.length - head, tail); - } - } - return result; - } -} diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/ObjectRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/ObjectRingBufferTest.java deleted file mode 100644 index 30f21d6c748..00000000000 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/ObjectRingBufferTest.java +++ /dev/null @@ -1,399 +0,0 @@ -/** - * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit CharRingBufferTest and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -package io.deephaven.base.ringbuffer; - -import junit.framework.TestCase; - -import java.util.NoSuchElementException; - -public class ObjectRingBufferTest extends TestCase { - - final Object SENTINEL = "SENTINEL"; - - private void assertEmpty(ObjectRingBuffer rb) { - assertTrue(rb.isEmpty()); - assertEquals(0, rb.size()); - - assertEquals(SENTINEL, rb.peek(SENTINEL)); - try { - rb.element(); - fail("queue should be empty"); - } catch (NoSuchElementException x) { - // expected - } - - assertEquals(SENTINEL, rb.poll(SENTINEL)); - try { - rb.remove(); - fail("queue should be empty"); - } catch (NoSuchElementException x) { - // expected - } - } - - private void assertFull(ObjectRingBuffer rb) { - assertFalse(rb.isEmpty()); - assertEquals(rb.capacity(), rb.size()); - } - - private void assertNotEmpty(ObjectRingBuffer rb, int expectedSize, Object expectedHead) { - assertFalse(rb.isEmpty()); - assertEquals(expectedSize, rb.size()); - - assertTrue(expectedHead == rb.peek(SENTINEL)); - try { - assertTrue(expectedHead == rb.element()); - } catch (NoSuchElementException x) { - fail("queue should not be empty"); - } - } - - private void assertAdd(ObjectRingBuffer rb, Object newHead, int expectedSize, Object expectedHead) { - assertTrue(rb.add(newHead)); - assertNotEmpty(rb, expectedSize, expectedHead); - } - - private void assertOffer(ObjectRingBuffer rb, Object newHead, int expectedSize, Object expectedHead) { - assertTrue(rb.offer(newHead)); - assertNotEmpty(rb, expectedSize, expectedHead); - } - - private void assertPoll(ObjectRingBuffer rb, int expectedSize, Object expectedHead) { - assertNotEmpty(rb, expectedSize, expectedHead); - assertTrue(expectedHead == rb.poll(SENTINEL)); - } - - private void assertRemove(ObjectRingBuffer rb, int expectedSize, Object expectedHead) { - assertNotEmpty(rb, expectedSize, expectedHead); - try { - assertTrue(expectedHead == rb.remove()); - } catch (NoSuchElementException x) { - fail("queue should not be empty"); - } - } - - private void assertContents(ObjectRingBuffer rb, Object[] expectedData) { - final Object[] data = rb.getAll(); - assertEquals(data.length, expectedData.length); - for (int ii = 0; ii < data.length; ii++) { - assertEquals(data[ii], expectedData[ii]); - } - } - - Object A = 'A'; - Object B = 'B'; - Object C = 'C'; - Object D = 'D'; - Object E = 'E'; - Object F = 'F'; - - public void testAddRemove() { - - ObjectRingBuffer rb = new ObjectRingBuffer(3); - - assertEmpty(rb); - - assertAdd(rb, A, 1, A); - assertAdd(rb, B, 2, A); - assertAdd(rb, C, 3, A); - assertContents(rb, new Object[] {A, B, C}); - assertFull(rb); - - assertRemove(rb, 3, A); - assertContents(rb, new Object[] {B, C}); - - assertRemove(rb, 2, B); - assertContents(rb, new Object[] {C}); - - assertRemove(rb, 1, C); - assertContents(rb, new Object[0]); - assertEmpty(rb); - - assertAdd(rb, A, 1, A); - assertAdd(rb, B, 2, A); - assertContents(rb, new Object[] {A, B}); - - assertRemove(rb, 2, A); - assertContents(rb, new Object[] {B}); - - assertAdd(rb, C, 2, B); - assertContents(rb, new Object[] {B, C}); - - assertRemove(rb, 2, B); - assertContents(rb, new Object[] {C}); - - assertRemove(rb, 1, C); - assertContents(rb, new Object[0]); - - assertEmpty(rb); - - assertAdd(rb, A, 1, A); - assertRemove(rb, 1, A); - assertEmpty(rb); - assertAdd(rb, B, 1, B); - assertRemove(rb, 1, B); - assertEmpty(rb); - assertAdd(rb, C, 1, C); - assertRemove(rb, 1, C); - assertEmpty(rb); - - assertAdd(rb, D, 1, D); - assertRemove(rb, 1, D); - assertEmpty(rb); - assertAdd(rb, E, 1, E); - assertRemove(rb, 1, E); - assertEmpty(rb); - assertAdd(rb, F, 1, F); - assertRemove(rb, 1, F); - assertEmpty(rb); - - assertAdd(rb, A, 1, A); - assertAdd(rb, B, 2, A); - assertAdd(rb, C, 3, A); - assertContents(rb, new Object[] {A, B, C}); - assertFull(rb); - - assertAdd(rb, D, 4, A); - assertAdd(rb, E, 5, A); - assertAdd(rb, F, 6, A); - assertContents(rb, new Object[] {A, B, C, D, E, F}); - - assertRemove(rb, 6, A); - assertRemove(rb, 5, B); - assertRemove(rb, 4, C); - assertRemove(rb, 3, D); - assertRemove(rb, 2, E); - assertRemove(rb, 1, F); - assertContents(rb, new Object[0]); - assertEmpty(rb); - } - - public void testOfferPoll() { - ObjectRingBuffer rb = new ObjectRingBuffer(3); - - assertEmpty(rb); - - assertOffer(rb, A, 1, A); - assertOffer(rb, B, 2, A); - assertOffer(rb, C, 3, A); - - assertFull(rb); - - assertPoll(rb, 3, A); - assertPoll(rb, 2, B); - assertPoll(rb, 1, C); - assertEmpty(rb); - - assertOffer(rb, A, 1, A); - assertOffer(rb, B, 2, A); - assertPoll(rb, 2, A); - assertOffer(rb, C, 2, B); - assertPoll(rb, 2, B); - assertPoll(rb, 1, C); - assertEmpty(rb); - - assertOffer(rb, A, 1, A); - assertPoll(rb, 1, A); - assertEmpty(rb); - assertOffer(rb, B, 1, B); - assertPoll(rb, 1, B); - assertEmpty(rb); - assertOffer(rb, C, 1, C); - assertPoll(rb, 1, C); - assertEmpty(rb); - - assertOffer(rb, D, 1, D); - assertPoll(rb, 1, D); - assertEmpty(rb); - assertOffer(rb, E, 1, E); - assertPoll(rb, 1, E); - assertEmpty(rb); - assertOffer(rb, F, 1, F); - assertPoll(rb, 1, F); - assertEmpty(rb); - - assertOffer(rb, A, 1, A); - assertOffer(rb, B, 2, A); - assertOffer(rb, C, 3, A); - assertFull(rb); - - assertAdd(rb, D, 4, A); // need one add to grow it - assertOffer(rb, E, 5, A); // NOTE: assumes capacity grows by at least a factor of two - assertOffer(rb, F, 6, A); - - assertPoll(rb, 6, A); - assertPoll(rb, 5, B); - assertPoll(rb, 4, C); - assertPoll(rb, 3, D); - assertPoll(rb, 2, E); - assertPoll(rb, 1, F); - assertEmpty(rb); - } - - public void testGrowSimple() { - ObjectRingBuffer rb = new ObjectRingBuffer(5); - - assertAdd(rb, A, 1, A); - assertAdd(rb, B, 2, A); - assertAdd(rb, C, 3, A); - assertAdd(rb, D, 4, A); - assertAdd(rb, E, 5, A); - assertFull(rb); - - // this will grow; the elements are in a single contiguous block - assertAdd(rb, F, 6, A); - - assertRemove(rb, 6, A); - assertRemove(rb, 5, B); - assertRemove(rb, 4, C); - assertRemove(rb, 3, D); - assertRemove(rb, 2, E); - assertRemove(rb, 1, F); - assertEmpty(rb); - } - - public void testGrowComplex() { - ObjectRingBuffer rb = new ObjectRingBuffer(5); - - assertAdd(rb, A, 1, A); - assertAdd(rb, B, 2, A); - assertAdd(rb, C, 3, A); - assertRemove(rb, 3, A); - assertRemove(rb, 2, B); - assertRemove(rb, 1, C); - assertEmpty(rb); - - assertAdd(rb, A, 1, A); - assertAdd(rb, B, 2, A); - assertAdd(rb, C, 3, A); - assertAdd(rb, D, 4, A); - assertAdd(rb, E, 5, A); - assertFull(rb); - - // this will grow; the elements are in two blocks - assertAdd(rb, F, 6, A); - - assertRemove(rb, 6, A); - assertRemove(rb, 5, B); - assertRemove(rb, 4, C); - assertRemove(rb, 3, D); - assertRemove(rb, 2, E); - assertRemove(rb, 1, F); - assertEmpty(rb); - } - - public void testIterator() { - ObjectRingBuffer rb = new ObjectRingBuffer(3); - - ObjectRingBuffer.Iterator iter = rb.iterator(); - assertFalse(iter.hasNext()); - - assertAdd(rb, A, 1, A); - assertAdd(rb, B, 2, A); - assertAdd(rb, C, 3, A); - assertFull(rb); - - iter = rb.iterator(); - assertTrue(iter.hasNext()); - assertEquals(A, iter.next()); - assertTrue(iter.hasNext()); - assertEquals(B, iter.next()); - assertTrue(iter.hasNext()); - assertEquals(C, iter.next()); - assertFalse(iter.hasNext()); - - assertRemove(rb, 3, A); - assertAdd(rb, D, 3, B); - - iter = rb.iterator(); - assertTrue(iter.hasNext()); - assertEquals(B, iter.next()); - assertTrue(iter.hasNext()); - assertEquals(C, iter.next()); - assertTrue(iter.hasNext()); - assertEquals(D, iter.next()); - assertFalse(iter.hasNext()); - - assertRemove(rb, 3, B); - - iter = rb.iterator(); - assertEquals(C, iter.next()); - assertEquals(D, iter.next()); - assertFalse(iter.hasNext()); - - assertAdd(rb, E, 3, C); - assertAdd(rb, F, 4, C); - - iter = rb.iterator(); - assertTrue(iter.hasNext()); - assertEquals(C, iter.next()); - assertTrue(iter.hasNext()); - assertEquals(D, iter.next()); - assertTrue(iter.hasNext()); - assertEquals(E, iter.next()); - assertTrue(iter.hasNext()); - assertEquals(F, iter.next()); - assertFalse(iter.hasNext()); - } - - public void testBack() { - ObjectRingBuffer rb = new ObjectRingBuffer(5); - - assertAdd(rb, A, 1, A); - assertEquals(rb.back(), A); - - assertAdd(rb, B, 2, A); - assertAdd(rb, C, 3, A); - assertEquals(rb.back(), C); - } - - public void testBackWhenEmpty() { - ObjectRingBuffer rb = new ObjectRingBuffer(5); - try { - rb.back(); - fail("expected a NoSuchElement exception"); - } catch (NoSuchElementException x) { - // expected - } - } - - public void testBackTailIsZero() { - ObjectRingBuffer rb = new ObjectRingBuffer(5); - - assertAdd(rb, A, 1, A); - assertAdd(rb, B, 2, A); - assertAdd(rb, C, 3, A); - assertAdd(rb, D, 4, A); - assertAdd(rb, E, 5, A); - assertFull(rb); - - assertRemove(rb, 5, A); - assertAdd(rb, F, 5, B); - assertEquals(rb.back(), F); - } - - public void testLargeAmounts() { - ObjectRingBuffer rb = new ObjectRingBuffer(3); - - for (int i = 0; i < 100; i++) - rb.add((Object) i); - - for (int i = 100; i < 200; i++) { - rb.add((Object) i); - assertEquals((Object) (i - 100 + 1), rb.front(1)); - assertEquals((Object) (i - 100), rb.poll(SENTINEL)); - } - - for (int i = 200; i < 300; i++) { - if (i < 299) - assertEquals((Object) (i - 100 + 1), rb.front(1)); - assertEquals((Object) (i - 100), rb.poll(SENTINEL)); - } - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectColumnLayer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectColumnLayer.java index 1ebbd1fe155..9f6e2247856 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectColumnLayer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectColumnLayer.java @@ -21,6 +21,7 @@ import io.deephaven.engine.table.impl.sources.ChunkedBackingStoreExposedWritableSource; import io.deephaven.engine.table.impl.sources.ReinterpretUtils; import io.deephaven.engine.table.impl.util.ChunkUtils; +import io.deephaven.engine.table.impl.util.JobScheduler; import io.deephaven.engine.updategraph.DynamicNode; import io.deephaven.engine.updategraph.UpdateCommitterEx; import io.deephaven.engine.updategraph.UpdateGraphProcessor; diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java index 8ba3b2f529a..4d383b87666 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java @@ -8,6 +8,11 @@ import io.deephaven.engine.table.PartitionedTable; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.impl.*; +import io.deephaven.engine.testutil.EvalNugget; +import io.deephaven.engine.testutil.GenerateTableUpdates; +import io.deephaven.engine.testutil.TstUtils; +import io.deephaven.engine.testutil.generator.Generator; +import io.deephaven.engine.testutil.generator.SortedDateTimeGenerator; import io.deephaven.engine.updategraph.UpdateGraphProcessor; import io.deephaven.test.types.OutOfBandTest; import io.deephaven.time.DateTime; @@ -23,9 +28,10 @@ import java.util.List; import java.util.Random; -import static io.deephaven.engine.table.impl.GenerateTableUpdates.generateAppends; -import static io.deephaven.engine.table.impl.RefreshingTableTestCase.simulateShiftAwareStep; -import static io.deephaven.engine.table.impl.TstUtils.*; +import static io.deephaven.engine.testutil.GenerateTableUpdates.generateAppends; +import static io.deephaven.engine.testutil.TstUtils.assertTableEquals; +import static io.deephaven.engine.testutil.TstUtils.testTable; +import static io.deephaven.engine.testutil.testcase.RefreshingTableTestCase.simulateShiftAwareStep; import static io.deephaven.engine.util.TableTools.*; import static io.deephaven.function.Basic.isNull; import static io.deephaven.time.DateTimeUtils.convertDateTime; @@ -84,7 +90,7 @@ public void testStaticZeroKeyFwdRevWindow() { @Test public void testStaticZeroKeyTimed() { final QueryTable t = createTestTable(10000, false, false, false, 0xFFFABBBC, - new String[] {"ts"}, new TstUtils.Generator[] {new TstUtils.SortedDateTimeGenerator( + new String[] {"ts"}, new Generator[] {new SortedDateTimeGenerator( convertDateTime("2022-03-09T09:00:00.000 NY"), convertDateTime("2022-03-09T16:30:00.000 NY"))}).t; @@ -120,7 +126,7 @@ public void testStaticZeroKeyTimed() { @Test public void testStaticZeroKeyFwdWindowTimed() { final QueryTable t = createTestTable(10000, false, false, false, 0xFFFABBBC, - new String[] {"ts"}, new TstUtils.Generator[] {new TstUtils.SortedDateTimeGenerator( + new String[] {"ts"}, new Generator[] {new SortedDateTimeGenerator( convertDateTime("2022-03-09T09:00:00.000 NY"), convertDateTime("2022-03-09T16:30:00.000 NY"))}).t; @@ -156,7 +162,7 @@ public void testStaticZeroKeyFwdWindowTimed() { @Test public void testStaticZeroKeyFwdRevWindowTimed() { final QueryTable t = createTestTable(10000, false, false, false, 0xFFFABBBC, - new String[] {"ts"}, new TstUtils.Generator[] {new TstUtils.SortedDateTimeGenerator( + new String[] {"ts"}, new Generator[] {new SortedDateTimeGenerator( convertDateTime("2022-03-09T09:00:00.000 NY"), convertDateTime("2022-03-09T16:30:00.000 NY"))}).t; @@ -269,7 +275,7 @@ public void testStaticBucketedFwdRevWindowTimed() { private void doTestStaticBucketedTimed(boolean grouped, Duration prevTime, Duration postTime) { final QueryTable t = createTestTable(10000, true, grouped, false, 0xFFFABBBC, - new String[] {"ts"}, new TstUtils.Generator[] {new TstUtils.SortedDateTimeGenerator( + new String[] {"ts"}, new Generator[] {new SortedDateTimeGenerator( convertDateTime("2022-03-09T09:00:00.000 NY"), convertDateTime("2022-03-09T16:30:00.000 NY"))}).t; @@ -348,7 +354,7 @@ public void testBucketedAppendOnlyTimed() { private void doTestAppendOnlyTimed(boolean bucketed) { final CreateResult result = createTestTable(10000, bucketed, false, true, 0x31313131, - new String[] {"ts"}, new TstUtils.Generator[] {new TstUtils.SortedDateTimeGenerator( + new String[] {"ts"}, new Generator[] {new SortedDateTimeGenerator( convertDateTime("2022-03-09T09:00:00.000 NY"), convertDateTime("2022-03-09T16:30:00.000 NY"))}); final QueryTable t = result.t; diff --git a/replication/static/src/main/java/io/deephaven/replicators/ReplicateRingBuffers.java b/replication/static/src/main/java/io/deephaven/replicators/ReplicateRingBuffers.java index 6d2ff4ac0d5..493cde76e05 100644 --- a/replication/static/src/main/java/io/deephaven/replicators/ReplicateRingBuffers.java +++ b/replication/static/src/main/java/io/deephaven/replicators/ReplicateRingBuffers.java @@ -17,17 +17,11 @@ public class ReplicateRingBuffers { public static void main(String... args) throws IOException { - // replicate ring buffers + // replicate ring buffers to all but Object (since RingBuffer<> already exisits) charToAllButBoolean("Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java"); - String objectResult = ReplicatePrimitiveCode.charToObject( - "Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java"); // replicate the tests charToAllButBoolean("Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java"); - objectResult = ReplicatePrimitiveCode.charToObject( - "Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java"); - replaceLines(objectResult, - "SENTINEL = Object.MIN_VALUE", "SENTINEL = \"SENTINEL\""); } private static void replaceLines(String fileResult, String... replacements) throws IOException { From d853cd18a5b69baf3fb4891e617ad9ff45ed4655 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Mon, 19 Dec 2022 13:28:41 -0800 Subject: [PATCH 058/123] modified RingBuffer to throw exception on add() when full and growing disabled, improved test coverage --- .../base/ringbuffer/ByteRingBuffer.java | 42 +++++++++- .../base/ringbuffer/CharRingBuffer.java | 42 +++++++++- .../base/ringbuffer/DoubleRingBuffer.java | 42 +++++++++- .../base/ringbuffer/FloatRingBuffer.java | 42 +++++++++- .../base/ringbuffer/IntRingBuffer.java | 42 +++++++++- .../base/ringbuffer/LongRingBuffer.java | 42 +++++++++- .../base/ringbuffer/ShortRingBuffer.java | 42 +++++++++- .../base/ringbuffer/ByteRingBufferTest.java | 83 ++++++++++++++++--- .../base/ringbuffer/CharRingBufferTest.java | 83 ++++++++++++++++--- .../base/ringbuffer/DoubleRingBufferTest.java | 83 ++++++++++++++++--- .../base/ringbuffer/FloatRingBufferTest.java | 83 ++++++++++++++++--- .../base/ringbuffer/IntRingBufferTest.java | 83 ++++++++++++++++--- .../base/ringbuffer/LongRingBufferTest.java | 83 ++++++++++++++++--- .../base/ringbuffer/ShortRingBufferTest.java | 83 ++++++++++++++++--- .../engine/table/impl/SortListener.java | 8 +- 15 files changed, 789 insertions(+), 94 deletions(-) diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java index 2c7644a4afa..0e7b8544819 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java @@ -35,8 +35,6 @@ private void grow() { } head = 0; storage = newStorage; - } else { - head = (head + 1) % storage.length; } } @@ -44,10 +42,23 @@ public boolean isFull() { return (tail + 1) % storage.length == head; } + /** + * Create an unbounded-growth ring buffer of byte primitives + * + * @param capacity minimum capacity of ring buffer + */ public ByteRingBuffer(int capacity) { this(capacity, true); } + /** + * Create a ring buffer of byte primitives + * + * @param capacity minimum capacity of ring buffer + * @param growable whether to allow growth when the buffer is full. If this is {@code false} and {@code add()} is + * called, an + * + */ public ByteRingBuffer(int capacity, boolean growable) { this.growable = growable; this.storage = new byte[capacity + 1]; @@ -74,15 +85,33 @@ public void clear() { tail = head = 0; } + /** + * Adds an entry to the ring buffer, will throw an {@link UnsupportedOperationException} if buffer is full. For a + * graceful failure, use {@link #offer(byte)} + * + * @param e the byte to be added to the buffer + * @return true if the byte was added successfully + */ public boolean add(byte e) { if (isFull()) { - grow(); + if (!growable) { + throw new UnsupportedOperationException("Ring buffer is full and growth is disabled"); + } else { + grow(); + } } storage[tail] = e; tail = (tail + 1) % storage.length; return true; } + /** + * Add an entry to the ring buffer. If the buffer is full, will overwrite the oldest entry with the new one. + * + * @param e the byte to be added to the buffer + * @param notFullResult value to return is the buffer is not full + * @return the overwritten entry if the buffer is full, the provided value otherwise + */ public byte addOverwrite(byte e, byte notFullResult) { byte result = notFullResult; if (isFull()) { @@ -93,6 +122,13 @@ public byte addOverwrite(byte e, byte notFullResult) { return result; } + /** + * Attempt to add an entry to the ring buffer. If the buffer is full, the write will fail and the buffer will not + * grow even if allowed. + * + * @param e the byte to be added to the buffer + * @return true if the value was added successfully, false otherwise + */ public boolean offer(byte e) { if (isFull()) { return false; diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java index 34d1060f38d..f23fe51c2f3 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java @@ -30,8 +30,6 @@ private void grow() { } head = 0; storage = newStorage; - } else { - head = (head + 1) % storage.length; } } @@ -39,10 +37,23 @@ public boolean isFull() { return (tail + 1) % storage.length == head; } + /** + * Create an unbounded-growth ring buffer of char primitives + * + * @param capacity minimum capacity of ring buffer + */ public CharRingBuffer(int capacity) { this(capacity, true); } + /** + * Create a ring buffer of char primitives + * + * @param capacity minimum capacity of ring buffer + * @param growable whether to allow growth when the buffer is full. If this is {@code false} and {@code add()} is + * called, an + * + */ public CharRingBuffer(int capacity, boolean growable) { this.growable = growable; this.storage = new char[capacity + 1]; @@ -69,15 +80,33 @@ public void clear() { tail = head = 0; } + /** + * Adds an entry to the ring buffer, will throw an {@link UnsupportedOperationException} if buffer is full. For a + * graceful failure, use {@link #offer(char)} + * + * @param e the char to be added to the buffer + * @return true if the char was added successfully + */ public boolean add(char e) { if (isFull()) { - grow(); + if (!growable) { + throw new UnsupportedOperationException("Ring buffer is full and growth is disabled"); + } else { + grow(); + } } storage[tail] = e; tail = (tail + 1) % storage.length; return true; } + /** + * Add an entry to the ring buffer. If the buffer is full, will overwrite the oldest entry with the new one. + * + * @param e the char to be added to the buffer + * @param notFullResult value to return is the buffer is not full + * @return the overwritten entry if the buffer is full, the provided value otherwise + */ public char addOverwrite(char e, char notFullResult) { char result = notFullResult; if (isFull()) { @@ -88,6 +117,13 @@ public char addOverwrite(char e, char notFullResult) { return result; } + /** + * Attempt to add an entry to the ring buffer. If the buffer is full, the write will fail and the buffer will not + * grow even if allowed. + * + * @param e the char to be added to the buffer + * @return true if the value was added successfully, false otherwise + */ public boolean offer(char e) { if (isFull()) { return false; diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java index e39f94e46e6..b729908c95f 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java @@ -35,8 +35,6 @@ private void grow() { } head = 0; storage = newStorage; - } else { - head = (head + 1) % storage.length; } } @@ -44,10 +42,23 @@ public boolean isFull() { return (tail + 1) % storage.length == head; } + /** + * Create an unbounded-growth ring buffer of double primitives + * + * @param capacity minimum capacity of ring buffer + */ public DoubleRingBuffer(int capacity) { this(capacity, true); } + /** + * Create a ring buffer of double primitives + * + * @param capacity minimum capacity of ring buffer + * @param growable whether to allow growth when the buffer is full. If this is {@code false} and {@code add()} is + * called, an + * + */ public DoubleRingBuffer(int capacity, boolean growable) { this.growable = growable; this.storage = new double[capacity + 1]; @@ -74,15 +85,33 @@ public void clear() { tail = head = 0; } + /** + * Adds an entry to the ring buffer, will throw an {@link UnsupportedOperationException} if buffer is full. For a + * graceful failure, use {@link #offer(double)} + * + * @param e the double to be added to the buffer + * @return true if the double was added successfully + */ public boolean add(double e) { if (isFull()) { - grow(); + if (!growable) { + throw new UnsupportedOperationException("Ring buffer is full and growth is disabled"); + } else { + grow(); + } } storage[tail] = e; tail = (tail + 1) % storage.length; return true; } + /** + * Add an entry to the ring buffer. If the buffer is full, will overwrite the oldest entry with the new one. + * + * @param e the double to be added to the buffer + * @param notFullResult value to return is the buffer is not full + * @return the overwritten entry if the buffer is full, the provided value otherwise + */ public double addOverwrite(double e, double notFullResult) { double result = notFullResult; if (isFull()) { @@ -93,6 +122,13 @@ public double addOverwrite(double e, double notFullResult) { return result; } + /** + * Attempt to add an entry to the ring buffer. If the buffer is full, the write will fail and the buffer will not + * grow even if allowed. + * + * @param e the double to be added to the buffer + * @return true if the value was added successfully, false otherwise + */ public boolean offer(double e) { if (isFull()) { return false; diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java index b62b4a7d36b..38fd409287a 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java @@ -35,8 +35,6 @@ private void grow() { } head = 0; storage = newStorage; - } else { - head = (head + 1) % storage.length; } } @@ -44,10 +42,23 @@ public boolean isFull() { return (tail + 1) % storage.length == head; } + /** + * Create an unbounded-growth ring buffer of float primitives + * + * @param capacity minimum capacity of ring buffer + */ public FloatRingBuffer(int capacity) { this(capacity, true); } + /** + * Create a ring buffer of float primitives + * + * @param capacity minimum capacity of ring buffer + * @param growable whether to allow growth when the buffer is full. If this is {@code false} and {@code add()} is + * called, an + * + */ public FloatRingBuffer(int capacity, boolean growable) { this.growable = growable; this.storage = new float[capacity + 1]; @@ -74,15 +85,33 @@ public void clear() { tail = head = 0; } + /** + * Adds an entry to the ring buffer, will throw an {@link UnsupportedOperationException} if buffer is full. For a + * graceful failure, use {@link #offer(float)} + * + * @param e the float to be added to the buffer + * @return true if the float was added successfully + */ public boolean add(float e) { if (isFull()) { - grow(); + if (!growable) { + throw new UnsupportedOperationException("Ring buffer is full and growth is disabled"); + } else { + grow(); + } } storage[tail] = e; tail = (tail + 1) % storage.length; return true; } + /** + * Add an entry to the ring buffer. If the buffer is full, will overwrite the oldest entry with the new one. + * + * @param e the float to be added to the buffer + * @param notFullResult value to return is the buffer is not full + * @return the overwritten entry if the buffer is full, the provided value otherwise + */ public float addOverwrite(float e, float notFullResult) { float result = notFullResult; if (isFull()) { @@ -93,6 +122,13 @@ public float addOverwrite(float e, float notFullResult) { return result; } + /** + * Attempt to add an entry to the ring buffer. If the buffer is full, the write will fail and the buffer will not + * grow even if allowed. + * + * @param e the float to be added to the buffer + * @return true if the value was added successfully, false otherwise + */ public boolean offer(float e) { if (isFull()) { return false; diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java index 89218f977df..dc98823b506 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java @@ -35,8 +35,6 @@ private void grow() { } head = 0; storage = newStorage; - } else { - head = (head + 1) % storage.length; } } @@ -44,10 +42,23 @@ public boolean isFull() { return (tail + 1) % storage.length == head; } + /** + * Create an unbounded-growth ring buffer of int primitives + * + * @param capacity minimum capacity of ring buffer + */ public IntRingBuffer(int capacity) { this(capacity, true); } + /** + * Create a ring buffer of int primitives + * + * @param capacity minimum capacity of ring buffer + * @param growable whether to allow growth when the buffer is full. If this is {@code false} and {@code add()} is + * called, an + * + */ public IntRingBuffer(int capacity, boolean growable) { this.growable = growable; this.storage = new int[capacity + 1]; @@ -74,15 +85,33 @@ public void clear() { tail = head = 0; } + /** + * Adds an entry to the ring buffer, will throw an {@link UnsupportedOperationException} if buffer is full. For a + * graceful failure, use {@link #offer(int)} + * + * @param e the int to be added to the buffer + * @return true if the int was added successfully + */ public boolean add(int e) { if (isFull()) { - grow(); + if (!growable) { + throw new UnsupportedOperationException("Ring buffer is full and growth is disabled"); + } else { + grow(); + } } storage[tail] = e; tail = (tail + 1) % storage.length; return true; } + /** + * Add an entry to the ring buffer. If the buffer is full, will overwrite the oldest entry with the new one. + * + * @param e the int to be added to the buffer + * @param notFullResult value to return is the buffer is not full + * @return the overwritten entry if the buffer is full, the provided value otherwise + */ public int addOverwrite(int e, int notFullResult) { int result = notFullResult; if (isFull()) { @@ -93,6 +122,13 @@ public int addOverwrite(int e, int notFullResult) { return result; } + /** + * Attempt to add an entry to the ring buffer. If the buffer is full, the write will fail and the buffer will not + * grow even if allowed. + * + * @param e the int to be added to the buffer + * @return true if the value was added successfully, false otherwise + */ public boolean offer(int e) { if (isFull()) { return false; diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java index 9011c0eef97..17433ffd3be 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java @@ -35,8 +35,6 @@ private void grow() { } head = 0; storage = newStorage; - } else { - head = (head + 1) % storage.length; } } @@ -44,10 +42,23 @@ public boolean isFull() { return (tail + 1) % storage.length == head; } + /** + * Create an unbounded-growth ring buffer of long primitives + * + * @param capacity minimum capacity of ring buffer + */ public LongRingBuffer(int capacity) { this(capacity, true); } + /** + * Create a ring buffer of long primitives + * + * @param capacity minimum capacity of ring buffer + * @param growable whether to allow growth when the buffer is full. If this is {@code false} and {@code add()} is + * called, an + * + */ public LongRingBuffer(int capacity, boolean growable) { this.growable = growable; this.storage = new long[capacity + 1]; @@ -74,15 +85,33 @@ public void clear() { tail = head = 0; } + /** + * Adds an entry to the ring buffer, will throw an {@link UnsupportedOperationException} if buffer is full. For a + * graceful failure, use {@link #offer(long)} + * + * @param e the long to be added to the buffer + * @return true if the long was added successfully + */ public boolean add(long e) { if (isFull()) { - grow(); + if (!growable) { + throw new UnsupportedOperationException("Ring buffer is full and growth is disabled"); + } else { + grow(); + } } storage[tail] = e; tail = (tail + 1) % storage.length; return true; } + /** + * Add an entry to the ring buffer. If the buffer is full, will overwrite the oldest entry with the new one. + * + * @param e the long to be added to the buffer + * @param notFullResult value to return is the buffer is not full + * @return the overwritten entry if the buffer is full, the provided value otherwise + */ public long addOverwrite(long e, long notFullResult) { long result = notFullResult; if (isFull()) { @@ -93,6 +122,13 @@ public long addOverwrite(long e, long notFullResult) { return result; } + /** + * Attempt to add an entry to the ring buffer. If the buffer is full, the write will fail and the buffer will not + * grow even if allowed. + * + * @param e the long to be added to the buffer + * @return true if the value was added successfully, false otherwise + */ public boolean offer(long e) { if (isFull()) { return false; diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java index dcf75d3e6c7..ac5241c15fc 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java @@ -35,8 +35,6 @@ private void grow() { } head = 0; storage = newStorage; - } else { - head = (head + 1) % storage.length; } } @@ -44,10 +42,23 @@ public boolean isFull() { return (tail + 1) % storage.length == head; } + /** + * Create an unbounded-growth ring buffer of short primitives + * + * @param capacity minimum capacity of ring buffer + */ public ShortRingBuffer(int capacity) { this(capacity, true); } + /** + * Create a ring buffer of short primitives + * + * @param capacity minimum capacity of ring buffer + * @param growable whether to allow growth when the buffer is full. If this is {@code false} and {@code add()} is + * called, an + * + */ public ShortRingBuffer(int capacity, boolean growable) { this.growable = growable; this.storage = new short[capacity + 1]; @@ -74,15 +85,33 @@ public void clear() { tail = head = 0; } + /** + * Adds an entry to the ring buffer, will throw an {@link UnsupportedOperationException} if buffer is full. For a + * graceful failure, use {@link #offer(short)} + * + * @param e the short to be added to the buffer + * @return true if the short was added successfully + */ public boolean add(short e) { if (isFull()) { - grow(); + if (!growable) { + throw new UnsupportedOperationException("Ring buffer is full and growth is disabled"); + } else { + grow(); + } } storage[tail] = e; tail = (tail + 1) % storage.length; return true; } + /** + * Add an entry to the ring buffer. If the buffer is full, will overwrite the oldest entry with the new one. + * + * @param e the short to be added to the buffer + * @param notFullResult value to return is the buffer is not full + * @return the overwritten entry if the buffer is full, the provided value otherwise + */ public short addOverwrite(short e, short notFullResult) { short result = notFullResult; if (isFull()) { @@ -93,6 +122,13 @@ public short addOverwrite(short e, short notFullResult) { return result; } + /** + * Attempt to add an entry to the ring buffer. If the buffer is full, the write will fail and the buffer will not + * grow even if allowed. + * + * @param e the short to be added to the buffer + * @return true if the value was added successfully, false otherwise + */ public boolean offer(short e) { if (isFull()) { return false; diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/ByteRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/ByteRingBufferTest.java index f67ba44d124..89e61661f42 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/ByteRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/ByteRingBufferTest.java @@ -9,9 +9,13 @@ package io.deephaven.base.ringbuffer; import junit.framework.TestCase; +import org.junit.Assert; +import org.junit.Test; import java.util.NoSuchElementException; +import static org.junit.Assert.assertThrows; + public class ByteRingBufferTest extends TestCase { final byte SENTINEL = Byte.MIN_VALUE; @@ -78,7 +82,7 @@ private void assertRemove(ByteRingBuffer rb, int expectedSize, byte expectedHead } } - private void assertContents(ByteRingBuffer rb, byte[] expectedData) { + private void assertContents(ByteRingBuffer rb, byte... expectedData) { final byte[] data = rb.getAll(); assertEquals(data.length, expectedData.length); for (int ii = 0; ii < data.length; ii++) { @@ -102,31 +106,32 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertContents(rb, new byte[] {A, B, C}); + assertContents(rb, A, B, C); assertFull(rb); assertRemove(rb, 3, A); - assertContents(rb, new byte[] {B, C}); + assertContents(rb, B, C); assertRemove(rb, 2, B); - assertContents(rb, new byte[] {C}); + assertContents(rb, C); assertRemove(rb, 1, C); assertContents(rb, new byte[0]); + assertEmpty(rb); assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); - assertContents(rb, new byte[] {A, B}); + assertContents(rb, A, B); assertRemove(rb, 2, A); - assertContents(rb, new byte[] {B}); + assertContents(rb, B); assertAdd(rb, C, 2, B); - assertContents(rb, new byte[] {B, C}); + assertContents(rb, B, C); assertRemove(rb, 2, B); - assertContents(rb, new byte[] {C}); + assertContents(rb, C); assertRemove(rb, 1, C); assertContents(rb, new byte[0]); @@ -156,13 +161,13 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertContents(rb, new byte[] {A, B, C}); + assertContents(rb, A, B, C); assertFull(rb); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); assertAdd(rb, F, 6, A); - assertContents(rb, new byte[] {A, B, C, D, E, F}); + assertContents(rb, A, B, C, D, E, F); assertRemove(rb, 6, A); assertRemove(rb, 5, B); @@ -340,6 +345,11 @@ public void testIterator() { assertTrue(iter.hasNext()); assertEquals(F, iter.next()); assertFalse(iter.hasNext()); + + final ByteRingBuffer.Iterator iterFinal = rb.iterator(); + + assertThrows(UnsupportedOperationException.class, + () -> iterFinal.remove()); } public void testBack() { @@ -396,4 +406,57 @@ public void testLargeAmounts() { assertEquals((byte) (i - 100), rb.poll(SENTINEL)); } } + + public void testAddExceptionWhenFull() { + ByteRingBuffer rb = new ByteRingBuffer(3, false); + assert (rb.add(A)); + assert (rb.add(B)); + assert (rb.add(C)); + + // this should throw + assertThrows(UnsupportedOperationException.class, + () -> rb.add(D)); + } + + public void testAddOverwriteAndOffer() { + ByteRingBuffer rb = new ByteRingBuffer(3, false); + assert (3 == rb.remaining()); + + assert (F == rb.addOverwrite(A, F)); + assert (2 == rb.remaining()); + + assert (F == rb.addOverwrite(B, F)); + assert (1 == rb.remaining()); + + assert (F == rb.addOverwrite(C, F)); + assert (0 == rb.remaining()); + assert (rb.isFull()); + + // now full, should return first value + assert (A == rb.addOverwrite(D, F)); + assert (B == rb.addOverwrite(E, F)); + assert (rb.isFull()); + + // offer() testing + assert (false == rb.offer(F)); + assert (C == rb.remove()); + assert (true == rb.offer(F)); + + // peek testing + assert (D == rb.front()); + assert (E == rb.front(1)); + assert (F == rb.front(2)); + // this should throw + assertThrows(NoSuchElementException.class, + () -> rb.front(99)); + + assert (F == rb.peekBack(A)); + + // clear() testing + rb.clear(); + assert (rb.isEmpty()); + + assert (A == rb.peekBack(A)); + + } } diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java index c968881bb3e..3cbf2eb050d 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java @@ -4,9 +4,13 @@ package io.deephaven.base.ringbuffer; import junit.framework.TestCase; +import org.junit.Assert; +import org.junit.Test; import java.util.NoSuchElementException; +import static org.junit.Assert.assertThrows; + public class CharRingBufferTest extends TestCase { final char SENTINEL = Character.MIN_VALUE; @@ -73,7 +77,7 @@ private void assertRemove(CharRingBuffer rb, int expectedSize, char expectedHead } } - private void assertContents(CharRingBuffer rb, char[] expectedData) { + private void assertContents(CharRingBuffer rb, char... expectedData) { final char[] data = rb.getAll(); assertEquals(data.length, expectedData.length); for (int ii = 0; ii < data.length; ii++) { @@ -97,31 +101,32 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertContents(rb, new char[] {A, B, C}); + assertContents(rb, A, B, C); assertFull(rb); assertRemove(rb, 3, A); - assertContents(rb, new char[] {B, C}); + assertContents(rb, B, C); assertRemove(rb, 2, B); - assertContents(rb, new char[] {C}); + assertContents(rb, C); assertRemove(rb, 1, C); assertContents(rb, new char[0]); + assertEmpty(rb); assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); - assertContents(rb, new char[] {A, B}); + assertContents(rb, A, B); assertRemove(rb, 2, A); - assertContents(rb, new char[] {B}); + assertContents(rb, B); assertAdd(rb, C, 2, B); - assertContents(rb, new char[] {B, C}); + assertContents(rb, B, C); assertRemove(rb, 2, B); - assertContents(rb, new char[] {C}); + assertContents(rb, C); assertRemove(rb, 1, C); assertContents(rb, new char[0]); @@ -151,13 +156,13 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertContents(rb, new char[] {A, B, C}); + assertContents(rb, A, B, C); assertFull(rb); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); assertAdd(rb, F, 6, A); - assertContents(rb, new char[] {A, B, C, D, E, F}); + assertContents(rb, A, B, C, D, E, F); assertRemove(rb, 6, A); assertRemove(rb, 5, B); @@ -335,6 +340,11 @@ public void testIterator() { assertTrue(iter.hasNext()); assertEquals(F, iter.next()); assertFalse(iter.hasNext()); + + final CharRingBuffer.Iterator iterFinal = rb.iterator(); + + assertThrows(UnsupportedOperationException.class, + () -> iterFinal.remove()); } public void testBack() { @@ -391,4 +401,57 @@ public void testLargeAmounts() { assertEquals((char) (i - 100), rb.poll(SENTINEL)); } } + + public void testAddExceptionWhenFull() { + CharRingBuffer rb = new CharRingBuffer(3, false); + assert (rb.add(A)); + assert (rb.add(B)); + assert (rb.add(C)); + + // this should throw + assertThrows(UnsupportedOperationException.class, + () -> rb.add(D)); + } + + public void testAddOverwriteAndOffer() { + CharRingBuffer rb = new CharRingBuffer(3, false); + assert (3 == rb.remaining()); + + assert (F == rb.addOverwrite(A, F)); + assert (2 == rb.remaining()); + + assert (F == rb.addOverwrite(B, F)); + assert (1 == rb.remaining()); + + assert (F == rb.addOverwrite(C, F)); + assert (0 == rb.remaining()); + assert (rb.isFull()); + + // now full, should return first value + assert (A == rb.addOverwrite(D, F)); + assert (B == rb.addOverwrite(E, F)); + assert (rb.isFull()); + + // offer() testing + assert (false == rb.offer(F)); + assert (C == rb.remove()); + assert (true == rb.offer(F)); + + // peek testing + assert (D == rb.front()); + assert (E == rb.front(1)); + assert (F == rb.front(2)); + // this should throw + assertThrows(NoSuchElementException.class, + () -> rb.front(99)); + + assert (F == rb.peekBack(A)); + + // clear() testing + rb.clear(); + assert (rb.isEmpty()); + + assert (A == rb.peekBack(A)); + + } } diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/DoubleRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/DoubleRingBufferTest.java index 494341d7015..9cac78dd348 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/DoubleRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/DoubleRingBufferTest.java @@ -9,9 +9,13 @@ package io.deephaven.base.ringbuffer; import junit.framework.TestCase; +import org.junit.Assert; +import org.junit.Test; import java.util.NoSuchElementException; +import static org.junit.Assert.assertThrows; + public class DoubleRingBufferTest extends TestCase { final double SENTINEL = Double.MIN_VALUE; @@ -78,7 +82,7 @@ private void assertRemove(DoubleRingBuffer rb, int expectedSize, double expected } } - private void assertContents(DoubleRingBuffer rb, double[] expectedData) { + private void assertContents(DoubleRingBuffer rb, double... expectedData) { final double[] data = rb.getAll(); assertEquals(data.length, expectedData.length); for (int ii = 0; ii < data.length; ii++) { @@ -102,31 +106,32 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertContents(rb, new double[] {A, B, C}); + assertContents(rb, A, B, C); assertFull(rb); assertRemove(rb, 3, A); - assertContents(rb, new double[] {B, C}); + assertContents(rb, B, C); assertRemove(rb, 2, B); - assertContents(rb, new double[] {C}); + assertContents(rb, C); assertRemove(rb, 1, C); assertContents(rb, new double[0]); + assertEmpty(rb); assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); - assertContents(rb, new double[] {A, B}); + assertContents(rb, A, B); assertRemove(rb, 2, A); - assertContents(rb, new double[] {B}); + assertContents(rb, B); assertAdd(rb, C, 2, B); - assertContents(rb, new double[] {B, C}); + assertContents(rb, B, C); assertRemove(rb, 2, B); - assertContents(rb, new double[] {C}); + assertContents(rb, C); assertRemove(rb, 1, C); assertContents(rb, new double[0]); @@ -156,13 +161,13 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertContents(rb, new double[] {A, B, C}); + assertContents(rb, A, B, C); assertFull(rb); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); assertAdd(rb, F, 6, A); - assertContents(rb, new double[] {A, B, C, D, E, F}); + assertContents(rb, A, B, C, D, E, F); assertRemove(rb, 6, A); assertRemove(rb, 5, B); @@ -340,6 +345,11 @@ public void testIterator() { assertTrue(iter.hasNext()); assertEquals(F, iter.next()); assertFalse(iter.hasNext()); + + final DoubleRingBuffer.Iterator iterFinal = rb.iterator(); + + assertThrows(UnsupportedOperationException.class, + () -> iterFinal.remove()); } public void testBack() { @@ -396,4 +406,57 @@ public void testLargeAmounts() { assertEquals((double) (i - 100), rb.poll(SENTINEL)); } } + + public void testAddExceptionWhenFull() { + DoubleRingBuffer rb = new DoubleRingBuffer(3, false); + assert (rb.add(A)); + assert (rb.add(B)); + assert (rb.add(C)); + + // this should throw + assertThrows(UnsupportedOperationException.class, + () -> rb.add(D)); + } + + public void testAddOverwriteAndOffer() { + DoubleRingBuffer rb = new DoubleRingBuffer(3, false); + assert (3 == rb.remaining()); + + assert (F == rb.addOverwrite(A, F)); + assert (2 == rb.remaining()); + + assert (F == rb.addOverwrite(B, F)); + assert (1 == rb.remaining()); + + assert (F == rb.addOverwrite(C, F)); + assert (0 == rb.remaining()); + assert (rb.isFull()); + + // now full, should return first value + assert (A == rb.addOverwrite(D, F)); + assert (B == rb.addOverwrite(E, F)); + assert (rb.isFull()); + + // offer() testing + assert (false == rb.offer(F)); + assert (C == rb.remove()); + assert (true == rb.offer(F)); + + // peek testing + assert (D == rb.front()); + assert (E == rb.front(1)); + assert (F == rb.front(2)); + // this should throw + assertThrows(NoSuchElementException.class, + () -> rb.front(99)); + + assert (F == rb.peekBack(A)); + + // clear() testing + rb.clear(); + assert (rb.isEmpty()); + + assert (A == rb.peekBack(A)); + + } } diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/FloatRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/FloatRingBufferTest.java index cc6f13d78f0..c3435fa7792 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/FloatRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/FloatRingBufferTest.java @@ -9,9 +9,13 @@ package io.deephaven.base.ringbuffer; import junit.framework.TestCase; +import org.junit.Assert; +import org.junit.Test; import java.util.NoSuchElementException; +import static org.junit.Assert.assertThrows; + public class FloatRingBufferTest extends TestCase { final float SENTINEL = Float.MIN_VALUE; @@ -78,7 +82,7 @@ private void assertRemove(FloatRingBuffer rb, int expectedSize, float expectedHe } } - private void assertContents(FloatRingBuffer rb, float[] expectedData) { + private void assertContents(FloatRingBuffer rb, float... expectedData) { final float[] data = rb.getAll(); assertEquals(data.length, expectedData.length); for (int ii = 0; ii < data.length; ii++) { @@ -102,31 +106,32 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertContents(rb, new float[] {A, B, C}); + assertContents(rb, A, B, C); assertFull(rb); assertRemove(rb, 3, A); - assertContents(rb, new float[] {B, C}); + assertContents(rb, B, C); assertRemove(rb, 2, B); - assertContents(rb, new float[] {C}); + assertContents(rb, C); assertRemove(rb, 1, C); assertContents(rb, new float[0]); + assertEmpty(rb); assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); - assertContents(rb, new float[] {A, B}); + assertContents(rb, A, B); assertRemove(rb, 2, A); - assertContents(rb, new float[] {B}); + assertContents(rb, B); assertAdd(rb, C, 2, B); - assertContents(rb, new float[] {B, C}); + assertContents(rb, B, C); assertRemove(rb, 2, B); - assertContents(rb, new float[] {C}); + assertContents(rb, C); assertRemove(rb, 1, C); assertContents(rb, new float[0]); @@ -156,13 +161,13 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertContents(rb, new float[] {A, B, C}); + assertContents(rb, A, B, C); assertFull(rb); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); assertAdd(rb, F, 6, A); - assertContents(rb, new float[] {A, B, C, D, E, F}); + assertContents(rb, A, B, C, D, E, F); assertRemove(rb, 6, A); assertRemove(rb, 5, B); @@ -340,6 +345,11 @@ public void testIterator() { assertTrue(iter.hasNext()); assertEquals(F, iter.next()); assertFalse(iter.hasNext()); + + final FloatRingBuffer.Iterator iterFinal = rb.iterator(); + + assertThrows(UnsupportedOperationException.class, + () -> iterFinal.remove()); } public void testBack() { @@ -396,4 +406,57 @@ public void testLargeAmounts() { assertEquals((float) (i - 100), rb.poll(SENTINEL)); } } + + public void testAddExceptionWhenFull() { + FloatRingBuffer rb = new FloatRingBuffer(3, false); + assert (rb.add(A)); + assert (rb.add(B)); + assert (rb.add(C)); + + // this should throw + assertThrows(UnsupportedOperationException.class, + () -> rb.add(D)); + } + + public void testAddOverwriteAndOffer() { + FloatRingBuffer rb = new FloatRingBuffer(3, false); + assert (3 == rb.remaining()); + + assert (F == rb.addOverwrite(A, F)); + assert (2 == rb.remaining()); + + assert (F == rb.addOverwrite(B, F)); + assert (1 == rb.remaining()); + + assert (F == rb.addOverwrite(C, F)); + assert (0 == rb.remaining()); + assert (rb.isFull()); + + // now full, should return first value + assert (A == rb.addOverwrite(D, F)); + assert (B == rb.addOverwrite(E, F)); + assert (rb.isFull()); + + // offer() testing + assert (false == rb.offer(F)); + assert (C == rb.remove()); + assert (true == rb.offer(F)); + + // peek testing + assert (D == rb.front()); + assert (E == rb.front(1)); + assert (F == rb.front(2)); + // this should throw + assertThrows(NoSuchElementException.class, + () -> rb.front(99)); + + assert (F == rb.peekBack(A)); + + // clear() testing + rb.clear(); + assert (rb.isEmpty()); + + assert (A == rb.peekBack(A)); + + } } diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/IntRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/IntRingBufferTest.java index f0239fecdc8..eed54c9d3a7 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/IntRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/IntRingBufferTest.java @@ -9,9 +9,13 @@ package io.deephaven.base.ringbuffer; import junit.framework.TestCase; +import org.junit.Assert; +import org.junit.Test; import java.util.NoSuchElementException; +import static org.junit.Assert.assertThrows; + public class IntRingBufferTest extends TestCase { final int SENTINEL = Integer.MIN_VALUE; @@ -78,7 +82,7 @@ private void assertRemove(IntRingBuffer rb, int expectedSize, int expectedHead) } } - private void assertContents(IntRingBuffer rb, int[] expectedData) { + private void assertContents(IntRingBuffer rb, int... expectedData) { final int[] data = rb.getAll(); assertEquals(data.length, expectedData.length); for (int ii = 0; ii < data.length; ii++) { @@ -102,31 +106,32 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertContents(rb, new int[] {A, B, C}); + assertContents(rb, A, B, C); assertFull(rb); assertRemove(rb, 3, A); - assertContents(rb, new int[] {B, C}); + assertContents(rb, B, C); assertRemove(rb, 2, B); - assertContents(rb, new int[] {C}); + assertContents(rb, C); assertRemove(rb, 1, C); assertContents(rb, new int[0]); + assertEmpty(rb); assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); - assertContents(rb, new int[] {A, B}); + assertContents(rb, A, B); assertRemove(rb, 2, A); - assertContents(rb, new int[] {B}); + assertContents(rb, B); assertAdd(rb, C, 2, B); - assertContents(rb, new int[] {B, C}); + assertContents(rb, B, C); assertRemove(rb, 2, B); - assertContents(rb, new int[] {C}); + assertContents(rb, C); assertRemove(rb, 1, C); assertContents(rb, new int[0]); @@ -156,13 +161,13 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertContents(rb, new int[] {A, B, C}); + assertContents(rb, A, B, C); assertFull(rb); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); assertAdd(rb, F, 6, A); - assertContents(rb, new int[] {A, B, C, D, E, F}); + assertContents(rb, A, B, C, D, E, F); assertRemove(rb, 6, A); assertRemove(rb, 5, B); @@ -340,6 +345,11 @@ public void testIterator() { assertTrue(iter.hasNext()); assertEquals(F, iter.next()); assertFalse(iter.hasNext()); + + final IntRingBuffer.Iterator iterFinal = rb.iterator(); + + assertThrows(UnsupportedOperationException.class, + () -> iterFinal.remove()); } public void testBack() { @@ -396,4 +406,57 @@ public void testLargeAmounts() { assertEquals((int) (i - 100), rb.poll(SENTINEL)); } } + + public void testAddExceptionWhenFull() { + IntRingBuffer rb = new IntRingBuffer(3, false); + assert (rb.add(A)); + assert (rb.add(B)); + assert (rb.add(C)); + + // this should throw + assertThrows(UnsupportedOperationException.class, + () -> rb.add(D)); + } + + public void testAddOverwriteAndOffer() { + IntRingBuffer rb = new IntRingBuffer(3, false); + assert (3 == rb.remaining()); + + assert (F == rb.addOverwrite(A, F)); + assert (2 == rb.remaining()); + + assert (F == rb.addOverwrite(B, F)); + assert (1 == rb.remaining()); + + assert (F == rb.addOverwrite(C, F)); + assert (0 == rb.remaining()); + assert (rb.isFull()); + + // now full, should return first value + assert (A == rb.addOverwrite(D, F)); + assert (B == rb.addOverwrite(E, F)); + assert (rb.isFull()); + + // offer() testing + assert (false == rb.offer(F)); + assert (C == rb.remove()); + assert (true == rb.offer(F)); + + // peek testing + assert (D == rb.front()); + assert (E == rb.front(1)); + assert (F == rb.front(2)); + // this should throw + assertThrows(NoSuchElementException.class, + () -> rb.front(99)); + + assert (F == rb.peekBack(A)); + + // clear() testing + rb.clear(); + assert (rb.isEmpty()); + + assert (A == rb.peekBack(A)); + + } } diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/LongRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/LongRingBufferTest.java index 72e79963ce6..cc4fd085f38 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/LongRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/LongRingBufferTest.java @@ -9,9 +9,13 @@ package io.deephaven.base.ringbuffer; import junit.framework.TestCase; +import org.junit.Assert; +import org.junit.Test; import java.util.NoSuchElementException; +import static org.junit.Assert.assertThrows; + public class LongRingBufferTest extends TestCase { final long SENTINEL = Long.MIN_VALUE; @@ -78,7 +82,7 @@ private void assertRemove(LongRingBuffer rb, int expectedSize, long expectedHead } } - private void assertContents(LongRingBuffer rb, long[] expectedData) { + private void assertContents(LongRingBuffer rb, long... expectedData) { final long[] data = rb.getAll(); assertEquals(data.length, expectedData.length); for (int ii = 0; ii < data.length; ii++) { @@ -102,31 +106,32 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertContents(rb, new long[] {A, B, C}); + assertContents(rb, A, B, C); assertFull(rb); assertRemove(rb, 3, A); - assertContents(rb, new long[] {B, C}); + assertContents(rb, B, C); assertRemove(rb, 2, B); - assertContents(rb, new long[] {C}); + assertContents(rb, C); assertRemove(rb, 1, C); assertContents(rb, new long[0]); + assertEmpty(rb); assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); - assertContents(rb, new long[] {A, B}); + assertContents(rb, A, B); assertRemove(rb, 2, A); - assertContents(rb, new long[] {B}); + assertContents(rb, B); assertAdd(rb, C, 2, B); - assertContents(rb, new long[] {B, C}); + assertContents(rb, B, C); assertRemove(rb, 2, B); - assertContents(rb, new long[] {C}); + assertContents(rb, C); assertRemove(rb, 1, C); assertContents(rb, new long[0]); @@ -156,13 +161,13 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertContents(rb, new long[] {A, B, C}); + assertContents(rb, A, B, C); assertFull(rb); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); assertAdd(rb, F, 6, A); - assertContents(rb, new long[] {A, B, C, D, E, F}); + assertContents(rb, A, B, C, D, E, F); assertRemove(rb, 6, A); assertRemove(rb, 5, B); @@ -340,6 +345,11 @@ public void testIterator() { assertTrue(iter.hasNext()); assertEquals(F, iter.next()); assertFalse(iter.hasNext()); + + final LongRingBuffer.Iterator iterFinal = rb.iterator(); + + assertThrows(UnsupportedOperationException.class, + () -> iterFinal.remove()); } public void testBack() { @@ -396,4 +406,57 @@ public void testLargeAmounts() { assertEquals((long) (i - 100), rb.poll(SENTINEL)); } } + + public void testAddExceptionWhenFull() { + LongRingBuffer rb = new LongRingBuffer(3, false); + assert (rb.add(A)); + assert (rb.add(B)); + assert (rb.add(C)); + + // this should throw + assertThrows(UnsupportedOperationException.class, + () -> rb.add(D)); + } + + public void testAddOverwriteAndOffer() { + LongRingBuffer rb = new LongRingBuffer(3, false); + assert (3 == rb.remaining()); + + assert (F == rb.addOverwrite(A, F)); + assert (2 == rb.remaining()); + + assert (F == rb.addOverwrite(B, F)); + assert (1 == rb.remaining()); + + assert (F == rb.addOverwrite(C, F)); + assert (0 == rb.remaining()); + assert (rb.isFull()); + + // now full, should return first value + assert (A == rb.addOverwrite(D, F)); + assert (B == rb.addOverwrite(E, F)); + assert (rb.isFull()); + + // offer() testing + assert (false == rb.offer(F)); + assert (C == rb.remove()); + assert (true == rb.offer(F)); + + // peek testing + assert (D == rb.front()); + assert (E == rb.front(1)); + assert (F == rb.front(2)); + // this should throw + assertThrows(NoSuchElementException.class, + () -> rb.front(99)); + + assert (F == rb.peekBack(A)); + + // clear() testing + rb.clear(); + assert (rb.isEmpty()); + + assert (A == rb.peekBack(A)); + + } } diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/ShortRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/ShortRingBufferTest.java index 9d1222fa7e0..ff12c564d71 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/ShortRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/ShortRingBufferTest.java @@ -9,9 +9,13 @@ package io.deephaven.base.ringbuffer; import junit.framework.TestCase; +import org.junit.Assert; +import org.junit.Test; import java.util.NoSuchElementException; +import static org.junit.Assert.assertThrows; + public class ShortRingBufferTest extends TestCase { final short SENTINEL = Short.MIN_VALUE; @@ -78,7 +82,7 @@ private void assertRemove(ShortRingBuffer rb, int expectedSize, short expectedHe } } - private void assertContents(ShortRingBuffer rb, short[] expectedData) { + private void assertContents(ShortRingBuffer rb, short... expectedData) { final short[] data = rb.getAll(); assertEquals(data.length, expectedData.length); for (int ii = 0; ii < data.length; ii++) { @@ -102,31 +106,32 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertContents(rb, new short[] {A, B, C}); + assertContents(rb, A, B, C); assertFull(rb); assertRemove(rb, 3, A); - assertContents(rb, new short[] {B, C}); + assertContents(rb, B, C); assertRemove(rb, 2, B); - assertContents(rb, new short[] {C}); + assertContents(rb, C); assertRemove(rb, 1, C); assertContents(rb, new short[0]); + assertEmpty(rb); assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); - assertContents(rb, new short[] {A, B}); + assertContents(rb, A, B); assertRemove(rb, 2, A); - assertContents(rb, new short[] {B}); + assertContents(rb, B); assertAdd(rb, C, 2, B); - assertContents(rb, new short[] {B, C}); + assertContents(rb, B, C); assertRemove(rb, 2, B); - assertContents(rb, new short[] {C}); + assertContents(rb, C); assertRemove(rb, 1, C); assertContents(rb, new short[0]); @@ -156,13 +161,13 @@ public void testAddRemove() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertContents(rb, new short[] {A, B, C}); + assertContents(rb, A, B, C); assertFull(rb); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); assertAdd(rb, F, 6, A); - assertContents(rb, new short[] {A, B, C, D, E, F}); + assertContents(rb, A, B, C, D, E, F); assertRemove(rb, 6, A); assertRemove(rb, 5, B); @@ -340,6 +345,11 @@ public void testIterator() { assertTrue(iter.hasNext()); assertEquals(F, iter.next()); assertFalse(iter.hasNext()); + + final ShortRingBuffer.Iterator iterFinal = rb.iterator(); + + assertThrows(UnsupportedOperationException.class, + () -> iterFinal.remove()); } public void testBack() { @@ -396,4 +406,57 @@ public void testLargeAmounts() { assertEquals((short) (i - 100), rb.poll(SENTINEL)); } } + + public void testAddExceptionWhenFull() { + ShortRingBuffer rb = new ShortRingBuffer(3, false); + assert (rb.add(A)); + assert (rb.add(B)); + assert (rb.add(C)); + + // this should throw + assertThrows(UnsupportedOperationException.class, + () -> rb.add(D)); + } + + public void testAddOverwriteAndOffer() { + ShortRingBuffer rb = new ShortRingBuffer(3, false); + assert (3 == rb.remaining()); + + assert (F == rb.addOverwrite(A, F)); + assert (2 == rb.remaining()); + + assert (F == rb.addOverwrite(B, F)); + assert (1 == rb.remaining()); + + assert (F == rb.addOverwrite(C, F)); + assert (0 == rb.remaining()); + assert (rb.isFull()); + + // now full, should return first value + assert (A == rb.addOverwrite(D, F)); + assert (B == rb.addOverwrite(E, F)); + assert (rb.isFull()); + + // offer() testing + assert (false == rb.offer(F)); + assert (C == rb.remove()); + assert (true == rb.offer(F)); + + // peek testing + assert (D == rb.front()); + assert (E == rb.front(1)); + assert (F == rb.front(2)); + // this should throw + assertThrows(NoSuchElementException.class, + () -> rb.front(99)); + + assert (F == rb.peekBack(A)); + + // clear() testing + rb.clear(); + assert (rb.isEmpty()); + + assert (A == rb.peekBack(A)); + + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/SortListener.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/SortListener.java index 2b0ae5c3ec1..026284582ed 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/SortListener.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/SortListener.java @@ -700,15 +700,17 @@ private static class EffortTracker { } public void add(long numWrites, long numRequestedAdds) { - if (writes.size() == writes.capacity()) { + if (writes.isFull()) { totalNumWrites -= writes.remove(); totalNumRequestedAdds -= requestedAdds.remove(); } totalNumWrites += numWrites; totalNumRequestedAdds += numRequestedAdds; - writes.add(numWrites); - requestedAdds.add(numRequestedAdds); + + // assert that the buffers are not overwritten + Assert.eqTrue(writes.offer(numWrites), "writes.offer(numWrites)"); + Assert.eqTrue(requestedAdds.add(numRequestedAdds), "requestedAdds.add(numRequestedAdds)"); } String summarize() { From e1d96cccce8f67cb5e17c5989da898acce786899 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Mon, 19 Dec 2022 13:58:42 -0800 Subject: [PATCH 059/123] changed dense ArraySources per PR comments --- .../table/impl/sources/ArraySourceHelper.java | 19 +++++------- .../table/impl/sources/ByteArraySource.java | 15 +++------ .../impl/sources/CharacterArraySource.java | 17 +++------- .../table/impl/sources/DoubleArraySource.java | 31 +++++++------------ .../table/impl/sources/FloatArraySource.java | 15 +++------ .../impl/sources/IntegerArraySource.java | 15 +++------ .../table/impl/sources/ShortArraySource.java | 15 +++------ .../barrage/table/BarrageRedirectedTable.java | 8 ++--- 8 files changed, 45 insertions(+), 90 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ArraySourceHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ArraySourceHelper.java index 4a301ffe20d..2593843eb9a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ArraySourceHelper.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ArraySourceHelper.java @@ -3,28 +3,25 @@ */ package io.deephaven.engine.table.impl.sources; +import gnu.trove.list.array.TIntArrayList; import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.ChunkType; +import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.table.SharedContext; +import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.SharedContext; import io.deephaven.engine.table.WritableSourceWithPrepareForParallelPopulation; -import io.deephaven.engine.updategraph.LogicalClock; -import io.deephaven.util.datastructures.LongSizedDataStructure; -import io.deephaven.chunk.*; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.updategraph.UpdateCommitter; import io.deephaven.engine.table.impl.util.copy.CopyKernel; +import io.deephaven.engine.updategraph.UpdateCommitter; import io.deephaven.util.SoftRecycler; -import gnu.trove.list.array.TIntArrayList; +import io.deephaven.util.datastructures.LongSizedDataStructure; import org.apache.commons.lang3.mutable.MutableInt; import org.jetbrains.annotations.NotNull; import java.util.Arrays; -import static io.deephaven.engine.table.impl.sources.sparse.SparseConstants.*; -import static io.deephaven.engine.table.impl.sources.sparse.SparseConstants.IN_USE_MASK; - abstract class ArraySourceHelper extends ArrayBackedColumnSource implements WritableSourceWithPrepareForParallelPopulation { /** diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ByteArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ByteArraySource.java index e06a3eec7f3..e92f8197f44 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ByteArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ByteArraySource.java @@ -10,28 +10,21 @@ import gnu.trove.list.array.TIntArrayList; import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.impl.MutableColumnSourceGetDefaults; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeyRanges; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.table.impl.MutableColumnSourceGetDefaults; import io.deephaven.engine.updategraph.LogicalClock; -import io.deephaven.util.compare.ByteComparisons; -import io.deephaven.chunk.*; -import io.deephaven.chunk.ResettableWritableChunk; -import io.deephaven.chunk.WritableByteChunk; -import io.deephaven.chunk.WritableChunk; -import io.deephaven.chunk.LongChunk; -import io.deephaven.engine.rowset.RowSequence; import io.deephaven.util.SoftRecycler; +import io.deephaven.util.compare.ByteComparisons; import org.jetbrains.annotations.NotNull; import java.util.Arrays; -import static io.deephaven.engine.table.impl.sources.sparse.SparseConstants.*; -import static io.deephaven.engine.table.impl.sources.sparse.SparseConstants.BLOCK2_MASK; import static io.deephaven.util.QueryConstants.NULL_BYTE; import static io.deephaven.util.type.TypeUtils.box; import static io.deephaven.util.type.TypeUtils.unbox; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/CharacterArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/CharacterArraySource.java index fe84494e539..14faedfbac4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/CharacterArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/CharacterArraySource.java @@ -5,28 +5,21 @@ import gnu.trove.list.array.TIntArrayList; import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.impl.MutableColumnSourceGetDefaults; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeyRanges; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.table.impl.MutableColumnSourceGetDefaults; import io.deephaven.engine.updategraph.LogicalClock; -import io.deephaven.util.compare.CharComparisons; -import io.deephaven.chunk.*; -import io.deephaven.chunk.ResettableWritableChunk; -import io.deephaven.chunk.WritableCharChunk; -import io.deephaven.chunk.WritableChunk; -import io.deephaven.chunk.LongChunk; -import io.deephaven.engine.rowset.RowSequence; import io.deephaven.util.SoftRecycler; +import io.deephaven.util.compare.CharComparisons; import org.jetbrains.annotations.NotNull; import java.util.Arrays; -import static io.deephaven.engine.table.impl.sources.sparse.SparseConstants.*; -import static io.deephaven.engine.table.impl.sources.sparse.SparseConstants.BLOCK2_MASK; import static io.deephaven.util.QueryConstants.NULL_CHAR; import static io.deephaven.util.type.TypeUtils.box; import static io.deephaven.util.type.TypeUtils.unbox; @@ -115,7 +108,7 @@ public void prepareForParallelPopulation(RowSet changedIndices) { it.getNextRowSequenceThrough(maxKeyInCurrentBlock).forAllRowKeys(key -> { final int nextIndexWithinBlock = (int) (key & INDEX_MASK); final int nextIndexWithinInUse = nextIndexWithinBlock >> LOG_INUSE_BITSET_SIZE; - final long nextMaskWithinInUse = 1L << (nextIndexWithinBlock & IN_USE_MASK); + final long nextMaskWithinInUse = 1L << nextIndexWithinBlock; prevBlocks[block][nextIndexWithinBlock] = blocks[block][nextIndexWithinBlock]; inUse[nextIndexWithinInUse] |= nextMaskWithinInUse; }); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/DoubleArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/DoubleArraySource.java index 014776ea032..8700bf1ffc1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/DoubleArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/DoubleArraySource.java @@ -10,28 +10,21 @@ import gnu.trove.list.array.TIntArrayList; import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.impl.MutableColumnSourceGetDefaults; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeyRanges; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.table.impl.MutableColumnSourceGetDefaults; import io.deephaven.engine.updategraph.LogicalClock; -import io.deephaven.util.compare.DoubleComparisons; -import io.deephaven.chunk.*; -import io.deephaven.chunk.ResettableWritableChunk; -import io.deephaven.chunk.WritableDoubleChunk; -import io.deephaven.chunk.WritableChunk; -import io.deephaven.chunk.LongChunk; -import io.deephaven.engine.rowset.RowSequence; import io.deephaven.util.SoftRecycler; +import io.deephaven.util.compare.DoubleComparisons; import org.jetbrains.annotations.NotNull; import java.util.Arrays; -import static io.deephaven.engine.table.impl.sources.sparse.SparseConstants.*; -import static io.deephaven.engine.table.impl.sources.sparse.SparseConstants.BLOCK2_MASK; import static io.deephaven.util.QueryConstants.NULL_DOUBLE; import static io.deephaven.util.type.TypeUtils.box; import static io.deephaven.util.type.TypeUtils.unbox; @@ -70,25 +63,25 @@ public void ensureCapacity(long capacity, boolean nullFill) { /** * This version of `prepareForParallelPopulation` will internally call {@link #ensureCapacity(long, boolean)} to - * make sure there is room for the incoming values. + * create room for incoming values. * - * @param changedIndices indices in the dense table + * @param changedRows indices in the dense table */ @Override - public void prepareForParallelPopulation(RowSet changedIndices) { + public void prepareForParallelPopulation(RowSet changedRows) { final long currentStep = LogicalClock.DEFAULT.currentStep(); if (ensurePreviousClockCycle == currentStep) { throw new IllegalStateException("May not call ensurePrevious twice on one clock cycle!"); } ensurePreviousClockCycle = currentStep; - if (changedIndices.isEmpty()) { + if (changedRows.isEmpty()) { return; } - // ensure that this source will have sufficient capacity to store these indices, does not need to be - // null-filled as the values will be immediately written - ensureCapacity(changedIndices.lastRowKey() + 1, false); + // ensure that this source will have sufficient capacity to store these rows, does not need to be + // null-filled as the values will be immediately overwritten + ensureCapacity(changedRows.lastRowKey() + 1, false); if (prevFlusher != null) { prevFlusher.maybeActivate(); @@ -97,7 +90,7 @@ public void prepareForParallelPopulation(RowSet changedIndices) { return; } - try (final RowSequence.Iterator it = changedIndices.getRowSequenceIterator()) { + try (final RowSequence.Iterator it = changedRows.getRowSequenceIterator()) { do { final long firstKey = it.peekNextKey(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/FloatArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/FloatArraySource.java index 57c1cd1c9b4..b9f3b5e019d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/FloatArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/FloatArraySource.java @@ -10,28 +10,21 @@ import gnu.trove.list.array.TIntArrayList; import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.impl.MutableColumnSourceGetDefaults; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeyRanges; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.table.impl.MutableColumnSourceGetDefaults; import io.deephaven.engine.updategraph.LogicalClock; -import io.deephaven.util.compare.FloatComparisons; -import io.deephaven.chunk.*; -import io.deephaven.chunk.ResettableWritableChunk; -import io.deephaven.chunk.WritableFloatChunk; -import io.deephaven.chunk.WritableChunk; -import io.deephaven.chunk.LongChunk; -import io.deephaven.engine.rowset.RowSequence; import io.deephaven.util.SoftRecycler; +import io.deephaven.util.compare.FloatComparisons; import org.jetbrains.annotations.NotNull; import java.util.Arrays; -import static io.deephaven.engine.table.impl.sources.sparse.SparseConstants.*; -import static io.deephaven.engine.table.impl.sources.sparse.SparseConstants.BLOCK2_MASK; import static io.deephaven.util.QueryConstants.NULL_FLOAT; import static io.deephaven.util.type.TypeUtils.box; import static io.deephaven.util.type.TypeUtils.unbox; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/IntegerArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/IntegerArraySource.java index f1cae208688..0c09009e5c7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/IntegerArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/IntegerArraySource.java @@ -10,28 +10,21 @@ import gnu.trove.list.array.TIntArrayList; import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.impl.MutableColumnSourceGetDefaults; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeyRanges; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.table.impl.MutableColumnSourceGetDefaults; import io.deephaven.engine.updategraph.LogicalClock; -import io.deephaven.util.compare.IntComparisons; -import io.deephaven.chunk.*; -import io.deephaven.chunk.ResettableWritableChunk; -import io.deephaven.chunk.WritableIntChunk; -import io.deephaven.chunk.WritableChunk; -import io.deephaven.chunk.LongChunk; -import io.deephaven.engine.rowset.RowSequence; import io.deephaven.util.SoftRecycler; +import io.deephaven.util.compare.IntComparisons; import org.jetbrains.annotations.NotNull; import java.util.Arrays; -import static io.deephaven.engine.table.impl.sources.sparse.SparseConstants.*; -import static io.deephaven.engine.table.impl.sources.sparse.SparseConstants.BLOCK2_MASK; import static io.deephaven.util.QueryConstants.NULL_INT; import static io.deephaven.util.type.TypeUtils.box; import static io.deephaven.util.type.TypeUtils.unbox; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ShortArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ShortArraySource.java index 84701305262..ee51a3b77f4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ShortArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ShortArraySource.java @@ -10,28 +10,21 @@ import gnu.trove.list.array.TIntArrayList; import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.impl.MutableColumnSourceGetDefaults; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeyRanges; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.table.impl.MutableColumnSourceGetDefaults; import io.deephaven.engine.updategraph.LogicalClock; -import io.deephaven.util.compare.ShortComparisons; -import io.deephaven.chunk.*; -import io.deephaven.chunk.ResettableWritableChunk; -import io.deephaven.chunk.WritableShortChunk; -import io.deephaven.chunk.WritableChunk; -import io.deephaven.chunk.LongChunk; -import io.deephaven.engine.rowset.RowSequence; import io.deephaven.util.SoftRecycler; +import io.deephaven.util.compare.ShortComparisons; import org.jetbrains.annotations.NotNull; import java.util.Arrays; -import static io.deephaven.engine.table.impl.sources.sparse.SparseConstants.*; -import static io.deephaven.engine.table.impl.sources.sparse.SparseConstants.BLOCK2_MASK; import static io.deephaven.util.QueryConstants.NULL_SHORT; import static io.deephaven.util.type.TypeUtils.box; import static io.deephaven.util.type.TypeUtils.unbox; diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/table/BarrageRedirectedTable.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/table/BarrageRedirectedTable.java index bf4b0e3684d..c68faeeb3ce 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/table/BarrageRedirectedTable.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/table/BarrageRedirectedTable.java @@ -147,14 +147,14 @@ private UpdateCoalescer processUpdate(final BarrageMessage update, final UpdateC // update the table with the rowsIncluded set (in manageable batch sizes) try (final RowSequence.Iterator rowsIncludedIterator = update.rowsIncluded.getRowSequenceIterator(); - final ChunkSink.FillFromContext redirHelper = + final ChunkSink.FillFromContext redirContext = rowRedirection.makeFillFromContext(addBatchSize)) { while (rowsIncludedIterator.hasMore()) { final RowSequence rowsToRedirect = rowsIncludedIterator.getNextRowSequenceWithLength(addBatchSize); try (final RowSet newRows = getFreeRows(rowsToRedirect.intSize())) { // Update redirection mapping: - rowRedirection.fillFromChunk(redirHelper, newRows.asRowKeyChunk(), rowsToRedirect); + rowRedirection.fillFromChunk(redirContext, newRows.asRowKeyChunk(), rowsToRedirect); // add these rows to the final destination set destinationRowSet.insert(newRows); } @@ -200,7 +200,7 @@ private UpdateCoalescer processUpdate(final BarrageMessage update, final UpdateC final int modBatchSize = (int) Math.min(column.rowsModified.size(), BATCH_SIZE); modifiedColumnSet.setColumnWithIndex(ii); - try (final ChunkSource.FillContext redirHelper = rowRedirection.makeFillContext(modBatchSize, null); + try (final ChunkSource.FillContext redirContext = rowRedirection.makeFillContext(modBatchSize, null); final ChunkSink.FillFromContext fillContext = destSources[ii].makeFillFromContext(modBatchSize); final WritableLongChunk keys = WritableLongChunk.makeWritableChunk(modBatchSize); final RowSequence.Iterator destIterator = column.rowsModified.getRowSequenceIterator()) { @@ -214,7 +214,7 @@ private UpdateCoalescer processUpdate(final BarrageMessage update, final UpdateC int effectiveBatchSize = Math.min(modBatchSize, chunk.size() - chunkOffset); final RowSequence chunkKeys = destIterator.getNextRowSequenceWithLength(effectiveBatchSize); // fill the key chunk with the keys from this rowset - rowRedirection.fillChunk(redirHelper, keys, chunkKeys); + rowRedirection.fillChunk(redirContext, keys, chunkKeys); Chunk slicedChunk = chunk.slice(chunkOffset, effectiveBatchSize); destSources[ii].fillFromChunkUnordered(fillContext, slicedChunk, keys); From 72688285e5f454139f20303155cd32ecd17171c1 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Thu, 29 Dec 2022 12:35:58 -0800 Subject: [PATCH 060/123] Renamed 'TimeScale' to 'WindowScale' to better represent the Ticks/Time spectrum --- .../java/io/deephaven/engine/table/Table.java | 1 + .../client/impl/UpdateByBuilder.java | 10 ++--- .../server/table/ops/UpdateByGrpcImpl.java | 15 +++---- .../api/updateby/UpdateByOperation.java | 16 ++++---- .../deephaven/api/updateby/spec/EmaSpec.java | 22 +++++----- .../api/updateby/spec/RollingSumSpec.java | 40 +++++++++---------- .../spec/{TimeScale.java => WindowScale.java} | 10 ++--- 7 files changed, 55 insertions(+), 59 deletions(-) rename table-api/src/main/java/io/deephaven/api/updateby/spec/{TimeScale.java => WindowScale.java} (78%) 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 e0cb72ccc90..acc0d842c49 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 @@ -140,6 +140,7 @@ public interface Table extends *
                *
              1. {@link #groupBy} is unsupported *
              2. {@link #partitionBy} is unsupported
              3. + *
              4. {@link #partitionedAggBy(Collection, boolean, Table, String...) partitionedAggBy} is unsupported
              5. *
              6. {@link #rollup(Collection, boolean, ColumnName...) rollup()} is unsupported if * {@code includeConstituents == true}
              7. *
              8. {@link #treeTable(String, String) treeTable()} is unsupported
              9. diff --git a/java-client/session/src/main/java/io/deephaven/client/impl/UpdateByBuilder.java b/java-client/session/src/main/java/io/deephaven/client/impl/UpdateByBuilder.java index a3816ab21cb..9905ca77f87 100644 --- a/java-client/session/src/main/java/io/deephaven/client/impl/UpdateByBuilder.java +++ b/java-client/session/src/main/java/io/deephaven/client/impl/UpdateByBuilder.java @@ -92,18 +92,18 @@ private static UpdateByEmaOptions adapt(OperationControl control) { return builder.build(); } - private static UpdateByEmaTimescale adapt(TimeScale timeScale) { - if (timeScale.isTimeBased()) { + private static UpdateByEmaTimescale adapt(WindowScale windowScale) { + if (windowScale.isTimeBased()) { return UpdateByEmaTimescale.newBuilder() .setTime(UpdateByEmaTime.newBuilder() - .setColumn(timeScale.timestampCol()) - .setPeriodNanos(timeScale.timescaleUnits()) + .setColumn(windowScale.timestampCol()) + .setPeriodNanos(windowScale.timescaleUnits()) .build()) .build(); } else { return UpdateByEmaTimescale.newBuilder() .setTicks(UpdateByEmaTicks.newBuilder() - .setTicks(timeScale.timescaleUnits()) + .setTicks(windowScale.timescaleUnits()) .build()) .build(); } diff --git a/server/src/main/java/io/deephaven/server/table/ops/UpdateByGrpcImpl.java b/server/src/main/java/io/deephaven/server/table/ops/UpdateByGrpcImpl.java index 6c0a9855342..60af7eab881 100644 --- a/server/src/main/java/io/deephaven/server/table/ops/UpdateByGrpcImpl.java +++ b/server/src/main/java/io/deephaven/server/table/ops/UpdateByGrpcImpl.java @@ -8,13 +8,8 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.api.updateby.UpdateByControl; import io.deephaven.api.updateby.UpdateByOperation; -import io.deephaven.api.updateby.spec.CumMinMaxSpec; -import io.deephaven.api.updateby.spec.CumProdSpec; -import io.deephaven.api.updateby.spec.CumSumSpec; -import io.deephaven.api.updateby.spec.EmaSpec; -import io.deephaven.api.updateby.spec.FillBySpec; -import io.deephaven.api.updateby.spec.TimeScale; -import io.deephaven.api.updateby.spec.UpdateBySpec; +import io.deephaven.api.updateby.spec.*; +import io.deephaven.api.updateby.spec.WindowScale; import io.deephaven.auth.codegen.impl.TableServiceContextualAuthWiring; import io.deephaven.base.verify.Assert; import io.deephaven.engine.table.Table; @@ -227,12 +222,12 @@ private static RoundingMode adaptRoundingMode( } } - private static TimeScale adaptTimescale(UpdateByEma.UpdateByEmaTimescale timescale) { + private static WindowScale adaptTimescale(UpdateByEma.UpdateByEmaTimescale timescale) { switch (timescale.getTypeCase()) { case TICKS: - return TimeScale.ofTicks(timescale.getTicks().getTicks()); + return WindowScale.ofTicks(timescale.getTicks().getTicks()); case TIME: - return TimeScale.ofTime(timescale.getTime().getColumn(), timescale.getTime().getPeriodNanos()); + return WindowScale.ofTime(timescale.getTime().getColumn(), timescale.getTime().getPeriodNanos()); case TYPE_NOT_SET: default: throw new IllegalArgumentException("Unexpected timescale type: " + timescale.getTypeCase()); diff --git a/table-api/src/main/java/io/deephaven/api/updateby/UpdateByOperation.java b/table-api/src/main/java/io/deephaven/api/updateby/UpdateByOperation.java index 500b03188ad..16b664a6321 100644 --- a/table-api/src/main/java/io/deephaven/api/updateby/UpdateByOperation.java +++ b/table-api/src/main/java/io/deephaven/api/updateby/UpdateByOperation.java @@ -34,7 +34,7 @@ static ColumnUpdateOperation of(final UpdateBySpec spec, final Pair... columns) } /** - * Create an {@link CumSumSpec cumulative sum} for the supplied column name pairs. + * Create a {@link CumSumSpec cumulative sum} for the supplied column name pairs. * * @param pairs The input/output column name pairs * @return The aggregation @@ -44,7 +44,7 @@ static UpdateByOperation CumSum(String... pairs) { } /** - * Create an {@link CumProdSpec cumulative produce} for the supplied column name pairs. + * Create a {@link CumProdSpec cumulative product} for the supplied column name pairs. * * @param pairs The input/output column name pairs * @return The aggregation @@ -54,7 +54,7 @@ static UpdateByOperation CumProd(String... pairs) { } /** - * Create an {@link CumMinMaxSpec cumulative minimum} for the supplied column name pairs. + * Create a {@link CumMinMaxSpec cumulative minimum} for the supplied column name pairs. * * @param pairs The input/output column name pairs * @return The aggregation @@ -64,7 +64,7 @@ static UpdateByOperation CumMin(String... pairs) { } /** - * Create an {@link CumMinMaxSpec cumulative maximum} for the supplied column name pairs. + * Create a {@link CumMinMaxSpec cumulative maximum} for the supplied column name pairs. * * @param pairs The input/output column name pairs * @return The aggregation @@ -74,7 +74,7 @@ static UpdateByOperation CumMax(String... pairs) { } /** - * Create an {@link FillBySpec fill by} for the supplied column name pairs. + * Create a {@link FillBySpec forward fill} for the supplied column name pairs. * * @param pairs The input/output column name pairs * @return The aggregation @@ -216,7 +216,7 @@ static UpdateByOperation Ema(OperationControl control, String timestampColumn, D } /** - * Create an {@link RollingSumSpec rolling sum} for the supplied column name pairs, using ticks as the windowing + * Create a {@link RollingSumSpec rolling sum} for the supplied column name pairs, using ticks as the windowing * unit. Uses the default OperationControl settings. * * @param prevTimeTicks the look-behind window size (in rows/ticks) @@ -228,7 +228,7 @@ static UpdateByOperation RollingSum(long prevTimeTicks, String... pairs) { } /** - * Create an {@link RollingSumSpec rolling sum} for the supplied column name pairs, using ticks as the windowing + * Create a {@link RollingSumSpec rolling sum} for the supplied column name pairs, using ticks as the windowing * unit. Uses the default OperationControl settings. * * @param prevTimeTicks the look-behind window size (in rows/ticks) @@ -241,7 +241,7 @@ static UpdateByOperation RollingSum(long prevTimeTicks, long fwdTimeTicks, Strin } /** - * Create an {@link RollingSumSpec rolling sum} for the supplied column name pairs, using time as the windowing + * Create a {@link RollingSumSpec rolling sum} for the supplied column name pairs, using time as the windowing * unit. Uses the default OperationControl settings. * * @param prevWindowDuration the look-behind window size (in Duration) diff --git a/table-api/src/main/java/io/deephaven/api/updateby/spec/EmaSpec.java b/table-api/src/main/java/io/deephaven/api/updateby/spec/EmaSpec.java index 536f91085c4..c58ac2a764b 100644 --- a/table-api/src/main/java/io/deephaven/api/updateby/spec/EmaSpec.java +++ b/table-api/src/main/java/io/deephaven/api/updateby/spec/EmaSpec.java @@ -14,45 +14,45 @@ @BuildableStyle public abstract class EmaSpec extends UpdateBySpecBase { - public static EmaSpec of(OperationControl control, TimeScale timeScale) { - return ImmutableEmaSpec.builder().control(control).timeScale(timeScale).build(); + public static EmaSpec of(OperationControl control, WindowScale windowScale) { + return ImmutableEmaSpec.builder().control(control).timeScale(windowScale).build(); } - public static EmaSpec of(TimeScale timeScale) { - return ImmutableEmaSpec.builder().timeScale(timeScale).build(); + public static EmaSpec of(WindowScale windowScale) { + return ImmutableEmaSpec.builder().timeScale(windowScale).build(); } public static EmaSpec ofTime(final OperationControl control, final String timestampCol, long timeScaleNanos) { - return of(control, TimeScale.ofTime(timestampCol, timeScaleNanos)); + return of(control, WindowScale.ofTime(timestampCol, timeScaleNanos)); } public static EmaSpec ofTime(final String timestampCol, long timeScaleNanos) { - return of(TimeScale.ofTime(timestampCol, timeScaleNanos)); + return of(WindowScale.ofTime(timestampCol, timeScaleNanos)); } public static EmaSpec ofTime(final OperationControl control, final String timestampCol, Duration emaDuration) { - return of(control, TimeScale.ofTime(timestampCol, emaDuration)); + return of(control, WindowScale.ofTime(timestampCol, emaDuration)); } public static EmaSpec ofTime(final String timestampCol, Duration emaDuration) { - return of(TimeScale.ofTime(timestampCol, emaDuration)); + return of(WindowScale.ofTime(timestampCol, emaDuration)); } public static EmaSpec ofTicks(OperationControl control, long tickWindow) { - return of(control, TimeScale.ofTicks(tickWindow)); + return of(control, WindowScale.ofTicks(tickWindow)); } public static EmaSpec ofTicks(long tickWindow) { - return of(TimeScale.ofTicks(tickWindow)); + return of(WindowScale.ofTicks(tickWindow)); } public abstract Optional control(); - public abstract TimeScale timeScale(); + public abstract WindowScale timeScale(); public final OperationControl controlOrDefault() { return control().orElseGet(OperationControl::defaultInstance); diff --git a/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java b/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java index a641d24e048..eb242f247a1 100644 --- a/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java +++ b/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java @@ -18,38 +18,38 @@ public abstract class RollingSumSpec extends UpdateBySpecBase { // most common usages first, will complete the list later public static RollingSumSpec ofTicks(long tickWindow) { - return of(TimeScale.ofTicks(tickWindow)); + return of(WindowScale.ofTicks(tickWindow)); } public static RollingSumSpec ofTicks(long prevTickWindow, long fwdTickWindow) { - return of(TimeScale.ofTicks(prevTickWindow), TimeScale.ofTicks(fwdTickWindow)); + return of(WindowScale.ofTicks(prevTickWindow), WindowScale.ofTicks(fwdTickWindow)); } public static RollingSumSpec ofTime(final String timestampCol, Duration prevWindowDuration) { - return of(TimeScale.ofTime(timestampCol, prevWindowDuration)); + return of(WindowScale.ofTime(timestampCol, prevWindowDuration)); } public static RollingSumSpec ofTime(final String timestampCol, Duration prevWindowDuration, Duration fwdWindowDuration) { - return of(TimeScale.ofTime(timestampCol, prevWindowDuration), - TimeScale.ofTime(timestampCol, fwdWindowDuration)); + return of(WindowScale.ofTime(timestampCol, prevWindowDuration), + WindowScale.ofTime(timestampCol, fwdWindowDuration)); } // general use constructors - public static RollingSumSpec of(TimeScale prevTimeScale) { - return ImmutableRollingSumSpec.builder().prevTimeScale(prevTimeScale).build(); + public static RollingSumSpec of(WindowScale prevWindowScale) { + return ImmutableRollingSumSpec.builder().prevTimeScale(prevWindowScale).build(); } - public static RollingSumSpec of(TimeScale prevTimeScale, TimeScale fwdTimeScale) { - return ImmutableRollingSumSpec.builder().prevTimeScale(prevTimeScale).fwdTimeScale(fwdTimeScale).build(); + public static RollingSumSpec of(WindowScale prevWindowScale, WindowScale fwdWindowScale) { + return ImmutableRollingSumSpec.builder().prevTimeScale(prevWindowScale).fwdTimeScale(fwdWindowScale).build(); } - // public static RollingSumSpec of(TimeScale prevTimeScale) { + // public static RollingSumSpec of(WindowScale prevTimeScale) { // return ImmutableWindowedOpSpec.builder().prevTimeScale(prevTimeScale).build(); // } // - // public static RollingSumSpec of(OperationControl control, TimeScale prevTimeScale, TimeScale fwdTimeScale) { + // public static RollingSumSpec of(OperationControl control, WindowScale prevTimeScale, WindowScale fwdTimeScale) { // return // ImmutableWindowedOpSpec.builder().control(control).prevTimeScale(prevTimeScale).fwdTimeScale(fwdTimeScale).build(); // } @@ -57,21 +57,21 @@ public static RollingSumSpec of(TimeScale prevTimeScale, TimeScale fwdTimeScale) // public static RollingSumSpec ofTime(final OperationControl control, // final String timestampCol, // long prevWindowTimeScaleNanos) { - // return of(control, TimeScale.ofTime(timestampCol, prevWindowTimeScaleNanos)); + // return of(control, WindowScale.ofTime(timestampCol, prevWindowTimeScaleNanos)); // } // // public static RollingSumSpec ofTime(final OperationControl control, // final String timestampCol, // long prevWindowTimeScaleNanos, // long fwdWindowTimeScaleNanos) { - // return of(control, TimeScale.ofTime(timestampCol, prevWindowTimeScaleNanos), TimeScale.ofTime(timestampCol, + // return of(control, WindowScale.ofTime(timestampCol, prevWindowTimeScaleNanos), WindowScale.ofTime(timestampCol, // fwdWindowTimeScaleNanos)); // } // // public static RollingSumSpec ofTime(final OperationControl control, // final String timestampCol, // Duration prevWindowDuration) { - // return of(control, TimeScale.ofTime(timestampCol, prevWindowDuration)); + // return of(control, WindowScale.ofTime(timestampCol, prevWindowDuration)); // } // // @@ -79,27 +79,27 @@ public static RollingSumSpec of(TimeScale prevTimeScale, TimeScale fwdTimeScale) // final String timestampCol, // Duration prevWindowDuration, // Duration fwdWindowDuration) { - // return of(control, TimeScale.ofTime(timestampCol, prevWindowDuration), TimeScale.ofTime(timestampCol, + // return of(control, WindowScale.ofTime(timestampCol, prevWindowDuration), WindowScale.ofTime(timestampCol, // fwdWindowDuration)); // } // // public static RollingSumSpec ofTicks(OperationControl control, long prevTickWindow) { - // return of(control, TimeScale.ofTicks(prevTickWindow)); + // return of(control, WindowScale.ofTicks(prevTickWindow)); // } // // public static RollingSumSpec ofTicks(OperationControl control, long prevTickWindow, long fwdTickWindow) { - // return of(control, TimeScale.ofTicks(prevTickWindow), TimeScale.ofTicks(fwdTickWindow)); + // return of(control, WindowScale.ofTicks(prevTickWindow), WindowScale.ofTicks(fwdTickWindow)); // } public abstract Optional control(); - public abstract TimeScale prevTimeScale(); + public abstract WindowScale prevTimeScale(); // provide a default forward-looking timescale @Value.Default - public TimeScale fwdTimeScale() { - return TimeScale.ofTicks(0); + public WindowScale fwdTimeScale() { + return WindowScale.ofTicks(0); } public final OperationControl controlOrDefault() { diff --git a/table-api/src/main/java/io/deephaven/api/updateby/spec/TimeScale.java b/table-api/src/main/java/io/deephaven/api/updateby/spec/WindowScale.java similarity index 78% rename from table-api/src/main/java/io/deephaven/api/updateby/spec/TimeScale.java rename to table-api/src/main/java/io/deephaven/api/updateby/spec/WindowScale.java index e867c834b5c..b78666bfb77 100644 --- a/table-api/src/main/java/io/deephaven/api/updateby/spec/TimeScale.java +++ b/table-api/src/main/java/io/deephaven/api/updateby/spec/WindowScale.java @@ -10,16 +10,16 @@ @Immutable @SimpleStyle -public abstract class TimeScale { - public static TimeScale ofTime(final String timestampCol, long timeScaleNanos) { +public abstract class WindowScale { + public static WindowScale ofTime(final String timestampCol, long timeScaleNanos) { return ImmutableTimeScale.of(timestampCol, timeScaleNanos); } - public static TimeScale ofTime(final String timestampCol, Duration duration) { + public static WindowScale ofTime(final String timestampCol, Duration duration) { return ImmutableTimeScale.of(timestampCol, duration.toNanos()); } - public static TimeScale ofTicks(long tickWindow) { + public static WindowScale ofTicks(long tickWindow) { return ImmutableTimeScale.of(null, tickWindow); } @@ -51,7 +51,7 @@ public final long getTicks() { @Value.Check final void checkTimestampColEmpty() { if (timestampCol() != null && timestampCol().isEmpty()) { - throw new IllegalArgumentException("TimeScale.timestampCol() must not be an empty string"); + throw new IllegalArgumentException("WindowScale.timestampCol() must not be an empty string"); } } } From 58844ced1361037e025ecf189f8c18e1702984ea Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 30 Dec 2022 12:52:20 -0800 Subject: [PATCH 061/123] Fxed a bug in Fwd Time windows (when timestamp was modified) --- .../engine/table/impl/updateby/UpdateByWindow.java | 3 --- .../table/impl/updateby/UpdateByWindowTime.java | 13 ++++++++++--- .../table/impl/updateby/TestUpdateByGeneral.java | 13 +++++++++++-- .../deephaven/api/updateby/spec/RollingSumSpec.java | 3 ++- .../io/deephaven/api/updateby/spec/WindowScale.java | 6 +++--- 5 files changed, 26 insertions(+), 12 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java index f611dfe3946..0d705b73fe1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java @@ -263,9 +263,6 @@ public void assignInputSources(final UpdateByWindowContext context, final Column */ protected void prepareValuesChunkForSource(final UpdateByWindowContext context, final int srcIdx, final RowSequence rs) { - if (rs.isEmpty()) { - return; - } if (!context.inputSourceChunkPopulated[srcIdx]) { context.inputSourceChunks[srcIdx] = context.inputSources[srcIdx].getChunk(context.inputSourceGetContexts[srcIdx], rs); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java index 65d925f3d05..00a8b765312 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java @@ -154,7 +154,7 @@ private static WritableRowSet computeInfluencerRowsTime(final RowSet sourceSet, Assert.eqTrue(ssaIt.hasNext() && ssaIt.nextValue() >= head, "SSA Iterator outside of window"); - // step through the SSA and collect keys until outside of the window + // step through the SSA and collect keys until outside the window while (ssaIt.hasNext() && ssaIt.nextValue() <= tail) { builder.appendKey(ssaIt.nextKey()); ssaIt.next(); @@ -285,16 +285,23 @@ public void computeAffectedRowsAndOperators(UpdateByWindowContext context, @NotN WritableRowSet tmpAffected = computeAffectedRowsTime(ctx.sourceRowSet, changed, prevUnits, fwdUnits, ctx.timestampColumnSource, ctx.timestampSsa, false); - // other rows can be affected by removes + // other rows can be affected by removes or mods if (upstream.removed().isNonempty()) { try (final RowSet prev = ctx.sourceRowSet.copyPrev(); final WritableRowSet affectedByRemoves = computeAffectedRowsTime(prev, upstream.removed(), prevUnits, fwdUnits, - ctx.timestampColumnSource, ctx.timestampSsa, true)) { + ctx.timestampColumnSource, ctx.timestampSsa, true); + final WritableRowSet affectedByModifies = + computeAffectedRowsTime(prev, upstream.getModifiedPreShift(), prevUnits, fwdUnits, + ctx.timestampColumnSource, ctx.timestampSsa, true); + ) { // we used the SSA (post-shift) to get these keys, no need to shift // retain only the rows that still exist in the sourceRowSet affectedByRemoves.retain(ctx.sourceRowSet); + affectedByModifies.retain(ctx.sourceRowSet); + tmpAffected.insert(affectedByRemoves); + tmpAffected.insert(affectedByModifies); } } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java index ca2ea6ee8ed..a01ab2f8542 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java @@ -91,9 +91,18 @@ protected Table e() { final Collection clauses = List.of( UpdateByOperation.Fill(), UpdateByOperation.RollingSum(100, 0, - makeOpColNames(columnNamesArray, "_rollsumticks", "Sym", "ts", "boolCol")), + makeOpColNames(columnNamesArray, "_rollsumticksrev", "Sym", "ts", "boolCol")), UpdateByOperation.RollingSum("ts", Duration.ofMinutes(15), Duration.ofMinutes(0), - makeOpColNames(columnNamesArray, "_rollsumtime", "Sym", "ts", "boolCol")), + makeOpColNames(columnNamesArray, "_rollsumtimerev", "Sym", "ts", "boolCol")), + UpdateByOperation.RollingSum(0, 100, + makeOpColNames(columnNamesArray, "_rollsumticksfwd", "Sym", "ts", "boolCol")), + UpdateByOperation.RollingSum("ts", Duration.ofMinutes(0), Duration.ofMinutes(15), + makeOpColNames(columnNamesArray, "_rollsumtimefwd", "Sym", "ts", "boolCol")), + UpdateByOperation.RollingSum(50, 50, + makeOpColNames(columnNamesArray, "_rollsumticksfwdrev", "Sym", "ts", "boolCol")), + UpdateByOperation.RollingSum("ts", Duration.ofMinutes(5), Duration.ofMinutes(5), + makeOpColNames(columnNamesArray, "_rollsumtimebothfwdrev", "Sym", "ts", "boolCol")), + UpdateByOperation.Ema(skipControl, "ts", 10 * MINUTE, makeOpColNames(columnNamesArray, "_ema", "Sym", "ts", "boolCol")), UpdateByOperation.CumSum(makeOpColNames(columnNamesArray, "_sum", "Sym", "ts")), diff --git a/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java b/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java index eb242f247a1..0ffcfdd5db4 100644 --- a/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java +++ b/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java @@ -96,7 +96,8 @@ public static RollingSumSpec of(WindowScale prevWindowScale, WindowScale fwdWind public abstract WindowScale prevTimeScale(); - // provide a default forward-looking timescale + /** provide a default forward-looking timescale + */ @Value.Default public WindowScale fwdTimeScale() { return WindowScale.ofTicks(0); diff --git a/table-api/src/main/java/io/deephaven/api/updateby/spec/WindowScale.java b/table-api/src/main/java/io/deephaven/api/updateby/spec/WindowScale.java index b78666bfb77..f6f0dbcbd35 100644 --- a/table-api/src/main/java/io/deephaven/api/updateby/spec/WindowScale.java +++ b/table-api/src/main/java/io/deephaven/api/updateby/spec/WindowScale.java @@ -12,15 +12,15 @@ @SimpleStyle public abstract class WindowScale { public static WindowScale ofTime(final String timestampCol, long timeScaleNanos) { - return ImmutableTimeScale.of(timestampCol, timeScaleNanos); + return ImmutableWindowScale.of(timestampCol, timeScaleNanos); } public static WindowScale ofTime(final String timestampCol, Duration duration) { - return ImmutableTimeScale.of(timestampCol, duration.toNanos()); + return ImmutableWindowScale.of(timestampCol, duration.toNanos()); } public static WindowScale ofTicks(long tickWindow) { - return ImmutableTimeScale.of(null, tickWindow); + return ImmutableWindowScale.of(null, tickWindow); } @Parameter From 647708dc5abdbbde26d1ccbce9a59c02c8cebe11 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 3 Jan 2023 10:11:17 -0800 Subject: [PATCH 062/123] RingBuffer documentation and renamed InverseRowRedirectionImpl.java --- .../base/ringbuffer/ByteRingBuffer.java | 2 +- .../base/ringbuffer/CharRingBuffer.java | 2 +- .../base/ringbuffer/DoubleRingBuffer.java | 2 +- .../base/ringbuffer/FloatRingBuffer.java | 2 +- .../base/ringbuffer/IntRingBuffer.java | 2 +- .../base/ringbuffer/LongRingBuffer.java | 2 +- .../base/ringbuffer/ShortRingBuffer.java | 2 +- .../deephaven/engine/table/impl/UpdateBy.java | 8 +++--- ...eWrappedRowSetWritableRowRedirection.java} | 11 ++++---- .../engine/table/impl/util/JobScheduler.java | 25 +++++++++++-------- 10 files changed, 32 insertions(+), 26 deletions(-) rename engine/table/src/main/java/io/deephaven/engine/table/impl/util/{InverseRowRedirectionImpl.java => InverseWrappedRowSetWritableRowRedirection.java} (90%) diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java index 0e7b8544819..5f56c101a03 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java @@ -56,7 +56,7 @@ public ByteRingBuffer(int capacity) { * * @param capacity minimum capacity of ring buffer * @param growable whether to allow growth when the buffer is full. If this is {@code false} and {@code add()} is - * called, an + * called, an {@link UnsupportedOperationException} will be thrown * */ public ByteRingBuffer(int capacity, boolean growable) { diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java index f23fe51c2f3..ce83c6a4332 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java @@ -51,7 +51,7 @@ public CharRingBuffer(int capacity) { * * @param capacity minimum capacity of ring buffer * @param growable whether to allow growth when the buffer is full. If this is {@code false} and {@code add()} is - * called, an + * called, an {@link UnsupportedOperationException} will be thrown * */ public CharRingBuffer(int capacity, boolean growable) { diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java index b729908c95f..111b9fe700c 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java @@ -56,7 +56,7 @@ public DoubleRingBuffer(int capacity) { * * @param capacity minimum capacity of ring buffer * @param growable whether to allow growth when the buffer is full. If this is {@code false} and {@code add()} is - * called, an + * called, an {@link UnsupportedOperationException} will be thrown * */ public DoubleRingBuffer(int capacity, boolean growable) { diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java index 38fd409287a..e853e1999f2 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java @@ -56,7 +56,7 @@ public FloatRingBuffer(int capacity) { * * @param capacity minimum capacity of ring buffer * @param growable whether to allow growth when the buffer is full. If this is {@code false} and {@code add()} is - * called, an + * called, an {@link UnsupportedOperationException} will be thrown * */ public FloatRingBuffer(int capacity, boolean growable) { diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java index dc98823b506..dfa66654a42 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java @@ -56,7 +56,7 @@ public IntRingBuffer(int capacity) { * * @param capacity minimum capacity of ring buffer * @param growable whether to allow growth when the buffer is full. If this is {@code false} and {@code add()} is - * called, an + * called, an {@link UnsupportedOperationException} will be thrown * */ public IntRingBuffer(int capacity, boolean growable) { diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java index 17433ffd3be..e28d363afdd 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java @@ -56,7 +56,7 @@ public LongRingBuffer(int capacity) { * * @param capacity minimum capacity of ring buffer * @param growable whether to allow growth when the buffer is full. If this is {@code false} and {@code add()} is - * called, an + * called, an {@link UnsupportedOperationException} will be thrown * */ public LongRingBuffer(int capacity, boolean growable) { diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java index ac5241c15fc..b3532e86757 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java @@ -56,7 +56,7 @@ public ShortRingBuffer(int capacity) { * * @param capacity minimum capacity of ring buffer * @param growable whether to allow growth when the buffer is full. If this is {@code false} and {@code add()} is - * called, an + * called, an {@link UnsupportedOperationException} will be thrown * */ public ShortRingBuffer(int capacity, boolean growable) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java index 1a386b4e34f..119dff9f462 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java @@ -82,7 +82,7 @@ public abstract class UpdateBy { /** ColumnSet transformer from source to downstream */ protected ModifiedColumnSet.Transformer transformer; - /** For refreshing sources, need a listener to react to upstream updates */ + /** Listener to react to upstream changes to refreshing source tables */ protected UpdateByListener listener; /** Store every bucket in this list for processing */ @@ -370,7 +370,7 @@ private void computeCachedColumnRowsets(final Runnable completeAction) { if (initialStep) { for (int srcIdx : cacheableSourceIndices) { if (inputSourceCacheNeeded[srcIdx]) { - // create a RowSet to be used by `InverseRowRedirectionImpl` + // create a RowSet to be used by `InverseWrappedRowSetWritableRowRedirection` inputSourceRowSets[srcIdx] = source.getRowSet().copy(); // record how many windows require this input source @@ -440,7 +440,7 @@ private void createCachedColumnSource(int srcIdx, final Runnable completeAction) innerSource.ensureCapacity(inputRowSet.size()); // there will be no updates to this cached column source, so use a simple redirection - final WritableRowRedirection rowRedirection = new InverseRowRedirectionImpl(inputRowSet); + final WritableRowRedirection rowRedirection = new InverseWrappedRowSetWritableRowRedirection(inputRowSet); final WritableColumnSource outputSource = new WritableRedirectedColumnSource<>(rowRedirection, innerSource, 0); @@ -834,7 +834,7 @@ public static Table updateBy(@NotNull final QueryTable source, if (!source.isRefreshing()) { if (!source.isFlat() && SparseConstants.sparseStructureExceedsOverhead(source.getRowSet(), control.maxStaticSparseMemoryOverheadOrDefault())) { - rowRedirection = new InverseRowRedirectionImpl(source.getRowSet()); + rowRedirection = new InverseWrappedRowSetWritableRowRedirection(source.getRowSet()); } else { rowRedirection = null; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/InverseRowRedirectionImpl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/InverseWrappedRowSetWritableRowRedirection.java similarity index 90% rename from engine/table/src/main/java/io/deephaven/engine/table/impl/util/InverseRowRedirectionImpl.java rename to engine/table/src/main/java/io/deephaven/engine/table/impl/util/InverseWrappedRowSetWritableRowRedirection.java index ec4d9c2d123..d18081ee6bc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/InverseRowRedirectionImpl.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/InverseWrappedRowSetWritableRowRedirection.java @@ -13,10 +13,11 @@ import org.apache.commons.lang3.mutable.MutableLong; import org.jetbrains.annotations.NotNull; -public class InverseRowRedirectionImpl implements WritableRowRedirection { +public class InverseWrappedRowSetWritableRowRedirection implements WritableRowRedirection { /** - * {@link RowSet} used to map from outer key (position in the index) to inner key. + * {@link RowSet} used to map from outer row key (row key in {@code wrappedRowSet}) to inner row key + * (row position in {@code wrappedRowSet}). */ private final RowSet wrappedRowSet; @@ -27,7 +28,7 @@ public class InverseRowRedirectionImpl implements WritableRowRedirection { * * @param wrappedRowSet the RowSet (or TrackingRowSet) to use as the redirection source */ - public InverseRowRedirectionImpl(final RowSet wrappedRowSet) { + public InverseWrappedRowSetWritableRowRedirection(final RowSet wrappedRowSet) { this.wrappedRowSet = wrappedRowSet; } @@ -70,8 +71,8 @@ public void fillChunk(@NotNull final ChunkSource.FillContext fillContext, public void fillPrevChunk(@NotNull final ChunkSource.FillContext fillContext, @NotNull final WritableLongChunk mappedKeysOut, @NotNull final RowSequence keysToMap) { - try (final RowSet prevWrappedIndex = wrappedRowSet.trackingCast().copyPrev(); - final RowSequence.Iterator prevOkIt = prevWrappedIndex.getRowSequenceIterator()) { + try (final RowSet prevWrappedRowSet = wrappedRowSet.trackingCast().copyPrev(); + final RowSequence.Iterator prevOkIt = prevWrappedRowSet.getRowSequenceIterator()) { doMapping(mappedKeysOut, keysToMap, prevOkIt); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java index 5fd58183cef..bcae48a3be2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java @@ -10,8 +10,9 @@ import java.util.function.Consumer; /** - * An interface for submitting jobs to be executed and accumulating their performance of all the tasks performed off - * thread. + * An interface for submitting jobs to be executed. Submitted jobs may be executed on the current thread, or in + * separate threads (thus allowing true parallelism). Performance metrics are accumulated for all executions off the + * current thread for inclusion in overall task metrics. */ public interface JobScheduler { /** @@ -29,7 +30,7 @@ void submit( final Consumer onError); /** - * The performance statistics of all runnables that have been completed off-thread, or null if it was executed in + * The performance statistics of all runnables that have been completed off-thread, or null if all were executed in * the current thread. */ BasePerformanceEntry getAccumulatedPerformance(); @@ -41,7 +42,7 @@ void submit( int threadCount(); /** - * Helper interface for {@code iterateSerial()} and {@code iterateParallel()}. This provides a callable interface + * Helper interface for {@code iterateSerial()} and {@code iterateParallel()}. This provides a functional interface * with {@code index} indicating which iteration to perform. When this returns, the scheduler will automatically * schedule the next iteration. */ @@ -51,13 +52,13 @@ interface IterateAction { } /** - * Helper interface for {@code iterateSerial()} and {@code iterateParallel()}. This provides a callable interface + * Helper interface for {@code iterateSerial()} and {@code iterateParallel()}. This provides a functional interface * with {@code index} indicating which iteration to perform and {@link Runnable resume} providing a mechanism to * inform the scheduler that the current task is complete. When {@code resume} is called, the scheduler will * automatically schedule the next iteration. *

                * NOTE: failing to call {@code resume} will result in the scheduler not scheduling all remaining iterations. This - * will not block the scheduler, but the {@code completeAction} {@link Runnable} will never be called + * will not block the scheduler, but the {@code completeAction} {@link Runnable} will never be called. */ @FunctionalInterface interface IterateResumeAction { @@ -90,8 +91,10 @@ default void iterateParallel(ExecutionContext executionContext, LogOutputAppenda final AtomicBoolean cancelRemainingExecution = new AtomicBoolean(false); final Consumer localError = exception -> { - cancelRemainingExecution.set(true); - onError.accept(exception); + // signal only on the first error + if (cancelRemainingExecution.compareAndSet(false, true)) { + onError.accept(exception); + } }; final Runnable task = () -> { @@ -155,8 +158,10 @@ default void iterateParallel(ExecutionContext executionContext, LogOutputAppenda final AtomicBoolean cancelRemainingExecution = new AtomicBoolean(false); final Consumer localError = exception -> { - cancelRemainingExecution.set(true); - onError.accept(exception); + // signal only on the first error + if (cancelRemainingExecution.compareAndSet(false, true)) { + onError.accept(exception); + } }; final Runnable resumeAction = () -> { From 0a69b50d94d4594b2f50d70f0bc9d2d62a8241e3 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 3 Jan 2023 13:32:06 -0800 Subject: [PATCH 063/123] Cleanup and spotless apply --- .../deephaven/engine/table/impl/UpdateBy.java | 112 +++++++++--------- .../table/impl/UpdateByBucketHelper.java | 15 +-- .../table/impl/updateby/UpdateByWindow.java | 2 +- .../impl/updateby/UpdateByWindowTime.java | 7 +- ...seWrappedRowSetWritableRowRedirection.java | 4 +- .../engine/table/impl/util/JobScheduler.java | 6 +- .../impl/updateby/TestUpdateByGeneral.java | 6 +- .../api/updateby/UpdateByOperation.java | 4 +- .../api/updateby/spec/RollingSumSpec.java | 3 +- 9 files changed, 80 insertions(+), 79 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java index 119dff9f462..4077db96235 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java @@ -94,7 +94,7 @@ public static class UpdateByRedirectionHelper { protected final WritableRowSet freeRows; protected long maxInnerRowKey; - public UpdateByRedirectionHelper(@Nullable final WritableRowRedirection rowRedirection) { + private UpdateByRedirectionHelper(@Nullable final WritableRowRedirection rowRedirection) { this.rowRedirection = rowRedirection; this.freeRows = rowRedirection == null ? null : RowSetFactory.empty(); this.maxInnerRowKey = 0; @@ -104,7 +104,7 @@ public boolean isRedirected() { return rowRedirection != null; } - public long requiredCapacity() { + private long requiredCapacity() { return maxInnerRowKey + 1; } @@ -113,7 +113,7 @@ public WritableRowRedirection getRowRedirection() { return rowRedirection; } - public void processUpdateForRedirection(@NotNull final TableUpdate upstream, + private void processUpdateForRedirection(@NotNull final TableUpdate upstream, final TrackingRowSet sourceRowSet) { assert rowRedirection != null; if (upstream.removed().isNonempty()) { @@ -172,7 +172,7 @@ private boolean shiftRedirectedKey(@NotNull final RowSet.SearchIterator iterator return !iterator.hasNext(); } - public RowSet getInnerKeys(final RowSet outerKeys) { + private RowSet getInnerKeys(final RowSet outerKeys) { if (rowRedirection == null) { return null; } @@ -237,25 +237,27 @@ private void releaseInputSources(int winIdx, ColumnSource[] maybeCachedInputS try (final ResettableWritableObjectChunk backingChunk = ResettableWritableObjectChunk.makeResettableChunk()) { for (int srcIdx : uniqueWindowSources) { - if (inputSourceCacheNeeded[srcIdx]) { - if (--inputSourceReferenceCounts[srcIdx] == 0) { - // do the cleanup immediately - inputSourceRowSets[srcIdx].close(); - inputSourceRowSets[srcIdx] = null; - - // release any objects we are holding in the cache - if (maybeCachedInputSources[srcIdx] instanceof ObjectArraySource) { - final long targetCapacity = inputSourceRowSets[srcIdx].size(); - for (long positionToNull = 0; positionToNull < targetCapacity; positionToNull += - backingChunk.size()) { - ((ObjectArraySource) maybeCachedInputSources[srcIdx]) - .resetWritableChunkToBackingStore(backingChunk, positionToNull); - backingChunk.fillWithNullValue(0, backingChunk.size()); - } - } + if (!inputSourceCacheNeeded[srcIdx]) { + continue; + } - maybeCachedInputSources[srcIdx] = null; + if (--inputSourceReferenceCounts[srcIdx] == 0) { + // release any objects we are holding in the cache + if (maybeCachedInputSources[srcIdx] instanceof ObjectArraySource) { + final long targetCapacity = inputSourceRowSets[srcIdx].size(); + for (long positionToNull = 0; positionToNull < targetCapacity; positionToNull += + backingChunk.size()) { + ((ObjectArraySource) maybeCachedInputSources[srcIdx]) + .resetWritableChunkToBackingStore(backingChunk, positionToNull); + backingChunk.fillWithNullValue(0, backingChunk.size()); + } } + + // release the row set + inputSourceRowSets[srcIdx].close(); + inputSourceRowSets[srcIdx] = null; + + maybeCachedInputSources[srcIdx] = null; } } } @@ -284,7 +286,7 @@ private void releaseInputSources(int winIdx, ColumnSource[] maybeCachedInputS */ class PhasedUpdateProcessor implements LogOutputAppendable { - final TableUpdate update; + final TableUpdate upstream; final boolean initialStep; final UpdateByBucketHelper[] dirtyBuckets; @@ -300,8 +302,8 @@ class PhasedUpdateProcessor implements LogOutputAppendable { WritableRowSet shiftedRows; - PhasedUpdateProcessor(TableUpdate update, boolean initialStep) { - this.update = update; + PhasedUpdateProcessor(TableUpdate upstream, boolean initialStep) { + this.upstream = upstream; this.initialStep = initialStep; // determine which buckets we'll examine during this update @@ -548,9 +550,9 @@ private void processWindowBuckets(int winIdx, final Runnable completedAction) { } /** - * Process all {@code windows} in a serial manner (to minimize cache memory usage). Will create cached input - * sources, process the buckets, then release the cached columns before starting the next window. Calls - * {@code completedAction} when the work is complete + * Process all {@code windows} in a serial manner (to minimize cache memory usage and to protect against races + * to fill the cached input sources). Will create cached input sources, process the buckets, then release the + * cached columns before starting the next window. Calls {@code completedAction} when the work is complete */ private void processWindows(final Runnable completeAction) { jobScheduler.iterateSerial(ExecutionContext.getContextToRecord(), this, 0, windows.length, @@ -570,7 +572,8 @@ private void processWindows(final Runnable completeAction) { } } } else { - // get the minimal set of rows to be updated for this window + // get the minimal set of rows to be updated for this window (shiftedRows is empty when + // using redirection) try (final WritableRowSet windowRowSet = shiftedRows.copy()) { for (UpdateByBucketHelper bucket : dirtyBuckets) { if (win.isWindowDirty(bucket.windowContexts[winIdx])) { @@ -587,10 +590,10 @@ private void processWindows(final Runnable completeAction) { } } - if (!redirHelper.isRedirected() && update.shifted().nonempty()) { + if (!redirHelper.isRedirected() && upstream.shifted().nonempty()) { // shift the non-redirected output sources now, after parallelPopulation try (final RowSet prevIdx = source.getRowSet().copyPrev()) { - update.shifted().apply((begin, end, delta) -> { + upstream.shifted().apply((begin, end, delta) -> { try (final RowSet subRowSet = prevIdx.subSetByKeyRange(begin, end)) { for (UpdateByOperator op : win.getOperators()) { op.applyOutputShift(subRowSet, delta); @@ -665,23 +668,21 @@ private void cleanUpAfterError() { private TableUpdate computeDownstreamUpdate() { final TableUpdateImpl downstream = new TableUpdateImpl(); - downstream.added = update.added().copy(); - downstream.removed = update.removed().copy(); - downstream.shifted = update.shifted(); + // get the adds/removes/shifts from upstream, make a copy since TableUpdateImpl#reset will + // close them with the upstream update + downstream.added = upstream.added().copy(); + downstream.removed = upstream.removed().copy(); + downstream.shifted = upstream.shifted(); // union the modifies from all the tables (including source) downstream.modifiedColumnSet = result().getModifiedColumnSetForUpdates(); downstream.modifiedColumnSet.clear(); - // get the adds/removes/shifts from upstream, make a copy since TableUpdateImpl#reset will - // close them with the upstream update - - WritableRowSet modifiedRowSet = RowSetFactory.empty(); + WritableRowSet modifiedRowSet = upstream.modified().copy(); downstream.modified = modifiedRowSet; - if (update.modified().isNonempty()) { - modifiedRowSet.insert(update.modified()); - transformer.transform(update.modifiedColumnSet(), downstream.modifiedColumnSet); + if (upstream.modified().isNonempty()) { + transformer.transform(upstream.modifiedColumnSet(), downstream.modifiedColumnSet); } for (UpdateByBucketHelper bucket : dirtyBuckets) { @@ -717,23 +718,23 @@ public void processUpdate() { if (redirHelper.isRedirected()) { // this call does all the work needed for redirected output sources, for sparse output sources // we will process shifts only after a call to `prepareForParallelPopulation()` on each source - redirHelper.processUpdateForRedirection(update, source.getRowSet()); + redirHelper.processUpdateForRedirection(upstream, source.getRowSet()); shiftedRows = RowSetFactory.empty(); } else { // for our sparse array output sources, we need to identify which rows will be affected by the upstream // shifts and include them in our parallel update preparations - if (update.shifted().nonempty()) { + if (upstream.shifted().nonempty()) { try (final RowSet prev = source.getRowSet().copyPrev(); final RowSequence.Iterator it = prev.getRowSequenceIterator()) { final RowSetBuilderSequential builder = RowSetFactory.builderSequential(); - final int size = update.shifted().size(); + final int size = upstream.shifted().size(); // get these in ascending order and use a sequential builder for (int ii = 0; ii < size; ii++) { - final long begin = update.shifted().getBeginRange(ii); - final long end = update.shifted().getEndRange(ii); - final long delta = update.shifted().getShiftDelta(ii); + final long begin = upstream.shifted().getBeginRange(ii); + final long end = upstream.shifted().getEndRange(ii); + final long delta = upstream.shifted().getShiftDelta(ii); it.advance(begin); final RowSequence rs = it.getNextRowSequenceThrough(end); @@ -780,7 +781,8 @@ public void processUpdate() { } /** - * The Listener for apply to the constituent table updates + * The Listener that is called when all input tables (source and constituent) are satisfied. This listener will + * initiate UpdateBy operator processing in parallel by bucket */ class UpdateByListener extends InstrumentedTableUpdateListenerAdapter { public UpdateByListener(@Nullable String description) { @@ -816,7 +818,7 @@ public UpdateByListener newUpdateByListener(@NotNull final String description) { /** * Apply the specified operations to each group of rows in the source table and produce a result table with the same - * index as the source with each operator applied. + * row set as the source with each operator applied. * * @param source the source to apply to. * @param clauses the operations to apply. @@ -879,7 +881,6 @@ public static Table updateBy(@NotNull final QueryTable source, String timestampColumnName = null; final Set problems = new LinkedHashSet<>(); - // noinspection rawtypes final Map> opResultSources = new LinkedHashMap<>(); for (int opIdx = 0; opIdx < opArr.length; opIdx++) { final UpdateByOperator op = opArr[opIdx]; @@ -895,9 +896,8 @@ public static Table updateBy(@NotNull final QueryTable source, } else { if (!timestampColumnName.equals(op.getTimestampColumnName())) { throw new UncheckedTableException( - "Cannot reference more than one timestamp source on a single UpdateByBucketHelper operation {" - + - timestampColumnName + ", " + op.getTimestampColumnName() + "}"); + "Cannot reference more than one timestamp source on a single UpdateBy call {" + + timestampColumnName + ", " + op.getTimestampColumnName() + "}"); } } } @@ -912,11 +912,11 @@ public static Table updateBy(@NotNull final QueryTable source, for (int opIdx = 0; opIdx < opArr.length; opIdx++) { final String[] inputColumnNames = opArr[opIdx].getInputColumnNames(); + // add a new entry for this operator + operatorInputSourceSlotArr[opIdx] = new int[inputColumnNames.length]; for (int colIdx = 0; colIdx < inputColumnNames.length; colIdx++) { final ColumnSource input = source.getColumnSource(inputColumnNames[colIdx]); final int maybeExistingSlot = sourceToSlotMap.get(input); - // add a new entry for this operator - operatorInputSourceSlotArr[opIdx] = new int[inputColumnNames.length]; if (maybeExistingSlot == sourceToSlotMap.getNoEntryValue()) { int srcIdx = inputSourceList.size(); // create a new input source and map the operator to it @@ -928,7 +928,7 @@ public static Table updateBy(@NotNull final QueryTable source, } } } - final ColumnSource[] inputSourceArr = inputSourceList.toArray(new ColumnSource[0]); + final ColumnSource[] inputSourceArr = inputSourceList.toArray(ColumnSource.ZERO_LENGTH_COLUMN_SOURCE_ARRAY); // now we want to divide the operators into similar windows for efficient processing TIntObjectHashMap windowHashToOperatorIndicesMap = new TIntObjectHashMap<>(); @@ -975,7 +975,6 @@ public static Table updateBy(@NotNull final QueryTable source, return true; }); - // noinspection rawtypes final Map> resultSources = new LinkedHashMap<>(source.getColumnSourceMap()); resultSources.putAll(opResultSources); @@ -1008,7 +1007,6 @@ public static Table updateBy(@NotNull final QueryTable source, for (final MatchPair byColumn : pairs) { if (!source.hasColumns(byColumn.rightColumn)) { problems.add(byColumn.rightColumn); - continue; } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java index 9d09b3185d3..e7724ff28d7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java @@ -47,7 +47,7 @@ class UpdateByBucketHelper extends IntrusiveDoublyLinkedNode.Impl keysChunk, LongChunk ssaValues.add(ts); ssaKeys.add(keysChunk.get(i)); } + // store the current ts for comparison + lastTimestamp.setValue(ts); } } @@ -284,7 +283,7 @@ public boolean isDirty() { * use of cached input sources instead of the original input sources. * * @param winIdx the index of the window to modify - * @param inputSources the input sources for the + * @param inputSources the input sources for the operators */ public void assignInputSources(final int winIdx, final ColumnSource[] inputSources) { windows[winIdx].assignInputSources(windowContexts[winIdx], inputSources); @@ -295,7 +294,7 @@ public void assignInputSources(final int winIdx, final ColumnSource[] inputSo * {@code assignInputSources()} call. * * @param winIdx the index of the window to modify - * @param initialStep the input sources for the + * @param initialStep indicates whether this is part of the creation phase */ public void processWindow(final int winIdx, final boolean initialStep) { if (!windows[winIdx].isWindowDirty(windowContexts[winIdx])) { @@ -316,7 +315,9 @@ public void finalizeUpdate() { } /** - * The Listener for apply an upstream {@link InstrumentedTableUpdateListenerAdapter#onUpdate(TableUpdate) update} + * The Listener that accepts an {@link InstrumentedTableUpdateListenerAdapter#onUpdate(TableUpdate) update} and + * prepares this bucket for processing. This includes determination of `isDirty` status and the computation of + * `affected` and `influencer` row sets for this processing cycle. */ class UpdateByBucketHelperListener extends InstrumentedTableUpdateListenerAdapter { public UpdateByBucketHelperListener(@Nullable String description, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java index 0d705b73fe1..c7364d3dca7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java @@ -359,7 +359,7 @@ public static int hashCodeFromOperator(final UpdateByOperator op) { op.getInputColumnNames(), op.getTimestampColumnName(), op.getPrevWindowUnits(), - op.getPrevWindowUnits()); + op.getFwdWindowUnits()); } /** diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java index 00a8b765312..a60935117ef 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java @@ -291,10 +291,9 @@ public void computeAffectedRowsAndOperators(UpdateByWindowContext context, @NotN final WritableRowSet affectedByRemoves = computeAffectedRowsTime(prev, upstream.removed(), prevUnits, fwdUnits, ctx.timestampColumnSource, ctx.timestampSsa, true); - final WritableRowSet affectedByModifies = - computeAffectedRowsTime(prev, upstream.getModifiedPreShift(), prevUnits, fwdUnits, - ctx.timestampColumnSource, ctx.timestampSsa, true); - ) { + final WritableRowSet affectedByModifies = + computeAffectedRowsTime(prev, upstream.getModifiedPreShift(), prevUnits, fwdUnits, + ctx.timestampColumnSource, ctx.timestampSsa, true);) { // we used the SSA (post-shift) to get these keys, no need to shift // retain only the rows that still exist in the sourceRowSet affectedByRemoves.retain(ctx.sourceRowSet); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/InverseWrappedRowSetWritableRowRedirection.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/InverseWrappedRowSetWritableRowRedirection.java index d18081ee6bc..f1ebab217b7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/InverseWrappedRowSetWritableRowRedirection.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/InverseWrappedRowSetWritableRowRedirection.java @@ -16,8 +16,8 @@ public class InverseWrappedRowSetWritableRowRedirection implements WritableRowRedirection { /** - * {@link RowSet} used to map from outer row key (row key in {@code wrappedRowSet}) to inner row key - * (row position in {@code wrappedRowSet}). + * {@link RowSet} used to map from outer row key (row key in {@code wrappedRowSet}) to inner row key (row position + * in {@code wrappedRowSet}). */ private final RowSet wrappedRowSet; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java index bcae48a3be2..bd2d267151c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java @@ -10,9 +10,9 @@ import java.util.function.Consumer; /** - * An interface for submitting jobs to be executed. Submitted jobs may be executed on the current thread, or in - * separate threads (thus allowing true parallelism). Performance metrics are accumulated for all executions off the - * current thread for inclusion in overall task metrics. + * An interface for submitting jobs to be executed. Submitted jobs may be executed on the current thread, or in separate + * threads (thus allowing true parallelism). Performance metrics are accumulated for all executions off the current + * thread for inclusion in overall task metrics. */ public interface JobScheduler { /** diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java index a01ab2f8542..c16555feb9e 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java @@ -99,9 +99,11 @@ protected Table e() { UpdateByOperation.RollingSum("ts", Duration.ofMinutes(0), Duration.ofMinutes(15), makeOpColNames(columnNamesArray, "_rollsumtimefwd", "Sym", "ts", "boolCol")), UpdateByOperation.RollingSum(50, 50, - makeOpColNames(columnNamesArray, "_rollsumticksfwdrev", "Sym", "ts", "boolCol")), + makeOpColNames(columnNamesArray, "_rollsumticksfwdrev", "Sym", "ts", + "boolCol")), UpdateByOperation.RollingSum("ts", Duration.ofMinutes(5), Duration.ofMinutes(5), - makeOpColNames(columnNamesArray, "_rollsumtimebothfwdrev", "Sym", "ts", "boolCol")), + makeOpColNames(columnNamesArray, "_rollsumtimebothfwdrev", "Sym", "ts", + "boolCol")), UpdateByOperation.Ema(skipControl, "ts", 10 * MINUTE, makeOpColNames(columnNamesArray, "_ema", "Sym", "ts", "boolCol")), diff --git a/table-api/src/main/java/io/deephaven/api/updateby/UpdateByOperation.java b/table-api/src/main/java/io/deephaven/api/updateby/UpdateByOperation.java index 16b664a6321..2fa345a031e 100644 --- a/table-api/src/main/java/io/deephaven/api/updateby/UpdateByOperation.java +++ b/table-api/src/main/java/io/deephaven/api/updateby/UpdateByOperation.java @@ -241,8 +241,8 @@ static UpdateByOperation RollingSum(long prevTimeTicks, long fwdTimeTicks, Strin } /** - * Create a {@link RollingSumSpec rolling sum} for the supplied column name pairs, using time as the windowing - * unit. Uses the default OperationControl settings. + * Create a {@link RollingSumSpec rolling sum} for the supplied column name pairs, using time as the windowing unit. + * Uses the default OperationControl settings. * * @param prevWindowDuration the look-behind window size (in Duration) * @param fwdWindowDuration the look-ahead window size (in Duration) diff --git a/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java b/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java index 0ffcfdd5db4..217d47bea43 100644 --- a/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java +++ b/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java @@ -96,7 +96,8 @@ public static RollingSumSpec of(WindowScale prevWindowScale, WindowScale fwdWind public abstract WindowScale prevTimeScale(); - /** provide a default forward-looking timescale + /** + * provide a default forward-looking timescale */ @Value.Default public WindowScale fwdTimeScale() { From 1bcc109583c6f83d476f7c4f56ed829cf14d9877 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 3 Jan 2023 13:58:42 -0800 Subject: [PATCH 064/123] Low hanging fruit resolved, more work ahead --- .../impl/UpdateByCumulativeOperator.java | 4 +-- .../engine/table/impl/UpdateByOperator.java | 35 +++++++++---------- .../table/impl/UpdateByWindowedOperator.java | 2 +- .../internal/BaseByteUpdateByOperator.java | 9 +++-- .../internal/BaseCharUpdateByOperator.java | 13 ++++--- .../internal/BaseDoubleUpdateByOperator.java | 9 +++-- .../internal/BaseFloatUpdateByOperator.java | 9 +++-- .../internal/BaseIntUpdateByOperator.java | 13 ++++--- .../internal/BaseLongUpdateByOperator.java | 13 ++++--- .../internal/BaseObjectUpdateByOperator.java | 13 ++++--- .../internal/BaseShortUpdateByOperator.java | 13 ++++--- .../BaseWindowedByteUpdateByOperator.java | 7 ++-- .../BaseWindowedCharUpdateByOperator.java | 11 +++--- .../BaseWindowedDoubleUpdateByOperator.java | 11 +++--- .../BaseWindowedFloatUpdateByOperator.java | 11 +++--- .../BaseWindowedIntUpdateByOperator.java | 11 +++--- .../BaseWindowedLongUpdateByOperator.java | 11 +++--- .../BaseWindowedObjectUpdateByOperator.java | 11 +++--- .../BaseWindowedShortUpdateByOperator.java | 11 +++--- .../PairwiseDoubleRingBufferTest.java | 2 +- .../PairwiseFloatRingBufferTest.java | 2 +- .../replicators/ReplicateUpdateBy.java | 2 +- 22 files changed, 103 insertions(+), 120 deletions(-) rename engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/{ => internal}/PairwiseDoubleRingBufferTest.java (99%) rename engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/{ => internal}/PairwiseFloatRingBufferTest.java (99%) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java index 82e5fd9b3b1..6da7b64f6a9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java @@ -11,10 +11,10 @@ import static io.deephaven.util.QueryConstants.NULL_LONG; public abstract class UpdateByCumulativeOperator extends UpdateByOperator { - public abstract class Context implements UpdateContext { + public abstract static class Context implements UpdateContext { public long curTimestamp; - protected Context(final int chunkSize) { + protected Context() { curTimestamp = NULL_LONG; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java index 9f1fa44dc08..24bce1160a8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java @@ -38,16 +38,19 @@ public abstract class UpdateByOperator { protected final String[] affectingColumns; protected final UpdateBy.UpdateByRedirectionHelper redirHelper; - // these will be used by the timestamp-aware operators (EMA for example) protected OperationControl control; protected long reverseTimeScaleUnits; protected long forwardTimeScaleUnits; protected String timestampColumnName; - // individual input modifiedColumnSet for this operator + /** + * The input modifiedColumnSet for this operator + */ protected ModifiedColumnSet inputModifiedColumnSet; - // individual output modifiedColumnSet for this operators + /** + * The output modifiedColumnSet for this operator + */ protected ModifiedColumnSet outputModifiedColumnSet; /** @@ -55,9 +58,9 @@ public abstract class UpdateByOperator { */ public interface UpdateContext extends SafeCloseable { - void setValuesChunk(@NotNull final Chunk valuesChunk); + void setValuesChunk(@NotNull Chunk valuesChunk); - void setTimestampChunk(@NotNull final LongChunk valuesChunk); + void setTimestampChunk(@NotNull LongChunk valuesChunk); /** * Add a value to the operators current data set @@ -110,9 +113,9 @@ protected UpdateByOperator(@NotNull final MatchPair pair, } /** - * Get the name of the input column(s) this operator depends on. + * Get the names of the input column(s) for this operator. * - * @return the name of the input column + * @return the names of the input column */ @NotNull public String[] getInputColumnNames() { @@ -120,9 +123,9 @@ public String[] getInputColumnNames() { } /** - * Get the name of the timestamp column this operator depends on. + * Get the name of the timestamp column for this operator (or null if the operator does not require timestamp data). * - * @return the name of the input column + * @return the name of the timestamp column */ @Nullable public String getTimestampColumnName() { @@ -130,18 +133,14 @@ public String getTimestampColumnName() { } /** - * Get the value of the backward-looking window (might be nanos or ticks). - * - * @return the name of the input column + * Get the value of the backward-looking window (might be nanoseconds or ticks). */ public long getPrevWindowUnits() { return reverseTimeScaleUnits; } /** - * Get the value of the forward-looking window (might be nanos or ticks). - * - * @return the name of the input column + * Get the value of the forward-looking window (might be nanoseconds or ticks). */ public long getFwdWindowUnits() { return forwardTimeScaleUnits; @@ -190,7 +189,7 @@ public String[] getOutputColumnNames() { public abstract UpdateContext makeUpdateContext(final int chunkSize); /** - * Perform and bookkeeping required at the end of a single part of the update. This is always preceded with a call + * Perform any bookkeeping required at the end of a single part of the update. This is always preceded with a call * to {@code #initializeUpdate(UpdateContext)} (specialized for each type of operator) * * @param context the context object @@ -200,12 +199,12 @@ public String[] getOutputColumnNames() { /** * Apply a shift to the operation. */ - public abstract void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta); + public abstract void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long delta); /** * Prepare this operator output column for parallel updated. */ - public abstract void prepareForParallelPopulation(final RowSet added); + public abstract void prepareForParallelPopulation(final RowSet changedRows); /** * Create the modified column set for the input columns of this operator. diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java index 0ee64ab93ca..15ff3895e22 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java @@ -11,7 +11,7 @@ public abstract class UpdateByWindowedOperator extends UpdateByOperator { - public abstract class Context implements UpdateContext { + public abstract static class Context implements UpdateContext { public int nullCount = 0; @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java index 5545e74bdef..8257c61e5ed 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java @@ -20,7 +20,6 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import java.util.Collections; @@ -44,7 +43,7 @@ protected abstract class Context extends UpdateByCumulativeOperator.Context { public byte curVal = NULL_BYTE; protected Context(final int chunkSize) { - super(chunkSize); + super(); this.outputFillContext = outputSource.makeFillFromContext(chunkSize); this.outputValues = WritableByteChunk.makeWritableChunk(chunkSize); } @@ -175,11 +174,11 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d // endregion Shifts @Override - public void prepareForParallelPopulation(final RowSet added) { + public void prepareForParallelPopulation(final RowSet changedRows) { if (redirHelper.isRedirected()) { - ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); + ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { - ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); + ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java index ae35df55ab3..03b0dcd9e4e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java @@ -10,7 +10,6 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import java.util.Collections; @@ -33,7 +32,7 @@ protected abstract class Context extends UpdateByCumulativeOperator.Context { public char curVal = NULL_CHAR; protected Context(final int chunkSize) { - super(chunkSize); + super(); this.outputFillContext = outputSource.makeFillFromContext(chunkSize); this.outputValues = WritableCharChunk.makeWritableChunk(chunkSize); } @@ -141,17 +140,17 @@ public void startTrackingPrev() { // region Shifts @Override - public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { - ((CharacterSparseArraySource)outputSource).shift(subIndexToShift, delta); + public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long delta) { + ((CharacterSparseArraySource)outputSource).shift(subRowSetToShift, delta); } // endregion Shifts @Override - public void prepareForParallelPopulation(final RowSet added) { + public void prepareForParallelPopulation(final RowSet changedRows) { if (redirHelper.isRedirected()) { - ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); + ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { - ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); + ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java index 166a1c87722..96d9b181d4a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java @@ -14,7 +14,6 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import java.util.Collections; @@ -37,7 +36,7 @@ protected abstract class Context extends UpdateByCumulativeOperator.Context { public double curVal = NULL_DOUBLE; protected Context(final int chunkSize) { - super(chunkSize); + super(); this.outputFillContext = outputSource.makeFillFromContext(chunkSize); this.outputValues = WritableDoubleChunk.makeWritableChunk(chunkSize); } @@ -150,11 +149,11 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long // endregion @Override - public void prepareForParallelPopulation(final RowSet added) { + public void prepareForParallelPopulation(final RowSet changedRows) { if (redirHelper.isRedirected()) { - ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); + ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { - ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); + ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java index 66c1910df3b..be69c5982af 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java @@ -9,7 +9,6 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import java.util.Collections; @@ -32,7 +31,7 @@ protected abstract class Context extends UpdateByCumulativeOperator.Context { public float curVal = NULL_FLOAT; protected Context(final int chunkSize) { - super(chunkSize); + super(); this.outputFillContext = outputSource.makeFillFromContext(chunkSize); this.outputValues = WritableFloatChunk.makeWritableChunk(chunkSize); } @@ -145,11 +144,11 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long // endregion @Override - public void prepareForParallelPopulation(final RowSet added) { + public void prepareForParallelPopulation(final RowSet changedRows) { if (redirHelper.isRedirected()) { - ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); + ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { - ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); + ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java index 98499651302..c117acb1cf8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java @@ -15,7 +15,6 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import java.util.Collections; @@ -38,7 +37,7 @@ protected abstract class Context extends UpdateByCumulativeOperator.Context { public int curVal = NULL_INT; protected Context(final int chunkSize) { - super(chunkSize); + super(); this.outputFillContext = outputSource.makeFillFromContext(chunkSize); this.outputValues = WritableIntChunk.makeWritableChunk(chunkSize); } @@ -146,17 +145,17 @@ public void startTrackingPrev() { // region Shifts @Override - public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { - ((IntegerSparseArraySource)outputSource).shift(subIndexToShift, delta); + public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long delta) { + ((IntegerSparseArraySource)outputSource).shift(subRowSetToShift, delta); } // endregion Shifts @Override - public void prepareForParallelPopulation(final RowSet added) { + public void prepareForParallelPopulation(final RowSet changedRows) { if (redirHelper.isRedirected()) { - ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); + ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { - ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); + ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java index b5fe7b19d7e..6a2512ab3c2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java @@ -15,7 +15,6 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import java.util.Collections; @@ -38,7 +37,7 @@ protected abstract class Context extends UpdateByCumulativeOperator.Context { public long curVal = NULL_LONG; protected Context(final int chunkSize) { - super(chunkSize); + super(); this.outputFillContext = outputSource.makeFillFromContext(chunkSize); this.outputValues = WritableLongChunk.makeWritableChunk(chunkSize); } @@ -146,17 +145,17 @@ public void startTrackingPrev() { // region Shifts @Override - public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { - ((LongSparseArraySource)outputSource).shift(subIndexToShift, delta); + public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long delta) { + ((LongSparseArraySource)outputSource).shift(subRowSetToShift, delta); } // endregion Shifts @Override - public void prepareForParallelPopulation(final RowSet added) { + public void prepareForParallelPopulation(final RowSet changedRows) { if (redirHelper.isRedirected()) { - ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); + ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { - ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); + ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java index 9e7ee33cf4c..7c058aff5e4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java @@ -15,7 +15,6 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import java.util.Collections; @@ -39,7 +38,7 @@ protected abstract class Context extends UpdateByCumulativeOperator.Context { public T curVal = null; protected Context(final int chunkSize) { - super(chunkSize); + super(); this.outputFillContext = outputSource.makeFillFromContext(chunkSize); this.outputValues = WritableObjectChunk.makeWritableChunk(chunkSize); } @@ -149,17 +148,17 @@ public void startTrackingPrev() { // region Shifts @Override - public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { - ((ObjectSparseArraySource)outputSource).shift(subIndexToShift, delta); + public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long delta) { + ((ObjectSparseArraySource)outputSource).shift(subRowSetToShift, delta); } // endregion Shifts @Override - public void prepareForParallelPopulation(final RowSet added) { + public void prepareForParallelPopulation(final RowSet changedRows) { if (redirHelper.isRedirected()) { - ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); + ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { - ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); + ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java index 9588043982c..fcf5b10aeed 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java @@ -15,7 +15,6 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import java.util.Collections; @@ -38,7 +37,7 @@ protected abstract class Context extends UpdateByCumulativeOperator.Context { public short curVal = NULL_SHORT; protected Context(final int chunkSize) { - super(chunkSize); + super(); this.outputFillContext = outputSource.makeFillFromContext(chunkSize); this.outputValues = WritableShortChunk.makeWritableChunk(chunkSize); } @@ -146,17 +145,17 @@ public void startTrackingPrev() { // region Shifts @Override - public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { - ((ShortSparseArraySource)outputSource).shift(subIndexToShift, delta); + public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long delta) { + ((ShortSparseArraySource)outputSource).shift(subRowSetToShift, delta); } // endregion Shifts @Override - public void prepareForParallelPopulation(final RowSet added) { + public void prepareForParallelPopulation(final RowSet changedRows) { if (redirHelper.isRedirected()) { - ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); + ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { - ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); + ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java index d7e68988bf4..8378eed9a67 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java @@ -21,7 +21,6 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -181,11 +180,11 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d // endregion Shifts @Override - public void prepareForParallelPopulation(final RowSet added) { + public void prepareForParallelPopulation(final RowSet changedRows) { if (redirHelper.isRedirected()) { - ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); + ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { - ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); + ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java index 52bc1978c59..9dc6df11edf 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java @@ -11,7 +11,6 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -147,17 +146,17 @@ public void startTrackingPrev() { // region Shifts @Override - public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { - ((CharacterSparseArraySource)outputSource).shift(subIndexToShift, delta); + public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long delta) { + ((CharacterSparseArraySource)outputSource).shift(subRowSetToShift, delta); } // endregion Shifts @Override - public void prepareForParallelPopulation(final RowSet added) { + public void prepareForParallelPopulation(final RowSet changedRows) { if (redirHelper.isRedirected()) { - ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); + ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { - ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); + ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java index d2f9340735b..df3e319426e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java @@ -15,7 +15,6 @@ import io.deephaven.engine.table.impl.sources.DoubleArraySource; import io.deephaven.engine.table.impl.sources.DoubleSparseArraySource; import io.deephaven.engine.table.impl.sources.WritableRedirectedColumnSource; -import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -151,17 +150,17 @@ public void startTrackingPrev() { // region Shifts @Override - public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { - ((DoubleSparseArraySource)outputSource).shift(subIndexToShift, delta); + public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long delta) { + ((DoubleSparseArraySource)outputSource).shift(subRowSetToShift, delta); } // endregion Shifts @Override - public void prepareForParallelPopulation(final RowSet added) { + public void prepareForParallelPopulation(final RowSet changedRows) { if (redirHelper.isRedirected()) { - ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); + ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { - ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); + ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java index 08a538dfa39..9e8a10c10c0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java @@ -10,7 +10,6 @@ import io.deephaven.engine.table.impl.sources.FloatArraySource; import io.deephaven.engine.table.impl.sources.FloatSparseArraySource; import io.deephaven.engine.table.impl.sources.WritableRedirectedColumnSource; -import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -146,17 +145,17 @@ public void startTrackingPrev() { // region Shifts @Override - public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { - ((FloatSparseArraySource)outputSource).shift(subIndexToShift, delta); + public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long delta) { + ((FloatSparseArraySource)outputSource).shift(subRowSetToShift, delta); } // endregion Shifts @Override - public void prepareForParallelPopulation(final RowSet added) { + public void prepareForParallelPopulation(final RowSet changedRows) { if (redirHelper.isRedirected()) { - ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); + ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { - ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); + ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java index cd1d51d0c64..8ffa00c5eb3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java @@ -16,7 +16,6 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -152,17 +151,17 @@ public void startTrackingPrev() { // region Shifts @Override - public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { - ((IntegerSparseArraySource)outputSource).shift(subIndexToShift, delta); + public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long delta) { + ((IntegerSparseArraySource)outputSource).shift(subRowSetToShift, delta); } // endregion Shifts @Override - public void prepareForParallelPopulation(final RowSet added) { + public void prepareForParallelPopulation(final RowSet changedRows) { if (redirHelper.isRedirected()) { - ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); + ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { - ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); + ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java index b4cbbceed53..60100e7d180 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java @@ -16,7 +16,6 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -152,17 +151,17 @@ public void startTrackingPrev() { // region Shifts @Override - public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { - ((LongSparseArraySource)outputSource).shift(subIndexToShift, delta); + public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long delta) { + ((LongSparseArraySource)outputSource).shift(subRowSetToShift, delta); } // endregion Shifts @Override - public void prepareForParallelPopulation(final RowSet added) { + public void prepareForParallelPopulation(final RowSet changedRows) { if (redirHelper.isRedirected()) { - ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); + ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { - ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); + ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java index 1d0d81d82ad..53e1d4cdeba 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java @@ -16,7 +16,6 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -155,17 +154,17 @@ public void startTrackingPrev() { // region Shifts @Override - public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { - ((ObjectSparseArraySource)outputSource).shift(subIndexToShift, delta); + public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long delta) { + ((ObjectSparseArraySource)outputSource).shift(subRowSetToShift, delta); } // endregion Shifts @Override - public void prepareForParallelPopulation(final RowSet added) { + public void prepareForParallelPopulation(final RowSet changedRows) { if (redirHelper.isRedirected()) { - ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); + ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { - ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); + ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java index a5ff7212f3f..42a5ca6470b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java @@ -16,7 +16,6 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -152,17 +151,17 @@ public void startTrackingPrev() { // region Shifts @Override - public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { - ((ShortSparseArraySource)outputSource).shift(subIndexToShift, delta); + public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long delta) { + ((ShortSparseArraySource)outputSource).shift(subRowSetToShift, delta); } // endregion Shifts @Override - public void prepareForParallelPopulation(final RowSet added) { + public void prepareForParallelPopulation(final RowSet changedRows) { if (redirHelper.isRedirected()) { - ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(added); + ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { - ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(added); + ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); } } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/PairwiseDoubleRingBufferTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBufferTest.java similarity index 99% rename from engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/PairwiseDoubleRingBufferTest.java rename to engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBufferTest.java index 96c48fba846..513ab148150 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/PairwiseDoubleRingBufferTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBufferTest.java @@ -6,7 +6,7 @@ * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit PairwiseFloatRingBufferTest and regenerate * --------------------------------------------------------------------------------------------------------------------- */ -package io.deephaven.engine.table.impl.updateby; +package io.deephaven.engine.table.impl.updateby.internal; import io.deephaven.engine.table.impl.updateby.internal.PairwiseDoubleRingBuffer; import io.deephaven.test.types.OutOfBandTest; diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/PairwiseFloatRingBufferTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBufferTest.java similarity index 99% rename from engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/PairwiseFloatRingBufferTest.java rename to engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBufferTest.java index 5112a54532b..e5bfb6d2264 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/PairwiseFloatRingBufferTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBufferTest.java @@ -1,7 +1,7 @@ /** * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending */ -package io.deephaven.engine.table.impl.updateby; +package io.deephaven.engine.table.impl.updateby.internal; import io.deephaven.engine.table.impl.updateby.internal.PairwiseFloatRingBuffer; import io.deephaven.test.types.OutOfBandTest; diff --git a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java index 2a68ba21de4..7359843c6b6 100644 --- a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java +++ b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java @@ -110,7 +110,7 @@ public static void main(String[] args) throws IOException { "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java", exemptions); ReplicatePrimitiveCode.floatToAllFloatingPoints( - "engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/PairwiseFloatRingBufferTest.java", + "engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBufferTest.java", exemptions); } From 26ae4b71bbceff14897dc1df4c8b594f712f44eb Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 3 Jan 2023 21:24:39 -0800 Subject: [PATCH 065/123] Added new base operator constructor for specifying timestamp column. UpdateBy cleanup --- .../deephaven/engine/table/impl/UpdateBy.java | 29 +++---------------- .../impl/UpdateByCumulativeOperator.java | 19 ++++++++++++ .../engine/table/impl/UpdateByOperator.java | 8 ++--- .../ema/BasePrimitiveEMAOperator.java | 5 +--- .../updateby/ema/BigDecimalEMAOperator.java | 2 +- .../updateby/ema/BigIntegerEMAOperator.java | 2 +- .../updateby/ema/BigNumberEMAOperator.java | 5 +--- .../impl/updateby/ema/ByteEMAOperator.java | 2 +- .../impl/updateby/ema/DoubleEMAOperator.java | 2 +- .../impl/updateby/ema/FloatEMAOperator.java | 2 +- .../impl/updateby/ema/IntEMAOperator.java | 2 +- .../impl/updateby/ema/LongEMAOperator.java | 2 +- .../impl/updateby/ema/ShortEMAOperator.java | 2 +- .../internal/BaseByteUpdateByOperator.java | 26 ++++++++++++++++- .../internal/BaseCharUpdateByOperator.java | 26 ++++++++++++++++- .../internal/BaseDoubleUpdateByOperator.java | 27 ++++++++++++++++- .../internal/BaseFloatUpdateByOperator.java | 27 ++++++++++++++++- .../internal/BaseIntUpdateByOperator.java | 26 ++++++++++++++++- .../internal/BaseLongUpdateByOperator.java | 26 ++++++++++++++++- .../internal/BaseObjectUpdateByOperator.java | 27 ++++++++++++++++- .../internal/BaseShortUpdateByOperator.java | 26 ++++++++++++++++- .../replicators/ReplicateUpdateBy.java | 3 +- 22 files changed, 242 insertions(+), 54 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java index 4077db96235..d16a49ce0b2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java @@ -90,9 +90,9 @@ public abstract class UpdateBy { public static class UpdateByRedirectionHelper { @Nullable - protected final WritableRowRedirection rowRedirection; - protected final WritableRowSet freeRows; - protected long maxInnerRowKey; + private final WritableRowRedirection rowRedirection; + private final WritableRowSet freeRows; + private long maxInnerRowKey; private UpdateByRedirectionHelper(@Nullable final WritableRowRedirection rowRedirection) { this.rowRedirection = rowRedirection; @@ -125,28 +125,7 @@ private void processUpdateForRedirection(@NotNull final TableUpdate upstream, if (upstream.shifted().nonempty()) { try (final WritableRowSet prevRowSetLessRemoves = sourceRowSet.copyPrev()) { prevRowSetLessRemoves.remove(upstream.removed()); - final RowSet.SearchIterator fwdIt = prevRowSetLessRemoves.searchIterator(); - - upstream.shifted().apply((start, end, delta) -> { - if (delta < 0 && fwdIt.advance(start)) { - for (long key = fwdIt.currentValue(); fwdIt.currentValue() <= end; key = fwdIt.nextLong()) { - if (shiftRedirectedKey(fwdIt, delta, key)) { - break; - } - } - } else { - try (final RowSet.SearchIterator revIt = prevRowSetLessRemoves.reverseIterator()) { - if (revIt.advance(end)) { - for (long key = revIt.currentValue(); revIt.currentValue() >= start; key = - revIt.nextLong()) { - if (shiftRedirectedKey(revIt, delta, key)) { - break; - } - } - } - } - } - }); + rowRedirection.applyShift(prevRowSetLessRemoves, upstream.shifted()); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java index 6da7b64f6a9..848d63fdede 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java @@ -8,6 +8,8 @@ import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; +import javax.annotation.Nullable; + import static io.deephaven.util.QueryConstants.NULL_LONG; public abstract class UpdateByCumulativeOperator extends UpdateByOperator { @@ -50,6 +52,23 @@ public UpdateByCumulativeOperator(@NotNull final MatchPair pair, super(pair, affectingColumns, null, null, 0L, 0L, redirHelper); } + /** + * An operator that computes a cumulative operation from a column while providing an optional timestamp column name + * and a + * + * @param pair the {@link MatchPair} that defines the input/output for this operation + * @param affectingColumns the names of the columns that affect this operation + * @param redirHelper the row redirection context to use for the operation + */ + public UpdateByCumulativeOperator(@NotNull final MatchPair pair, + @NotNull final String[] affectingColumns, + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, + @Nullable final String timestampColumnName, + @NotNull final long reverseTimeScaleUnits) { + super(pair, affectingColumns, null, timestampColumnName, reverseTimeScaleUnits, 0L, redirHelper); + } + + /** * Initialize the bucket context for this cumulative operator */ diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java index 24bce1160a8..ae178125265 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java @@ -39,10 +39,10 @@ public abstract class UpdateByOperator { protected final UpdateBy.UpdateByRedirectionHelper redirHelper; // these will be used by the timestamp-aware operators (EMA for example) - protected OperationControl control; - protected long reverseTimeScaleUnits; - protected long forwardTimeScaleUnits; - protected String timestampColumnName; + protected final OperationControl control; + protected final long reverseTimeScaleUnits; + protected final long forwardTimeScaleUnits; + protected final String timestampColumnName; /** * The input modifiedColumnSet for this operator diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java index c3907b4c621..e3b5723ca95 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java @@ -17,7 +17,6 @@ public abstract class BasePrimitiveEMAOperator extends BaseDoubleUpdateByOperator { protected final OperationControl control; - protected final double timeScaleUnits; protected final double alpha; protected double oneMinusAlpha; @@ -60,10 +59,8 @@ public BasePrimitiveEMAOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long timeScaleUnits, @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper) { - super(pair, affectingColumns, redirHelper); + super(pair, affectingColumns, redirHelper, timestampColumnName, timeScaleUnits); this.control = control; - this.timestampColumnName = timestampColumnName; - this.timeScaleUnits = (double) timeScaleUnits; alpha = Math.exp(-1.0 / (double) timeScaleUnits); oneMinusAlpha = 1 - alpha; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java index 85f710200c0..8a2a8e2d861 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java @@ -68,7 +68,7 @@ public void accumulate(RowSequence inputKeys, handleBadTime(this, dt); } else { // alpha is dynamic, based on time - BigDecimal alpha = BigDecimal.valueOf(Math.exp(-dt / timeScaleUnits)); + BigDecimal alpha = BigDecimal.valueOf(Math.exp(-dt / (double)reverseTimeScaleUnits)); BigDecimal oneMinusAlpha = BigDecimal.ONE.subtract(alpha, control.bigValueContextOrDefault()); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java index e241a17864b..450335c9edb 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java @@ -71,7 +71,7 @@ public void accumulate(RowSequence inputKeys, handleBadTime(this, dt); } else { // alpha is dynamic, based on time - BigDecimal alpha = BigDecimal.valueOf(Math.exp(-dt / timeScaleUnits)); + BigDecimal alpha = BigDecimal.valueOf(Math.exp(-dt / (double)reverseTimeScaleUnits)); BigDecimal oneMinusAlpha = BigDecimal.ONE.subtract(alpha, control.bigValueContextOrDefault()); curVal = curVal.multiply(alpha, control.bigValueContextOrDefault()) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java index a1bb12c402b..6d2eabc230d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java @@ -22,7 +22,6 @@ public abstract class BigNumberEMAOperator extends BaseObjectUpdateByOperator { protected final ColumnSource valueSource; protected final OperationControl control; - protected final double timeScaleUnits; protected final BigDecimal alpha; protected final BigDecimal oneMinusAlpha; @@ -75,11 +74,9 @@ public BigNumberEMAOperator(@NotNull final MatchPair pair, final long timeScaleUnits, @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, final ColumnSource valueSource) { - super(pair, affectingColumns, redirHelper, BigDecimal.class); + super(pair, affectingColumns, redirHelper, timestampColumnName, timeScaleUnits, BigDecimal.class); this.control = control; - this.timestampColumnName = timestampColumnName; - this.timeScaleUnits = (double) timeScaleUnits; this.valueSource = valueSource; alpha = BigDecimal.valueOf(Math.exp(-1.0 / (double) timeScaleUnits)); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java index 99565c5479c..bd043bf375c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java @@ -77,7 +77,7 @@ public void accumulate(RowSequence inputKeys, handleBadTime(this, dt); } else { // alpha is dynamic, based on time - final double alpha = Math.exp(-dt / timeScaleUnits); + final double alpha = Math.exp(-dt / (double)reverseTimeScaleUnits); curVal = alpha * curVal + ((1 - alpha) * input); lastStamp = timestamp; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java index 35b54f23ff4..3fe520338cb 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java @@ -85,7 +85,7 @@ public void accumulate(RowSequence inputKeys, if (dt <= 0) { handleBadTime(this, dt); } else if (!currentPoisoned) { - final double alpha = Math.exp(-dt / timeScaleUnits); + final double alpha = Math.exp(-dt / (double)reverseTimeScaleUnits); curVal = alpha * curVal + ((1 - alpha) * input); lastStamp = timestamp; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java index 79c9c27f3b2..cb1874feb29 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java @@ -80,7 +80,7 @@ public void accumulate(RowSequence inputKeys, if (dt <= 0) { handleBadTime(this, dt); } else if (!currentPoisoned) { - final double alpha = Math.exp(-dt / timeScaleUnits); + final double alpha = Math.exp(-dt / (double)reverseTimeScaleUnits); curVal = alpha * curVal + ((1 - alpha) * input); lastStamp = timestamp; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java index 4807d44f878..7d7c4988edf 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java @@ -77,7 +77,7 @@ public void accumulate(RowSequence inputKeys, handleBadTime(this, dt); } else { // alpha is dynamic, based on time - final double alpha = Math.exp(-dt / timeScaleUnits); + final double alpha = Math.exp(-dt / (double)reverseTimeScaleUnits); curVal = alpha * curVal + ((1 - alpha) * input); lastStamp = timestamp; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java index ba56d6152c0..36181b2b9d1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java @@ -77,7 +77,7 @@ public void accumulate(RowSequence inputKeys, handleBadTime(this, dt); } else { // alpha is dynamic, based on time - final double alpha = Math.exp(-dt / timeScaleUnits); + final double alpha = Math.exp(-dt / (double)reverseTimeScaleUnits); curVal = alpha * curVal + ((1 - alpha) * input); lastStamp = timestamp; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java index d209b07d1c5..4471cb3f678 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java @@ -72,7 +72,7 @@ public void accumulate(RowSequence inputKeys, handleBadTime(this, dt); } else { // alpha is dynamic, based on time - final double alpha = Math.exp(-dt / timeScaleUnits); + final double alpha = Math.exp(-dt / (double)reverseTimeScaleUnits); curVal = alpha * curVal + ((1 - alpha) * input); lastStamp = timestamp; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java index 8257c61e5ed..18f50b46c78 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java @@ -21,6 +21,7 @@ import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.util.Collections; import java.util.Map; @@ -109,8 +110,31 @@ public BaseByteUpdateByOperator(@NotNull final MatchPair pair, @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args + ) { + this(pair, affectingColumns, redirHelper, null, 0); + } + + /** + * Construct a base operator for operations that produce byte outputs. + * + * @param pair the {@link MatchPair} that defines the input/output for this operation + * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result + * of this operator. + * @param redirHelper the {@link UpdateBy.UpdateByRedirectionHelper} for the overall update + * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in + * integer ticks. + * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is + * measured in ticks, otherwise it is measured in nanoseconds. + */ + public BaseByteUpdateByOperator(@NotNull final MatchPair pair, + @NotNull final String[] affectingColumns, + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, + @Nullable final String timestampColumnName, + final long timeScaleUnits + // region extra-constructor-args + // endregion extra-constructor-args ) { - super(pair, affectingColumns, redirHelper); + super(pair, affectingColumns, redirHelper, timestampColumnName, timeScaleUnits); if(this.redirHelper.isRedirected()) { // region create-dense this.maybeInnerSource = makeDenseSource(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java index 03b0dcd9e4e..7af8e9dc26e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java @@ -11,6 +11,7 @@ import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.util.Collections; import java.util.Map; @@ -98,8 +99,31 @@ public BaseCharUpdateByOperator(@NotNull final MatchPair pair, @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args + ) { + this(pair, affectingColumns, redirHelper, null, 0); + } + + /** + * Construct a base operator for operations that produce char outputs. + * + * @param pair the {@link MatchPair} that defines the input/output for this operation + * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result + * of this operator. + * @param redirHelper the {@link UpdateBy.UpdateByRedirectionHelper} for the overall update + * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in + * integer ticks. + * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is + * measured in ticks, otherwise it is measured in nanoseconds. + */ + public BaseCharUpdateByOperator(@NotNull final MatchPair pair, + @NotNull final String[] affectingColumns, + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, + @Nullable final String timestampColumnName, + final long timeScaleUnits + // region extra-constructor-args + // endregion extra-constructor-args ) { - super(pair, affectingColumns, redirHelper); + super(pair, affectingColumns, redirHelper, timestampColumnName, timeScaleUnits); if(this.redirHelper.isRedirected()) { // region create-dense this.maybeInnerSource = new CharacterArraySource(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java index 96d9b181d4a..60e42b68bfa 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java @@ -15,6 +15,7 @@ import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.util.Collections; import java.util.Map; @@ -89,6 +90,7 @@ public void reset() { } } + /** * Construct a base operator for operations that produce double outputs. * @@ -102,8 +104,31 @@ public BaseDoubleUpdateByOperator(@NotNull final MatchPair pair, @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args + ) { + this(pair, affectingColumns, redirHelper, null, 0); + } + + /** + * Construct a base operator for operations that produce double outputs. + * + * @param pair the {@link MatchPair} that defines the input/output for this operation + * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result + * of this operator. + * @param redirHelper the {@link UpdateBy.UpdateByRedirectionHelper} for the overall update + * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in + * integer ticks. + * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is + * measured in ticks, otherwise it is measured in nanoseconds. + */ + public BaseDoubleUpdateByOperator(@NotNull final MatchPair pair, + @NotNull final String[] affectingColumns, + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, + @Nullable final String timestampColumnName, + final long timeScaleUnits + // region extra-constructor-args + // endregion extra-constructor-args ) { - super(pair, affectingColumns, redirHelper); + super(pair, affectingColumns, redirHelper, timestampColumnName, timeScaleUnits); if(this.redirHelper.isRedirected()) { // region create-dense this.maybeInnerSource = new DoubleArraySource(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java index be69c5982af..5e7e7c5ae7a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java @@ -10,6 +10,7 @@ import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.util.Collections; import java.util.Map; @@ -84,6 +85,7 @@ public void reset() { } } + /** * Construct a base operator for operations that produce float outputs. * @@ -97,8 +99,31 @@ public BaseFloatUpdateByOperator(@NotNull final MatchPair pair, @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args + ) { + this(pair, affectingColumns, redirHelper, null, 0); + } + + /** + * Construct a base operator for operations that produce float outputs. + * + * @param pair the {@link MatchPair} that defines the input/output for this operation + * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result + * of this operator. + * @param redirHelper the {@link UpdateBy.UpdateByRedirectionHelper} for the overall update + * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in + * integer ticks. + * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is + * measured in ticks, otherwise it is measured in nanoseconds. + */ + public BaseFloatUpdateByOperator(@NotNull final MatchPair pair, + @NotNull final String[] affectingColumns, + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, + @Nullable final String timestampColumnName, + final long timeScaleUnits + // region extra-constructor-args + // endregion extra-constructor-args ) { - super(pair, affectingColumns, redirHelper); + super(pair, affectingColumns, redirHelper, timestampColumnName, timeScaleUnits); if(this.redirHelper.isRedirected()) { // region create-dense this.maybeInnerSource = new FloatArraySource(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java index c117acb1cf8..3fb666197af 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java @@ -16,6 +16,7 @@ import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.util.Collections; import java.util.Map; @@ -103,8 +104,31 @@ public BaseIntUpdateByOperator(@NotNull final MatchPair pair, @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args + ) { + this(pair, affectingColumns, redirHelper, null, 0); + } + + /** + * Construct a base operator for operations that produce int outputs. + * + * @param pair the {@link MatchPair} that defines the input/output for this operation + * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result + * of this operator. + * @param redirHelper the {@link UpdateBy.UpdateByRedirectionHelper} for the overall update + * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in + * integer ticks. + * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is + * measured in ticks, otherwise it is measured in nanoseconds. + */ + public BaseIntUpdateByOperator(@NotNull final MatchPair pair, + @NotNull final String[] affectingColumns, + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, + @Nullable final String timestampColumnName, + final long timeScaleUnits + // region extra-constructor-args + // endregion extra-constructor-args ) { - super(pair, affectingColumns, redirHelper); + super(pair, affectingColumns, redirHelper, timestampColumnName, timeScaleUnits); if(this.redirHelper.isRedirected()) { // region create-dense this.maybeInnerSource = new IntegerArraySource(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java index 6a2512ab3c2..0818064b48e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java @@ -16,6 +16,7 @@ import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.util.Collections; import java.util.Map; @@ -103,8 +104,31 @@ public BaseLongUpdateByOperator(@NotNull final MatchPair pair, @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args + ) { + this(pair, affectingColumns, redirHelper, null, 0); + } + + /** + * Construct a base operator for operations that produce long outputs. + * + * @param pair the {@link MatchPair} that defines the input/output for this operation + * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result + * of this operator. + * @param redirHelper the {@link UpdateBy.UpdateByRedirectionHelper} for the overall update + * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in + * integer ticks. + * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is + * measured in ticks, otherwise it is measured in nanoseconds. + */ + public BaseLongUpdateByOperator(@NotNull final MatchPair pair, + @NotNull final String[] affectingColumns, + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, + @Nullable final String timestampColumnName, + final long timeScaleUnits + // region extra-constructor-args + // endregion extra-constructor-args ) { - super(pair, affectingColumns, redirHelper); + super(pair, affectingColumns, redirHelper, timestampColumnName, timeScaleUnits); if(this.redirHelper.isRedirected()) { // region create-dense this.maybeInnerSource = new LongArraySource(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java index 7c058aff5e4..361aaf62cb6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java @@ -16,6 +16,7 @@ import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.util.Collections; import java.util.Map; @@ -105,8 +106,32 @@ public BaseObjectUpdateByOperator(@NotNull final MatchPair pair, // region extra-constructor-args , final Class colType // endregion extra-constructor-args + ) { + this(pair, affectingColumns, redirHelper, null, 0, colType); + } + + /** + * Construct a base operator for operations that produce Object outputs. + * + * @param pair the {@link MatchPair} that defines the input/output for this operation + * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result + * of this operator. + * @param redirHelper the {@link UpdateBy.UpdateByRedirectionHelper} for the overall update + * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in + * integer ticks. + * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is + * measured in ticks, otherwise it is measured in nanoseconds. + */ + public BaseObjectUpdateByOperator(@NotNull final MatchPair pair, + @NotNull final String[] affectingColumns, + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, + @Nullable final String timestampColumnName, + final long timeScaleUnits + // region extra-constructor-args + , final Class colType + // endregion extra-constructor-args ) { - super(pair, affectingColumns, redirHelper); + super(pair, affectingColumns, redirHelper, timestampColumnName, timeScaleUnits); if(this.redirHelper.isRedirected()) { // region create-dense this.maybeInnerSource = new ObjectArraySource(colType); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java index fcf5b10aeed..42efd265094 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java @@ -16,6 +16,7 @@ import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.util.Collections; import java.util.Map; @@ -103,8 +104,31 @@ public BaseShortUpdateByOperator(@NotNull final MatchPair pair, @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper // region extra-constructor-args // endregion extra-constructor-args + ) { + this(pair, affectingColumns, redirHelper, null, 0); + } + + /** + * Construct a base operator for operations that produce short outputs. + * + * @param pair the {@link MatchPair} that defines the input/output for this operation + * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result + * of this operator. + * @param redirHelper the {@link UpdateBy.UpdateByRedirectionHelper} for the overall update + * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in + * integer ticks. + * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is + * measured in ticks, otherwise it is measured in nanoseconds. + */ + public BaseShortUpdateByOperator(@NotNull final MatchPair pair, + @NotNull final String[] affectingColumns, + @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, + @Nullable final String timestampColumnName, + final long timeScaleUnits + // region extra-constructor-args + // endregion extra-constructor-args ) { - super(pair, affectingColumns, redirHelper); + super(pair, affectingColumns, redirHelper, timestampColumnName, timeScaleUnits); if(this.redirHelper.isRedirected()) { // region create-dense this.maybeInnerSource = new ShortArraySource(); diff --git a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java index 7359843c6b6..569fed4affb 100644 --- a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java +++ b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java @@ -295,7 +295,8 @@ private static void fixupStandardObject(String objectResult, final String classN "Object previousVal", "T previousVal", "Object singletonVal", "T singletonVal", "getObject", "get", - "getPrevObject", "getPrev"); + "getPrevObject", "getPrev", + "this\\(pair, affectingColumns, redirHelper, null, 0\\);", "this(pair, affectingColumns, redirHelper, null, 0, colType);"); if (extraReplacements != null && extraReplacements.length > 0) { lines = globalReplacements(lines, extraReplacements); } From 5813d962ffc4d9ecb6d4f7f235a62f9353ccd133 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 3 Jan 2023 22:44:08 -0800 Subject: [PATCH 066/123] Refactored UpdateByRedirectionHelper and operators to use rowRedirection directly --- .../BucketedPartitionedUpdateByManager.java | 9 +- .../deephaven/engine/table/impl/UpdateBy.java | 17 ++- .../table/impl/UpdateByBucketHelper.java | 9 +- .../impl/UpdateByCumulativeOperator.java | 15 +-- .../engine/table/impl/UpdateByOperator.java | 7 +- .../table/impl/UpdateByOperatorFactory.java | 107 +++++++++--------- .../table/impl/UpdateByWindowedOperator.java | 7 +- .../table/impl/ZeroKeyUpdateByManager.java | 13 ++- .../ema/BasePrimitiveEMAOperator.java | 7 +- .../updateby/ema/BigDecimalEMAOperator.java | 9 +- .../updateby/ema/BigIntegerEMAOperator.java | 7 +- .../updateby/ema/BigNumberEMAOperator.java | 5 +- .../impl/updateby/ema/ByteEMAOperator.java | 5 +- .../impl/updateby/ema/DoubleEMAOperator.java | 5 +- .../impl/updateby/ema/FloatEMAOperator.java | 5 +- .../impl/updateby/ema/IntEMAOperator.java | 5 +- .../impl/updateby/ema/LongEMAOperator.java | 5 +- .../impl/updateby/ema/ShortEMAOperator.java | 5 +- .../updateby/fill/BooleanFillByOperator.java | 6 +- .../updateby/fill/ByteFillByOperator.java | 6 +- .../updateby/fill/CharFillByOperator.java | 6 +- .../updateby/fill/DoubleFillByOperator.java | 6 +- .../updateby/fill/FloatFillByOperator.java | 6 +- .../impl/updateby/fill/IntFillByOperator.java | 6 +- .../updateby/fill/LongFillByOperator.java | 6 +- .../updateby/fill/ObjectFillByOperator.java | 6 +- .../updateby/fill/ShortFillByOperator.java | 6 +- .../internal/BaseByteUpdateByOperator.java | 21 ++-- .../internal/BaseCharUpdateByOperator.java | 21 ++-- .../internal/BaseDoubleUpdateByOperator.java | 21 ++-- .../internal/BaseFloatUpdateByOperator.java | 21 ++-- .../internal/BaseIntUpdateByOperator.java | 21 ++-- .../internal/BaseLongUpdateByOperator.java | 21 ++-- .../internal/BaseObjectBinaryOperator.java | 6 +- .../internal/BaseObjectUpdateByOperator.java | 21 ++-- .../internal/BaseShortUpdateByOperator.java | 21 ++-- .../BaseWindowedByteUpdateByOperator.java | 13 ++- .../BaseWindowedCharUpdateByOperator.java | 13 ++- .../BaseWindowedDoubleUpdateByOperator.java | 13 ++- .../BaseWindowedFloatUpdateByOperator.java | 13 ++- .../BaseWindowedIntUpdateByOperator.java | 13 ++- .../BaseWindowedLongUpdateByOperator.java | 13 ++- .../BaseWindowedObjectUpdateByOperator.java | 13 ++- .../BaseWindowedShortUpdateByOperator.java | 13 ++- .../minmax/ByteCumMinMaxOperator.java | 6 +- .../minmax/ComparableCumMinMaxOperator.java | 6 +- .../minmax/DoubleCumMinMaxOperator.java | 6 +- .../minmax/FloatCumMinMaxOperator.java | 6 +- .../updateby/minmax/IntCumMinMaxOperator.java | 6 +- .../minmax/LongCumMinMaxOperator.java | 6 +- .../minmax/ShortCumMinMaxOperator.java | 6 +- .../prod/BigDecimalCumProdOperator.java | 6 +- .../prod/BigIntegerCumProdOperator.java | 6 +- .../updateby/prod/ByteCumProdOperator.java | 6 +- .../updateby/prod/DoubleCumProdOperator.java | 6 +- .../updateby/prod/FloatCumProdOperator.java | 6 +- .../updateby/prod/IntCumProdOperator.java | 6 +- .../updateby/prod/LongCumProdOperator.java | 6 +- .../updateby/prod/ShortCumProdOperator.java | 6 +- .../BigDecimalRollingSumOperator.java | 5 +- .../BigIntegerRollingSumOperator.java | 5 +- .../rollingsum/ByteRollingSumOperator.java | 5 +- .../rollingsum/DoubleRollingSumOperator.java | 5 +- .../rollingsum/FloatRollingSumOperator.java | 5 +- .../rollingsum/IntRollingSumOperator.java | 5 +- .../rollingsum/LongRollingSumOperator.java | 5 +- .../rollingsum/ShortRollingSumOperator.java | 5 +- .../sum/BigDecimalCumSumOperator.java | 6 +- .../sum/BigIntegerCumSumOperator.java | 6 +- .../impl/updateby/sum/ByteCumSumOperator.java | 6 +- .../updateby/sum/DoubleCumSumOperator.java | 6 +- .../updateby/sum/FloatCumSumOperator.java | 6 +- .../impl/updateby/sum/IntCumSumOperator.java | 6 +- .../impl/updateby/sum/LongCumSumOperator.java | 6 +- .../updateby/sum/ShortCumSumOperator.java | 6 +- .../replicators/ReplicateUpdateBy.java | 11 +- 76 files changed, 435 insertions(+), 328 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java index 92a56a265b9..82eb3b6baca 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java @@ -6,6 +6,7 @@ import io.deephaven.engine.rowset.*; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.updateby.UpdateByWindow; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -46,10 +47,10 @@ protected BucketedPartitionedUpdateByManager( @NotNull final Map> resultSources, @NotNull final Collection byColumns, @Nullable final String timestampColumnName, - @NotNull final UpdateByRedirectionHelper redirHelper, + @Nullable final WritableRowRedirection rowRedirection, @NotNull final UpdateByControl control) { - super(source, operators, windows, inputSources, operatorInputSourceSlots, timestampColumnName, redirHelper, - control); + super(source, operators, windows, inputSources, operatorInputSourceSlots, timestampColumnName, + rowRedirection, control); // this table will always have the rowset of the source result = new QueryTable(source.getRowSet(), resultSources); @@ -85,7 +86,7 @@ protected BucketedPartitionedUpdateByManager( operatorInputSourceSlots, resultSources, timestampColumnName, - redirHelper, + redirHelper.getRowRedirection(), control); // add this to the bucket list diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java index d16a49ce0b2..ff3d211ce70 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java @@ -100,7 +100,7 @@ private UpdateByRedirectionHelper(@Nullable final WritableRowRedirection rowRedi this.maxInnerRowKey = 0; } - public boolean isRedirected() { + boolean isRedirected() { return rowRedirection != null; } @@ -109,7 +109,7 @@ private long requiredCapacity() { } @Nullable - public WritableRowRedirection getRowRedirection() { + WritableRowRedirection getRowRedirection() { return rowRedirection; } @@ -177,7 +177,7 @@ protected UpdateBy( @NotNull final ColumnSource[] inputSources, @NotNull final int[][] operatorInputSourceSlots, @Nullable String timestampColumnName, - @NotNull final UpdateByRedirectionHelper redirHelper, + @Nullable final WritableRowRedirection rowRedirection, @NotNull final UpdateByControl control) { if (operators.length == 0) { @@ -190,7 +190,7 @@ protected UpdateBy( this.inputSources = inputSources; this.operatorInputSourceSlots = operatorInputSourceSlots; this.timestampColumnName = timestampColumnName; - this.redirHelper = redirHelper; + this.redirHelper = new UpdateByRedirectionHelper(rowRedirection); this.control = control; this.inputSourceCacheNeeded = new boolean[inputSources.length]; @@ -837,15 +837,12 @@ public static Table updateBy(@NotNull final QueryTable source, rowRedirection = null; } - // create an UpdateByRedirectionHelper for use by the UpdateByBucketHelper objects - UpdateByRedirectionHelper redirHelper = new UpdateByRedirectionHelper(rowRedirection); - // TODO(deephaven-core#2693): Improve UpdateByBucketHelper implementation for ColumnName // generate a MatchPair array for use by the existing algorithm MatchPair[] pairs = MatchPair.fromPairs(byColumns); final UpdateByOperatorFactory updateByOperatorFactory = - new UpdateByOperatorFactory(source, pairs, redirHelper, control); + new UpdateByOperatorFactory(source, pairs, rowRedirection, control); final Collection ops = updateByOperatorFactory.getOperators(clauses); final UpdateByOperator[] opArr = ops.toArray(UpdateByOperator.ZERO_LENGTH_OP_ARRAY); @@ -968,7 +965,7 @@ public static Table updateBy(@NotNull final QueryTable source, source, resultSources, timestampColumnName, - redirHelper, + rowRedirection, control); if (source.isRefreshing()) { @@ -1004,7 +1001,7 @@ public static Table updateBy(@NotNull final QueryTable source, resultSources, byColumns, timestampColumnName, - redirHelper, + rowRedirection, control); if (source.isRefreshing()) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java index e7724ff28d7..99d353a583f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java @@ -10,6 +10,7 @@ import io.deephaven.engine.table.impl.sources.ReinterpretUtils; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.updateby.UpdateByWindow; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import io.deephaven.util.datastructures.linked.IntrusiveDoublyLinkedNode; import org.apache.commons.lang3.mutable.MutableLong; import org.jetbrains.annotations.NotNull; @@ -30,7 +31,7 @@ class UpdateByBucketHelper extends IntrusiveDoublyLinkedNode.Impl> resultSources, @Nullable String timestampColumnName, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, + @Nullable final WritableRowRedirection rowRedirection, @NotNull final UpdateByControl control) { this.source = source; @@ -76,7 +77,7 @@ protected UpdateByBucketHelper(@NotNull final String description, this.windows = windows; this.inputSources = inputSources; this.operatorInputSourceSlots = operatorInputSourceSlots; - this.redirHelper = redirHelper; + this.rowRedirection = rowRedirection; this.control = control; result = new QueryTable(source.getRowSet(), resultSources); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java index 848d63fdede..b1f5d316310 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java @@ -5,6 +5,7 @@ import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; @@ -48,8 +49,8 @@ public abstract void accumulate(RowSequence inputKeys, */ public UpdateByCumulativeOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper) { - super(pair, affectingColumns, null, null, 0L, 0L, redirHelper); + @Nullable final WritableRowRedirection rowRedirection) { + super(pair, affectingColumns, null, null, 0L, 0L, rowRedirection); } /** @@ -61,11 +62,11 @@ public UpdateByCumulativeOperator(@NotNull final MatchPair pair, * @param redirHelper the row redirection context to use for the operation */ public UpdateByCumulativeOperator(@NotNull final MatchPair pair, - @NotNull final String[] affectingColumns, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, - @Nullable final String timestampColumnName, - @NotNull final long reverseTimeScaleUnits) { - super(pair, affectingColumns, null, timestampColumnName, reverseTimeScaleUnits, 0L, redirHelper); + @NotNull final String[] affectingColumns, + @Nullable final WritableRowRedirection rowRedirection, + @Nullable final String timestampColumnName, + @NotNull final long reverseTimeScaleUnits) { + super(pair, affectingColumns, null, timestampColumnName, reverseTimeScaleUnits, 0L, rowRedirection); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java index ae178125265..13246c1594e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java @@ -11,6 +11,7 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.ModifiedColumnSet; import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import io.deephaven.util.SafeCloseable; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -36,7 +37,7 @@ public abstract class UpdateByOperator { protected final MatchPair pair; protected final String[] affectingColumns; - protected final UpdateBy.UpdateByRedirectionHelper redirHelper; + protected final WritableRowRedirection rowRedirection; // these will be used by the timestamp-aware operators (EMA for example) protected final OperationControl control; @@ -102,10 +103,10 @@ protected UpdateByOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper) { + @Nullable final WritableRowRedirection rowRedirection) { this.pair = pair; this.affectingColumns = affectingColumns; - this.redirHelper = redirHelper; + this.rowRedirection = rowRedirection; this.timestampColumnName = timestampColumnName; this.control = control; this.reverseTimeScaleUnits = reverseTimeScaleUnits; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java index cc77bfd0a40..da7e3f6023e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java @@ -15,6 +15,7 @@ import io.deephaven.engine.table.impl.updateby.prod.*; import io.deephaven.engine.table.impl.updateby.rollingsum.*; import io.deephaven.engine.table.impl.updateby.sum.*; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import io.deephaven.time.DateTime; import org.jetbrains.annotations.NotNull; @@ -34,17 +35,17 @@ public class UpdateByOperatorFactory { private final TableDefaults source; private final MatchPair[] groupByColumns; @NotNull - private final UpdateBy.UpdateByRedirectionHelper redirHelper; + private final WritableRowRedirection rowRedirection; @NotNull private final UpdateByControl control; public UpdateByOperatorFactory(@NotNull final TableDefaults source, @NotNull final MatchPair[] groupByColumns, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, + @NotNull final WritableRowRedirection rowRedirection, @NotNull final UpdateByControl control) { this.source = source; this.groupByColumns = groupByColumns; - this.redirHelper = redirHelper; + this.rowRedirection = rowRedirection; this.control = control; } @@ -228,28 +229,28 @@ private UpdateByOperator makeEmaOperator(@NotNull final MatchPair pair, if (csType == byte.class || csType == Byte.class) { return new ByteEMAOperator(pair, affectingColumns, control, - ema.timeScale().timestampCol(), timeScaleUnits, redirHelper, columnSource); + ema.timeScale().timestampCol(), timeScaleUnits, rowRedirection, columnSource); } else if (csType == short.class || csType == Short.class) { return new ShortEMAOperator(pair, affectingColumns, control, - ema.timeScale().timestampCol(), timeScaleUnits, redirHelper, columnSource); + ema.timeScale().timestampCol(), timeScaleUnits, rowRedirection, columnSource); } else if (csType == int.class || csType == Integer.class) { return new IntEMAOperator(pair, affectingColumns, control, - ema.timeScale().timestampCol(), timeScaleUnits, redirHelper, columnSource); + ema.timeScale().timestampCol(), timeScaleUnits, rowRedirection, columnSource); } else if (csType == long.class || csType == Long.class) { return new LongEMAOperator(pair, affectingColumns, control, - ema.timeScale().timestampCol(), timeScaleUnits, redirHelper, columnSource); + ema.timeScale().timestampCol(), timeScaleUnits, rowRedirection, columnSource); } else if (csType == float.class || csType == Float.class) { return new FloatEMAOperator(pair, affectingColumns, control, - ema.timeScale().timestampCol(), timeScaleUnits, redirHelper, columnSource); + ema.timeScale().timestampCol(), timeScaleUnits, rowRedirection, columnSource); } else if (csType == double.class || csType == Double.class) { return new DoubleEMAOperator(pair, affectingColumns, control, - ema.timeScale().timestampCol(), timeScaleUnits, redirHelper, columnSource); + ema.timeScale().timestampCol(), timeScaleUnits, rowRedirection, columnSource); } else if (csType == BigDecimal.class) { return new BigDecimalEMAOperator(pair, affectingColumns, control, - ema.timeScale().timestampCol(), timeScaleUnits, redirHelper, columnSource); + ema.timeScale().timestampCol(), timeScaleUnits, rowRedirection, columnSource); } else if (csType == BigInteger.class) { return new BigIntegerEMAOperator(pair, affectingColumns, control, - ema.timeScale().timestampCol(), timeScaleUnits, redirHelper, columnSource); + ema.timeScale().timestampCol(), timeScaleUnits, rowRedirection, columnSource); } throw new IllegalArgumentException("Can not perform EMA on type " + csType); @@ -258,21 +259,21 @@ private UpdateByOperator makeEmaOperator(@NotNull final MatchPair pair, private UpdateByOperator makeCumProdOperator(MatchPair fc, TableDefaults source) { final Class csType = source.getColumnSource(fc.rightColumn).getType(); if (csType == byte.class || csType == Byte.class) { - return new ByteCumProdOperator(fc, redirHelper); + return new ByteCumProdOperator(fc, rowRedirection); } else if (csType == short.class || csType == Short.class) { - return new ShortCumProdOperator(fc, redirHelper); + return new ShortCumProdOperator(fc, rowRedirection); } else if (csType == int.class || csType == Integer.class) { - return new IntCumProdOperator(fc, redirHelper); + return new IntCumProdOperator(fc, rowRedirection); } else if (csType == long.class || csType == Long.class) { - return new LongCumProdOperator(fc, redirHelper); + return new LongCumProdOperator(fc, rowRedirection); } else if (csType == float.class || csType == Float.class) { - return new FloatCumProdOperator(fc, redirHelper); + return new FloatCumProdOperator(fc, rowRedirection); } else if (csType == double.class || csType == Double.class) { - return new DoubleCumProdOperator(fc, redirHelper); + return new DoubleCumProdOperator(fc, rowRedirection); } else if (csType == BigDecimal.class) { - return new BigDecimalCumProdOperator(fc, redirHelper, control.mathContextOrDefault()); + return new BigDecimalCumProdOperator(fc, rowRedirection, control.mathContextOrDefault()); } else if (csType == BigInteger.class) { - return new BigIntegerCumProdOperator(fc, redirHelper); + return new BigIntegerCumProdOperator(fc, rowRedirection); } throw new IllegalArgumentException("Can not perform Cumulative Min/Max on type " + csType); @@ -282,20 +283,20 @@ private UpdateByOperator makeCumMinMaxOperator(MatchPair fc, TableDefaults sourc final ColumnSource columnSource = source.getColumnSource(fc.rightColumn); final Class csType = columnSource.getType(); if (csType == byte.class || csType == Byte.class) { - return new ByteCumMinMaxOperator(fc, isMax, redirHelper); + return new ByteCumMinMaxOperator(fc, isMax, rowRedirection); } else if (csType == short.class || csType == Short.class) { - return new ShortCumMinMaxOperator(fc, isMax, redirHelper); + return new ShortCumMinMaxOperator(fc, isMax, rowRedirection); } else if (csType == int.class || csType == Integer.class) { - return new IntCumMinMaxOperator(fc, isMax, redirHelper); + return new IntCumMinMaxOperator(fc, isMax, rowRedirection); } else if (csType == long.class || csType == Long.class || isTimeType(csType)) { - return new LongCumMinMaxOperator(fc, isMax, redirHelper, csType); + return new LongCumMinMaxOperator(fc, isMax, rowRedirection, csType); } else if (csType == float.class || csType == Float.class) { - return new FloatCumMinMaxOperator(fc, isMax, redirHelper); + return new FloatCumMinMaxOperator(fc, isMax, rowRedirection); } else if (csType == double.class || csType == Double.class) { - return new DoubleCumMinMaxOperator(fc, isMax, redirHelper); + return new DoubleCumMinMaxOperator(fc, isMax, rowRedirection); } else if (Comparable.class.isAssignableFrom(csType)) { // noinspection unchecked,rawtypes - return new ComparableCumMinMaxOperator(csType, fc, isMax, redirHelper); + return new ComparableCumMinMaxOperator(csType, fc, isMax, rowRedirection); } throw new IllegalArgumentException("Can not perform Cumulative Min/Max on type " + csType); @@ -304,23 +305,23 @@ private UpdateByOperator makeCumMinMaxOperator(MatchPair fc, TableDefaults sourc private UpdateByOperator makeCumSumOperator(MatchPair fc, TableDefaults source) { final Class csType = source.getColumnSource(fc.rightColumn).getType(); if (csType == Boolean.class || csType == boolean.class) { - return new ByteCumSumOperator(fc, redirHelper, NULL_BOOLEAN_AS_BYTE); + return new ByteCumSumOperator(fc, rowRedirection, NULL_BOOLEAN_AS_BYTE); } else if (csType == byte.class || csType == Byte.class) { - return new ByteCumSumOperator(fc, redirHelper, NULL_BYTE); + return new ByteCumSumOperator(fc, rowRedirection, NULL_BYTE); } else if (csType == short.class || csType == Short.class) { - return new ShortCumSumOperator(fc, redirHelper); + return new ShortCumSumOperator(fc, rowRedirection); } else if (csType == int.class || csType == Integer.class) { - return new IntCumSumOperator(fc, redirHelper); + return new IntCumSumOperator(fc, rowRedirection); } else if (csType == long.class || csType == Long.class) { - return new LongCumSumOperator(fc, redirHelper); + return new LongCumSumOperator(fc, rowRedirection); } else if (csType == float.class || csType == Float.class) { - return new FloatCumSumOperator(fc, redirHelper); + return new FloatCumSumOperator(fc, rowRedirection); } else if (csType == double.class || csType == Double.class) { - return new DoubleCumSumOperator(fc, redirHelper); + return new DoubleCumSumOperator(fc, rowRedirection); } else if (csType == BigDecimal.class) { - return new BigDecimalCumSumOperator(fc, redirHelper, control.mathContextOrDefault()); + return new BigDecimalCumSumOperator(fc, rowRedirection, control.mathContextOrDefault()); } else if (csType == BigInteger.class) { - return new BigIntegerCumSumOperator(fc, redirHelper); + return new BigIntegerCumSumOperator(fc, rowRedirection); } throw new IllegalArgumentException("Can not perform Cumulative Sum on type " + csType); @@ -330,23 +331,23 @@ private UpdateByOperator makeForwardFillOperator(MatchPair fc, TableDefaults sou final ColumnSource columnSource = source.getColumnSource(fc.rightColumn); final Class csType = columnSource.getType(); if (csType == char.class || csType == Character.class) { - return new CharFillByOperator(fc, redirHelper); + return new CharFillByOperator(fc, rowRedirection); } else if (csType == byte.class || csType == Byte.class) { - return new ByteFillByOperator(fc, redirHelper); + return new ByteFillByOperator(fc, rowRedirection); } else if (csType == short.class || csType == Short.class) { - return new ShortFillByOperator(fc, redirHelper); + return new ShortFillByOperator(fc, rowRedirection); } else if (csType == int.class || csType == Integer.class) { - return new IntFillByOperator(fc, redirHelper); + return new IntFillByOperator(fc, rowRedirection); } else if (csType == long.class || csType == Long.class || isTimeType(csType)) { - return new LongFillByOperator(fc, redirHelper, csType); + return new LongFillByOperator(fc, rowRedirection, csType); } else if (csType == float.class || csType == Float.class) { - return new FloatFillByOperator(fc, redirHelper); + return new FloatFillByOperator(fc, rowRedirection); } else if (csType == double.class || csType == Double.class) { - return new DoubleFillByOperator(fc, redirHelper); + return new DoubleFillByOperator(fc, rowRedirection); } else if (csType == boolean.class || csType == Boolean.class) { - return new BooleanFillByOperator(fc, redirHelper); + return new BooleanFillByOperator(fc, rowRedirection); } else { - return new ObjectFillByOperator<>(fc, redirHelper, csType); + return new ObjectFillByOperator<>(fc, rowRedirection, csType); } } @@ -370,40 +371,40 @@ private UpdateByOperator makeRollingSumOperator(@NotNull final MatchPair pair, if (csType == Boolean.class || csType == boolean.class) { return new ByteRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, - redirHelper, NULL_BOOLEAN_AS_BYTE); + rowRedirection, NULL_BOOLEAN_AS_BYTE); } else if (csType == byte.class || csType == Byte.class) { return new ByteRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, - redirHelper, NULL_BYTE); + rowRedirection, NULL_BYTE); } else if (csType == short.class || csType == Short.class) { return new ShortRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, - redirHelper); + rowRedirection); } else if (csType == int.class || csType == Integer.class) { return new IntRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, - redirHelper); + rowRedirection); } else if (csType == long.class || csType == Long.class) { return new LongRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, - redirHelper); + rowRedirection); } else if (csType == float.class || csType == Float.class) { return new FloatRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, - redirHelper); + rowRedirection); } else if (csType == double.class || csType == Double.class) { return new DoubleRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), - prevTimeScaleUnits, fwdTimeScaleUnits, redirHelper); + prevTimeScaleUnits, fwdTimeScaleUnits, rowRedirection); } else if (csType == BigDecimal.class) { return new BigDecimalRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), - prevTimeScaleUnits, fwdTimeScaleUnits, redirHelper, + prevTimeScaleUnits, fwdTimeScaleUnits, rowRedirection, control.mathContextOrDefault()); } else if (csType == BigInteger.class) { return new BigIntegerRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), rs.prevTimeScale().timestampCol(), - prevTimeScaleUnits, fwdTimeScaleUnits, redirHelper); + prevTimeScaleUnits, fwdTimeScaleUnits, rowRedirection); } throw new IllegalArgumentException("Can not perform RollingSum on type " + csType); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java index 15ff3895e22..aa7be8cc9b7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java @@ -6,6 +6,7 @@ import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -33,7 +34,7 @@ public abstract void accumulate(RowSequence inputKeys, * @param timestampColumnName the optional time stamp column for windowing (uses ticks if not provided) * @param reverseTimeScaleUnits the time (us) or ticks to extend the window backwards * @param forwardTimeScaleUnits the time (us) or ticks to extend the window forwards - * @param redirHelper the row redirection context to use for the operation + * @param rowRedirection the row redirection to use for the operator output columns */ public UpdateByWindowedOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @@ -41,9 +42,9 @@ public UpdateByWindowedOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper) { + @Nullable final WritableRowRedirection rowRedirection) { super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, - redirHelper); + rowRedirection); } /** diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateByManager.java index 653e291b44e..5ec689c26f1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateByManager.java @@ -4,6 +4,7 @@ import io.deephaven.engine.rowset.*; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.updateby.UpdateByWindow; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -28,7 +29,7 @@ public class ZeroKeyUpdateByManager extends UpdateBy { * @param source the source table * @param resultSources the result sources * @param timestampColumnName the column to use for all time-aware operators - * @param redirHelper the row redirection helper for dense output sources + * @param rowRedirection the row redirection for dense output sources * @param control the control object. */ protected ZeroKeyUpdateByManager( @@ -40,9 +41,9 @@ protected ZeroKeyUpdateByManager( @NotNull QueryTable source, @NotNull final Map> resultSources, @Nullable String timestampColumnName, - @NotNull UpdateByRedirectionHelper redirHelper, + @Nullable WritableRowRedirection rowRedirection, @NotNull UpdateByControl control) { - super(source, operators, windows, inputSources, operatorInputSourceSlots, timestampColumnName, redirHelper, + super(source, operators, windows, inputSources, operatorInputSourceSlots, timestampColumnName, rowRedirection, control); if (source.isRefreshing()) { @@ -62,7 +63,8 @@ protected ZeroKeyUpdateByManager( // create an updateby bucket instance directly from the source table zeroKeyUpdateBy = new UpdateByBucketHelper(description, source, operators, windows, inputSources, - operatorInputSourceSlots, resultSources, timestampColumnName, redirHelper, control); + operatorInputSourceSlots, resultSources, timestampColumnName, redirHelper.getRowRedirection(), + control); buckets.offer(zeroKeyUpdateBy); // make the source->result transformer @@ -72,7 +74,8 @@ protected ZeroKeyUpdateByManager( result.addParentReference(zeroKeyUpdateBy); } else { zeroKeyUpdateBy = new UpdateByBucketHelper(description, source, operators, windows, inputSources, - operatorInputSourceSlots, resultSources, timestampColumnName, redirHelper, control); + operatorInputSourceSlots, resultSources, timestampColumnName, redirHelper.getRowRedirection(), + control); result = zeroKeyUpdateBy.result; buckets.offer(zeroKeyUpdateBy); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java index e3b5723ca95..f2ab189e6ff 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java @@ -8,6 +8,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.updateby.internal.BaseDoubleUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -51,15 +52,15 @@ public void reset() { * integer ticks. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is * measured in ticks, otherwise it is measured in nanoseconds. - * @param redirHelper the row redirection context to use for the EMA + * @param rowRedirection the row redirection to use for the EMA output columns */ public BasePrimitiveEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper) { - super(pair, affectingColumns, redirHelper, timestampColumnName, timeScaleUnits); + @Nullable final WritableRowRedirection rowRedirection) { + super(pair, affectingColumns, rowRedirection, timestampColumnName, timeScaleUnits); this.control = control; alpha = Math.exp(-1.0 / (double) timeScaleUnits); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java index 8a2a8e2d861..6d5fd4b3b12 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java @@ -8,6 +8,7 @@ import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -68,7 +69,7 @@ public void accumulate(RowSequence inputKeys, handleBadTime(this, dt); } else { // alpha is dynamic, based on time - BigDecimal alpha = BigDecimal.valueOf(Math.exp(-dt / (double)reverseTimeScaleUnits)); + BigDecimal alpha = BigDecimal.valueOf(Math.exp(-dt / (double) reverseTimeScaleUnits)); BigDecimal oneMinusAlpha = BigDecimal.ONE.subtract(alpha, control.bigValueContextOrDefault()); @@ -103,16 +104,16 @@ public void push(long key, int pos) { * integer ticks. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timeRecorder} is provided, this is measured * in ticks, otherwise it is measured in nanoseconds - * @param redirHelper the row redirection context to use for the EMA + * @param rowRedirection the row redirection for the EMA output column */ public BigDecimalEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, + @Nullable final WritableRowRedirection rowRedirection, final ColumnSource valueSource) { - super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, redirHelper, valueSource); + super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, rowRedirection, valueSource); } @NotNull diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java index 450335c9edb..6d178c48574 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java @@ -8,6 +8,7 @@ import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -105,16 +106,16 @@ public void push(long key, int pos) { * integer ticks. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timeRecorder} is provided, this is measured * in ticks, otherwise it is measured in nanoseconds - * @param redirHelper the row redirection context to use for the EMA + * @param rowRedirection the row redirection for the EMA output column */ public BigIntegerEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, + @Nullable final WritableRowRedirection rowRedirection, final ColumnSource valueSource) { - super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, redirHelper, valueSource); + super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, rowRedirection, valueSource); } @NotNull diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java index 6d2eabc230d..11864c20471 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java @@ -11,6 +11,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -72,9 +73,9 @@ public BigNumberEMAOperator(@NotNull final MatchPair pair, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, + @Nullable final WritableRowRedirection rowRedirection, final ColumnSource valueSource) { - super(pair, affectingColumns, redirHelper, timestampColumnName, timeScaleUnits, BigDecimal.class); + super(pair, affectingColumns, rowRedirection, timestampColumnName, timeScaleUnits, BigDecimal.class); this.control = control; this.valueSource = valueSource; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java index bd043bf375c..c6b238fc44b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java @@ -14,6 +14,7 @@ import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -121,12 +122,12 @@ public ByteEMAOperator(@NotNull final MatchPair pair, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, + @Nullable final WritableRowRedirection rowRedirection, final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits,redirHelper); + super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, rowRedirection); this.valueSource = valueSource; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java index 3fe520338cb..2ff5ab481af 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java @@ -15,6 +15,7 @@ import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -137,12 +138,12 @@ public DoubleEMAOperator(@NotNull final MatchPair pair, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, + @Nullable final WritableRowRedirection rowRedirection, final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, redirHelper); + super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, rowRedirection); this.valueSource = valueSource; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java index cb1874feb29..ad044cabdae 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java @@ -10,6 +10,7 @@ import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -132,12 +133,12 @@ public FloatEMAOperator(@NotNull final MatchPair pair, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, + @Nullable final WritableRowRedirection rowRedirection, final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, redirHelper); + super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, rowRedirection); this.valueSource = valueSource; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java index 7d7c4988edf..eec63a73999 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java @@ -14,6 +14,7 @@ import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -121,12 +122,12 @@ public IntEMAOperator(@NotNull final MatchPair pair, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, + @Nullable final WritableRowRedirection rowRedirection, final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits,redirHelper); + super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, rowRedirection); this.valueSource = valueSource; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java index 36181b2b9d1..ace39363324 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java @@ -14,6 +14,7 @@ import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -121,12 +122,12 @@ public LongEMAOperator(@NotNull final MatchPair pair, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, + @Nullable final WritableRowRedirection rowRedirection, final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits,redirHelper); + super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, rowRedirection); this.valueSource = valueSource; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java index 4471cb3f678..59e18e50acc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java @@ -9,6 +9,7 @@ import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -116,12 +117,12 @@ public ShortEMAOperator(@NotNull final MatchPair pair, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, + @Nullable final WritableRowRedirection rowRedirection, final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits,redirHelper); + super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, rowRedirection); this.valueSource = valueSource; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java index d61d76d1759..82b4c1e4fd0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java @@ -18,7 +18,9 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseByteUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import static io.deephaven.util.BooleanUtils.NULL_BOOLEAN_AS_BYTE; @@ -48,11 +50,11 @@ public void push(long key, int pos) { } public BooleanFillByOperator(@NotNull final MatchPair fillPair, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, redirHelper); + super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java index c1271d06ebd..2758a80741d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java @@ -11,7 +11,9 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseByteUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.NULL_BYTE; @@ -41,11 +43,11 @@ public void push(long key, int pos) { } public ByteFillByOperator(@NotNull final MatchPair fillPair, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, redirHelper); + super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java index b4326f8fc90..22892bc5d75 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java @@ -6,7 +6,9 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseCharUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.NULL_CHAR; @@ -36,11 +38,11 @@ public void push(long key, int pos) { } public CharFillByOperator(@NotNull final MatchPair fillPair, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, redirHelper); + super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java index 7245bc3a440..7692ecf41dc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java @@ -11,7 +11,9 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseDoubleUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.NULL_DOUBLE; @@ -41,11 +43,11 @@ public void push(long key, int pos) { } public DoubleFillByOperator(@NotNull final MatchPair fillPair, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, redirHelper); + super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java index db23fcec3f7..83603c27be6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java @@ -11,7 +11,9 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseFloatUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.NULL_FLOAT; @@ -41,11 +43,11 @@ public void push(long key, int pos) { } public FloatFillByOperator(@NotNull final MatchPair fillPair, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, redirHelper); + super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java index fd477ddcb34..c02822f5d94 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java @@ -11,7 +11,9 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseIntUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.NULL_INT; @@ -41,11 +43,11 @@ public void push(long key, int pos) { } public IntFillByOperator(@NotNull final MatchPair fillPair, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, redirHelper); + super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java index a436838ebcc..fa537de0d7d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java @@ -18,7 +18,9 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.NULL_LONG; @@ -49,12 +51,12 @@ public void push(long key, int pos) { } public LongFillByOperator(@NotNull final MatchPair fillPair, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args ,@NotNull final Class type // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, redirHelper); + super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection); // region constructor this.type = type; // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java index 033fd88117c..8ac7d27ed74 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java @@ -11,7 +11,9 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; public class ObjectFillByOperator extends BaseObjectUpdateByOperator { @@ -40,12 +42,12 @@ public void push(long key, int pos) { } public ObjectFillByOperator(@NotNull final MatchPair fillPair, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args , final Class colType // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, redirHelper, colType); + super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection, colType); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java index bfe6f5b54f5..b8b78a39a1a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java @@ -11,7 +11,9 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseShortUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.NULL_SHORT; @@ -41,11 +43,11 @@ public void push(long key, int pos) { } public ShortFillByOperator(@NotNull final MatchPair fillPair, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, redirHelper); + super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java index 18f50b46c78..c6497667fd7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java @@ -20,6 +20,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -103,15 +104,15 @@ public void reset() { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param redirHelper the {@link UpdateBy.UpdateByRedirectionHelper} for the overall update + * @param rowRedirection the {@link WritableRowRedirection} for the output column */ public BaseByteUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - this(pair, affectingColumns, redirHelper, null, 0); + this(pair, affectingColumns, rowRedirection, null, 0); } /** @@ -120,7 +121,7 @@ public BaseByteUpdateByOperator(@NotNull final MatchPair pair, * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param redirHelper the {@link UpdateBy.UpdateByRedirectionHelper} for the overall update + * @param rowRedirection the {@link WritableRowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is @@ -128,18 +129,18 @@ public BaseByteUpdateByOperator(@NotNull final MatchPair pair, */ public BaseByteUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, + @Nullable final WritableRowRedirection rowRedirection, @Nullable final String timestampColumnName, final long timeScaleUnits // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, redirHelper, timestampColumnName, timeScaleUnits); - if(this.redirHelper.isRedirected()) { + super(pair, affectingColumns, rowRedirection, timestampColumnName, timeScaleUnits); + if(rowRedirection != null) { // region create-dense this.maybeInnerSource = makeDenseSource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(this.redirHelper.getRowRedirection(), maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -181,7 +182,7 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (redirHelper.isRedirected()) { + if (rowRedirection != null) { maybeInnerSource.startTrackingPrevValues(); } } @@ -199,7 +200,7 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d @Override public void prepareForParallelPopulation(final RowSet changedRows) { - if (redirHelper.isRedirected()) { + if (rowRedirection != null) { ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java index 7af8e9dc26e..be25128af4d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java @@ -10,6 +10,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -92,15 +93,15 @@ public void reset() { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param redirHelper the {@link UpdateBy.UpdateByRedirectionHelper} for the overall update + * @param rowRedirection the {@link WritableRowRedirection} for the output column */ public BaseCharUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - this(pair, affectingColumns, redirHelper, null, 0); + this(pair, affectingColumns, rowRedirection, null, 0); } /** @@ -109,7 +110,7 @@ public BaseCharUpdateByOperator(@NotNull final MatchPair pair, * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param redirHelper the {@link UpdateBy.UpdateByRedirectionHelper} for the overall update + * @param rowRedirection the {@link WritableRowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is @@ -117,18 +118,18 @@ public BaseCharUpdateByOperator(@NotNull final MatchPair pair, */ public BaseCharUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, + @Nullable final WritableRowRedirection rowRedirection, @Nullable final String timestampColumnName, final long timeScaleUnits // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, redirHelper, timestampColumnName, timeScaleUnits); - if(this.redirHelper.isRedirected()) { + super(pair, affectingColumns, rowRedirection, timestampColumnName, timeScaleUnits); + if(rowRedirection != null) { // region create-dense this.maybeInnerSource = new CharacterArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(this.redirHelper.getRowRedirection(), maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -157,7 +158,7 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (redirHelper.isRedirected()) { + if (rowRedirection != null) { maybeInnerSource.startTrackingPrevValues(); } } @@ -171,7 +172,7 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long @Override public void prepareForParallelPopulation(final RowSet changedRows) { - if (redirHelper.isRedirected()) { + if (rowRedirection != null) { ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java index 60e42b68bfa..318a765cac1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java @@ -14,6 +14,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -97,15 +98,15 @@ public void reset() { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param redirHelper the {@link UpdateBy.UpdateByRedirectionHelper} for the overall update + * @param rowRedirection the {@link WritableRowRedirection} for the output column */ public BaseDoubleUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - this(pair, affectingColumns, redirHelper, null, 0); + this(pair, affectingColumns, rowRedirection, null, 0); } /** @@ -114,7 +115,7 @@ public BaseDoubleUpdateByOperator(@NotNull final MatchPair pair, * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param redirHelper the {@link UpdateBy.UpdateByRedirectionHelper} for the overall update + * @param rowRedirection the {@link WritableRowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is @@ -122,18 +123,18 @@ public BaseDoubleUpdateByOperator(@NotNull final MatchPair pair, */ public BaseDoubleUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, + @Nullable final WritableRowRedirection rowRedirection, @Nullable final String timestampColumnName, final long timeScaleUnits // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, redirHelper, timestampColumnName, timeScaleUnits); - if(this.redirHelper.isRedirected()) { + super(pair, affectingColumns, rowRedirection, timestampColumnName, timeScaleUnits); + if(rowRedirection != null) { // region create-dense this.maybeInnerSource = new DoubleArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(this.redirHelper.getRowRedirection(), maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -161,7 +162,7 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (redirHelper.isRedirected()) { + if (rowRedirection != null) { maybeInnerSource.startTrackingPrevValues(); } } @@ -175,7 +176,7 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long @Override public void prepareForParallelPopulation(final RowSet changedRows) { - if (redirHelper.isRedirected()) { + if (rowRedirection != null) { ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java index 5e7e7c5ae7a..7d1d39f89c5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java @@ -9,6 +9,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -92,15 +93,15 @@ public void reset() { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param redirHelper the {@link UpdateBy.UpdateByRedirectionHelper} for the overall update + * @param rowRedirection the {@link WritableRowRedirection} for the output column */ public BaseFloatUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - this(pair, affectingColumns, redirHelper, null, 0); + this(pair, affectingColumns, rowRedirection, null, 0); } /** @@ -109,7 +110,7 @@ public BaseFloatUpdateByOperator(@NotNull final MatchPair pair, * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param redirHelper the {@link UpdateBy.UpdateByRedirectionHelper} for the overall update + * @param rowRedirection the {@link WritableRowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is @@ -117,18 +118,18 @@ public BaseFloatUpdateByOperator(@NotNull final MatchPair pair, */ public BaseFloatUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, + @Nullable final WritableRowRedirection rowRedirection, @Nullable final String timestampColumnName, final long timeScaleUnits // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, redirHelper, timestampColumnName, timeScaleUnits); - if(this.redirHelper.isRedirected()) { + super(pair, affectingColumns, rowRedirection, timestampColumnName, timeScaleUnits); + if(rowRedirection != null) { // region create-dense this.maybeInnerSource = new FloatArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(this.redirHelper.getRowRedirection(), maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -156,7 +157,7 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (redirHelper.isRedirected()) { + if (rowRedirection != null) { maybeInnerSource.startTrackingPrevValues(); } } @@ -170,7 +171,7 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long @Override public void prepareForParallelPopulation(final RowSet changedRows) { - if (redirHelper.isRedirected()) { + if (rowRedirection != null) { ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java index 3fb666197af..637e4cfab83 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java @@ -15,6 +15,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -97,15 +98,15 @@ public void reset() { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param redirHelper the {@link UpdateBy.UpdateByRedirectionHelper} for the overall update + * @param rowRedirection the {@link WritableRowRedirection} for the output column */ public BaseIntUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - this(pair, affectingColumns, redirHelper, null, 0); + this(pair, affectingColumns, rowRedirection, null, 0); } /** @@ -114,7 +115,7 @@ public BaseIntUpdateByOperator(@NotNull final MatchPair pair, * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param redirHelper the {@link UpdateBy.UpdateByRedirectionHelper} for the overall update + * @param rowRedirection the {@link WritableRowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is @@ -122,18 +123,18 @@ public BaseIntUpdateByOperator(@NotNull final MatchPair pair, */ public BaseIntUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, + @Nullable final WritableRowRedirection rowRedirection, @Nullable final String timestampColumnName, final long timeScaleUnits // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, redirHelper, timestampColumnName, timeScaleUnits); - if(this.redirHelper.isRedirected()) { + super(pair, affectingColumns, rowRedirection, timestampColumnName, timeScaleUnits); + if(rowRedirection != null) { // region create-dense this.maybeInnerSource = new IntegerArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(this.redirHelper.getRowRedirection(), maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -162,7 +163,7 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (redirHelper.isRedirected()) { + if (rowRedirection != null) { maybeInnerSource.startTrackingPrevValues(); } } @@ -176,7 +177,7 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long @Override public void prepareForParallelPopulation(final RowSet changedRows) { - if (redirHelper.isRedirected()) { + if (rowRedirection != null) { ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java index 0818064b48e..94f45913384 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java @@ -15,6 +15,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -97,15 +98,15 @@ public void reset() { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param redirHelper the {@link UpdateBy.UpdateByRedirectionHelper} for the overall update + * @param rowRedirection the {@link WritableRowRedirection} for the output column */ public BaseLongUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - this(pair, affectingColumns, redirHelper, null, 0); + this(pair, affectingColumns, rowRedirection, null, 0); } /** @@ -114,7 +115,7 @@ public BaseLongUpdateByOperator(@NotNull final MatchPair pair, * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param redirHelper the {@link UpdateBy.UpdateByRedirectionHelper} for the overall update + * @param rowRedirection the {@link WritableRowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is @@ -122,18 +123,18 @@ public BaseLongUpdateByOperator(@NotNull final MatchPair pair, */ public BaseLongUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, + @Nullable final WritableRowRedirection rowRedirection, @Nullable final String timestampColumnName, final long timeScaleUnits // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, redirHelper, timestampColumnName, timeScaleUnits); - if(this.redirHelper.isRedirected()) { + super(pair, affectingColumns, rowRedirection, timestampColumnName, timeScaleUnits); + if(rowRedirection != null) { // region create-dense this.maybeInnerSource = new LongArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(this.redirHelper.getRowRedirection(), maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -162,7 +163,7 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (redirHelper.isRedirected()) { + if (rowRedirection != null) { maybeInnerSource.startTrackingPrevValues(); } } @@ -176,7 +177,7 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long @Override public void prepareForParallelPopulation(final RowSet changedRows) { - if (redirHelper.isRedirected()) { + if (rowRedirection != null) { ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java index 1499781c842..9e3e89bf2ea 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java @@ -5,7 +5,9 @@ import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; public abstract class BaseObjectBinaryOperator extends BaseObjectUpdateByOperator { protected class Context extends BaseObjectUpdateByOperator.Context { @@ -40,8 +42,8 @@ public void reset() { public BaseObjectBinaryOperator(@NotNull final Class type, @NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper) { - super(pair, affectingColumns, redirHelper, type); + @Nullable final WritableRowRedirection rowRedirection) { + super(pair, affectingColumns, rowRedirection, type); } protected abstract T doOperation(T bucketCurVal, T chunkCurVal); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java index 361aaf62cb6..7b6c0297ad4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java @@ -15,6 +15,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -98,16 +99,16 @@ public void reset() { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param redirHelper the {@link UpdateBy.UpdateByRedirectionHelper} for the overall update + * @param rowRedirection the {@link WritableRowRedirection} for the output column */ public BaseObjectUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args , final Class colType // endregion extra-constructor-args ) { - this(pair, affectingColumns, redirHelper, null, 0, colType); + this(pair, affectingColumns, rowRedirection, null, 0, colType); } /** @@ -116,7 +117,7 @@ public BaseObjectUpdateByOperator(@NotNull final MatchPair pair, * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param redirHelper the {@link UpdateBy.UpdateByRedirectionHelper} for the overall update + * @param rowRedirection the {@link WritableRowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is @@ -124,19 +125,19 @@ public BaseObjectUpdateByOperator(@NotNull final MatchPair pair, */ public BaseObjectUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, + @Nullable final WritableRowRedirection rowRedirection, @Nullable final String timestampColumnName, final long timeScaleUnits // region extra-constructor-args , final Class colType // endregion extra-constructor-args ) { - super(pair, affectingColumns, redirHelper, timestampColumnName, timeScaleUnits); - if(this.redirHelper.isRedirected()) { + super(pair, affectingColumns, rowRedirection, timestampColumnName, timeScaleUnits); + if(rowRedirection != null) { // region create-dense this.maybeInnerSource = new ObjectArraySource(colType); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(this.redirHelper.getRowRedirection(), maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -166,7 +167,7 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (redirHelper.isRedirected()) { + if (rowRedirection != null) { maybeInnerSource.startTrackingPrevValues(); } } @@ -180,7 +181,7 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long @Override public void prepareForParallelPopulation(final RowSet changedRows) { - if (redirHelper.isRedirected()) { + if (rowRedirection != null) { ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java index 42efd265094..d41f44b38f0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java @@ -15,6 +15,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -97,15 +98,15 @@ public void reset() { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param redirHelper the {@link UpdateBy.UpdateByRedirectionHelper} for the overall update + * @param rowRedirection the {@link WritableRowRedirection} for the output column */ public BaseShortUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - this(pair, affectingColumns, redirHelper, null, 0); + this(pair, affectingColumns, rowRedirection, null, 0); } /** @@ -114,7 +115,7 @@ public BaseShortUpdateByOperator(@NotNull final MatchPair pair, * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param redirHelper the {@link UpdateBy.UpdateByRedirectionHelper} for the overall update + * @param rowRedirection the {@link WritableRowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is @@ -122,18 +123,18 @@ public BaseShortUpdateByOperator(@NotNull final MatchPair pair, */ public BaseShortUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, + @Nullable final WritableRowRedirection rowRedirection, @Nullable final String timestampColumnName, final long timeScaleUnits // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, redirHelper, timestampColumnName, timeScaleUnits); - if(this.redirHelper.isRedirected()) { + super(pair, affectingColumns, rowRedirection, timestampColumnName, timeScaleUnits); + if(rowRedirection != null) { // region create-dense this.maybeInnerSource = new ShortArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(this.redirHelper.getRowRedirection(), maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -162,7 +163,7 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (redirHelper.isRedirected()) { + if (rowRedirection != null) { maybeInnerSource.startTrackingPrevValues(); } } @@ -176,7 +177,7 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long @Override public void prepareForParallelPopulation(final RowSet changedRows) { - if (redirHelper.isRedirected()) { + if (rowRedirection != null) { ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java index 8378eed9a67..e649c2f08f0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java @@ -21,6 +21,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -119,16 +120,16 @@ public BaseWindowedByteUpdateByOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirHelper); - if (this.redirHelper.isRedirected()) { + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); + if (rowRedirection != null) { // region create-dense this.maybeInnerSource = makeDenseSource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(this.redirHelper.getRowRedirection(), maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -163,7 +164,7 @@ public void initializeUpdate(@NotNull UpdateContext context) { @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (redirHelper.isRedirected()) { + if (rowRedirection != null) { maybeInnerSource.startTrackingPrevValues(); } } @@ -181,7 +182,7 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d @Override public void prepareForParallelPopulation(final RowSet changedRows) { - if (redirHelper.isRedirected()) { + if (rowRedirection != null) { ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java index 9dc6df11edf..e02e2aba336 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java @@ -11,6 +11,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -108,16 +109,16 @@ public BaseWindowedCharUpdateByOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirHelper); - if (this.redirHelper.isRedirected()) { + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); + if (rowRedirection != null) { // region create-dense this.maybeInnerSource = new CharacterArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(this.redirHelper.getRowRedirection(), maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -139,7 +140,7 @@ public void initializeUpdate(@NotNull UpdateContext context) { @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (redirHelper.isRedirected()) { + if (rowRedirection != null) { maybeInnerSource.startTrackingPrevValues(); } } @@ -153,7 +154,7 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long @Override public void prepareForParallelPopulation(final RowSet changedRows) { - if (redirHelper.isRedirected()) { + if (rowRedirection != null) { ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java index df3e319426e..a16a75c3036 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java @@ -15,6 +15,7 @@ import io.deephaven.engine.table.impl.sources.DoubleArraySource; import io.deephaven.engine.table.impl.sources.DoubleSparseArraySource; import io.deephaven.engine.table.impl.sources.WritableRedirectedColumnSource; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -112,16 +113,16 @@ public BaseWindowedDoubleUpdateByOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirHelper); - if(this.redirHelper.isRedirected()) { + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); + if(rowRedirection != null) { // region create-dense this.maybeInnerSource = new DoubleArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(this.redirHelper.getRowRedirection(), maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -143,7 +144,7 @@ public void initializeUpdate(@NotNull UpdateContext context) { @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (redirHelper.isRedirected()) { + if (rowRedirection != null) { maybeInnerSource.startTrackingPrevValues(); } } @@ -157,7 +158,7 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long @Override public void prepareForParallelPopulation(final RowSet changedRows) { - if (redirHelper.isRedirected()) { + if (rowRedirection != null) { ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java index 9e8a10c10c0..162b6353695 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java @@ -10,6 +10,7 @@ import io.deephaven.engine.table.impl.sources.FloatArraySource; import io.deephaven.engine.table.impl.sources.FloatSparseArraySource; import io.deephaven.engine.table.impl.sources.WritableRedirectedColumnSource; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -107,16 +108,16 @@ public BaseWindowedFloatUpdateByOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirHelper); - if(this.redirHelper.isRedirected()) { + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); + if(rowRedirection != null) { // region create-dense this.maybeInnerSource = new FloatArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(this.redirHelper.getRowRedirection(), maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -138,7 +139,7 @@ public void initializeUpdate(@NotNull UpdateContext context) { @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (redirHelper.isRedirected()) { + if (rowRedirection != null) { maybeInnerSource.startTrackingPrevValues(); } } @@ -152,7 +153,7 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long @Override public void prepareForParallelPopulation(final RowSet changedRows) { - if (redirHelper.isRedirected()) { + if (rowRedirection != null) { ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java index 8ffa00c5eb3..93cd65b0b98 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java @@ -16,6 +16,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -113,16 +114,16 @@ public BaseWindowedIntUpdateByOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirHelper); - if (this.redirHelper.isRedirected()) { + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); + if (rowRedirection != null) { // region create-dense this.maybeInnerSource = new IntegerArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(this.redirHelper.getRowRedirection(), maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -144,7 +145,7 @@ public void initializeUpdate(@NotNull UpdateContext context) { @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (redirHelper.isRedirected()) { + if (rowRedirection != null) { maybeInnerSource.startTrackingPrevValues(); } } @@ -158,7 +159,7 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long @Override public void prepareForParallelPopulation(final RowSet changedRows) { - if (redirHelper.isRedirected()) { + if (rowRedirection != null) { ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java index 60100e7d180..b115804c91b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java @@ -16,6 +16,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -113,16 +114,16 @@ public BaseWindowedLongUpdateByOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirHelper); - if (this.redirHelper.isRedirected()) { + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); + if (rowRedirection != null) { // region create-dense this.maybeInnerSource = new LongArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(this.redirHelper.getRowRedirection(), maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -144,7 +145,7 @@ public void initializeUpdate(@NotNull UpdateContext context) { @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (redirHelper.isRedirected()) { + if (rowRedirection != null) { maybeInnerSource.startTrackingPrevValues(); } } @@ -158,7 +159,7 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long @Override public void prepareForParallelPopulation(final RowSet changedRows) { - if (redirHelper.isRedirected()) { + if (rowRedirection != null) { ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java index 53e1d4cdeba..7ee72763fa7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java @@ -16,6 +16,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -114,17 +115,17 @@ public BaseWindowedObjectUpdateByOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args , final Class colType // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirHelper); - if (this.redirHelper.isRedirected()) { + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); + if (rowRedirection != null) { // region create-dense this.maybeInnerSource = new ObjectArraySource(colType); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(this.redirHelper.getRowRedirection(), maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -147,7 +148,7 @@ public void initializeUpdate(@NotNull UpdateContext context) { @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (redirHelper.isRedirected()) { + if (rowRedirection != null) { maybeInnerSource.startTrackingPrevValues(); } } @@ -161,7 +162,7 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long @Override public void prepareForParallelPopulation(final RowSet changedRows) { - if (redirHelper.isRedirected()) { + if (rowRedirection != null) { ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java index 42a5ca6470b..b1352d7da6a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java @@ -16,6 +16,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -113,16 +114,16 @@ public BaseWindowedShortUpdateByOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirHelper); - if (this.redirHelper.isRedirected()) { + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); + if (rowRedirection != null) { // region create-dense this.maybeInnerSource = new ShortArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(this.redirHelper.getRowRedirection(), maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -144,7 +145,7 @@ public void initializeUpdate(@NotNull UpdateContext context) { @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); - if (redirHelper.isRedirected()) { + if (rowRedirection != null) { maybeInnerSource.startTrackingPrevValues(); } } @@ -158,7 +159,7 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long @Override public void prepareForParallelPopulation(final RowSet changedRows) { - if (redirHelper.isRedirected()) { + if (rowRedirection != null) { ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java index 1713bf7ee28..c3f291f6ed3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java @@ -11,7 +11,9 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseByteUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.*; @@ -51,11 +53,11 @@ public void push(long key, int pos) { public ByteCumMinMaxOperator(@NotNull final MatchPair pair, final boolean isMax, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, redirHelper); + super(pair, new String[] { pair.rightColumn }, rowRedirection); this.isMax = isMax; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ComparableCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ComparableCumMinMaxOperator.java index d4117ce378a..8892e1436c8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ComparableCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ComparableCumMinMaxOperator.java @@ -3,7 +3,9 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectBinaryOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; public final class ComparableCumMinMaxOperator> extends BaseObjectBinaryOperator { private final boolean isMax; @@ -11,8 +13,8 @@ public final class ComparableCumMinMaxOperator> extends public ComparableCumMinMaxOperator(final Class colType, @NotNull final MatchPair inputPair, final boolean isMax, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper) { - super(colType, inputPair, new String[] {inputPair.rightColumn}, redirHelper); + @Nullable final WritableRowRedirection rowRedirection) { + super(colType, inputPair, new String[] {inputPair.rightColumn}, rowRedirection); this.isMax = isMax; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java index 3d36b84f9b1..b34648109da 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java @@ -11,7 +11,9 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseDoubleUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.NULL_DOUBLE; @@ -51,11 +53,11 @@ public void push(long key, int pos) { public DoubleCumMinMaxOperator(@NotNull final MatchPair pair, final boolean isMax, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, redirHelper); + super(pair, new String[] { pair.rightColumn }, rowRedirection); this.isMax = isMax; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java index d43cda243b9..25529eade98 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java @@ -6,7 +6,9 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseFloatUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.NULL_FLOAT; @@ -46,11 +48,11 @@ public void push(long key, int pos) { public FloatCumMinMaxOperator(@NotNull final MatchPair pair, final boolean isMax, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, redirHelper); + super(pair, new String[] { pair.rightColumn }, rowRedirection); this.isMax = isMax; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java index 6d07c45d278..2f8a583d4af 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java @@ -11,7 +11,9 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseIntUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.*; @@ -51,11 +53,11 @@ public void push(long key, int pos) { public IntCumMinMaxOperator(@NotNull final MatchPair pair, final boolean isMax, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, redirHelper); + super(pair, new String[] { pair.rightColumn }, rowRedirection); this.isMax = isMax; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java index 4a9496aabff..0818f71673c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java @@ -18,7 +18,9 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.*; @@ -59,12 +61,12 @@ public void push(long key, int pos) { public LongCumMinMaxOperator(@NotNull final MatchPair pair, final boolean isMax, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args ,@NotNull final Class type // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, redirHelper); + super(pair, new String[] { pair.rightColumn }, rowRedirection); this.isMax = isMax; // region constructor this.type = type; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java index c450e15ff62..750a220244b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java @@ -6,7 +6,9 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseShortUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.*; @@ -46,11 +48,11 @@ public void push(long key, int pos) { public ShortCumMinMaxOperator(@NotNull final MatchPair pair, final boolean isMax, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, redirHelper); + super(pair, new String[] { pair.rightColumn }, rowRedirection); this.isMax = isMax; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java index f41d6211cc0..d42d18d8d1b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java @@ -6,7 +6,9 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.math.BigDecimal; import java.math.MathContext; @@ -49,9 +51,9 @@ public void reset() { } public BigDecimalCumProdOperator(@NotNull final MatchPair inputPair, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, + @Nullable final WritableRowRedirection rowRedirection, @NotNull final MathContext mathContext) { - super(inputPair, new String[] {inputPair.rightColumn}, redirHelper, BigDecimal.class); + super(inputPair, new String[] {inputPair.rightColumn}, rowRedirection, BigDecimal.class); this.mathContext = mathContext; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java index eb8ef6f62ec..9334e92bbdb 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java @@ -6,7 +6,9 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.math.BigInteger; @@ -45,8 +47,8 @@ public void reset() { } public BigIntegerCumProdOperator(@NotNull final MatchPair inputPair, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper) { - super(inputPair, new String[] {inputPair.rightColumn}, redirHelper, BigInteger.class); + @Nullable final WritableRowRedirection rowRedirection) { + super(inputPair, new String[] {inputPair.rightColumn}, rowRedirection, BigInteger.class); } @NotNull diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java index 7f71f13f9b2..b275d217403 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java @@ -11,7 +11,9 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.*; @@ -45,11 +47,11 @@ public void push(long key, int pos) { } public ByteCumProdOperator(@NotNull final MatchPair pair, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, redirHelper); + super(pair, new String[] { pair.rightColumn }, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java index 3b7e1ad5983..d41804f83b6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java @@ -11,7 +11,9 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseDoubleUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.NULL_DOUBLE; @@ -45,11 +47,11 @@ public void push(long key, int pos) { } public DoubleCumProdOperator(@NotNull final MatchPair pair, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, redirHelper); + super(pair, new String[] { pair.rightColumn }, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java index a625da06b7e..67c67bc49a7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java @@ -6,7 +6,9 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseFloatUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.NULL_FLOAT; @@ -40,11 +42,11 @@ public void push(long key, int pos) { } public FloatCumProdOperator(@NotNull final MatchPair pair, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, redirHelper); + super(pair, new String[] { pair.rightColumn }, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java index c0497983d23..c997fcba590 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java @@ -11,7 +11,9 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.*; @@ -45,11 +47,11 @@ public void push(long key, int pos) { } public IntCumProdOperator(@NotNull final MatchPair pair, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, redirHelper); + super(pair, new String[] { pair.rightColumn }, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java index 47213098646..0f19edd73fc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java @@ -11,7 +11,9 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.*; @@ -45,11 +47,11 @@ public void push(long key, int pos) { } public LongCumProdOperator(@NotNull final MatchPair pair, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, redirHelper); + super(pair, new String[] { pair.rightColumn }, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java index dcbfbb462db..c653eb01aff 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java @@ -6,7 +6,9 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.*; @@ -40,11 +42,11 @@ public void push(long key, int pos) { } public ShortCumProdOperator(@NotNull final MatchPair pair, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, redirHelper); + super(pair, new String[] { pair.rightColumn }, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java index 979019adff6..9255561c700 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java @@ -8,6 +8,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedObjectUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -90,10 +91,10 @@ public BigDecimalRollingSumOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, + @Nullable final WritableRowRedirection rowRedirection, @NotNull final MathContext mathContext) { super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, - redirHelper, BigDecimal.class); + rowRedirection, BigDecimal.class); this.mathContext = mathContext; } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java index c11c6f7cefa..bd15e1bebb2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java @@ -8,6 +8,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedObjectUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -86,11 +87,11 @@ public BigIntegerRollingSumOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirHelper, BigInteger.class); + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection, BigInteger.class); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java index ca21babcc67..07e69a39ea2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java @@ -13,6 +13,7 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedLongUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -96,12 +97,12 @@ public ByteRollingSumOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args ,final byte nullValue // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirHelper); + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); // region constructor this.nullValue = nullValue; // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java index 6f413e2e751..eb89e05490c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java @@ -12,6 +12,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedDoubleUpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.PairwiseDoubleRingBuffer; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -95,11 +96,11 @@ public DoubleRollingSumOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirHelper); + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java index ca61d385909..ad03a30337c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java @@ -7,6 +7,7 @@ import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedFloatUpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.PairwiseFloatRingBuffer; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -90,11 +91,11 @@ public FloatRollingSumOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirHelper); + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java index 783a575bed7..46f6df8cace 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java @@ -13,6 +13,7 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedLongUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -95,11 +96,11 @@ public IntRollingSumOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirHelper); + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java index 56a8e608db7..90deb4d5b4c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java @@ -13,6 +13,7 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedLongUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -95,11 +96,11 @@ public LongRollingSumOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirHelper); + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java index 5267e5f3546..d7662ac8c5c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java @@ -8,6 +8,7 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedLongUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -90,11 +91,11 @@ public ShortRollingSumOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, redirHelper); + super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java index d498f600dcc..f718e59f0ed 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java @@ -6,7 +6,9 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.math.BigDecimal; import java.math.MathContext; @@ -45,9 +47,9 @@ public void push(long key, int pos) { } public BigDecimalCumSumOperator(@NotNull final MatchPair inputPair, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper, + @Nullable final WritableRowRedirection rowRedirection, @NotNull final MathContext mathContext) { - super(inputPair, new String[] {inputPair.rightColumn}, redirHelper, BigDecimal.class); + super(inputPair, new String[] {inputPair.rightColumn}, rowRedirection, BigDecimal.class); this.mathContext = mathContext; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java index 9d6667941ff..02d5068faad 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java @@ -6,7 +6,9 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.math.BigInteger; @@ -40,8 +42,8 @@ public void push(long key, int pos) { } public BigIntegerCumSumOperator(@NotNull final MatchPair inputPair, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper) { - super(inputPair, new String[] {inputPair.rightColumn}, redirHelper, BigInteger.class); + @Nullable final WritableRowRedirection rowRedirection) { + super(inputPair, new String[] {inputPair.rightColumn}, rowRedirection, BigInteger.class); } @NotNull diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java index f503861abcd..63bb1a32648 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java @@ -10,7 +10,9 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.*; @@ -45,12 +47,12 @@ public void push(long key, int pos) { } public ByteCumSumOperator(@NotNull final MatchPair pair, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args ,final byte nullValue // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, redirHelper); + super(pair, new String[] { pair.rightColumn }, rowRedirection); // region constructor this.nullValue = nullValue; // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java index c73729aeaf5..0a945c33ab6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java @@ -11,7 +11,9 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseDoubleUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.NULL_DOUBLE; @@ -45,11 +47,11 @@ public void push(long key, int pos) { } public DoubleCumSumOperator(@NotNull final MatchPair pair, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, redirHelper); + super(pair, new String[] { pair.rightColumn }, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java index 199538d402d..7385a43a361 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java @@ -6,7 +6,9 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseFloatUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.NULL_FLOAT; @@ -40,11 +42,11 @@ public void push(long key, int pos) { } public FloatCumSumOperator(@NotNull final MatchPair pair, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, redirHelper); + super(pair, new String[] { pair.rightColumn }, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java index 0c23f60b076..d4643c0dd0a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java @@ -10,7 +10,9 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.*; @@ -44,11 +46,11 @@ public void push(long key, int pos) { } public IntCumSumOperator(@NotNull final MatchPair pair, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, redirHelper); + super(pair, new String[] { pair.rightColumn }, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java index de9aabb1273..5164ee1d6c2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java @@ -10,7 +10,9 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.*; @@ -44,11 +46,11 @@ public void push(long key, int pos) { } public LongCumSumOperator(@NotNull final MatchPair pair, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, redirHelper); + super(pair, new String[] { pair.rightColumn }, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java index bc400f434e3..54630c37b1f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java @@ -5,7 +5,9 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.*; @@ -39,11 +41,11 @@ public void push(long key, int pos) { } public ShortCumSumOperator(@NotNull final MatchPair pair, - @NotNull final UpdateBy.UpdateByRedirectionHelper redirHelper + @Nullable final WritableRowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, redirHelper); + super(pair, new String[] { pair.rightColumn }, rowRedirection); // region constructor // endregion constructor } diff --git a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java index 569fed4affb..dbc85aee911 100644 --- a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java +++ b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java @@ -34,8 +34,8 @@ public static void main(String[] args) throws IOException { String objectResult = ReplicatePrimitiveCode.charToObject( "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java"); fixupStandardObject(objectResult, "ObjectFillByOperator", false, - "super\\(fillPair, new String\\[\\] \\{ fillPair.rightColumn \\}, redirHelper\\);", - "super(fillPair, new String[] { fillPair.rightColumn }, redirHelper, colType);", + "super\\(fillPair, new String\\[\\] \\{ fillPair.rightColumn \\}, rowRedirection\\);", + "super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection, colType);", " BaseObjectUpdateByOperator", " BaseObjectUpdateByOperator", "public ObjectChunk 0) { lines = globalReplacements(lines, extraReplacements); } From 4db151f4e7fc4e42fe197a7a8c5e67ca70046ad5 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 4 Jan 2023 09:28:05 -0800 Subject: [PATCH 067/123] Corrected off-by-one error in UpdateBy row redirection --- .../main/java/io/deephaven/engine/table/impl/UpdateBy.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java index ff3d211ce70..e6490094d21 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java @@ -105,7 +105,7 @@ boolean isRedirected() { } private long requiredCapacity() { - return maxInnerRowKey + 1; + return maxInnerRowKey; } @Nullable @@ -133,7 +133,7 @@ private void processUpdateForRedirection(@NotNull final TableUpdate upstream, final MutableLong lastAllocated = new MutableLong(0); final WritableRowSet.Iterator freeIt = freeRows.iterator(); upstream.added().forAllRowKeys(outerKey -> { - final long innerKey = freeIt.hasNext() ? freeIt.nextLong() : ++maxInnerRowKey; + final long innerKey = freeIt.hasNext() ? freeIt.nextLong() : maxInnerRowKey++; lastAllocated.setValue(innerKey); rowRedirection.put(outerKey, innerKey); }); From 0acdaaeee82a9e9118ddf4d9704a3076a1d5cdce Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 4 Jan 2023 09:55:00 -0800 Subject: [PATCH 068/123] JobScheduler simplifications --- .../engine/table/impl/util/JobScheduler.java | 82 ++++--------------- 1 file changed, 14 insertions(+), 68 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java index bd2d267151c..cf1328cf407 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java @@ -79,54 +79,13 @@ interface IterateResumeAction { @FinalDefault default void iterateParallel(ExecutionContext executionContext, LogOutputAppendable description, int start, int count, IterateAction action, Runnable completeAction, final Consumer onError) { - - if (count == 0) { - // no work to do - completeAction.run(); - } - - final AtomicInteger nextIndex = new AtomicInteger(start); - final AtomicInteger remaining = new AtomicInteger(count); - - final AtomicBoolean cancelRemainingExecution = new AtomicBoolean(false); - - final Consumer localError = exception -> { - // signal only on the first error - if (cancelRemainingExecution.compareAndSet(false, true)) { - onError.accept(exception); - } - }; - - final Runnable task = () -> { - // this will run until all tasks have started - while (true) { - if (cancelRemainingExecution.get()) { - return; - } - int idx = nextIndex.getAndIncrement(); - if (idx < start + count) { - // do the work + iterateParallel(executionContext, description, start, count, + (final int idx, final Runnable resume) -> + { action.run(idx); - - // check for completion - if (remaining.decrementAndGet() == 0) { - completeAction.run(); - return; - } - } else { - // no more work to do - return; - } - } - }; - - // create multiple tasks but not more than one per scheduler thread - for (int i = 0; i < Math.min(count, threadCount()); i++) { - submit(executionContext, - task, - description, - localError); - } + resume.run(); + }, + completeAction, onError); } /** @@ -220,39 +179,29 @@ default void iterateSerial(ExecutionContext executionContext, LogOutputAppendabl completeAction.run(); } - final AtomicInteger nextIndex = new AtomicInteger(start); - final AtomicInteger remaining = new AtomicInteger(count); - - final AtomicBoolean cancelRemainingExecution = new AtomicBoolean(false); - - final Consumer localError = exception -> { - cancelRemainingExecution.set(true); - onError.accept(exception); - }; - // no lambda, need the `this` reference to re-execute final Runnable resumeAction = new Runnable() { + int nextIndex = start + 1; + int remaining = count; + @Override public void run() { // check for completion - if (remaining.decrementAndGet() == 0) { + if (--remaining == 0) { completeAction.run(); } else { - if (cancelRemainingExecution.get()) { - return; - } // schedule the next task submit(executionContext, () -> { - int idx = nextIndex.getAndIncrement(); + int idx = nextIndex++; if (idx < start + count) { // do the work action.run(idx, this); } }, description, - localError); + onError); } } @@ -261,12 +210,9 @@ public void run() { // create a single task submit(executionContext, () -> { - int idx = nextIndex.getAndIncrement(); - if (idx < start + count) { - action.run(idx, resumeAction); - } + action.run(start, resumeAction); }, description, - localError); + onError); } } From a13dfb4e8b84d3567920334178abed57679e8219 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 4 Jan 2023 12:26:08 -0800 Subject: [PATCH 069/123] Removed magic numbers, replaced with descriptive constants --- .../deephaven/engine/table/impl/UpdateBy.java | 24 ++- .../table/impl/UpdateByBucketHelper.java | 144 +++++++++--------- .../impl/updateby/UpdateByWindowTicks.java | 4 +- .../impl/updateby/UpdateByWindowTime.java | 3 +- .../BigDecimalRollingSumOperator.java | 11 +- .../BigIntegerRollingSumOperator.java | 11 +- .../rollingsum/ByteRollingSumOperator.java | 7 +- .../rollingsum/DoubleRollingSumOperator.java | 23 ++- .../rollingsum/FloatRollingSumOperator.java | 23 ++- .../rollingsum/IntRollingSumOperator.java | 7 +- .../rollingsum/LongRollingSumOperator.java | 7 +- .../rollingsum/ShortRollingSumOperator.java | 7 +- .../engine/table/impl/util/JobScheduler.java | 7 +- 13 files changed, 136 insertions(+), 142 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java index e6490094d21..4e92c671775 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java @@ -28,7 +28,6 @@ import io.deephaven.util.datastructures.linked.IntrusiveDoublyLinkedNode; import io.deephaven.util.datastructures.linked.IntrusiveDoublyLinkedQueue; import org.apache.commons.lang3.mutable.MutableInt; -import org.apache.commons.lang3.mutable.MutableLong; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -51,6 +50,9 @@ public abstract class UpdateBy { public static final int PARALLEL_CACHE_CHUNK_SIZE = Configuration.getInstance().getIntegerWithDefault("UpdateBy.parallelCacheChunkSize", 1 << 16); + /** When extracting keys from the redirection, what size chunks to use? (2K default) */ + public static final int REDIRECTION_CHUNK_SIZE = 1 << 11; + /** Input sources may be reused by multiple operators, only store and cache unique ones (post-reinterpret) */ protected final ColumnSource[] inputSources; /** Map operators to input sources, note some operators need more than one input, WAvg e.g. */ @@ -130,25 +132,17 @@ private void processUpdateForRedirection(@NotNull final TableUpdate upstream, } if (upstream.added().isNonempty()) { - final MutableLong lastAllocated = new MutableLong(0); final WritableRowSet.Iterator freeIt = freeRows.iterator(); upstream.added().forAllRowKeys(outerKey -> { final long innerKey = freeIt.hasNext() ? freeIt.nextLong() : maxInnerRowKey++; - lastAllocated.setValue(innerKey); rowRedirection.put(outerKey, innerKey); }); - freeRows.removeRange(0, lastAllocated.longValue()); - } - } - - private boolean shiftRedirectedKey(@NotNull final RowSet.SearchIterator iterator, final long delta, - final long key) { - assert rowRedirection != null; - final long inner = rowRedirection.remove(key); - if (inner != NULL_ROW_KEY) { - rowRedirection.put(key + delta, inner); + if (freeIt.hasNext()) { + freeRows.removeRange(0, freeIt.nextLong() - 1); + } else { + freeRows.clear(); + } } - return !iterator.hasNext(); } private RowSet getInnerKeys(final RowSet outerKeys) { @@ -156,7 +150,7 @@ private RowSet getInnerKeys(final RowSet outerKeys) { return null; } RowSetBuilderRandom builder = RowSetFactory.builderRandom(); - final int chunkSize = Math.min(outerKeys.intSize(), 4096); + final int chunkSize = Math.min(outerKeys.intSize(), REDIRECTION_CHUNK_SIZE); try (final RowSequence.Iterator it = outerKeys.getRowSequenceIterator(); ChunkSource.FillContext fillContext = rowRedirection.makeFillContext(chunkSize, null); WritableLongChunk chunk = WritableLongChunk.makeWritableChunk(chunkSize)) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java index 99d353a583f..b2df7065e69 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java @@ -25,6 +25,7 @@ * bucket of rows. */ class UpdateByBucketHelper extends IntrusiveDoublyLinkedNode.Impl { + public static final int SSA_LEAF_SIZE = 4096; protected final ColumnSource[] inputSources; // some columns will have multiple inputs, such as time-based and Weighted computations final int[][] operatorInputSourceSlots; @@ -85,7 +86,7 @@ protected UpdateByBucketHelper(@NotNull final String description, // do we need a timestamp SSA? this.timestampColumnName = timestampColumnName; if (timestampColumnName != null) { - this.timestampSsa = new LongSegmentedSortedArray(4096); + this.timestampSsa = new LongSegmentedSortedArray(SSA_LEAF_SIZE); this.timestampColumnSource = ReinterpretUtils.maybeConvertToPrimitive(source.getColumnSource(this.timestampColumnName)); this.timestampColumnSet = source.newModifiedColumnSet(timestampColumnName); @@ -120,87 +121,82 @@ UpdateByBucketHelperListener newListener(@NotNull final String description) { private void processUpdateForSsa(TableUpdate upstream) { final boolean stampModified = upstream.modifiedColumnSet().containsAny(timestampColumnSet); - final RowSet restampRemovals; - final RowSet restampAdditions; - - // modifies are remove + add operations - if (stampModified) { - restampAdditions = upstream.added().union(upstream.modified()); - restampRemovals = upstream.removed().union(upstream.getModifiedPreShift()); - } else { - restampAdditions = upstream.added(); - restampRemovals = upstream.removed(); - } - - // removes - if (restampRemovals.isNonempty()) { - final int size = (int) Math.min(restampRemovals.size(), 4096); - try (final RowSequence.Iterator it = restampRemovals.getRowSequenceIterator(); - final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(size); - final WritableLongChunk ssaValues = WritableLongChunk.makeWritableChunk(size); - final WritableLongChunk ssaKeys = WritableLongChunk.makeWritableChunk(size)) { - - MutableLong lastTimestamp = new MutableLong(NULL_LONG); - while (it.hasMore()) { - RowSequence chunkRs = it.getNextRowSequenceWithLength(4096); - - // get the chunks for values and keys - LongChunk valuesChunk = - timestampColumnSource.getPrevChunk(context, chunkRs).asLongChunk(); - LongChunk keysChunk = chunkRs.asRowKeyChunk(); - - // push only non-null values/keys into the Ssa - fillChunkWithNonNull(keysChunk, valuesChunk, ssaKeys, ssaValues, lastTimestamp); - timestampSsa.remove(ssaValues, ssaKeys); + try (final RowSet restampAdditions = + stampModified ? upstream.added().union(upstream.modified()) : upstream.added().copy(); + final RowSet restampRemovals = stampModified ? upstream.removed().union(upstream.getModifiedPreShift()) + : upstream.removed().copy()) { + // removes + if (restampRemovals.isNonempty()) { + final int size = (int) Math.min(restampRemovals.size(), 4096); + try (final RowSequence.Iterator it = restampRemovals.getRowSequenceIterator(); + final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(size); + final WritableLongChunk ssaValues = WritableLongChunk.makeWritableChunk(size); + final WritableLongChunk ssaKeys = WritableLongChunk.makeWritableChunk(size)) { + + MutableLong lastTimestamp = new MutableLong(NULL_LONG); + while (it.hasMore()) { + RowSequence chunkRs = it.getNextRowSequenceWithLength(4096); + + // get the chunks for values and keys + LongChunk valuesChunk = + timestampColumnSource.getPrevChunk(context, chunkRs).asLongChunk(); + LongChunk keysChunk = chunkRs.asRowKeyChunk(); + + // push only non-null values/keys into the Ssa + fillChunkWithNonNull(keysChunk, valuesChunk, ssaKeys, ssaValues, lastTimestamp); + timestampSsa.remove(ssaValues, ssaKeys); + } } } - } - // shifts - if (upstream.shifted().nonempty()) { - final int size = Math.max( - upstream.modified().intSize() + Math.max(upstream.added().intSize(), upstream.removed().intSize()), - (int) upstream.shifted().getEffectiveSize()); - try (final RowSet fullPrevRowSet = source.getRowSet().copyPrev(); - final WritableRowSet previousToShift = fullPrevRowSet.minus(restampRemovals); - final ColumnSource.GetContext getContext = timestampColumnSource.makeGetContext(size)) { - - final RowSetShiftData.Iterator sit = upstream.shifted().applyIterator(); - while (sit.hasNext()) { - sit.next(); - try (final RowSet subRowSet = previousToShift.subSetByKeyRange(sit.beginRange(), sit.endRange())) { - if (subRowSet.isEmpty()) { - continue; + // shifts + if (upstream.shifted().nonempty()) { + final int size = Math.max( + upstream.modified().intSize() + + Math.max(upstream.added().intSize(), upstream.removed().intSize()), + (int) upstream.shifted().getEffectiveSize()); + try (final RowSet fullPrevRowSet = source.getRowSet().copyPrev(); + final WritableRowSet previousToShift = fullPrevRowSet.minus(restampRemovals); + final ColumnSource.GetContext getContext = timestampColumnSource.makeGetContext(size)) { + + final RowSetShiftData.Iterator sit = upstream.shifted().applyIterator(); + while (sit.hasNext()) { + sit.next(); + try (final RowSet subRowSet = + previousToShift.subSetByKeyRange(sit.beginRange(), sit.endRange())) { + if (subRowSet.isEmpty()) { + continue; + } + + final LongChunk shiftValues = + timestampColumnSource.getPrevChunk(getContext, subRowSet).asLongChunk(); + + timestampSsa.applyShiftReverse(shiftValues, subRowSet.asRowKeyChunk(), sit.shiftDelta()); } - - final LongChunk shiftValues = - timestampColumnSource.getPrevChunk(getContext, subRowSet).asLongChunk(); - - timestampSsa.applyShiftReverse(shiftValues, subRowSet.asRowKeyChunk(), sit.shiftDelta()); } } } - } - // adds - if (restampAdditions.isNonempty()) { - final int size = (int) Math.min(restampAdditions.size(), 4096); - try (final RowSequence.Iterator it = restampAdditions.getRowSequenceIterator(); - final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(size); - final WritableLongChunk ssaValues = WritableLongChunk.makeWritableChunk(size); - final WritableLongChunk ssaKeys = WritableLongChunk.makeWritableChunk(size)) { - MutableLong lastTimestamp = new MutableLong(NULL_LONG); - while (it.hasMore()) { - RowSequence chunkRs = it.getNextRowSequenceWithLength(4096); - - // get the chunks for values and keys - LongChunk valuesChunk = - timestampColumnSource.getChunk(context, chunkRs).asLongChunk(); - LongChunk keysChunk = chunkRs.asRowKeyChunk(); - - // push only non-null values/keys into the Ssa - fillChunkWithNonNull(keysChunk, valuesChunk, ssaKeys, ssaValues, lastTimestamp); - timestampSsa.insert(ssaValues, ssaKeys); + // adds + if (restampAdditions.isNonempty()) { + final int size = (int) Math.min(restampAdditions.size(), 4096); + try (final RowSequence.Iterator it = restampAdditions.getRowSequenceIterator(); + final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(size); + final WritableLongChunk ssaValues = WritableLongChunk.makeWritableChunk(size); + final WritableLongChunk ssaKeys = WritableLongChunk.makeWritableChunk(size)) { + MutableLong lastTimestamp = new MutableLong(NULL_LONG); + while (it.hasMore()) { + RowSequence chunkRs = it.getNextRowSequenceWithLength(4096); + + // get the chunks for values and keys + LongChunk valuesChunk = + timestampColumnSource.getChunk(context, chunkRs).asLongChunk(); + LongChunk keysChunk = chunkRs.asRowKeyChunk(); + + // push only non-null values/keys into the Ssa + fillChunkWithNonNull(keysChunk, valuesChunk, ssaKeys, ssaValues, lastTimestamp); + timestampSsa.insert(ssaValues, ssaKeys); + } } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java index 42ad04eebd6..d14bbae0d31 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java @@ -12,7 +12,6 @@ import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.TableUpdate; -import io.deephaven.engine.table.impl.QueryTable; import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; @@ -30,6 +29,7 @@ * of `influencer` values to add to the rolling window as the current row changes. */ public class UpdateByWindowTicks extends UpdateByWindow { + public static final int WINDOW_POS_BUFFER_INITIAL_CAPACITY = 512; protected final long prevUnits; protected final long fwdUnits; @@ -57,7 +57,7 @@ public UpdateByWindowTicksContext(final TrackingRowSet sourceRowSet, @Nullable final LongSegmentedSortedArray timestampSsa, final int chunkSize, final boolean initialStep) { super(sourceRowSet, null, null, chunkSize, initialStep); - currentWindowPositions = new IntRingBuffer(512, true); + currentWindowPositions = new IntRingBuffer(WINDOW_POS_BUFFER_INITIAL_CAPACITY, true); } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java index a60935117ef..7c69122ead4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java @@ -31,6 +31,7 @@ * a buffer of `influencer` values to add to the rolling window as the current row changes. */ public class UpdateByWindowTime extends UpdateByWindow { + public static final int WINDOW_TIMESTAMP_BUFFER_INITIAL_CAPACITY = 512; protected final long prevUnits; protected final long fwdUnits; @@ -56,7 +57,7 @@ public UpdateByWindowTimeContext(final TrackingRowSet sourceRowSet, super(sourceRowSet, timestampColumnSource, timestampSsa, chunkSize, initialStep); influencerTimestampContext = timestampColumnSource.makeGetContext(WINDOW_CHUNK_SIZE); - currentWindowTimestamps = new LongRingBuffer(512, true); + currentWindowTimestamps = new LongRingBuffer(WINDOW_TIMESTAMP_BUFFER_INITIAL_CAPACITY, true); } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java index 9255561c700..fe21481a64b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java @@ -2,10 +2,10 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.base.RingBuffer; -import io.deephaven.chunk.*; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedObjectUpdateByOperator; import io.deephaven.engine.table.impl.util.WritableRowRedirection; @@ -16,6 +16,7 @@ import java.math.MathContext; public final class BigDecimalRollingSumOperator extends BaseWindowedObjectUpdateByOperator { + public static final int RING_BUFFER_INITIAL_CAPACITY = 512; @NotNull private final MathContext mathContext; @@ -25,7 +26,7 @@ protected class Context extends BaseWindowedObjectUpdateByOperator.C protected Context(final int chunkSize) { super(chunkSize); - objectWindowValues = new RingBuffer<>(512); + objectWindowValues = new RingBuffer<>(RING_BUFFER_INITIAL_CAPACITY); } @Override @@ -59,7 +60,7 @@ public void push(long key, int pos) { @Override public void pop() { - BigDecimal val = val = objectWindowValues.remove(); + BigDecimal val = objectWindowValues.remove(); // reduce the running sum if (val != null) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java index bd15e1bebb2..d1644327d68 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java @@ -2,10 +2,10 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.base.RingBuffer; -import io.deephaven.chunk.*; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedObjectUpdateByOperator; import io.deephaven.engine.table.impl.util.WritableRowRedirection; @@ -15,13 +15,16 @@ import java.math.BigInteger; public final class BigIntegerRollingSumOperator extends BaseWindowedObjectUpdateByOperator { + + public static final int RING_BUFFER_INITIAL_CAPACITY = 512; + protected class Context extends BaseWindowedObjectUpdateByOperator.Context { protected ObjectChunk objectInfluencerValuesChunk; protected RingBuffer objectWindowValues; protected Context(final int chunkSize) { super(chunkSize); - objectWindowValues = new RingBuffer<>(512); + objectWindowValues = new RingBuffer<>(RING_BUFFER_INITIAL_CAPACITY); } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java index 07e69a39ea2..75c1e5106a0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java @@ -11,15 +11,16 @@ import io.deephaven.chunk.ByteChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedLongUpdateByOperator; import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.*; +import static io.deephaven.util.QueryConstants.NULL_LONG; +import static io.deephaven.util.QueryConstants.NULL_BYTE; public class ByteRollingSumOperator extends BaseWindowedLongUpdateByOperator { + public static final int RING_BUFFER_INITIAL_CAPACITY = 512; // region extra-fields final byte nullValue; // endregion extra-fields @@ -31,7 +32,7 @@ protected class Context extends BaseWindowedLongUpdateByOperator.Context { protected Context(int chunkSize) { super(chunkSize); - byteWindowValues = new ByteRingBuffer(512, true); + byteWindowValues = new ByteRingBuffer(RING_BUFFER_INITIAL_CAPACITY, true); } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java index eb89e05490c..47c3cad7548 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java @@ -6,10 +6,11 @@ package io.deephaven.engine.table.impl.updateby.rollingsum; import io.deephaven.api.updateby.OperationControl; -import io.deephaven.chunk.*; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.DoubleChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedDoubleUpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.PairwiseDoubleRingBuffer; import io.deephaven.engine.table.impl.util.WritableRowRedirection; @@ -22,6 +23,7 @@ import static io.deephaven.util.QueryConstants.NULL_DOUBLE; public class DoubleRollingSumOperator extends BaseWindowedDoubleUpdateByOperator { + public static final int PAIRWISE_BUFFER_INITIAL_SIZE = 64; // region extra-fields // endregion extra-fields @@ -31,16 +33,13 @@ protected class Context extends BaseWindowedDoubleUpdateByOperator.Context { protected Context(final int chunkSize) { super(chunkSize); - doublePairwiseSum = new PairwiseDoubleRingBuffer(64, 0.0f, new PairwiseDoubleRingBuffer.DoubleFunction() { - @Override - public double apply(double a, double b) { - if (a == NULL_DOUBLE) { - return b; - } else if (b == NULL_DOUBLE) { - return a; - } - return a + b; + doublePairwiseSum = new PairwiseDoubleRingBuffer(PAIRWISE_BUFFER_INITIAL_SIZE, 0.0f, (a, b) -> { + if (a == NULL_DOUBLE) { + return b; + } else if (b == NULL_DOUBLE) { + return a; } + return a + b; }); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java index ad03a30337c..0c6f6b48f6c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java @@ -1,10 +1,11 @@ package io.deephaven.engine.table.impl.updateby.rollingsum; import io.deephaven.api.updateby.OperationControl; -import io.deephaven.chunk.*; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.FloatChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.UpdateBy; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedFloatUpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.PairwiseFloatRingBuffer; import io.deephaven.engine.table.impl.util.WritableRowRedirection; @@ -17,6 +18,7 @@ import static io.deephaven.util.QueryConstants.NULL_FLOAT; public class FloatRollingSumOperator extends BaseWindowedFloatUpdateByOperator { + public static final int PAIRWISE_BUFFER_INITIAL_SIZE = 64; // region extra-fields // endregion extra-fields @@ -26,16 +28,13 @@ protected class Context extends BaseWindowedFloatUpdateByOperator.Context { protected Context(final int chunkSize) { super(chunkSize); - floatPairwiseSum = new PairwiseFloatRingBuffer(64, 0.0f, new PairwiseFloatRingBuffer.FloatFunction() { - @Override - public float apply(float a, float b) { - if (a == NULL_FLOAT) { - return b; - } else if (b == NULL_FLOAT) { - return a; - } - return a + b; + floatPairwiseSum = new PairwiseFloatRingBuffer(PAIRWISE_BUFFER_INITIAL_SIZE, 0.0f, (a, b) -> { + if (a == NULL_FLOAT) { + return b; + } else if (b == NULL_FLOAT) { + return a; } + return a + b; }); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java index 46f6df8cace..303d28201ce 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java @@ -11,15 +11,16 @@ import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedLongUpdateByOperator; import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.*; +import static io.deephaven.util.QueryConstants.NULL_LONG; +import static io.deephaven.util.QueryConstants.NULL_INT; public class IntRollingSumOperator extends BaseWindowedLongUpdateByOperator { + public static final int RING_BUFFER_INITIAL_CAPACITY = 512; // region extra-fields // endregion extra-fields @@ -30,7 +31,7 @@ protected class Context extends BaseWindowedLongUpdateByOperator.Context { protected Context(int chunkSize) { super(chunkSize); - intWindowValues = new IntRingBuffer(512, true); + intWindowValues = new IntRingBuffer(RING_BUFFER_INITIAL_CAPACITY, true); } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java index 90deb4d5b4c..db472652b3a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java @@ -11,15 +11,16 @@ import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedLongUpdateByOperator; import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.*; +import static io.deephaven.util.QueryConstants.NULL_LONG; +import static io.deephaven.util.QueryConstants.NULL_LONG; public class LongRollingSumOperator extends BaseWindowedLongUpdateByOperator { + public static final int RING_BUFFER_INITIAL_CAPACITY = 512; // region extra-fields // endregion extra-fields @@ -30,7 +31,7 @@ protected class Context extends BaseWindowedLongUpdateByOperator.Context { protected Context(int chunkSize) { super(chunkSize); - longWindowValues = new LongRingBuffer(512, true); + longWindowValues = new LongRingBuffer(RING_BUFFER_INITIAL_CAPACITY, true); } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java index d7662ac8c5c..322d3ca7140 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java @@ -6,15 +6,16 @@ import io.deephaven.chunk.ShortChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedLongUpdateByOperator; import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.*; +import static io.deephaven.util.QueryConstants.NULL_LONG; +import static io.deephaven.util.QueryConstants.NULL_SHORT; public class ShortRollingSumOperator extends BaseWindowedLongUpdateByOperator { + public static final int RING_BUFFER_INITIAL_CAPACITY = 512; // region extra-fields // endregion extra-fields @@ -25,7 +26,7 @@ protected class Context extends BaseWindowedLongUpdateByOperator.Context { protected Context(int chunkSize) { super(chunkSize); - shortWindowValues = new ShortRingBuffer(512, true); + shortWindowValues = new ShortRingBuffer(RING_BUFFER_INITIAL_CAPACITY, true); } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java index cf1328cf407..5ebef11dd27 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java @@ -80,8 +80,7 @@ interface IterateResumeAction { default void iterateParallel(ExecutionContext executionContext, LogOutputAppendable description, int start, int count, IterateAction action, Runnable completeAction, final Consumer onError) { iterateParallel(executionContext, description, start, count, - (final int idx, final Runnable resume) -> - { + (final int idx, final Runnable resume) -> { action.run(idx); resume.run(); }, @@ -209,9 +208,7 @@ public void run() { // create a single task submit(executionContext, - () -> { - action.run(start, resumeAction); - }, + () -> action.run(start, resumeAction), description, onError); } From e5107d3ef377e67672690fa6b9ac4a3baf6ccd47 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Thu, 5 Jan 2023 17:32:33 -0800 Subject: [PATCH 070/123] Moved UpdateBy classes, working version of clear output Object columns, converted to RowRedirection --- .../engine/table/impl/JoinControl.java | 2 +- .../engine/table/impl/QueryTable.java | 1 + .../engine/table/impl/UncoalescedTable.java | 1 + .../BucketedPartitionedUpdateByManager.java | 22 +- .../table/impl/{ => updateby}/UpdateBy.java | 237 ++++++++++-------- .../{ => updateby}/UpdateByBucketHelper.java | 32 ++- .../UpdateByCumulativeOperator.java | 14 +- .../impl/{ => updateby}/UpdateByOperator.java | 21 +- .../UpdateByOperatorFactory.java | 6 +- .../table/impl/updateby/UpdateByWindow.java | 98 +++----- .../updateby/UpdateByWindowCumulative.java | 13 +- .../impl/updateby/UpdateByWindowTicks.java | 35 ++- .../impl/updateby/UpdateByWindowTime.java | 37 ++- .../UpdateByWindowedOperator.java | 6 +- .../ZeroKeyUpdateByManager.java | 18 +- .../ema/BasePrimitiveEMAOperator.java | 8 +- .../updateby/ema/BigDecimalEMAOperator.java | 5 +- .../updateby/ema/BigIntegerEMAOperator.java | 7 +- .../updateby/ema/BigNumberEMAOperator.java | 8 +- .../impl/updateby/ema/ByteEMAOperator.java | 9 +- .../impl/updateby/ema/DoubleEMAOperator.java | 5 +- .../impl/updateby/ema/FloatEMAOperator.java | 5 +- .../impl/updateby/ema/IntEMAOperator.java | 9 +- .../impl/updateby/ema/LongEMAOperator.java | 9 +- .../impl/updateby/ema/ShortEMAOperator.java | 9 +- .../updateby/fill/BooleanFillByOperator.java | 5 +- .../updateby/fill/ByteFillByOperator.java | 5 +- .../updateby/fill/CharFillByOperator.java | 5 +- .../updateby/fill/DoubleFillByOperator.java | 5 +- .../updateby/fill/FloatFillByOperator.java | 5 +- .../impl/updateby/fill/IntFillByOperator.java | 5 +- .../updateby/fill/LongFillByOperator.java | 5 +- .../updateby/fill/ObjectFillByOperator.java | 7 +- .../updateby/fill/ShortFillByOperator.java | 5 +- .../UpdateByStateManagerTypedBase.java | 4 +- .../internal/BaseByteUpdateByOperator.java | 24 +- .../internal/BaseCharUpdateByOperator.java | 24 +- .../internal/BaseDoubleUpdateByOperator.java | 33 ++- .../internal/BaseFloatUpdateByOperator.java | 33 ++- .../internal/BaseIntUpdateByOperator.java | 24 +- .../internal/BaseLongUpdateByOperator.java | 24 +- .../internal/BaseObjectBinaryOperator.java | 5 +- .../internal/BaseObjectUpdateByOperator.java | 31 ++- .../internal/BaseShortUpdateByOperator.java | 24 +- .../BaseWindowedByteUpdateByOperator.java | 21 +- .../BaseWindowedCharUpdateByOperator.java | 21 +- .../BaseWindowedDoubleUpdateByOperator.java | 28 ++- .../BaseWindowedFloatUpdateByOperator.java | 28 ++- .../BaseWindowedIntUpdateByOperator.java | 21 +- .../BaseWindowedLongUpdateByOperator.java | 21 +- .../BaseWindowedObjectUpdateByOperator.java | 28 ++- .../BaseWindowedShortUpdateByOperator.java | 21 +- .../minmax/ByteCumMinMaxOperator.java | 7 +- .../minmax/ComparableCumMinMaxOperator.java | 5 +- .../minmax/DoubleCumMinMaxOperator.java | 5 +- .../minmax/FloatCumMinMaxOperator.java | 5 +- .../updateby/minmax/IntCumMinMaxOperator.java | 7 +- .../minmax/LongCumMinMaxOperator.java | 7 +- .../minmax/ShortCumMinMaxOperator.java | 7 +- .../prod/BigDecimalCumProdOperator.java | 5 +- .../prod/BigIntegerCumProdOperator.java | 5 +- .../updateby/prod/ByteCumProdOperator.java | 8 +- .../updateby/prod/DoubleCumProdOperator.java | 5 +- .../updateby/prod/FloatCumProdOperator.java | 5 +- .../updateby/prod/IntCumProdOperator.java | 8 +- .../updateby/prod/LongCumProdOperator.java | 8 +- .../updateby/prod/ShortCumProdOperator.java | 8 +- .../BigDecimalRollingSumOperator.java | 8 +- .../BigIntegerRollingSumOperator.java | 8 +- .../rollingsum/ByteRollingSumOperator.java | 9 +- .../rollingsum/DoubleRollingSumOperator.java | 6 +- .../rollingsum/FloatRollingSumOperator.java | 6 +- .../rollingsum/IntRollingSumOperator.java | 9 +- .../rollingsum/LongRollingSumOperator.java | 9 +- .../rollingsum/ShortRollingSumOperator.java | 9 +- .../sum/BigDecimalCumSumOperator.java | 5 +- .../sum/BigIntegerCumSumOperator.java | 5 +- .../impl/updateby/sum/ByteCumSumOperator.java | 11 +- .../updateby/sum/DoubleCumSumOperator.java | 5 +- .../updateby/sum/FloatCumSumOperator.java | 5 +- .../impl/updateby/sum/IntCumSumOperator.java | 11 +- .../impl/updateby/sum/LongCumSumOperator.java | 11 +- .../updateby/sum/ShortCumSumOperator.java | 11 +- .../replicators/ReplicateUpdateBy.java | 14 ++ 84 files changed, 731 insertions(+), 585 deletions(-) rename engine/table/src/main/java/io/deephaven/engine/table/impl/{ => updateby}/BucketedPartitionedUpdateByManager.java (88%) rename engine/table/src/main/java/io/deephaven/engine/table/impl/{ => updateby}/UpdateBy.java (87%) rename engine/table/src/main/java/io/deephaven/engine/table/impl/{ => updateby}/UpdateByBucketHelper.java (92%) rename engine/table/src/main/java/io/deephaven/engine/table/impl/{ => updateby}/UpdateByCumulativeOperator.java (85%) rename engine/table/src/main/java/io/deephaven/engine/table/impl/{ => updateby}/UpdateByOperator.java (91%) rename engine/table/src/main/java/io/deephaven/engine/table/impl/{ => updateby}/UpdateByOperatorFactory.java (99%) rename engine/table/src/main/java/io/deephaven/engine/table/impl/{ => updateby}/UpdateByWindowedOperator.java (92%) rename engine/table/src/main/java/io/deephaven/engine/table/impl/{ => updateby}/ZeroKeyUpdateByManager.java (90%) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/JoinControl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/JoinControl.java index 1f75b0fa797..46c7f0f9b6d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/JoinControl.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/JoinControl.java @@ -87,7 +87,7 @@ RedirectionType getRedirectionType(Table leftTable) { return getRedirectionType(leftTable, 4.0, true); } - static RedirectionType getRedirectionType(final @NotNull Table table, final double maximumOverhead, + public static RedirectionType getRedirectionType(final @NotNull Table table, final double maximumOverhead, final boolean allowSparseRedirection) { if (table.isFlat() && table.size() < Integer.MAX_VALUE) { if (table.isRefreshing()) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java index 0f28c31b7e7..4490221998e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java @@ -33,6 +33,7 @@ import io.deephaven.engine.table.impl.perf.QueryPerformanceNugget; import io.deephaven.engine.table.impl.select.MatchPairFactory; import io.deephaven.engine.table.impl.select.SelectColumnFactory; +import io.deephaven.engine.table.impl.updateby.UpdateBy; import io.deephaven.engine.table.impl.util.ImmediateJobScheduler; import io.deephaven.engine.table.impl.util.JobScheduler; import io.deephaven.engine.table.impl.util.OperationInitializationPoolJobScheduler; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UncoalescedTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UncoalescedTable.java index ddfbb8a5c8e..ff09baab5b2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UncoalescedTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UncoalescedTable.java @@ -16,6 +16,7 @@ import io.deephaven.engine.liveness.Liveness; import io.deephaven.engine.rowset.TrackingRowSet; import io.deephaven.engine.table.*; +import io.deephaven.engine.table.impl.updateby.UpdateBy; import io.deephaven.engine.table.iterators.*; import io.deephaven.api.util.ConcurrentMethod; import io.deephaven.util.QueryConstants; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java similarity index 88% rename from engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java rename to engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java index 82eb3b6baca..4e7120ad425 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketedPartitionedUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java @@ -1,16 +1,22 @@ -package io.deephaven.engine.table.impl; +package io.deephaven.engine.table.impl.updateby; import io.deephaven.api.ColumnName; import io.deephaven.api.updateby.UpdateByControl; -import io.deephaven.base.Pair; -import io.deephaven.engine.rowset.*; -import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.updateby.UpdateByWindow; +import io.deephaven.engine.rowset.RowSetFactory; +import io.deephaven.engine.rowset.RowSetShiftData; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.ModifiedColumnSet; +import io.deephaven.engine.table.PartitionedTable; +import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.impl.QueryTable; +import io.deephaven.engine.table.impl.TableUpdateImpl; import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.util.*; +import java.util.Collection; +import java.util.List; +import java.util.Map; /** * An implementation of {@link UpdateBy} dedicated to bucketed computation. @@ -34,7 +40,7 @@ class BucketedPartitionedUpdateByManager extends UpdateBy { * @param resultSources the result sources * @param byColumns the columns to use for the bucket keys * @param timestampColumnName the column to use for all time-aware operators - * @param redirHelper the row redirection helper for dense output sources + * @param rowRedirection the row redirection for dense output sources * @param control the control object. */ protected BucketedPartitionedUpdateByManager( @@ -86,7 +92,7 @@ protected BucketedPartitionedUpdateByManager( operatorInputSourceSlots, resultSources, timestampColumnName, - redirHelper.getRowRedirection(), + rowRedirection, control); // add this to the bucket list diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java similarity index 87% rename from engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java rename to engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java index 4e92c671775..cb937761f18 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java @@ -1,4 +1,4 @@ -package io.deephaven.engine.table.impl; +package io.deephaven.engine.table.impl.updateby; import gnu.trove.list.array.TIntArrayList; import gnu.trove.map.hash.TIntObjectHashMap; @@ -20,9 +20,9 @@ import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.*; +import io.deephaven.engine.table.impl.*; import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.sources.sparse.SparseConstants; -import io.deephaven.engine.table.impl.updateby.UpdateByWindow; import io.deephaven.engine.table.impl.util.*; import io.deephaven.engine.updategraph.UpdateGraphProcessor; import io.deephaven.util.datastructures.linked.IntrusiveDoublyLinkedNode; @@ -37,21 +37,19 @@ import java.util.concurrent.ExecutionException; import java.util.stream.IntStream; -import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; - /** * The core of the {@link Table#updateBy(UpdateByControl, Collection, Collection)} operation. */ public abstract class UpdateBy { /** When caching a column source, how many rows should we process in each parallel batch? (1M default) */ - public static final int PARALLEL_CACHE_BATCH_SIZE = + private static final int PARALLEL_CACHE_BATCH_SIZE = Configuration.getInstance().getIntegerWithDefault("UpdateBy.parallelCacheBatchSize", 1 << 20); /** When caching a column source, what size chunks should be used to move data to the cache? (64K default) */ - public static final int PARALLEL_CACHE_CHUNK_SIZE = + private static final int PARALLEL_CACHE_CHUNK_SIZE = Configuration.getInstance().getIntegerWithDefault("UpdateBy.parallelCacheChunkSize", 1 << 16); /** When extracting keys from the redirection, what size chunks to use? (2K default) */ - public static final int REDIRECTION_CHUNK_SIZE = 1 << 11; + private static final int REDIRECTION_CHUNK_SIZE = 1 << 11; /** Input sources may be reused by multiple operators, only store and cache unique ones (post-reinterpret) */ protected final ColumnSource[] inputSources; @@ -90,15 +88,15 @@ public abstract class UpdateBy { /** Store every bucket in this list for processing */ protected final IntrusiveDoublyLinkedQueue buckets; - public static class UpdateByRedirectionHelper { + static class UpdateByRedirectionHelper { @Nullable private final WritableRowRedirection rowRedirection; - private final WritableRowSet freeRows; + private final TrackingWritableRowSet freeRows; private long maxInnerRowKey; private UpdateByRedirectionHelper(@Nullable final WritableRowRedirection rowRedirection) { this.rowRedirection = rowRedirection; - this.freeRows = rowRedirection == null ? null : RowSetFactory.empty(); + this.freeRows = rowRedirection == null ? null : RowSetFactory.empty().toTracking(); this.maxInnerRowKey = 0; } @@ -162,6 +160,21 @@ private RowSet getInnerKeys(final RowSet outerKeys) { } return builder.build(); } + + /*** + * Compute the inner source keys that need to be cleared. These are rows that were removed this cycle and not + * replaced by added rows. These are in the dense key-space and must only be applied to the inner sources of the + * redirected output sources + * + * @return the set of rows that should be cleared from the inner (dense) sources + */ + WritableRowSet getRowsToClear() { + final WritableRowSet toClear = freeRows.copy(); + try (final RowSet prevKeys = freeRows.copyPrev()) { + toClear.remove(prevKeys); + } + return toClear; + } } protected UpdateBy( @@ -273,7 +286,15 @@ class PhasedUpdateProcessor implements LogOutputAppendable { final JobScheduler jobScheduler; final CompletableFuture waitForResult; - WritableRowSet shiftedRows; + /*** + * These rows will be changed because of shifts or removes and will need to be included in + * {@code prepareForParallelPopulation()} calls + */ + WritableRowSet changedRows; + /*** + * These rows will be unused after this cycle and Object columns should NULL these keys + */ + WritableRowSet toClear; PhasedUpdateProcessor(TableUpdate upstream, boolean initialStep) { this.upstream = upstream; @@ -289,7 +310,7 @@ class PhasedUpdateProcessor implements LogOutputAppendable { // set the uncacheable columns into the array for (int ii = 0; ii < inputSources.length; ii++) { - maybeCachedInputSources[ii] = inputSourceCacheNeeded[ii] ? inputSources[ii] : null; + maybeCachedInputSources[ii] = inputSourceCacheNeeded[ii] ? null : inputSources[ii]; } } else { maybeCachedInputSources = inputSources; @@ -311,6 +332,7 @@ class PhasedUpdateProcessor implements LogOutputAppendable { } else { jobScheduler = ImmediateJobScheduler.INSTANCE; } + waitForResult = null; } } @@ -335,9 +357,9 @@ private void onError(Exception error) { * Accumulate in parallel the dirty bucket rowsets for the cacheable input sources. Calls * {@code completedAction} when the work is complete */ - private void computeCachedColumnRowsets(final Runnable completeAction) { + private void computeCachedColumnRowsets(final Runnable resumeAction) { if (!inputCacheNeeded) { - completeAction.run(); + resumeAction.run(); return; } @@ -353,7 +375,7 @@ private void computeCachedColumnRowsets(final Runnable completeAction) { (int) Arrays.stream(windows).filter(win -> win.isSourceInUse(srcIdx)).count(); } } - completeAction.run(); + resumeAction.run(); return; } @@ -366,7 +388,7 @@ private void computeCachedColumnRowsets(final Runnable completeAction) { if (win.isSourceInUse(srcIdx)) { boolean srcNeeded = false; for (UpdateByBucketHelper bucket : dirtyBuckets) { - UpdateByWindow.UpdateByWindowContext winCtx = bucket.windowContexts[winIdx]; + UpdateByWindow.UpdateByWindowBucketContext winCtx = bucket.windowContexts[winIdx]; if (win.isWindowDirty(winCtx)) { // add this rowset to the running total for this input source @@ -386,7 +408,7 @@ private void computeCachedColumnRowsets(final Runnable completeAction) { } } }, - completeAction, + resumeAction, this::onError); } @@ -394,10 +416,10 @@ private void computeCachedColumnRowsets(final Runnable completeAction) { * Create a new input source cache and populate the required rows in parallel. Calls {@code completedAction} * when the work is complete */ - private void createCachedColumnSource(int srcIdx, final Runnable completeAction) { + private void createCachedColumnSource(int srcIdx, final Runnable resumeAction) { if (maybeCachedInputSources[srcIdx] != null || inputSourceRowSets[srcIdx] == null) { // already cached from another operator (or caching not needed) - completeAction.run(); + resumeAction.run(); return; } @@ -422,57 +444,44 @@ private void createCachedColumnSource(int srcIdx, final Runnable completeAction) // holding this reference should protect `rowDirection` and `innerSource` from GC maybeCachedInputSources[srcIdx] = outputSource; - if (inputRowSet.size() >= PARALLEL_CACHE_BATCH_SIZE) { - // divide the rowset into reasonable chunks and do the cache population in parallel - final ArrayList populationRowSets = new ArrayList<>(); - try (final RowSequence.Iterator rsIt = inputRowSet.getRowSequenceIterator()) { - while (rsIt.hasMore()) { - final RowSequence chunkOk = rsIt.getNextRowSequenceWithLength(PARALLEL_CACHE_BATCH_SIZE); - populationRowSets.add(chunkOk.asRowSet().copy()); - } - } - jobScheduler.iterateParallel(ExecutionContext.getContextToRecord(), this, - 0, populationRowSets.size(), - idx -> { - try (final RowSet chunkRs = populationRowSets.get(idx); - final RowSequence.Iterator rsIt = chunkRs.getRowSequenceIterator(); - final ChunkSink.FillFromContext ffc = - outputSource.makeFillFromContext(PARALLEL_CACHE_CHUNK_SIZE); - final ChunkSource.GetContext gc = - inputSource.makeGetContext(PARALLEL_CACHE_CHUNK_SIZE)) { - while (rsIt.hasMore()) { - final RowSequence chunkOk = - rsIt.getNextRowSequenceWithLength(PARALLEL_CACHE_CHUNK_SIZE); - final Chunk values = inputSource.getChunk(gc, chunkOk); - outputSource.fillFromChunk(ffc, values, chunkOk); - } + final int rowSetSize = inputRowSet.intSize(); + + // how many batches do we need? + final int taskCount = (rowSetSize + PARALLEL_CACHE_BATCH_SIZE - 1) / PARALLEL_CACHE_BATCH_SIZE; + + jobScheduler.iterateParallel(ExecutionContext.getContextToRecord(), this, + 0, taskCount, + idx -> { + try (final RowSequence.Iterator rsIt = inputRowSet.getRowSequenceIterator(); + final ChunkSink.FillFromContext ffc = + outputSource.makeFillFromContext(PARALLEL_CACHE_CHUNK_SIZE); + final ChunkSource.GetContext gc = + inputSource.makeGetContext(PARALLEL_CACHE_CHUNK_SIZE)) { + // advance to the first key of this block + rsIt.advance(inputRowSet.get(idx * PARALLEL_CACHE_BATCH_SIZE)); + int count = 0; + while (rsIt.hasMore() && count < PARALLEL_CACHE_BATCH_SIZE) { + final RowSequence chunkOk = + rsIt.getNextRowSequenceWithLength(PARALLEL_CACHE_CHUNK_SIZE); + final Chunk values = inputSource.getChunk(gc, chunkOk); + outputSource.fillFromChunk(ffc, values, chunkOk); + + // increment by the attempted stride, if this is the last block the iterator will + // be exhausted and hasMore() will return false + count += PARALLEL_CACHE_CHUNK_SIZE; } - }, () -> { - populationRowSets.clear(); - completeAction.run(); - }, - this::onError); - } else { - // run this in serial, not worth parallelization - try (final RowSequence.Iterator rsIt = inputRowSet.getRowSequenceIterator(); - final ChunkSink.FillFromContext ffc = - outputSource.makeFillFromContext(PARALLEL_CACHE_CHUNK_SIZE); - final ChunkSource.GetContext gc = inputSource.makeGetContext(PARALLEL_CACHE_CHUNK_SIZE)) { - while (rsIt.hasMore()) { - final RowSequence chunkOk = rsIt.getNextRowSequenceWithLength(PARALLEL_CACHE_CHUNK_SIZE); - final Chunk values = inputSource.getChunk(gc, chunkOk); - outputSource.fillFromChunk(ffc, values, chunkOk); - } - } - completeAction.run(); - } + } + }, () -> { + resumeAction.run(); + }, + this::onError); } /** * Create cached input sources for all input needed by {@code windows[winIdx]}. Calls {@code completedAction} * when the work is complete */ - private void cacheInputSources(final int winIdx, final Runnable completeAction) { + private void cacheInputSources(final int winIdx, final Runnable resumeAction) { if (inputCacheNeeded) { final UpdateByWindow win = windows[winIdx]; final int[] uniqueWindowSources = win.getUniqueSourceIndices(); @@ -480,45 +489,33 @@ private void cacheInputSources(final int winIdx, final Runnable completeAction) jobScheduler.iterateParallel(ExecutionContext.getContextToRecord(), this, 0, uniqueWindowSources.length, (idx, sourceComplete) -> { createCachedColumnSource(uniqueWindowSources[idx], sourceComplete); - }, completeAction, this::onError); + }, resumeAction, this::onError); } else { // no work to do, continue - completeAction.run(); + resumeAction.run(); } } /** * Divide the buckets for {@code windows[winIdx]} into sets and process each set in parallel. Calls - * {@code completedAction} when the work is complete + * {@code resumeAction} when the work is complete */ - private void processWindowBuckets(int winIdx, final Runnable completedAction) { + private void processWindowBuckets(int winIdx, final Runnable resumeAction) { if (jobScheduler.threadCount() > 1 && dirtyBuckets.length > 1) { // process the buckets in parallel - final int bucketsPerTask = Math.max(1, dirtyBuckets.length / jobScheduler.threadCount()); - final TIntArrayList offsetList = new TIntArrayList(); - final TIntArrayList countList = new TIntArrayList(); - - for (int ii = 0; ii < dirtyBuckets.length; ii += bucketsPerTask) { - offsetList.add(ii); - countList.add(Math.min(bucketsPerTask, dirtyBuckets.length - ii)); - } - - jobScheduler.iterateParallel(ExecutionContext.getContextToRecord(), this, 0, offsetList.size(), idx -> { - final int bucketOffset = offsetList.get(idx); - final int bucketCount = countList.get(idx); - for (int bucketIdx = bucketOffset; bucketIdx < bucketOffset + bucketCount; bucketIdx++) { - UpdateByBucketHelper bucket = dirtyBuckets[bucketIdx]; - bucket.assignInputSources(winIdx, maybeCachedInputSources); - bucket.processWindow(winIdx, initialStep); - } - }, completedAction, this::onError); + jobScheduler.iterateParallel(ExecutionContext.getContextToRecord(), this, 0, dirtyBuckets.length, + bucketIdx -> { + UpdateByBucketHelper bucket = dirtyBuckets[bucketIdx]; + bucket.assignInputSources(winIdx, maybeCachedInputSources); + bucket.processWindow(winIdx, initialStep); + }, resumeAction, this::onError); } else { // minimize overhead when running serially for (UpdateByBucketHelper bucket : dirtyBuckets) { bucket.assignInputSources(winIdx, maybeCachedInputSources); bucket.processWindow(winIdx, initialStep); } - completedAction.run(); + resumeAction.run(); } } @@ -527,7 +524,7 @@ private void processWindowBuckets(int winIdx, final Runnable completedAction) { * to fill the cached input sources). Will create cached input sources, process the buckets, then release the * cached columns before starting the next window. Calls {@code completedAction} when the work is complete */ - private void processWindows(final Runnable completeAction) { + private void processWindows(final Runnable resumeAction) { jobScheduler.iterateSerial(ExecutionContext.getContextToRecord(), this, 0, windows.length, (winIdx, windowComplete) -> { UpdateByWindow win = windows[winIdx]; @@ -540,14 +537,12 @@ private void processWindows(final Runnable completeAction) { try (final RowSet changedRows = redirHelper.isRedirected() ? RowSetFactory.flat(redirHelper.requiredCapacity()) : source.getRowSet().copy()) { - for (UpdateByOperator op : win.getOperators()) { - op.prepareForParallelPopulation(changedRows); - } + win.prepareForParallelPopulation(changedRows); } } else { // get the minimal set of rows to be updated for this window (shiftedRows is empty when // using redirection) - try (final WritableRowSet windowRowSet = shiftedRows.copy()) { + try (final WritableRowSet windowRowSet = changedRows.copy()) { for (UpdateByBucketHelper bucket : dirtyBuckets) { if (win.isWindowDirty(bucket.windowContexts[winIdx])) { windowRowSet.insert(win.getAffectedRows(bucket.windowContexts[winIdx])); @@ -556,9 +551,7 @@ private void processWindows(final Runnable completeAction) { try (final RowSet changedRows = redirHelper.isRedirected() ? redirHelper.getInnerKeys(windowRowSet) : windowRowSet.copy()) { - for (UpdateByOperator op : win.getOperators()) { - op.prepareForParallelPopulation(changedRows); - } + win.prepareForParallelPopulation(changedRows); } } } @@ -588,18 +581,14 @@ private void processWindows(final Runnable completeAction) { windowComplete.run(); }); }); - }, completeAction, this::onError); + }, resumeAction, this::onError); } /** * Clean up the resources created during this update and notify downstream if applicable. Calls * {@code completedAction} when the work is complete */ - private void cleanUpAndNotify(final Runnable completeAction) { - try (final RowSet ignoredRs = shiftedRows) { - // auto close these resources - } - + private void cleanUpAndNotify(final Runnable resumeAction) { // create the downstream before calling finalize() on the buckets (which releases resources) final TableUpdate downstream; if (!initialStep) { @@ -618,14 +607,28 @@ private void cleanUpAndNotify(final Runnable completeAction) { result().notifyListeners(downstream); } - completeAction.run(); + // clear the sparse output columns for rows that no longer exist + if (!initialStep && !redirHelper.isRedirected()) { + if (!toClear.isEmpty()) { + for (UpdateByOperator op : operators) { + op.clearOutputRows(toClear); + } + } + } + + try (final RowSet ignoredRs = changedRows; + final RowSet ignoredRs2 = toClear) { + // auto close these resources + } + resumeAction.run(); } /** * Clean up the resources created during this update. */ private void cleanUpAfterError() { - try (final RowSet ignoredRs = shiftedRows) { + try (final RowSet ignoredRs = changedRows; + final RowSet ignoredRs2 = toClear) { // auto close these resources } @@ -662,7 +665,7 @@ private TableUpdate computeDownstreamUpdate() { // retrieve the modified row and column sets from the windows for (int winIdx = 0; winIdx < windows.length; winIdx++) { UpdateByWindow win = windows[winIdx]; - UpdateByWindow.UpdateByWindowContext winCtx = bucket.windowContexts[winIdx]; + UpdateByWindow.UpdateByWindowBucketContext winCtx = bucket.windowContexts[winIdx]; if (win.isWindowDirty(winCtx)) { // add the window modified rows to this set @@ -689,11 +692,27 @@ private TableUpdate computeDownstreamUpdate() { */ public void processUpdate() { if (redirHelper.isRedirected()) { - // this call does all the work needed for redirected output sources, for sparse output sources - // we will process shifts only after a call to `prepareForParallelPopulation()` on each source + // this call does all the work needed for redirected output sources, including handling removed rows redirHelper.processUpdateForRedirection(upstream, source.getRowSet()); - shiftedRows = RowSetFactory.empty(); + changedRows = RowSetFactory.empty(); + + // identify which rows we need to clear in our Object columns. These rows will not intersect with rows + // we intend to modify later + toClear = redirHelper.getRowsToClear(); + + // clear them now and let them set their own prev states + if (!initialStep) { + if (!toClear.isEmpty()) { + for (UpdateByOperator op : operators) { + op.clearOutputRows(toClear); + } + } + } } else { + // identify which rows we need to clear in our Object columns (actual clearing will be performed later) + toClear = source.getRowSet().copyPrev(); + toClear.remove(source.getRowSet()); + // for our sparse array output sources, we need to identify which rows will be affected by the upstream // shifts and include them in our parallel update preparations if (upstream.shifted().nonempty()) { @@ -713,11 +732,13 @@ public void processUpdate() { final RowSequence rs = it.getNextRowSequenceThrough(end); builder.appendRowSequenceWithOffset(rs, delta); } - shiftedRows = builder.build(); + changedRows = builder.build(); } } else { - shiftedRows = RowSetFactory.empty(); + changedRows = RowSetFactory.empty(); } + // include the cleared rows in the calls to `prepareForParallelPopulation()` + changedRows.insert(toClear); } // this is where we leave single-threaded calls and rely on the scheduler to continue the work. Each diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java similarity index 92% rename from engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java rename to engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java index b2df7065e69..5b7a362944e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByBucketHelper.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java @@ -1,16 +1,22 @@ -package io.deephaven.engine.table.impl; +package io.deephaven.engine.table.impl.updateby; import io.deephaven.api.updateby.UpdateByControl; import io.deephaven.base.verify.Assert; -import io.deephaven.chunk.*; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.WritableLongChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.table.*; +import io.deephaven.engine.table.ChunkSource; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.ModifiedColumnSet; +import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.InstrumentedTableUpdateListenerAdapter; +import io.deephaven.engine.table.impl.QueryTable; +import io.deephaven.engine.table.impl.TableUpdateImpl; import io.deephaven.engine.table.impl.sources.ReinterpretUtils; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; -import io.deephaven.engine.table.impl.updateby.UpdateByWindow; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import io.deephaven.util.datastructures.linked.IntrusiveDoublyLinkedNode; import org.apache.commons.lang3.mutable.MutableLong; import org.jetbrains.annotations.NotNull; @@ -25,19 +31,19 @@ * bucket of rows. */ class UpdateByBucketHelper extends IntrusiveDoublyLinkedNode.Impl { - public static final int SSA_LEAF_SIZE = 4096; + private static final int SSA_LEAF_SIZE = 4096; protected final ColumnSource[] inputSources; // some columns will have multiple inputs, such as time-based and Weighted computations final int[][] operatorInputSourceSlots; final UpdateByOperator[] operators; final UpdateByWindow[] windows; final QueryTable source; - final WritableRowRedirection rowRedirection; + final RowRedirection rowRedirection; final UpdateByControl control; final QueryTable result; - /** An array of {@link UpdateByWindow.UpdateByWindowContext}s for each window */ - final UpdateByWindow.UpdateByWindowContext[] windowContexts; + /** An array of {@link UpdateByWindow.UpdateByWindowBucketContext}s for each window */ + final UpdateByWindow.UpdateByWindowBucketContext[] windowContexts; /** store timestamp data in an SSA (if needed) */ final String timestampColumnName; @@ -70,7 +76,7 @@ protected UpdateByBucketHelper(@NotNull final String description, @NotNull final int[][] operatorInputSourceSlots, @NotNull final Map> resultSources, @Nullable String timestampColumnName, - @Nullable final WritableRowRedirection rowRedirection, + @Nullable final RowRedirection rowRedirection, @NotNull final UpdateByControl control) { this.source = source; @@ -96,7 +102,7 @@ protected UpdateByBucketHelper(@NotNull final String description, this.timestampColumnSet = null; } - this.windowContexts = new UpdateByWindow.UpdateByWindowContext[windows.length]; + this.windowContexts = new UpdateByWindow.UpdateByWindowBucketContext[windows.length]; // make a fake update with the initial rows of the table final TableUpdateImpl initialUpdate = new TableUpdateImpl(source.getRowSet(), @@ -231,8 +237,8 @@ private void fillChunkWithNonNull(LongChunk keysChunk, LongChunk /** * Calling this function will prepare this bucket for computation, including making a - * {@link UpdateByWindow.UpdateByWindowContext} for each window and computing the affected and influencer rowsets - * for each window + * {@link UpdateByWindow.UpdateByWindowBucketContext} for each window and computing the affected and influencer + * rowsets for each window * * @param upstream The incoming update for which to prepare * @param initialStep Whether this update is part of the initial creation of the bucket diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByCumulativeOperator.java similarity index 85% rename from engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java rename to engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByCumulativeOperator.java index b1f5d316310..54e414a4785 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByCumulativeOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByCumulativeOperator.java @@ -1,11 +1,11 @@ -package io.deephaven.engine.table.impl; +package io.deephaven.engine.table.impl.updateby; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; @@ -45,11 +45,11 @@ public abstract void accumulate(RowSequence inputKeys, * * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this operation - * @param redirHelper the row redirection context to use for the operation + * @param rowRedirection the row redirection context to use for the operation */ public UpdateByCumulativeOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final WritableRowRedirection rowRedirection) { + @Nullable final RowRedirection rowRedirection) { super(pair, affectingColumns, null, null, 0L, 0L, rowRedirection); } @@ -59,13 +59,13 @@ public UpdateByCumulativeOperator(@NotNull final MatchPair pair, * * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this operation - * @param redirHelper the row redirection context to use for the operation + * @param rowRedirection the row redirection context to use for the operation */ public UpdateByCumulativeOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final WritableRowRedirection rowRedirection, + @Nullable final RowRedirection rowRedirection, @Nullable final String timestampColumnName, - @NotNull final long reverseTimeScaleUnits) { + final long reverseTimeScaleUnits) { super(pair, affectingColumns, null, timestampColumnName, reverseTimeScaleUnits, 0L, rowRedirection); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperator.java similarity index 91% rename from engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java rename to engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperator.java index 13246c1594e..d9a6a0868c2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperator.java @@ -1,4 +1,4 @@ -package io.deephaven.engine.table.impl; +package io.deephaven.engine.table.impl.updateby; import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; @@ -11,7 +11,8 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.ModifiedColumnSet; import io.deephaven.engine.table.Table; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.QueryTable; +import io.deephaven.engine.table.impl.util.RowRedirection; import io.deephaven.util.SafeCloseable; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -25,10 +26,9 @@ *

                  *
                1. {@link UpdateByCumulativeOperator#initializeUpdate(UpdateContext, long, long)} for cumulative operators or * {@link UpdateByWindowedOperator#initializeUpdate(UpdateContext)} for windowed operators
                2. - *
                3. {@link io.deephaven.engine.table.impl.UpdateByCumulativeOperator.Context#accumulate(RowSequence, Chunk[], LongChunk, int)} - * for cumulative operators or - * {@link UpdateByWindowedOperator.Context#accumulate(RowSequence, Chunk[], IntChunk, IntChunk, int)} for windowed - * operators
                4. + *
                5. {@link UpdateByCumulativeOperator.Context#accumulate(RowSequence, Chunk[], LongChunk, int)} for cumulative + * operators or {@link UpdateByWindowedOperator.Context#accumulate(RowSequence, Chunk[], IntChunk, IntChunk, int)} for + * windowed operators
                6. *
                7. {@link #finishUpdate(UpdateContext)}
                8. *
                */ @@ -37,7 +37,7 @@ public abstract class UpdateByOperator { protected final MatchPair pair; protected final String[] affectingColumns; - protected final WritableRowRedirection rowRedirection; + protected final RowRedirection rowRedirection; // these will be used by the timestamp-aware operators (EMA for example) protected final OperationControl control; @@ -103,7 +103,7 @@ protected UpdateByOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @Nullable final WritableRowRedirection rowRedirection) { + @Nullable final RowRedirection rowRedirection) { this.pair = pair; this.affectingColumns = affectingColumns; this.rowRedirection = rowRedirection; @@ -235,4 +235,9 @@ public ModifiedColumnSet getOutputModifiedColumnSet() { return outputModifiedColumnSet; } + /** + * Clear the output rows by setting value to NULL. Dense sources will apply removes to the inner source. + */ + public abstract void clearOutputRows(RowSet toClear); + } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperatorFactory.java similarity index 99% rename from engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java rename to engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperatorFactory.java index da7e3f6023e..a24cb388bc2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByOperatorFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperatorFactory.java @@ -1,4 +1,4 @@ -package io.deephaven.engine.table.impl; +package io.deephaven.engine.table.impl.updateby; import io.deephaven.api.agg.Pair; import io.deephaven.api.updateby.ColumnUpdateOperation; @@ -9,6 +9,7 @@ import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.impl.TableDefaults; import io.deephaven.engine.table.impl.updateby.ema.*; import io.deephaven.engine.table.impl.updateby.fill.*; import io.deephaven.engine.table.impl.updateby.minmax.*; @@ -208,7 +209,6 @@ public Void visit(@NotNull final RollingSumSpec rs) { return null; } - @SuppressWarnings("unchecked") private UpdateByOperator makeEmaOperator(@NotNull final MatchPair pair, @NotNull final TableDefaults source, @NotNull final EmaSpec ema) { @@ -295,7 +295,7 @@ private UpdateByOperator makeCumMinMaxOperator(MatchPair fc, TableDefaults sourc } else if (csType == double.class || csType == Double.class) { return new DoubleCumMinMaxOperator(fc, isMax, rowRedirection); } else if (Comparable.class.isAssignableFrom(csType)) { - // noinspection unchecked,rawtypes + // noinspection rawtypes return new ComparableCumMinMaxOperator(csType, fc, isMax, rowRedirection); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java index c7364d3dca7..c69e086d818 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java @@ -8,38 +8,39 @@ import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.TableUpdate; -import io.deephaven.engine.table.impl.UpdateByOperator; -import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.util.SafeCloseable; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import java.util.Arrays; +import java.util.Objects; + public abstract class UpdateByWindow { @Nullable protected final String timestampColumnName; - // store the operators for this window + /** The operators for this window */ protected final UpdateByOperator[] operators; - // store the index in the {@link UpdateBy.inputSources} + /** The indices in the UpdateBy input source collection for each operator input slots */ protected final int[][] operatorInputSourceSlots; protected int[] uniqueInputSourceIndices; /** This context will store the necessary info to process a single window for a single bucket */ - public abstract class UpdateByWindowContext implements SafeCloseable { - /** store a reference to the source rowset */ + public abstract class UpdateByWindowBucketContext implements SafeCloseable { + /** A reference to the source rowset */ protected final TrackingRowSet sourceRowSet; - /** the column source providing the timestamp data for this window */ + /** The column source providing the timestamp data for this window */ @Nullable protected final ColumnSource timestampColumnSource; - /** the timestamp SSA providing fast lookup for time windows */ + /** The timestamp SSA providing fast lookup for time windows */ @Nullable protected final LongSegmentedSortedArray timestampSsa; /** An array of context objects for each underlying operator */ protected final UpdateByOperator.UpdateContext[] opContext; - /** Whether this is the creation phase of this operator */ + /** Whether this is the creation phase of this window */ protected final boolean initialStep; /** An array of ColumnSources for each underlying operator */ @@ -63,9 +64,11 @@ public abstract class UpdateByWindowContext implements SafeCloseable { /** Indicates which sources are needed to process this window context */ protected int[] dirtySourceIndices; - public UpdateByWindowContext(final TrackingRowSet sourceRowSet, + public UpdateByWindowBucketContext(final TrackingRowSet sourceRowSet, @Nullable final ColumnSource timestampColumnSource, - @Nullable final LongSegmentedSortedArray timestampSsa, final int chunkSize, final boolean initialStep) { + @Nullable final LongSegmentedSortedArray timestampSsa, + final int chunkSize, + final boolean initialStep) { this.sourceRowSet = sourceRowSet; this.timestampColumnSource = timestampColumnSource; this.timestampSsa = timestampSsa; @@ -74,7 +77,6 @@ public UpdateByWindowContext(final TrackingRowSet sourceRowSet, this.workingChunkSize = chunkSize; this.initialStep = initialStep; - this.isDirty = false; } @Override @@ -102,7 +104,8 @@ public void close() { } } - public abstract UpdateByWindowContext makeWindowContext(final TrackingRowSet sourceRowSet, + + abstract UpdateByWindowBucketContext makeWindowContext(final TrackingRowSet sourceRowSet, final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa, final int chunkSize, @@ -169,13 +172,6 @@ public UpdateByOperator[] getOperators() { return operators; } - /** - * Returns the mapping from operator indices to input source indices - */ - public int[][] getOperatorInputSourceSlots() { - return operatorInputSourceSlots; - } - public int[] getUniqueSourceIndices() { if (uniqueInputSourceIndices == null) { final TIntHashSet set = new TIntHashSet(); @@ -205,9 +201,9 @@ public boolean isSourceInUse(int srcIdx) { } /** - * Pre-create all the modified/new rows in the output source so they can be updated in parallel tasks + * Pre-create all the modified/new rows in the output source for parallel update * - * @param changes the rowset indicating which rows will be modifed or added this cycle + * @param changes the rowset indicating which rows will be modified or added this cycle */ public void prepareForParallelPopulation(final RowSet changes) { for (UpdateByOperator operator : operators) { @@ -224,16 +220,9 @@ public void prepareForParallelPopulation(final RowSet changes) { * @param context the window context that will store the results. * @param upstream the update that indicates incoming changes to the data. */ - public abstract void computeAffectedRowsAndOperators(final UpdateByWindowContext context, + public abstract void computeAffectedRowsAndOperators(final UpdateByWindowBucketContext context, @NotNull final TableUpdate upstream); - /** - * Generate the contexts used by the operators for this bucket. - * - * @param context the window context that will store the results. - */ - protected abstract void makeOperatorContexts(final UpdateByWindowContext context); - /** * Accepts and stores the input source array that will be used for future computation. This call allows use of * cached input sources instead of potentially slow access to the original input sources @@ -241,11 +230,10 @@ public abstract void computeAffectedRowsAndOperators(final UpdateByWindowContext * @param context the window context that will store these sources. * @param inputSources the (potentially cached) input sources to use for processing. */ - public void assignInputSources(final UpdateByWindowContext context, final ColumnSource[] inputSources) { + public void assignInputSources(final UpdateByWindowBucketContext context, final ColumnSource[] inputSources) { context.inputSources = inputSources; context.inputSourceChunkPopulated = new boolean[inputSources.length]; context.inputSourceGetContexts = new ChunkSource.GetContext[inputSources.length]; - // noinspection unchecked context.inputSourceChunks = new WritableChunk[inputSources.length]; for (int srcIdx : context.dirtySourceIndices) { @@ -261,7 +249,7 @@ public void assignInputSources(final UpdateByWindowContext context, final Column * @param srcIdx the index of the input source. * @param rs the rows to retrieve. */ - protected void prepareValuesChunkForSource(final UpdateByWindowContext context, final int srcIdx, + protected void prepareValuesChunkForSource(final UpdateByWindowBucketContext context, final int srcIdx, final RowSequence rs) { if (!context.inputSourceChunkPopulated[srcIdx]) { context.inputSourceChunks[srcIdx] = @@ -276,41 +264,32 @@ protected void prepareValuesChunkForSource(final UpdateByWindowContext context, * @param context the window context that will manage the results. * @param initialStep whether this is the creation step of the table. */ - public abstract void processRows(final UpdateByWindowContext context, final boolean initialStep); + public abstract void processRows(final UpdateByWindowBucketContext context, final boolean initialStep); /** * Returns `true` if the window for this bucket needs to be processed this cycle. * * @param context the window context that will manage the results. */ - public boolean isWindowDirty(final UpdateByWindowContext context) { + public boolean isWindowDirty(final UpdateByWindowBucketContext context) { return context.isDirty; } /** - * Returns `true` if the window for this bucket needs to be processed this cycle. + * Returns the list of `dirty` operators that need to be processed this cycle. * * @param context the window context that will manage the results. */ - public int[] getDirtyOperators(final UpdateByWindowContext context) { + public int[] getDirtyOperators(final UpdateByWindowBucketContext context) { return context.dirtyOperatorIndices; } - /** - * Returns the list of input sources that will be needed to process the `dirty` operators for this bucket - * - * @param context the window context that will manage the results. - */ - public int[] getDirtySources(final UpdateByWindowContext context) { - return context.dirtySourceIndices; - } - /** * Returns the rows that will be recomputed for this bucket this cycle * * @param context the window context that will manage the results. */ - public RowSet getAffectedRows(final UpdateByWindowContext context) { + public RowSet getAffectedRows(final UpdateByWindowBucketContext context) { return context.affectedRows; } @@ -319,7 +298,7 @@ public RowSet getAffectedRows(final UpdateByWindowContext context) { * * @param context the window context that will manage the results. */ - public RowSet getInfluencerRows(final UpdateByWindowContext context) { + public RowSet getInfluencerRows(final UpdateByWindowBucketContext context) { return context.influencerRows; } @@ -338,14 +317,15 @@ protected static int hashCode(boolean windowed, @NotNull String[] inputColumnNam hash = 31 * hash + s.hashCode(); } + hash = 31 * hash + Boolean.hashCode(true); + // treat all cumulative ops with the same input columns as identical, even if they rely on timestamps if (!windowed) { - return 31 * hash + Boolean.hashCode(false); + return hash; } // windowed ops are unique per type (ticks/time-based) and window dimensions - hash = 31 * hash + Boolean.hashCode(true); - hash = 31 * hash + Boolean.hashCode(timestampColumnName != null); + hash = 31 * hash + Objects.hashCode(timestampColumnName); hash = 31 * hash + Long.hashCode(prevUnits); hash = 31 * hash + Long.hashCode(fwdUnits); return hash; @@ -366,18 +346,10 @@ public static int hashCodeFromOperator(final UpdateByOperator op) { * Returns `true` if two operators are compatible and can be executed as part of the same window */ public static boolean isEquivalentWindow(final UpdateByOperator opA, final UpdateByOperator opB) { - // verify input columns match - String[] opAInput = opA.getInputColumnNames(); - String[] opBInput = opB.getInputColumnNames(); - - if (opAInput.length != opBInput.length) { + // verify input columns are identical + if (!Arrays.equals(opA.getInputColumnNames(), opB.getInputColumnNames())) { return false; } - for (int ii = 0; ii < opAInput.length; ii++) { - if (!opAInput[ii].equals(opBInput[ii])) { - return false; - } - } final boolean aWindowed = opA instanceof UpdateByWindowedOperator; final boolean bWindowed = opB instanceof UpdateByWindowedOperator; @@ -385,9 +357,7 @@ public static boolean isEquivalentWindow(final UpdateByOperator opA, final Updat // equivalent if both are cumulative, not equivalent if only one is cumulative if (!aWindowed && !bWindowed) { return true; - } else if (!aWindowed) { - return false; - } else if (!bWindowed) { + } else if (aWindowed != bWindowed) { return false; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java index 995f793f60f..578ee302239 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java @@ -10,8 +10,6 @@ import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.TableUpdate; -import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; -import io.deephaven.engine.table.impl.UpdateByOperator; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -33,8 +31,7 @@ public UpdateByWindowCumulative(UpdateByOperator[] operators, int[][] operatorSo super(operators, operatorSourceSlots, timestampColumnName); } - @Override - protected void makeOperatorContexts(UpdateByWindowContext context) { + protected void makeOperatorContexts(UpdateByWindowBucketContext context) { // working chunk size need not be larger than affectedRows.size() context.workingChunkSize = Math.min(context.workingChunkSize, context.affectedRows.intSize()); @@ -44,18 +41,18 @@ protected void makeOperatorContexts(UpdateByWindowContext context) { } } - public UpdateByWindowContext makeWindowContext(final TrackingRowSet sourceRowSet, + public UpdateByWindowBucketContext makeWindowContext(final TrackingRowSet sourceRowSet, final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa, final int chunkSize, final boolean isInitializeStep) { - return new UpdateByWindowContext(sourceRowSet, timestampColumnSource, timestampSsa, + return new UpdateByWindowBucketContext(sourceRowSet, timestampColumnSource, timestampSsa, chunkSize, isInitializeStep) {}; } @Override - public void computeAffectedRowsAndOperators(UpdateByWindowContext context, @NotNull TableUpdate upstream) { + public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, @NotNull TableUpdate upstream) { // all rows are affected on the initial step if (context.initialStep) { @@ -114,7 +111,7 @@ public void computeAffectedRowsAndOperators(UpdateByWindowContext context, @NotN } @Override - public void processRows(UpdateByWindowContext context, final boolean initialStep) { + public void processRows(UpdateByWindowBucketContext context, final boolean initialStep) { Assert.neqNull(context.inputSources, "assignInputSources() must be called before processRow()"); // find the key before the first affected row diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java index d14bbae0d31..f8943903721 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java @@ -12,8 +12,6 @@ import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.TableUpdate; -import io.deephaven.engine.table.impl.UpdateByOperator; -import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.util.datastructures.LongSizedDataStructure; import org.apache.commons.lang3.mutable.MutableLong; @@ -29,11 +27,11 @@ * of `influencer` values to add to the rolling window as the current row changes. */ public class UpdateByWindowTicks extends UpdateByWindow { - public static final int WINDOW_POS_BUFFER_INITIAL_CAPACITY = 512; + private static final int WINDOW_POS_BUFFER_INITIAL_CAPACITY = 512; protected final long prevUnits; protected final long fwdUnits; - public class UpdateByWindowTicksContext extends UpdateByWindow.UpdateByWindowContext { + public class UpdateByWindowBucketTicksContext extends UpdateByWindowBucketContext { private static final int WINDOW_CHUNK_SIZE = 4096; protected final IntRingBuffer currentWindowPositions; @@ -52,7 +50,7 @@ public class UpdateByWindowTicksContext extends UpdateByWindow.UpdateByWindowCon protected long influencerPosChunkSize; protected int currentGetContextSize; - public UpdateByWindowTicksContext(final TrackingRowSet sourceRowSet, + public UpdateByWindowBucketTicksContext(final TrackingRowSet sourceRowSet, @Nullable final ColumnSource timestampColumnSource, @Nullable final LongSegmentedSortedArray timestampSsa, final int chunkSize, final boolean initialStep) { super(sourceRowSet, null, null, chunkSize, initialStep); @@ -81,11 +79,10 @@ public void close() { this.fwdUnits = fwdUnits; } - @Override - protected void makeOperatorContexts(UpdateByWindowContext context) { - UpdateByWindowTicksContext ctx = (UpdateByWindowTicksContext) context; + protected void makeOperatorContexts(UpdateByWindowBucketContext context) { + UpdateByWindowBucketTicksContext ctx = (UpdateByWindowBucketTicksContext) context; - ctx.workingChunkSize = UpdateByWindowTicksContext.WINDOW_CHUNK_SIZE; + ctx.workingChunkSize = UpdateByWindowBucketTicksContext.WINDOW_CHUNK_SIZE; ctx.currentGetContextSize = ctx.workingChunkSize; // create contexts for the affected operators @@ -94,12 +91,12 @@ protected void makeOperatorContexts(UpdateByWindowContext context) { } } - public UpdateByWindowContext makeWindowContext(final TrackingRowSet sourceRowSet, + public UpdateByWindowBucketContext makeWindowContext(final TrackingRowSet sourceRowSet, final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa, final int chunkSize, final boolean isInitializeStep) { - return new UpdateByWindowTicksContext(sourceRowSet, timestampColumnSource, timestampSsa, + return new UpdateByWindowBucketTicksContext(sourceRowSet, timestampColumnSource, timestampSsa, chunkSize, isInitializeStep); } @@ -133,7 +130,7 @@ private static WritableRowSet computeInfluencerRowsTicks(final RowSet sourceSet, } } - private void ensureGetContextSize(UpdateByWindowTicksContext ctx, long newSize) { + private void ensureGetContextSize(UpdateByWindowBucketTicksContext ctx, long newSize) { if (ctx.currentGetContextSize < newSize) { long size = ctx.currentGetContextSize; while (size < newSize) { @@ -167,7 +164,7 @@ private void ensureGetContextSize(UpdateByWindowTicksContext ctx, long newSize) * This function takes care of loading/preparing the next set of influencer data, in this case we load the next * chunk of key and position data and reset the index */ - private void loadNextInfluencerChunks(UpdateByWindowTicksContext ctx) { + private void loadNextInfluencerChunks(UpdateByWindowBucketTicksContext ctx) { if (!ctx.influencerIt.hasMore()) { ctx.nextInfluencerPos = Integer.MAX_VALUE; ctx.nextInfluencerKey = Long.MAX_VALUE; @@ -175,11 +172,11 @@ private void loadNextInfluencerChunks(UpdateByWindowTicksContext ctx) { } final RowSequence influencerRs = - ctx.influencerIt.getNextRowSequenceWithLength(UpdateByWindowTicksContext.WINDOW_CHUNK_SIZE); + ctx.influencerIt.getNextRowSequenceWithLength(UpdateByWindowBucketTicksContext.WINDOW_CHUNK_SIZE); ctx.influencerKeyChunk = influencerRs.asRowKeyChunk(); final RowSequence influencePosRs = - ctx.influencerPosIt.getNextRowSequenceWithLength(UpdateByWindowTicksContext.WINDOW_CHUNK_SIZE); + ctx.influencerPosIt.getNextRowSequenceWithLength(UpdateByWindowBucketTicksContext.WINDOW_CHUNK_SIZE); ctx.influencerPosChunk = influencePosRs.asRowKeyChunk(); Assert.eqTrue(influencePosRs.lastRowKey() < Integer.MAX_VALUE, @@ -201,9 +198,9 @@ private void loadNextInfluencerChunks(UpdateByWindowTicksContext ctx) { * these values (i.e. that fall within the window and will `influence` this computation). */ @Override - public void computeAffectedRowsAndOperators(UpdateByWindowContext context, @NotNull TableUpdate upstream) { + public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, @NotNull TableUpdate upstream) { - UpdateByWindowTicksContext ctx = (UpdateByWindowTicksContext) context; + UpdateByWindowBucketTicksContext ctx = (UpdateByWindowBucketTicksContext) context; // all rows are affected on the initial step if (ctx.initialStep) { @@ -295,8 +292,8 @@ public void computeAffectedRowsAndOperators(UpdateByWindowContext context, @NotN } @Override - public void processRows(UpdateByWindowContext context, boolean initialStep) { - UpdateByWindowTicksContext ctx = (UpdateByWindowTicksContext) context; + public void processRows(UpdateByWindowBucketContext context, boolean initialStep) { + UpdateByWindowBucketTicksContext ctx = (UpdateByWindowBucketTicksContext) context; Assert.neqNull(context.inputSources, "assignInputSources() must be called before processRow()"); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java index 7c69122ead4..acb68f4b7c5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java @@ -13,8 +13,6 @@ import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.TableUpdate; -import io.deephaven.engine.table.impl.UpdateByOperator; -import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.util.datastructures.LongSizedDataStructure; import org.jetbrains.annotations.NotNull; @@ -31,11 +29,11 @@ * a buffer of `influencer` values to add to the rolling window as the current row changes. */ public class UpdateByWindowTime extends UpdateByWindow { - public static final int WINDOW_TIMESTAMP_BUFFER_INITIAL_CAPACITY = 512; + private static final int WINDOW_TIMESTAMP_BUFFER_INITIAL_CAPACITY = 512; protected final long prevUnits; protected final long fwdUnits; - public class UpdateByWindowTimeContext extends UpdateByWindowContext { + public class UpdateByWindowBucketTimeContext extends UpdateByWindowBucketContext { private static final int WINDOW_CHUNK_SIZE = 4096; protected final ChunkSource.GetContext influencerTimestampContext; @@ -51,7 +49,7 @@ public class UpdateByWindowTimeContext extends UpdateByWindowContext { protected long influencerTimestampChunkSize; protected int currentGetContextSize; - public UpdateByWindowTimeContext(final TrackingRowSet sourceRowSet, + public UpdateByWindowBucketTimeContext(final TrackingRowSet sourceRowSet, @NotNull final ColumnSource timestampColumnSource, @Nullable final LongSegmentedSortedArray timestampSsa, final int chunkSize, final boolean initialStep) { super(sourceRowSet, timestampColumnSource, timestampSsa, chunkSize, initialStep); @@ -78,11 +76,10 @@ public void close() { this.fwdUnits = fwdUnits; } - @Override - protected void makeOperatorContexts(UpdateByWindowContext context) { - UpdateByWindowTimeContext ctx = (UpdateByWindowTimeContext) context; + protected void makeOperatorContexts(UpdateByWindowBucketContext context) { + UpdateByWindowBucketTimeContext ctx = (UpdateByWindowBucketTimeContext) context; - ctx.workingChunkSize = UpdateByWindowTimeContext.WINDOW_CHUNK_SIZE; + ctx.workingChunkSize = UpdateByWindowBucketTimeContext.WINDOW_CHUNK_SIZE; ctx.currentGetContextSize = ctx.workingChunkSize; // create contexts for the affected operators @@ -91,12 +88,12 @@ protected void makeOperatorContexts(UpdateByWindowContext context) { } } - public UpdateByWindowContext makeWindowContext(final TrackingRowSet sourceRowSet, + public UpdateByWindowBucketContext makeWindowContext(final TrackingRowSet sourceRowSet, final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa, final int chunkSize, final boolean isInitializeStep) { - return new UpdateByWindowTimeContext(sourceRowSet, timestampColumnSource, timestampSsa, chunkSize, + return new UpdateByWindowBucketTimeContext(sourceRowSet, timestampColumnSource, timestampSsa, chunkSize, isInitializeStep); } @@ -172,7 +169,7 @@ private static WritableRowSet computeInfluencerRowsTime(final RowSet sourceSet, } } - private void ensureGetContextSize(UpdateByWindowTimeContext ctx, long newSize) { + private void ensureGetContextSize(UpdateByWindowBucketTimeContext ctx, long newSize) { if (ctx.currentGetContextSize < newSize) { long size = ctx.currentGetContextSize; while (size < newSize) { @@ -206,7 +203,7 @@ private void ensureGetContextSize(UpdateByWindowTimeContext ctx, long newSize) { * This function takes care of loading/preparing the next set of influencer data, in this case we load the next * chunk of key and position data and reset the index */ - private void loadNextInfluencerChunks(UpdateByWindowTimeContext ctx) { + private void loadNextInfluencerChunks(UpdateByWindowBucketTimeContext ctx) { if (!ctx.influencerIt.hasMore()) { ctx.nextInfluencerTimestamp = Long.MAX_VALUE; ctx.nextInfluencerKey = Long.MAX_VALUE; @@ -214,7 +211,7 @@ private void loadNextInfluencerChunks(UpdateByWindowTimeContext ctx) { } final RowSequence influencerRs = - ctx.influencerIt.getNextRowSequenceWithLength(UpdateByWindowTimeContext.WINDOW_CHUNK_SIZE); + ctx.influencerIt.getNextRowSequenceWithLength(UpdateByWindowBucketTimeContext.WINDOW_CHUNK_SIZE); ctx.influencerKeyChunk = influencerRs.asRowKeyChunk(); ctx.influencerTimestampChunk = ctx.timestampColumnSource.getChunk(ctx.influencerTimestampContext, influencerRs).asLongChunk(); @@ -231,8 +228,8 @@ private void loadNextInfluencerChunks(UpdateByWindowTimeContext ctx) { // the rows that are affected by deletions (if any). After the affected rows have been identified, // determine which rows will be needed to compute new values for the affected rows (influencer rows) @Override - public void computeAffectedRowsAndOperators(UpdateByWindowContext context, @NotNull TableUpdate upstream) { - UpdateByWindowTimeContext ctx = (UpdateByWindowTimeContext) context; + public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, @NotNull TableUpdate upstream) { + UpdateByWindowBucketTimeContext ctx = (UpdateByWindowBucketTimeContext) context; // all rows are affected on the initial step if (ctx.initialStep) { @@ -244,7 +241,7 @@ public void computeAffectedRowsAndOperators(UpdateByWindowContext context, @NotN context.dirtySourceIndices = getUniqueSourceIndices(); makeOperatorContexts(ctx); - ctx.isDirty = !upstream.empty();; + ctx.isDirty = !upstream.empty(); return; } @@ -294,7 +291,7 @@ public void computeAffectedRowsAndOperators(UpdateByWindowContext context, @NotN ctx.timestampColumnSource, ctx.timestampSsa, true); final WritableRowSet affectedByModifies = computeAffectedRowsTime(prev, upstream.getModifiedPreShift(), prevUnits, fwdUnits, - ctx.timestampColumnSource, ctx.timestampSsa, true);) { + ctx.timestampColumnSource, ctx.timestampSsa, true)) { // we used the SSA (post-shift) to get these keys, no need to shift // retain only the rows that still exist in the sourceRowSet affectedByRemoves.retain(ctx.sourceRowSet); @@ -324,8 +321,8 @@ public void computeAffectedRowsAndOperators(UpdateByWindowContext context, @NotN * calls do not provide the popped data */ @Override - public void processRows(UpdateByWindowContext context, boolean initialStep) { - UpdateByWindowTimeContext ctx = (UpdateByWindowTimeContext) context; + public void processRows(UpdateByWindowBucketContext context, boolean initialStep) { + UpdateByWindowBucketTimeContext ctx = (UpdateByWindowBucketTimeContext) context; for (int opIdx : context.dirtyOperatorIndices) { UpdateByWindowedOperator winOp = (UpdateByWindowedOperator) operators[opIdx]; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowedOperator.java similarity index 92% rename from engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java rename to engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowedOperator.java index aa7be8cc9b7..113a8360622 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UpdateByWindowedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowedOperator.java @@ -1,4 +1,4 @@ -package io.deephaven.engine.table.impl; +package io.deephaven.engine.table.impl.updateby; import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; @@ -6,7 +6,7 @@ import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -42,7 +42,7 @@ public UpdateByWindowedOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @Nullable final WritableRowRedirection rowRedirection) { + @Nullable final RowRedirection rowRedirection) { super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java similarity index 90% rename from engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateByManager.java rename to engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java index 5ec689c26f1..bf56003a439 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ZeroKeyUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java @@ -1,14 +1,16 @@ -package io.deephaven.engine.table.impl; +package io.deephaven.engine.table.impl.updateby; import io.deephaven.api.updateby.UpdateByControl; -import io.deephaven.engine.rowset.*; -import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.updateby.UpdateByWindow; +import io.deephaven.engine.rowset.RowSetFactory; +import io.deephaven.engine.rowset.RowSetShiftData; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.ModifiedColumnSet; +import io.deephaven.engine.table.impl.QueryTable; +import io.deephaven.engine.table.impl.TableUpdateImpl; import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.util.LinkedList; import java.util.Map; public class ZeroKeyUpdateByManager extends UpdateBy { @@ -63,8 +65,7 @@ protected ZeroKeyUpdateByManager( // create an updateby bucket instance directly from the source table zeroKeyUpdateBy = new UpdateByBucketHelper(description, source, operators, windows, inputSources, - operatorInputSourceSlots, resultSources, timestampColumnName, redirHelper.getRowRedirection(), - control); + operatorInputSourceSlots, resultSources, timestampColumnName, rowRedirection, control); buckets.offer(zeroKeyUpdateBy); // make the source->result transformer @@ -74,8 +75,7 @@ protected ZeroKeyUpdateByManager( result.addParentReference(zeroKeyUpdateBy); } else { zeroKeyUpdateBy = new UpdateByBucketHelper(description, source, operators, windows, inputSources, - operatorInputSourceSlots, resultSources, timestampColumnName, redirHelper.getRowRedirection(), - control); + operatorInputSourceSlots, resultSources, timestampColumnName, rowRedirection, control); result = zeroKeyUpdateBy.result; buckets.offer(zeroKeyUpdateBy); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java index f2ab189e6ff..3db43d2488b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java @@ -5,10 +5,9 @@ import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.updateby.internal.BaseDoubleUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -59,7 +58,7 @@ public BasePrimitiveEMAOperator(@NotNull final MatchPair pair, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @Nullable final WritableRowRedirection rowRedirection) { + @Nullable final RowRedirection rowRedirection) { super(pair, affectingColumns, rowRedirection, timestampColumnName, timeScaleUnits); this.control = control; @@ -70,7 +69,8 @@ public BasePrimitiveEMAOperator(@NotNull final MatchPair pair, @Override public void initializeUpdate(@NotNull final UpdateContext updateContext, - @NotNull final long firstUnmodifiedKey, long firstUnmodifiedTimestamp) { + final long firstUnmodifiedKey, + final long firstUnmodifiedTimestamp) { super.initializeUpdate(updateContext, firstUnmodifiedKey, firstUnmodifiedTimestamp); final Context ctx = (Context) updateContext; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java index 6d5fd4b3b12..fb46e41ed15 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java @@ -7,8 +7,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -111,7 +110,7 @@ public BigDecimalEMAOperator(@NotNull final MatchPair pair, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @Nullable final WritableRowRedirection rowRedirection, + @Nullable final RowRedirection rowRedirection, final ColumnSource valueSource) { super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, rowRedirection, valueSource); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java index 6d178c48574..dd9e62edcef 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java @@ -7,15 +7,14 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import java.math.BigDecimal; import java.math.BigInteger; -import static io.deephaven.util.QueryConstants.*; +import static io.deephaven.util.QueryConstants.NULL_LONG; public class BigIntegerEMAOperator extends BigNumberEMAOperator { public class Context extends BigNumberEMAOperator.Context { @@ -113,7 +112,7 @@ public BigIntegerEMAOperator(@NotNull final MatchPair pair, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @Nullable final WritableRowRedirection rowRedirection, + @Nullable final RowRedirection rowRedirection, final ColumnSource valueSource) { super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, rowRedirection, valueSource); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java index 11864c20471..78df1531506 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java @@ -8,10 +8,9 @@ import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -73,7 +72,7 @@ public BigNumberEMAOperator(@NotNull final MatchPair pair, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @Nullable final WritableRowRedirection rowRedirection, + @Nullable final RowRedirection rowRedirection, final ColumnSource valueSource) { super(pair, affectingColumns, rowRedirection, timestampColumnName, timeScaleUnits, BigDecimal.class); @@ -87,7 +86,8 @@ public BigNumberEMAOperator(@NotNull final MatchPair pair, @Override public void initializeUpdate(@NotNull final UpdateContext updateContext, - @NotNull final long firstUnmodifiedKey, long firstUnmodifiedTimestamp) { + final long firstUnmodifiedKey, + final long firstUnmodifiedTimestamp) { super.initializeUpdate(updateContext, firstUnmodifiedKey, firstUnmodifiedTimestamp); final Context ctx = (Context) updateContext; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java index c6b238fc44b..ca7ba210750 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java @@ -13,8 +13,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -114,15 +113,15 @@ public void push(long key, int pos) { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this ema * @param control defines how to handle {@code null} input values. - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timeRecorder} is provided, this is measured - * in ticks, otherwise it is measured in nanoseconds + * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is + * measured in ticks, otherwise it is measured in nanoseconds */ public ByteEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @Nullable final WritableRowRedirection rowRedirection, + @Nullable final RowRedirection rowRedirection, final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java index 2ff5ab481af..8ce6abfb274 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java @@ -14,8 +14,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -138,7 +137,7 @@ public DoubleEMAOperator(@NotNull final MatchPair pair, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @Nullable final WritableRowRedirection rowRedirection, + @Nullable final RowRedirection rowRedirection, final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java index ad044cabdae..d92b522a26d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java @@ -9,8 +9,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -133,7 +132,7 @@ public FloatEMAOperator(@NotNull final MatchPair pair, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @Nullable final WritableRowRedirection rowRedirection, + @Nullable final RowRedirection rowRedirection, final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java index eec63a73999..3c374e6e9f7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java @@ -13,8 +13,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -114,15 +113,15 @@ public void push(long key, int pos) { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this ema * @param control defines how to handle {@code null} input values. - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timeRecorder} is provided, this is measured - * in ticks, otherwise it is measured in nanoseconds + * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is + * measured in ticks, otherwise it is measured in nanoseconds */ public IntEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @Nullable final WritableRowRedirection rowRedirection, + @Nullable final RowRedirection rowRedirection, final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java index ace39363324..a188b4dcfdf 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java @@ -13,8 +13,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -114,15 +113,15 @@ public void push(long key, int pos) { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this ema * @param control defines how to handle {@code null} input values. - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timeRecorder} is provided, this is measured - * in ticks, otherwise it is measured in nanoseconds + * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is + * measured in ticks, otherwise it is measured in nanoseconds */ public LongEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @Nullable final WritableRowRedirection rowRedirection, + @Nullable final RowRedirection rowRedirection, final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java index 59e18e50acc..781c04d1222 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java @@ -8,8 +8,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -109,15 +108,15 @@ public void push(long key, int pos) { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this ema * @param control defines how to handle {@code null} input values. - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timeRecorder} is provided, this is measured - * in ticks, otherwise it is measured in nanoseconds + * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is + * measured in ticks, otherwise it is measured in nanoseconds */ public ShortEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @NotNull final OperationControl control, @Nullable final String timestampColumnName, final long timeScaleUnits, - @Nullable final WritableRowRedirection rowRedirection, + @Nullable final RowRedirection rowRedirection, final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java index 82b4c1e4fd0..29f9de7e77e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java @@ -16,9 +16,8 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseByteUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -50,7 +49,7 @@ public void push(long key, int pos) { } public BooleanFillByOperator(@NotNull final MatchPair fillPair, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java index 2758a80741d..e686da6e843 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java @@ -9,9 +9,8 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseByteUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -43,7 +42,7 @@ public void push(long key, int pos) { } public ByteFillByOperator(@NotNull final MatchPair fillPair, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java index 22892bc5d75..1671b930172 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java @@ -4,9 +4,8 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseCharUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -38,7 +37,7 @@ public void push(long key, int pos) { } public CharFillByOperator(@NotNull final MatchPair fillPair, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java index 7692ecf41dc..e0dec94756d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java @@ -9,9 +9,8 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseDoubleUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -43,7 +42,7 @@ public void push(long key, int pos) { } public DoubleFillByOperator(@NotNull final MatchPair fillPair, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java index 83603c27be6..cdac08e5e5e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java @@ -9,9 +9,8 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseFloatUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -43,7 +42,7 @@ public void push(long key, int pos) { } public FloatFillByOperator(@NotNull final MatchPair fillPair, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java index c02822f5d94..3fc9b4e79f4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java @@ -9,9 +9,8 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseIntUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -43,7 +42,7 @@ public void push(long key, int pos) { } public IntFillByOperator(@NotNull final MatchPair fillPair, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java index fa537de0d7d..e61c3ccb7fd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java @@ -16,9 +16,8 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -51,7 +50,7 @@ public void push(long key, int pos) { } public LongFillByOperator(@NotNull final MatchPair fillPair, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args ,@NotNull final Class type // endregion extra-constructor-args diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java index 8ac7d27ed74..bb51017fff3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java @@ -5,13 +5,14 @@ */ package io.deephaven.engine.table.impl.updateby.fill; +import io.deephaven.engine.table.impl.util.ChunkUtils; + import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -42,7 +43,7 @@ public void push(long key, int pos) { } public ObjectFillByOperator(@NotNull final MatchPair fillPair, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args , final Class colType // endregion extra-constructor-args diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java index b8b78a39a1a..ec911cb650f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java @@ -9,9 +9,8 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseShortUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -43,7 +42,7 @@ public void push(long key, int pos) { } public ShortFillByOperator(@NotNull final MatchPair fillPair, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/hashing/UpdateByStateManagerTypedBase.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/hashing/UpdateByStateManagerTypedBase.java index a9f778d2b98..45a7df519b7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/hashing/UpdateByStateManagerTypedBase.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/hashing/UpdateByStateManagerTypedBase.java @@ -23,10 +23,10 @@ import static io.deephaven.engine.table.impl.util.TypedHasherUtil.getPrevKeyChunks; public abstract class UpdateByStateManagerTypedBase extends UpdateByStateManager { - public static final int CHUNK_SIZE = 4096; + private static final int CHUNK_SIZE = 4096; private static final long MAX_TABLE_SIZE = 1 << 30; // maximum array size - public static final int EMPTY_RIGHT_VALUE = QueryConstants.NULL_INT; + protected static final int EMPTY_RIGHT_VALUE = QueryConstants.NULL_INT; // the number of slots in our table protected int tableSize; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java index c6497667fd7..d6165638624 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java @@ -17,10 +17,9 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.updateby.UpdateByCumulativeOperator; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -104,11 +103,11 @@ public void reset() { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param rowRedirection the {@link WritableRowRedirection} for the output column + * @param rowRedirection the {@link RowRedirection} for the output column */ public BaseByteUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { @@ -121,7 +120,7 @@ public BaseByteUpdateByOperator(@NotNull final MatchPair pair, * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param rowRedirection the {@link WritableRowRedirection} for the output column + * @param rowRedirection the {@link RowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is @@ -129,7 +128,7 @@ public BaseByteUpdateByOperator(@NotNull final MatchPair pair, */ public BaseByteUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final WritableRowRedirection rowRedirection, + @Nullable final RowRedirection rowRedirection, @Nullable final String timestampColumnName, final long timeScaleUnits // region extra-constructor-args @@ -140,7 +139,7 @@ public BaseByteUpdateByOperator(@NotNull final MatchPair pair, // region create-dense this.maybeInnerSource = makeDenseSource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource<>(rowRedirection, maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -183,6 +182,7 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie public void startTrackingPrev() { outputSource.startTrackingPrevValues(); if (rowRedirection != null) { + assert maybeInnerSource != null; maybeInnerSource.startTrackingPrevValues(); } } @@ -201,6 +201,7 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d @Override public void prepareForParallelPopulation(final RowSet changedRows) { if (rowRedirection != null) { + assert maybeInnerSource != null; ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); @@ -212,4 +213,11 @@ public void prepareForParallelPopulation(final RowSet changedRows) { public Map> getOutputColumns() { return Collections.singletonMap(pair.leftColumn, outputSource); } + + // region clear-output + @Override + public void clearOutputRows(final RowSet toClear) { + // NOP for primitive types + } + // endregion clear-output } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java index be25128af4d..7ffbbaf1d8c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java @@ -7,10 +7,9 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.updateby.UpdateByCumulativeOperator; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -93,11 +92,11 @@ public void reset() { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param rowRedirection the {@link WritableRowRedirection} for the output column + * @param rowRedirection the {@link RowRedirection} for the output column */ public BaseCharUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { @@ -110,7 +109,7 @@ public BaseCharUpdateByOperator(@NotNull final MatchPair pair, * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param rowRedirection the {@link WritableRowRedirection} for the output column + * @param rowRedirection the {@link RowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is @@ -118,7 +117,7 @@ public BaseCharUpdateByOperator(@NotNull final MatchPair pair, */ public BaseCharUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final WritableRowRedirection rowRedirection, + @Nullable final RowRedirection rowRedirection, @Nullable final String timestampColumnName, final long timeScaleUnits // region extra-constructor-args @@ -129,7 +128,7 @@ public BaseCharUpdateByOperator(@NotNull final MatchPair pair, // region create-dense this.maybeInnerSource = new CharacterArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource<>(rowRedirection, maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -159,6 +158,7 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie public void startTrackingPrev() { outputSource.startTrackingPrevValues(); if (rowRedirection != null) { + assert maybeInnerSource != null; maybeInnerSource.startTrackingPrevValues(); } } @@ -173,6 +173,7 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long @Override public void prepareForParallelPopulation(final RowSet changedRows) { if (rowRedirection != null) { + assert maybeInnerSource != null; ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); @@ -184,4 +185,11 @@ public void prepareForParallelPopulation(final RowSet changedRows) { public Map> getOutputColumns() { return Collections.singletonMap(pair.leftColumn, outputSource); } + + // region clear-output + @Override + public void clearOutputRows(final RowSet toClear) { + // NOP for primitive types + } + // endregion clear-output } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java index 318a765cac1..e3a77c7c66f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java @@ -9,12 +9,14 @@ import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableDoubleChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.*; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; -import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.sources.DoubleArraySource; +import io.deephaven.engine.table.impl.sources.DoubleSparseArraySource; +import io.deephaven.engine.table.impl.sources.WritableRedirectedColumnSource; +import io.deephaven.engine.table.impl.updateby.UpdateByCumulativeOperator; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -45,7 +47,7 @@ protected Context(final int chunkSize) { @Override public void accumulate(RowSequence inputKeys, - Chunk valueChunkArr[], + Chunk[] valueChunkArr, LongChunk tsChunk, int len) { @@ -98,11 +100,11 @@ public void reset() { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param rowRedirection the {@link WritableRowRedirection} for the output column + * @param rowRedirection the {@link RowRedirection} for the output column */ public BaseDoubleUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { @@ -115,7 +117,7 @@ public BaseDoubleUpdateByOperator(@NotNull final MatchPair pair, * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param rowRedirection the {@link WritableRowRedirection} for the output column + * @param rowRedirection the {@link RowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is @@ -123,7 +125,7 @@ public BaseDoubleUpdateByOperator(@NotNull final MatchPair pair, */ public BaseDoubleUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final WritableRowRedirection rowRedirection, + @Nullable final RowRedirection rowRedirection, @Nullable final String timestampColumnName, final long timeScaleUnits // region extra-constructor-args @@ -134,7 +136,7 @@ public BaseDoubleUpdateByOperator(@NotNull final MatchPair pair, // region create-dense this.maybeInnerSource = new DoubleArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource<>(rowRedirection, maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -163,6 +165,7 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie public void startTrackingPrev() { outputSource.startTrackingPrevValues(); if (rowRedirection != null) { + assert maybeInnerSource != null; maybeInnerSource.startTrackingPrevValues(); } } @@ -177,6 +180,7 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long @Override public void prepareForParallelPopulation(final RowSet changedRows) { if (rowRedirection != null) { + assert maybeInnerSource != null; ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); @@ -188,4 +192,11 @@ public void prepareForParallelPopulation(final RowSet changedRows) { public Map> getOutputColumns() { return Collections.singletonMap(pair.leftColumn, outputSource); } + + // region clear-output + @Override + public void clearOutputRows(final RowSet toClear) { + // NOP for primitive types + } + // endregion clear-output } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java index 7d1d39f89c5..f0b555920f9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java @@ -4,12 +4,14 @@ import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableFloatChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.*; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; -import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.sources.FloatArraySource; +import io.deephaven.engine.table.impl.sources.FloatSparseArraySource; +import io.deephaven.engine.table.impl.sources.WritableRedirectedColumnSource; +import io.deephaven.engine.table.impl.updateby.UpdateByCumulativeOperator; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -40,7 +42,7 @@ protected Context(final int chunkSize) { @Override public void accumulate(RowSequence inputKeys, - Chunk valueChunkArr[], + Chunk[] valueChunkArr, LongChunk tsChunk, int len) { @@ -93,11 +95,11 @@ public void reset() { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param rowRedirection the {@link WritableRowRedirection} for the output column + * @param rowRedirection the {@link RowRedirection} for the output column */ public BaseFloatUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { @@ -110,7 +112,7 @@ public BaseFloatUpdateByOperator(@NotNull final MatchPair pair, * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param rowRedirection the {@link WritableRowRedirection} for the output column + * @param rowRedirection the {@link RowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is @@ -118,7 +120,7 @@ public BaseFloatUpdateByOperator(@NotNull final MatchPair pair, */ public BaseFloatUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final WritableRowRedirection rowRedirection, + @Nullable final RowRedirection rowRedirection, @Nullable final String timestampColumnName, final long timeScaleUnits // region extra-constructor-args @@ -129,7 +131,7 @@ public BaseFloatUpdateByOperator(@NotNull final MatchPair pair, // region create-dense this.maybeInnerSource = new FloatArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource<>(rowRedirection, maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -158,6 +160,7 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie public void startTrackingPrev() { outputSource.startTrackingPrevValues(); if (rowRedirection != null) { + assert maybeInnerSource != null; maybeInnerSource.startTrackingPrevValues(); } } @@ -172,6 +175,7 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long @Override public void prepareForParallelPopulation(final RowSet changedRows) { if (rowRedirection != null) { + assert maybeInnerSource != null; ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); @@ -183,4 +187,11 @@ public void prepareForParallelPopulation(final RowSet changedRows) { public Map> getOutputColumns() { return Collections.singletonMap(pair.leftColumn, outputSource); } + + // region clear-output + @Override + public void clearOutputRows(final RowSet toClear) { + // NOP for primitive types + } + // endregion clear-output } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java index 637e4cfab83..3b6f6d739cf 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java @@ -12,10 +12,9 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.updateby.UpdateByCumulativeOperator; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -98,11 +97,11 @@ public void reset() { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param rowRedirection the {@link WritableRowRedirection} for the output column + * @param rowRedirection the {@link RowRedirection} for the output column */ public BaseIntUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { @@ -115,7 +114,7 @@ public BaseIntUpdateByOperator(@NotNull final MatchPair pair, * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param rowRedirection the {@link WritableRowRedirection} for the output column + * @param rowRedirection the {@link RowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is @@ -123,7 +122,7 @@ public BaseIntUpdateByOperator(@NotNull final MatchPair pair, */ public BaseIntUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final WritableRowRedirection rowRedirection, + @Nullable final RowRedirection rowRedirection, @Nullable final String timestampColumnName, final long timeScaleUnits // region extra-constructor-args @@ -134,7 +133,7 @@ public BaseIntUpdateByOperator(@NotNull final MatchPair pair, // region create-dense this.maybeInnerSource = new IntegerArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource<>(rowRedirection, maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -164,6 +163,7 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie public void startTrackingPrev() { outputSource.startTrackingPrevValues(); if (rowRedirection != null) { + assert maybeInnerSource != null; maybeInnerSource.startTrackingPrevValues(); } } @@ -178,6 +178,7 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long @Override public void prepareForParallelPopulation(final RowSet changedRows) { if (rowRedirection != null) { + assert maybeInnerSource != null; ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); @@ -189,4 +190,11 @@ public void prepareForParallelPopulation(final RowSet changedRows) { public Map> getOutputColumns() { return Collections.singletonMap(pair.leftColumn, outputSource); } + + // region clear-output + @Override + public void clearOutputRows(final RowSet toClear) { + // NOP for primitive types + } + // endregion clear-output } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java index 94f45913384..6fa2f0d2939 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java @@ -12,10 +12,9 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.updateby.UpdateByCumulativeOperator; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -98,11 +97,11 @@ public void reset() { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param rowRedirection the {@link WritableRowRedirection} for the output column + * @param rowRedirection the {@link RowRedirection} for the output column */ public BaseLongUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { @@ -115,7 +114,7 @@ public BaseLongUpdateByOperator(@NotNull final MatchPair pair, * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param rowRedirection the {@link WritableRowRedirection} for the output column + * @param rowRedirection the {@link RowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is @@ -123,7 +122,7 @@ public BaseLongUpdateByOperator(@NotNull final MatchPair pair, */ public BaseLongUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final WritableRowRedirection rowRedirection, + @Nullable final RowRedirection rowRedirection, @Nullable final String timestampColumnName, final long timeScaleUnits // region extra-constructor-args @@ -134,7 +133,7 @@ public BaseLongUpdateByOperator(@NotNull final MatchPair pair, // region create-dense this.maybeInnerSource = new LongArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource<>(rowRedirection, maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -164,6 +163,7 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie public void startTrackingPrev() { outputSource.startTrackingPrevValues(); if (rowRedirection != null) { + assert maybeInnerSource != null; maybeInnerSource.startTrackingPrevValues(); } } @@ -178,6 +178,7 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long @Override public void prepareForParallelPopulation(final RowSet changedRows) { if (rowRedirection != null) { + assert maybeInnerSource != null; ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); @@ -189,4 +190,11 @@ public void prepareForParallelPopulation(final RowSet changedRows) { public Map> getOutputColumns() { return Collections.singletonMap(pair.leftColumn, outputSource); } + + // region clear-output + @Override + public void clearOutputRows(final RowSet toClear) { + // NOP for primitive types + } + // endregion clear-output } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java index 9e3e89bf2ea..59e57e13721 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java @@ -4,8 +4,7 @@ import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -42,7 +41,7 @@ public void reset() { public BaseObjectBinaryOperator(@NotNull final Class type, @NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final WritableRowRedirection rowRedirection) { + @Nullable final RowRedirection rowRedirection) { super(pair, affectingColumns, rowRedirection, type); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java index 7b6c0297ad4..910b3e8e9ef 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java @@ -5,6 +5,8 @@ */ package io.deephaven.engine.table.impl.updateby.internal; +import io.deephaven.engine.table.impl.util.ChunkUtils; + import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableObjectChunk; @@ -12,10 +14,9 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.updateby.UpdateByCumulativeOperator; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -99,11 +100,11 @@ public void reset() { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param rowRedirection the {@link WritableRowRedirection} for the output column + * @param rowRedirection the {@link RowRedirection} for the output column */ public BaseObjectUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args , final Class colType // endregion extra-constructor-args @@ -117,7 +118,7 @@ public BaseObjectUpdateByOperator(@NotNull final MatchPair pair, * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param rowRedirection the {@link WritableRowRedirection} for the output column + * @param rowRedirection the {@link RowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is @@ -125,7 +126,7 @@ public BaseObjectUpdateByOperator(@NotNull final MatchPair pair, */ public BaseObjectUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final WritableRowRedirection rowRedirection, + @Nullable final RowRedirection rowRedirection, @Nullable final String timestampColumnName, final long timeScaleUnits // region extra-constructor-args @@ -137,7 +138,7 @@ public BaseObjectUpdateByOperator(@NotNull final MatchPair pair, // region create-dense this.maybeInnerSource = new ObjectArraySource(colType); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource<>(rowRedirection, maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -168,6 +169,7 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie public void startTrackingPrev() { outputSource.startTrackingPrevValues(); if (rowRedirection != null) { + assert maybeInnerSource != null; maybeInnerSource.startTrackingPrevValues(); } } @@ -182,6 +184,7 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long @Override public void prepareForParallelPopulation(final RowSet changedRows) { if (rowRedirection != null) { + assert maybeInnerSource != null; ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); @@ -193,4 +196,16 @@ public void prepareForParallelPopulation(final RowSet changedRows) { public Map> getOutputColumns() { return Collections.singletonMap(pair.leftColumn, outputSource); } + + // region clear-output + @Override + public void clearOutputRows(final RowSet toClear) { + // if we are redirected, clear the inner source + if (rowRedirection != null) { + ChunkUtils.fillWithNullValue(maybeInnerSource, toClear); + } else { + ChunkUtils.fillWithNullValue(outputSource, toClear); + } + } + // endregion clear-output } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java index d41f44b38f0..07315a95827 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java @@ -12,10 +12,9 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.updateby.UpdateByCumulativeOperator; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -98,11 +97,11 @@ public void reset() { * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param rowRedirection the {@link WritableRowRedirection} for the output column + * @param rowRedirection the {@link RowRedirection} for the output column */ public BaseShortUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { @@ -115,7 +114,7 @@ public BaseShortUpdateByOperator(@NotNull final MatchPair pair, * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. - * @param rowRedirection the {@link WritableRowRedirection} for the output column + * @param rowRedirection the {@link RowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is @@ -123,7 +122,7 @@ public BaseShortUpdateByOperator(@NotNull final MatchPair pair, */ public BaseShortUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final WritableRowRedirection rowRedirection, + @Nullable final RowRedirection rowRedirection, @Nullable final String timestampColumnName, final long timeScaleUnits // region extra-constructor-args @@ -134,7 +133,7 @@ public BaseShortUpdateByOperator(@NotNull final MatchPair pair, // region create-dense this.maybeInnerSource = new ShortArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource<>(rowRedirection, maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -164,6 +163,7 @@ public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifie public void startTrackingPrev() { outputSource.startTrackingPrevValues(); if (rowRedirection != null) { + assert maybeInnerSource != null; maybeInnerSource.startTrackingPrevValues(); } } @@ -178,6 +178,7 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long @Override public void prepareForParallelPopulation(final RowSet changedRows) { if (rowRedirection != null) { + assert maybeInnerSource != null; ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); @@ -189,4 +190,11 @@ public void prepareForParallelPopulation(final RowSet changedRows) { public Map> getOutputColumns() { return Collections.singletonMap(pair.leftColumn, outputSource); } + + // region clear-output + @Override + public void clearOutputRows(final RowSet toClear) { + // NOP for primitive types + } + // endregion clear-output } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java index e649c2f08f0..c83a077a624 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java @@ -16,12 +16,12 @@ import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableByteChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.*; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.updateby.UpdateByWindowedOperator; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -120,7 +120,7 @@ public BaseWindowedByteUpdateByOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { @@ -129,7 +129,7 @@ public BaseWindowedByteUpdateByOperator(@NotNull final MatchPair pair, // region create-dense this.maybeInnerSource = makeDenseSource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource<>(rowRedirection, maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -165,6 +165,7 @@ public void initializeUpdate(@NotNull UpdateContext context) { public void startTrackingPrev() { outputSource.startTrackingPrevValues(); if (rowRedirection != null) { + assert maybeInnerSource != null; maybeInnerSource.startTrackingPrevValues(); } } @@ -183,6 +184,7 @@ public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long d @Override public void prepareForParallelPopulation(final RowSet changedRows) { if (rowRedirection != null) { + assert maybeInnerSource != null; ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); @@ -194,4 +196,11 @@ public void prepareForParallelPopulation(final RowSet changedRows) { public Map> getOutputColumns() { return Collections.singletonMap(pair.leftColumn, outputSource); } + + // region clear-output + @Override + public void clearOutputRows(final RowSet toClear) { + // NOP for primitive types + } + // endregion clear-output } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java index e02e2aba336..8bf6b9a0606 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java @@ -6,12 +6,12 @@ import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableCharChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.*; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.updateby.UpdateByWindowedOperator; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -109,7 +109,7 @@ public BaseWindowedCharUpdateByOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { @@ -118,7 +118,7 @@ public BaseWindowedCharUpdateByOperator(@NotNull final MatchPair pair, // region create-dense this.maybeInnerSource = new CharacterArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource<>(rowRedirection, maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -141,6 +141,7 @@ public void initializeUpdate(@NotNull UpdateContext context) { public void startTrackingPrev() { outputSource.startTrackingPrevValues(); if (rowRedirection != null) { + assert maybeInnerSource != null; maybeInnerSource.startTrackingPrevValues(); } } @@ -155,6 +156,7 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long @Override public void prepareForParallelPopulation(final RowSet changedRows) { if (rowRedirection != null) { + assert maybeInnerSource != null; ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); @@ -166,4 +168,11 @@ public void prepareForParallelPopulation(final RowSet changedRows) { public Map> getOutputColumns() { return Collections.singletonMap(pair.leftColumn, outputSource); } + + // region clear-output + @Override + public void clearOutputRows(final RowSet toClear) { + // NOP for primitive types + } + // endregion clear-output } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java index a16a75c3036..6b94780d9a5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java @@ -6,16 +6,19 @@ package io.deephaven.engine.table.impl.updateby.internal; import io.deephaven.api.updateby.OperationControl; -import io.deephaven.chunk.*; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.IntChunk; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.WritableDoubleChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.*; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.sources.DoubleArraySource; import io.deephaven.engine.table.impl.sources.DoubleSparseArraySource; import io.deephaven.engine.table.impl.sources.WritableRedirectedColumnSource; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.updateby.UpdateByWindowedOperator; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -45,7 +48,7 @@ protected Context(final int chunkSize) { @Override public void accumulate(RowSequence inputKeys, - Chunk influencerValueChunkArr[], + Chunk[] influencerValueChunkArr, IntChunk pushChunk, IntChunk popChunk, int len) { @@ -113,7 +116,7 @@ public BaseWindowedDoubleUpdateByOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { @@ -122,7 +125,7 @@ public BaseWindowedDoubleUpdateByOperator(@NotNull final MatchPair pair, // region create-dense this.maybeInnerSource = new DoubleArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource<>(rowRedirection, maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -145,6 +148,7 @@ public void initializeUpdate(@NotNull UpdateContext context) { public void startTrackingPrev() { outputSource.startTrackingPrevValues(); if (rowRedirection != null) { + assert maybeInnerSource != null; maybeInnerSource.startTrackingPrevValues(); } } @@ -159,6 +163,7 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long @Override public void prepareForParallelPopulation(final RowSet changedRows) { if (rowRedirection != null) { + assert maybeInnerSource != null; ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); @@ -170,4 +175,11 @@ public void prepareForParallelPopulation(final RowSet changedRows) { public Map> getOutputColumns() { return Collections.singletonMap(pair.leftColumn, outputSource); } + + // region clear-output + @Override + public void clearOutputRows(final RowSet toClear) { + // NOP for primitive types + } + // endregion clear-output } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java index 162b6353695..cf9c8064dc1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java @@ -1,16 +1,19 @@ package io.deephaven.engine.table.impl.updateby.internal; import io.deephaven.api.updateby.OperationControl; -import io.deephaven.chunk.*; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.IntChunk; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.WritableFloatChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.*; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.sources.FloatArraySource; import io.deephaven.engine.table.impl.sources.FloatSparseArraySource; import io.deephaven.engine.table.impl.sources.WritableRedirectedColumnSource; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.updateby.UpdateByWindowedOperator; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -40,7 +43,7 @@ protected Context(final int chunkSize) { @Override public void accumulate(RowSequence inputKeys, - Chunk influencerValueChunkArr[], + Chunk[] influencerValueChunkArr, IntChunk pushChunk, IntChunk popChunk, int len) { @@ -108,7 +111,7 @@ public BaseWindowedFloatUpdateByOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { @@ -117,7 +120,7 @@ public BaseWindowedFloatUpdateByOperator(@NotNull final MatchPair pair, // region create-dense this.maybeInnerSource = new FloatArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource<>(rowRedirection, maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -140,6 +143,7 @@ public void initializeUpdate(@NotNull UpdateContext context) { public void startTrackingPrev() { outputSource.startTrackingPrevValues(); if (rowRedirection != null) { + assert maybeInnerSource != null; maybeInnerSource.startTrackingPrevValues(); } } @@ -154,6 +158,7 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long @Override public void prepareForParallelPopulation(final RowSet changedRows) { if (rowRedirection != null) { + assert maybeInnerSource != null; ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); @@ -165,4 +170,11 @@ public void prepareForParallelPopulation(final RowSet changedRows) { public Map> getOutputColumns() { return Collections.singletonMap(pair.leftColumn, outputSource); } + + // region clear-output + @Override + public void clearOutputRows(final RowSet toClear) { + // NOP for primitive types + } + // endregion clear-output } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java index 93cd65b0b98..c918ea17b00 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java @@ -11,12 +11,12 @@ import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.*; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.updateby.UpdateByWindowedOperator; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -114,7 +114,7 @@ public BaseWindowedIntUpdateByOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { @@ -123,7 +123,7 @@ public BaseWindowedIntUpdateByOperator(@NotNull final MatchPair pair, // region create-dense this.maybeInnerSource = new IntegerArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource<>(rowRedirection, maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -146,6 +146,7 @@ public void initializeUpdate(@NotNull UpdateContext context) { public void startTrackingPrev() { outputSource.startTrackingPrevValues(); if (rowRedirection != null) { + assert maybeInnerSource != null; maybeInnerSource.startTrackingPrevValues(); } } @@ -160,6 +161,7 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long @Override public void prepareForParallelPopulation(final RowSet changedRows) { if (rowRedirection != null) { + assert maybeInnerSource != null; ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); @@ -171,4 +173,11 @@ public void prepareForParallelPopulation(final RowSet changedRows) { public Map> getOutputColumns() { return Collections.singletonMap(pair.leftColumn, outputSource); } + + // region clear-output + @Override + public void clearOutputRows(final RowSet toClear) { + // NOP for primitive types + } + // endregion clear-output } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java index b115804c91b..56b4a992600 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java @@ -11,12 +11,12 @@ import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableLongChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.*; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.updateby.UpdateByWindowedOperator; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -114,7 +114,7 @@ public BaseWindowedLongUpdateByOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { @@ -123,7 +123,7 @@ public BaseWindowedLongUpdateByOperator(@NotNull final MatchPair pair, // region create-dense this.maybeInnerSource = new LongArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource<>(rowRedirection, maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -146,6 +146,7 @@ public void initializeUpdate(@NotNull UpdateContext context) { public void startTrackingPrev() { outputSource.startTrackingPrevValues(); if (rowRedirection != null) { + assert maybeInnerSource != null; maybeInnerSource.startTrackingPrevValues(); } } @@ -160,6 +161,7 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long @Override public void prepareForParallelPopulation(final RowSet changedRows) { if (rowRedirection != null) { + assert maybeInnerSource != null; ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); @@ -171,4 +173,11 @@ public void prepareForParallelPopulation(final RowSet changedRows) { public Map> getOutputColumns() { return Collections.singletonMap(pair.leftColumn, outputSource); } + + // region clear-output + @Override + public void clearOutputRows(final RowSet toClear) { + // NOP for primitive types + } + // endregion clear-output } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java index 7ee72763fa7..55cfc68836c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java @@ -5,18 +5,20 @@ */ package io.deephaven.engine.table.impl.updateby.internal; +import io.deephaven.engine.table.impl.util.ChunkUtils; + import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.*; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.updateby.UpdateByWindowedOperator; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -115,7 +117,7 @@ public BaseWindowedObjectUpdateByOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args , final Class colType // endregion extra-constructor-args @@ -125,7 +127,7 @@ public BaseWindowedObjectUpdateByOperator(@NotNull final MatchPair pair, // region create-dense this.maybeInnerSource = new ObjectArraySource(colType); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource<>(rowRedirection, maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -149,6 +151,7 @@ public void initializeUpdate(@NotNull UpdateContext context) { public void startTrackingPrev() { outputSource.startTrackingPrevValues(); if (rowRedirection != null) { + assert maybeInnerSource != null; maybeInnerSource.startTrackingPrevValues(); } } @@ -163,6 +166,7 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long @Override public void prepareForParallelPopulation(final RowSet changedRows) { if (rowRedirection != null) { + assert maybeInnerSource != null; ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); @@ -174,4 +178,16 @@ public void prepareForParallelPopulation(final RowSet changedRows) { public Map> getOutputColumns() { return Collections.singletonMap(pair.leftColumn, outputSource); } + + // region clear-output + @Override + public void clearOutputRows(final RowSet toClear) { + // if we are redirected, clear the inner source + if (rowRedirection != null) { + ChunkUtils.fillWithNullValue(maybeInnerSource, toClear); + } else { + ChunkUtils.fillWithNullValue(outputSource, toClear); + } + } + // endregion clear-output } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java index b1352d7da6a..48a287d439c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java @@ -11,12 +11,12 @@ import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableShortChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.*; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.UpdateBy; -import io.deephaven.engine.table.impl.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.updateby.UpdateByWindowedOperator; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -114,7 +114,7 @@ public BaseWindowedShortUpdateByOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { @@ -123,7 +123,7 @@ public BaseWindowedShortUpdateByOperator(@NotNull final MatchPair pair, // region create-dense this.maybeInnerSource = new ShortArraySource(); // endregion create-dense - this.outputSource = new WritableRedirectedColumnSource(rowRedirection, maybeInnerSource, 0); + this.outputSource = new WritableRedirectedColumnSource<>(rowRedirection, maybeInnerSource, 0); } else { this.maybeInnerSource = null; // region create-sparse @@ -146,6 +146,7 @@ public void initializeUpdate(@NotNull UpdateContext context) { public void startTrackingPrev() { outputSource.startTrackingPrevValues(); if (rowRedirection != null) { + assert maybeInnerSource != null; maybeInnerSource.startTrackingPrevValues(); } } @@ -160,6 +161,7 @@ public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long @Override public void prepareForParallelPopulation(final RowSet changedRows) { if (rowRedirection != null) { + assert maybeInnerSource != null; ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); } else { ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); @@ -171,4 +173,11 @@ public void prepareForParallelPopulation(final RowSet changedRows) { public Map> getOutputColumns() { return Collections.singletonMap(pair.leftColumn, outputSource); } + + // region clear-output + @Override + public void clearOutputRows(final RowSet toClear) { + // NOP for primitive types + } + // endregion clear-output } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java index c3f291f6ed3..2899a5396ef 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java @@ -9,13 +9,12 @@ import io.deephaven.chunk.ByteChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseByteUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.*; +import static io.deephaven.util.QueryConstants.NULL_BYTE; public class ByteCumMinMaxOperator extends BaseByteUpdateByOperator { private final boolean isMax; @@ -53,7 +52,7 @@ public void push(long key, int pos) { public ByteCumMinMaxOperator(@NotNull final MatchPair pair, final boolean isMax, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ComparableCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ComparableCumMinMaxOperator.java index 8892e1436c8..754d93a8509 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ComparableCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ComparableCumMinMaxOperator.java @@ -1,9 +1,8 @@ package io.deephaven.engine.table.impl.updateby.minmax; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectBinaryOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -13,7 +12,7 @@ public final class ComparableCumMinMaxOperator> extends public ComparableCumMinMaxOperator(final Class colType, @NotNull final MatchPair inputPair, final boolean isMax, - @Nullable final WritableRowRedirection rowRedirection) { + @Nullable final RowRedirection rowRedirection) { super(colType, inputPair, new String[] {inputPair.rightColumn}, rowRedirection); this.isMax = isMax; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java index b34648109da..7b866e41c02 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java @@ -9,9 +9,8 @@ import io.deephaven.chunk.DoubleChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseDoubleUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -53,7 +52,7 @@ public void push(long key, int pos) { public DoubleCumMinMaxOperator(@NotNull final MatchPair pair, final boolean isMax, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java index 25529eade98..21b523cd9b3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java @@ -4,9 +4,8 @@ import io.deephaven.chunk.FloatChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseFloatUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -48,7 +47,7 @@ public void push(long key, int pos) { public FloatCumMinMaxOperator(@NotNull final MatchPair pair, final boolean isMax, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java index 2f8a583d4af..54ead8a9d9c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java @@ -9,13 +9,12 @@ import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseIntUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.*; +import static io.deephaven.util.QueryConstants.NULL_INT; public class IntCumMinMaxOperator extends BaseIntUpdateByOperator { private final boolean isMax; @@ -53,7 +52,7 @@ public void push(long key, int pos) { public IntCumMinMaxOperator(@NotNull final MatchPair pair, final boolean isMax, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java index 0818f71673c..ac848323665 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java @@ -16,13 +16,12 @@ import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.*; +import static io.deephaven.util.QueryConstants.NULL_LONG; public class LongCumMinMaxOperator extends BaseLongUpdateByOperator { private final boolean isMax; @@ -61,7 +60,7 @@ public void push(long key, int pos) { public LongCumMinMaxOperator(@NotNull final MatchPair pair, final boolean isMax, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args ,@NotNull final Class type // endregion extra-constructor-args diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java index 750a220244b..3b955581750 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java @@ -4,13 +4,12 @@ import io.deephaven.chunk.ShortChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseShortUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.*; +import static io.deephaven.util.QueryConstants.NULL_SHORT; public class ShortCumMinMaxOperator extends BaseShortUpdateByOperator { private final boolean isMax; @@ -48,7 +47,7 @@ public void push(long key, int pos) { public ShortCumMinMaxOperator(@NotNull final MatchPair pair, final boolean isMax, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java index d42d18d8d1b..ac9607aa0a2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java @@ -4,9 +4,8 @@ import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -51,7 +50,7 @@ public void reset() { } public BigDecimalCumProdOperator(@NotNull final MatchPair inputPair, - @Nullable final WritableRowRedirection rowRedirection, + @Nullable final RowRedirection rowRedirection, @NotNull final MathContext mathContext) { super(inputPair, new String[] {inputPair.rightColumn}, rowRedirection, BigDecimal.class); this.mathContext = mathContext; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java index 9334e92bbdb..21a5a737b5b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java @@ -4,9 +4,8 @@ import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -47,7 +46,7 @@ public void reset() { } public BigIntegerCumProdOperator(@NotNull final MatchPair inputPair, - @Nullable final WritableRowRedirection rowRedirection) { + @Nullable final RowRedirection rowRedirection) { super(inputPair, new String[] {inputPair.rightColumn}, rowRedirection, BigInteger.class); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java index b275d217403..0bdcc4d502b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java @@ -9,13 +9,13 @@ import io.deephaven.chunk.ByteChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.*; +import static io.deephaven.util.QueryConstants.NULL_LONG; +import static io.deephaven.util.QueryConstants.NULL_BYTE; public class ByteCumProdOperator extends BaseLongUpdateByOperator { // region extra-fields @@ -47,7 +47,7 @@ public void push(long key, int pos) { } public ByteCumProdOperator(@NotNull final MatchPair pair, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java index d41804f83b6..c179a412b28 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java @@ -9,9 +9,8 @@ import io.deephaven.chunk.DoubleChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseDoubleUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -47,7 +46,7 @@ public void push(long key, int pos) { } public DoubleCumProdOperator(@NotNull final MatchPair pair, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java index 67c67bc49a7..537f608c553 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java @@ -4,9 +4,8 @@ import io.deephaven.chunk.FloatChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseFloatUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -42,7 +41,7 @@ public void push(long key, int pos) { } public FloatCumProdOperator(@NotNull final MatchPair pair, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java index c997fcba590..5b1edb348f8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java @@ -9,13 +9,13 @@ import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.*; +import static io.deephaven.util.QueryConstants.NULL_LONG; +import static io.deephaven.util.QueryConstants.NULL_INT; public class IntCumProdOperator extends BaseLongUpdateByOperator { // region extra-fields @@ -47,7 +47,7 @@ public void push(long key, int pos) { } public IntCumProdOperator(@NotNull final MatchPair pair, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java index 0f19edd73fc..a546256f7bb 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java @@ -9,13 +9,13 @@ import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.*; +import static io.deephaven.util.QueryConstants.NULL_LONG; +import static io.deephaven.util.QueryConstants.NULL_LONG; public class LongCumProdOperator extends BaseLongUpdateByOperator { // region extra-fields @@ -47,7 +47,7 @@ public void push(long key, int pos) { } public LongCumProdOperator(@NotNull final MatchPair pair, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java index c653eb01aff..fe47c108379 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java @@ -4,13 +4,13 @@ import io.deephaven.chunk.ShortChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.*; +import static io.deephaven.util.QueryConstants.NULL_LONG; +import static io.deephaven.util.QueryConstants.NULL_SHORT; public class ShortCumProdOperator extends BaseLongUpdateByOperator { // region extra-fields @@ -42,7 +42,7 @@ public void push(long key, int pos) { } public ShortCumProdOperator(@NotNull final MatchPair pair, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java index fe21481a64b..f60ded6b5a1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java @@ -6,9 +6,9 @@ import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateByOperator; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedObjectUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -16,7 +16,7 @@ import java.math.MathContext; public final class BigDecimalRollingSumOperator extends BaseWindowedObjectUpdateByOperator { - public static final int RING_BUFFER_INITIAL_CAPACITY = 512; + private static final int RING_BUFFER_INITIAL_CAPACITY = 512; @NotNull private final MathContext mathContext; @@ -92,7 +92,7 @@ public BigDecimalRollingSumOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @Nullable final WritableRowRedirection rowRedirection, + @Nullable final RowRedirection rowRedirection, @NotNull final MathContext mathContext) { super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection, BigDecimal.class); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java index d1644327d68..eb4663bfcca 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java @@ -6,9 +6,9 @@ import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateByOperator; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedObjectUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -16,7 +16,7 @@ public final class BigIntegerRollingSumOperator extends BaseWindowedObjectUpdateByOperator { - public static final int RING_BUFFER_INITIAL_CAPACITY = 512; + private static final int RING_BUFFER_INITIAL_CAPACITY = 512; protected class Context extends BaseWindowedObjectUpdateByOperator.Context { protected ObjectChunk objectInfluencerValuesChunk; @@ -90,7 +90,7 @@ public BigIntegerRollingSumOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java index 75c1e5106a0..00f165eea5d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java @@ -12,15 +12,14 @@ import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedLongUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.NULL_LONG; -import static io.deephaven.util.QueryConstants.NULL_BYTE; +import static io.deephaven.util.QueryConstants.*; public class ByteRollingSumOperator extends BaseWindowedLongUpdateByOperator { - public static final int RING_BUFFER_INITIAL_CAPACITY = 512; + private static final int RING_BUFFER_INITIAL_CAPACITY = 512; // region extra-fields final byte nullValue; // endregion extra-fields @@ -98,7 +97,7 @@ public ByteRollingSumOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args ,final byte nullValue // endregion extra-constructor-args diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java index 47c3cad7548..54181987157 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java @@ -13,7 +13,7 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedDoubleUpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.PairwiseDoubleRingBuffer; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -23,7 +23,7 @@ import static io.deephaven.util.QueryConstants.NULL_DOUBLE; public class DoubleRollingSumOperator extends BaseWindowedDoubleUpdateByOperator { - public static final int PAIRWISE_BUFFER_INITIAL_SIZE = 64; + private static final int PAIRWISE_BUFFER_INITIAL_SIZE = 64; // region extra-fields // endregion extra-fields @@ -95,7 +95,7 @@ public DoubleRollingSumOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java index 0c6f6b48f6c..a8237b6c840 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java @@ -8,7 +8,7 @@ import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedFloatUpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.PairwiseFloatRingBuffer; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -18,7 +18,7 @@ import static io.deephaven.util.QueryConstants.NULL_FLOAT; public class FloatRollingSumOperator extends BaseWindowedFloatUpdateByOperator { - public static final int PAIRWISE_BUFFER_INITIAL_SIZE = 64; + private static final int PAIRWISE_BUFFER_INITIAL_SIZE = 64; // region extra-fields // endregion extra-fields @@ -90,7 +90,7 @@ public FloatRollingSumOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java index 303d28201ce..159352884ca 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java @@ -12,15 +12,14 @@ import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedLongUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.NULL_LONG; -import static io.deephaven.util.QueryConstants.NULL_INT; +import static io.deephaven.util.QueryConstants.*; public class IntRollingSumOperator extends BaseWindowedLongUpdateByOperator { - public static final int RING_BUFFER_INITIAL_CAPACITY = 512; + private static final int RING_BUFFER_INITIAL_CAPACITY = 512; // region extra-fields // endregion extra-fields @@ -97,7 +96,7 @@ public IntRollingSumOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java index db472652b3a..5abe8b76cc5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java @@ -12,15 +12,14 @@ import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedLongUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.NULL_LONG; -import static io.deephaven.util.QueryConstants.NULL_LONG; +import static io.deephaven.util.QueryConstants.*; public class LongRollingSumOperator extends BaseWindowedLongUpdateByOperator { - public static final int RING_BUFFER_INITIAL_CAPACITY = 512; + private static final int RING_BUFFER_INITIAL_CAPACITY = 512; // region extra-fields // endregion extra-fields @@ -97,7 +96,7 @@ public LongRollingSumOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java index 322d3ca7140..e70796f1eb6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java @@ -7,15 +7,14 @@ import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedLongUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.NULL_LONG; -import static io.deephaven.util.QueryConstants.NULL_SHORT; +import static io.deephaven.util.QueryConstants.*; public class ShortRollingSumOperator extends BaseWindowedLongUpdateByOperator { - public static final int RING_BUFFER_INITIAL_CAPACITY = 512; + private static final int RING_BUFFER_INITIAL_CAPACITY = 512; // region extra-fields // endregion extra-fields @@ -92,7 +91,7 @@ public ShortRollingSumOperator(@NotNull final MatchPair pair, @Nullable final String timestampColumnName, final long reverseTimeScaleUnits, final long forwardTimeScaleUnits, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java index f718e59f0ed..c435b0639ae 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java @@ -4,9 +4,8 @@ import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -47,7 +46,7 @@ public void push(long key, int pos) { } public BigDecimalCumSumOperator(@NotNull final MatchPair inputPair, - @Nullable final WritableRowRedirection rowRedirection, + @Nullable final RowRedirection rowRedirection, @NotNull final MathContext mathContext) { super(inputPair, new String[] {inputPair.rightColumn}, rowRedirection, BigDecimal.class); this.mathContext = mathContext; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java index 02d5068faad..05c154f19df 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java @@ -4,9 +4,8 @@ import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -42,7 +41,7 @@ public void push(long key, int pos) { } public BigIntegerCumSumOperator(@NotNull final MatchPair inputPair, - @Nullable final WritableRowRedirection rowRedirection) { + @Nullable final RowRedirection rowRedirection) { super(inputPair, new String[] {inputPair.rightColumn}, rowRedirection, BigInteger.class); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java index 63bb1a32648..b5f2141c925 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java @@ -5,16 +5,17 @@ */ package io.deephaven.engine.table.impl.updateby.sum; -import io.deephaven.chunk.*; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.ByteChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.*; +import static io.deephaven.util.QueryConstants.NULL_LONG; +import static io.deephaven.util.QueryConstants.NULL_BYTE; public class ByteCumSumOperator extends BaseLongUpdateByOperator { // region extra-fields @@ -47,7 +48,7 @@ public void push(long key, int pos) { } public ByteCumSumOperator(@NotNull final MatchPair pair, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args ,final byte nullValue // endregion extra-constructor-args diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java index 0a945c33ab6..03df794c6cc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java @@ -9,9 +9,8 @@ import io.deephaven.chunk.DoubleChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseDoubleUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -47,7 +46,7 @@ public void push(long key, int pos) { } public DoubleCumSumOperator(@NotNull final MatchPair pair, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java index 7385a43a361..58efcc76409 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java @@ -4,9 +4,8 @@ import io.deephaven.chunk.FloatChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseFloatUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -42,7 +41,7 @@ public void push(long key, int pos) { } public FloatCumSumOperator(@NotNull final MatchPair pair, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java index d4643c0dd0a..b2770ba525f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java @@ -5,16 +5,17 @@ */ package io.deephaven.engine.table.impl.updateby.sum; -import io.deephaven.chunk.*; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.*; +import static io.deephaven.util.QueryConstants.NULL_LONG; +import static io.deephaven.util.QueryConstants.NULL_INT; public class IntCumSumOperator extends BaseLongUpdateByOperator { // region extra-fields @@ -46,7 +47,7 @@ public void push(long key, int pos) { } public IntCumSumOperator(@NotNull final MatchPair pair, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java index 5164ee1d6c2..7dadcb6e1f6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java @@ -5,16 +5,17 @@ */ package io.deephaven.engine.table.impl.updateby.sum; -import io.deephaven.chunk.*; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.*; +import static io.deephaven.util.QueryConstants.NULL_LONG; +import static io.deephaven.util.QueryConstants.NULL_LONG; public class LongCumSumOperator extends BaseLongUpdateByOperator { // region extra-fields @@ -46,7 +47,7 @@ public void push(long key, int pos) { } public LongCumSumOperator(@NotNull final MatchPair pair, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java index 54630c37b1f..d9275ea207a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java @@ -1,15 +1,16 @@ package io.deephaven.engine.table.impl.updateby.sum; -import io.deephaven.chunk.*; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.ShortChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.UpdateBy; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.*; +import static io.deephaven.util.QueryConstants.NULL_LONG; +import static io.deephaven.util.QueryConstants.NULL_SHORT; public class ShortCumSumOperator extends BaseLongUpdateByOperator { // region extra-fields @@ -41,7 +42,7 @@ public void push(long key, int pos) { } public ShortCumSumOperator(@NotNull final MatchPair pair, - @Nullable final WritableRowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { diff --git a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java index dbc85aee911..e8c5d6b4fae 100644 --- a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java +++ b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java @@ -276,6 +276,7 @@ private static void fixupStandardObject(String objectResult, final String classN final File objectFile = new File(objectResult); List lines = FileUtils.readLines(objectFile, Charset.defaultCharset()); lines = fixupChunkAttributes(lines); + lines = ReplicationUtils.addImport(lines, "import io.deephaven.engine.table.impl.util.ChunkUtils;"); try { lines = removeImport(lines, "import static io.deephaven.util.QueryConstants.NULL_OBJECT;"); } catch (Exception e) { @@ -303,6 +304,19 @@ private static void fixupStandardObject(String objectResult, final String classN } lines = ReplicationUtils.replaceRegion(lines, "extra-constructor-args", Collections.singletonList(" , final Class colType")); + lines = ReplicationUtils.replaceRegion(lines, "clear-output", + Collections.singletonList( + " @Override\n" + + " public void clearOutputRows(final RowSet toClear) {\n" + + " // if we are redirected, clear the inner source\n" + + " if (rowRedirection != null) {\n" + + " ChunkUtils.fillWithNullValue(maybeInnerSource, toClear);\n" + + " } else {\n" + + " ChunkUtils.fillWithNullValue(outputSource, toClear);\n" + + " }\n" + + " }")); + + if (augmentConstructorAndFields) { lines = ReplicationUtils.replaceRegion(lines, "extra-fields", Collections.singletonList(" private final Class colType;")); From 15966dccf187bf4878f4c3dbecda1aea4e7c591a Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 6 Jan 2023 08:10:32 -0800 Subject: [PATCH 071/123] More cleanup --- .../base/ringbuffer/ByteRingBuffer.java | 11 +++++----- .../base/ringbuffer/CharRingBuffer.java | 11 +++++----- .../base/ringbuffer/DoubleRingBuffer.java | 11 +++++----- .../base/ringbuffer/FloatRingBuffer.java | 11 +++++----- .../base/ringbuffer/IntRingBuffer.java | 11 +++++----- .../base/ringbuffer/LongRingBuffer.java | 11 +++++----- .../base/ringbuffer/ShortRingBuffer.java | 11 +++++----- .../impl/updateby/UpdateByBucketHelper.java | 22 ++++++++++++------- 8 files changed, 49 insertions(+), 50 deletions(-) diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java index 5f56c101a03..a5fee8f7c01 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java @@ -55,9 +55,7 @@ public ByteRingBuffer(int capacity) { * Create a ring buffer of byte primitives * * @param capacity minimum capacity of ring buffer - * @param growable whether to allow growth when the buffer is full. If this is {@code false} and {@code add()} is - * called, an {@link UnsupportedOperationException} will be thrown - * + * @param growable whether to allow growth when the buffer is full. */ public ByteRingBuffer(int capacity, boolean growable) { this.growable = growable; @@ -86,11 +84,12 @@ public void clear() { } /** - * Adds an entry to the ring buffer, will throw an {@link UnsupportedOperationException} if buffer is full. For a - * graceful failure, use {@link #offer(byte)} + * Adds an entry to the ring buffer, will throw an exception if buffer is full. For a graceful failure, use + * {@link #offer(byte)} * * @param e the byte to be added to the buffer - * @return true if the byte was added successfully + * @throws UnsupportedOperationException when {@code growable} is {@code false} and buffer is full + * @return {@code true} if the byte was added successfully */ public boolean add(byte e) { if (isFull()) { diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java index ce83c6a4332..8ad9703be20 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java @@ -50,9 +50,7 @@ public CharRingBuffer(int capacity) { * Create a ring buffer of char primitives * * @param capacity minimum capacity of ring buffer - * @param growable whether to allow growth when the buffer is full. If this is {@code false} and {@code add()} is - * called, an {@link UnsupportedOperationException} will be thrown - * + * @param growable whether to allow growth when the buffer is full. */ public CharRingBuffer(int capacity, boolean growable) { this.growable = growable; @@ -81,11 +79,12 @@ public void clear() { } /** - * Adds an entry to the ring buffer, will throw an {@link UnsupportedOperationException} if buffer is full. For a - * graceful failure, use {@link #offer(char)} + * Adds an entry to the ring buffer, will throw an exception if buffer is full. For a graceful failure, use + * {@link #offer(char)} * * @param e the char to be added to the buffer - * @return true if the char was added successfully + * @throws UnsupportedOperationException when {@code growable} is {@code false} and buffer is full + * @return {@code true} if the char was added successfully */ public boolean add(char e) { if (isFull()) { diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java index 111b9fe700c..39ea7222ad1 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java @@ -55,9 +55,7 @@ public DoubleRingBuffer(int capacity) { * Create a ring buffer of double primitives * * @param capacity minimum capacity of ring buffer - * @param growable whether to allow growth when the buffer is full. If this is {@code false} and {@code add()} is - * called, an {@link UnsupportedOperationException} will be thrown - * + * @param growable whether to allow growth when the buffer is full. */ public DoubleRingBuffer(int capacity, boolean growable) { this.growable = growable; @@ -86,11 +84,12 @@ public void clear() { } /** - * Adds an entry to the ring buffer, will throw an {@link UnsupportedOperationException} if buffer is full. For a - * graceful failure, use {@link #offer(double)} + * Adds an entry to the ring buffer, will throw an exception if buffer is full. For a graceful failure, use + * {@link #offer(double)} * * @param e the double to be added to the buffer - * @return true if the double was added successfully + * @throws UnsupportedOperationException when {@code growable} is {@code false} and buffer is full + * @return {@code true} if the double was added successfully */ public boolean add(double e) { if (isFull()) { diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java index e853e1999f2..ee301dbf6c5 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java @@ -55,9 +55,7 @@ public FloatRingBuffer(int capacity) { * Create a ring buffer of float primitives * * @param capacity minimum capacity of ring buffer - * @param growable whether to allow growth when the buffer is full. If this is {@code false} and {@code add()} is - * called, an {@link UnsupportedOperationException} will be thrown - * + * @param growable whether to allow growth when the buffer is full. */ public FloatRingBuffer(int capacity, boolean growable) { this.growable = growable; @@ -86,11 +84,12 @@ public void clear() { } /** - * Adds an entry to the ring buffer, will throw an {@link UnsupportedOperationException} if buffer is full. For a - * graceful failure, use {@link #offer(float)} + * Adds an entry to the ring buffer, will throw an exception if buffer is full. For a graceful failure, use + * {@link #offer(float)} * * @param e the float to be added to the buffer - * @return true if the float was added successfully + * @throws UnsupportedOperationException when {@code growable} is {@code false} and buffer is full + * @return {@code true} if the float was added successfully */ public boolean add(float e) { if (isFull()) { diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java index dfa66654a42..7a39bb7b66d 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java @@ -55,9 +55,7 @@ public IntRingBuffer(int capacity) { * Create a ring buffer of int primitives * * @param capacity minimum capacity of ring buffer - * @param growable whether to allow growth when the buffer is full. If this is {@code false} and {@code add()} is - * called, an {@link UnsupportedOperationException} will be thrown - * + * @param growable whether to allow growth when the buffer is full. */ public IntRingBuffer(int capacity, boolean growable) { this.growable = growable; @@ -86,11 +84,12 @@ public void clear() { } /** - * Adds an entry to the ring buffer, will throw an {@link UnsupportedOperationException} if buffer is full. For a - * graceful failure, use {@link #offer(int)} + * Adds an entry to the ring buffer, will throw an exception if buffer is full. For a graceful failure, use + * {@link #offer(int)} * * @param e the int to be added to the buffer - * @return true if the int was added successfully + * @throws UnsupportedOperationException when {@code growable} is {@code false} and buffer is full + * @return {@code true} if the int was added successfully */ public boolean add(int e) { if (isFull()) { diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java index e28d363afdd..c3a17ffc1a5 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java @@ -55,9 +55,7 @@ public LongRingBuffer(int capacity) { * Create a ring buffer of long primitives * * @param capacity minimum capacity of ring buffer - * @param growable whether to allow growth when the buffer is full. If this is {@code false} and {@code add()} is - * called, an {@link UnsupportedOperationException} will be thrown - * + * @param growable whether to allow growth when the buffer is full. */ public LongRingBuffer(int capacity, boolean growable) { this.growable = growable; @@ -86,11 +84,12 @@ public void clear() { } /** - * Adds an entry to the ring buffer, will throw an {@link UnsupportedOperationException} if buffer is full. For a - * graceful failure, use {@link #offer(long)} + * Adds an entry to the ring buffer, will throw an exception if buffer is full. For a graceful failure, use + * {@link #offer(long)} * * @param e the long to be added to the buffer - * @return true if the long was added successfully + * @throws UnsupportedOperationException when {@code growable} is {@code false} and buffer is full + * @return {@code true} if the long was added successfully */ public boolean add(long e) { if (isFull()) { diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java index b3532e86757..b6197d7ceeb 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java @@ -55,9 +55,7 @@ public ShortRingBuffer(int capacity) { * Create a ring buffer of short primitives * * @param capacity minimum capacity of ring buffer - * @param growable whether to allow growth when the buffer is full. If this is {@code false} and {@code add()} is - * called, an {@link UnsupportedOperationException} will be thrown - * + * @param growable whether to allow growth when the buffer is full. */ public ShortRingBuffer(int capacity, boolean growable) { this.growable = growable; @@ -86,11 +84,12 @@ public void clear() { } /** - * Adds an entry to the ring buffer, will throw an {@link UnsupportedOperationException} if buffer is full. For a - * graceful failure, use {@link #offer(short)} + * Adds an entry to the ring buffer, will throw an exception if buffer is full. For a graceful failure, use + * {@link #offer(short)} * * @param e the short to be added to the buffer - * @return true if the short was added successfully + * @throws UnsupportedOperationException when {@code growable} is {@code false} and buffer is full + * @return {@code true} if the short was added successfully */ public boolean add(short e) { if (isFull()) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java index 5b7a362944e..57ce118553a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java @@ -127,10 +127,12 @@ UpdateByBucketHelperListener newListener(@NotNull final String description) { private void processUpdateForSsa(TableUpdate upstream) { final boolean stampModified = upstream.modifiedColumnSet().containsAny(timestampColumnSet); - try (final RowSet restampAdditions = - stampModified ? upstream.added().union(upstream.modified()) : upstream.added().copy(); - final RowSet restampRemovals = stampModified ? upstream.removed().union(upstream.getModifiedPreShift()) - : upstream.removed().copy()) { + try (final RowSet addedAndModified = stampModified ? upstream.added().union(upstream.modified()) : null; + final RowSet removedAndModifiedPreShift = + stampModified ? upstream.removed().union(upstream.getModifiedPreShift()) : null) { + final RowSet restampAdditions = stampModified ? addedAndModified : upstream.added(); + final RowSet restampRemovals = stampModified ? removedAndModifiedPreShift : upstream.removed(); + // removes if (restampRemovals.isNonempty()) { final int size = (int) Math.min(restampRemovals.size(), 4096); @@ -222,14 +224,18 @@ private void fillChunkWithNonNull(LongChunk keysChunk, LongChunk // add only non-null timestamps to this Ssa for (int i = 0; i < valuesChunk.size(); i++) { long ts = valuesChunk.get(i); + if (ts == NULL_LONG) { + // null timestamps will not cause problems + continue; + } if (ts < lastTimestamp.longValue()) { throw (new IllegalStateException( "updateBy time-based operators require non-descending timestamp values")); } - if (ts != NULL_LONG) { - ssaValues.add(ts); - ssaKeys.add(keysChunk.get(i)); - } + + ssaValues.add(ts); + ssaKeys.add(keysChunk.get(i)); + // store the current ts for comparison lastTimestamp.setValue(ts); } From 9ab165e1c5cfcb5197d757fec8144b48cccadb7a Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 6 Jan 2023 08:38:13 -0800 Subject: [PATCH 072/123] Cleanup in dense array sources --- .../table/impl/sources/ByteArraySource.java | 36 +++++++++---------- .../impl/sources/CharacterArraySource.java | 36 +++++++++---------- .../table/impl/sources/DoubleArraySource.java | 36 +++++++++---------- .../table/impl/sources/FloatArraySource.java | 36 +++++++++---------- .../impl/sources/IntegerArraySource.java | 36 +++++++++---------- .../table/impl/sources/ShortArraySource.java | 36 +++++++++---------- 6 files changed, 108 insertions(+), 108 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ByteArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ByteArraySource.java index f3ab171388d..9a33eb65ab5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ByteArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ByteArraySource.java @@ -65,23 +65,23 @@ public void ensureCapacity(long capacity, boolean nullFill) { * This version of `prepareForParallelPopulation` will internally call {@link #ensureCapacity(long, boolean)} to * make sure there is room for the incoming values. * - * @param changedIndices indices in the dense table + * @param changedRows row set in the dense table */ @Override - public void prepareForParallelPopulation(RowSet changedIndices) { + public void prepareForParallelPopulation(RowSet changedRows) { final long currentStep = LogicalClock.DEFAULT.currentStep(); if (ensurePreviousClockCycle == currentStep) { throw new IllegalStateException("May not call ensurePrevious twice on one clock cycle!"); } ensurePreviousClockCycle = currentStep; - if (changedIndices.isEmpty()) { + if (changedRows.isEmpty()) { return; } // ensure that this source will have sufficient capacity to store these indices, does not need to be // null-filled as the values will be immediately written - ensureCapacity(changedIndices.lastRowKey() + 1, false); + ensureCapacity(changedRows.lastRowKey() + 1, false); if (prevFlusher != null) { prevFlusher.maybeActivate(); @@ -90,7 +90,7 @@ public void prepareForParallelPopulation(RowSet changedIndices) { return; } - try (final RowSequence.Iterator it = changedIndices.getRowSequenceIterator()) { + try (final RowSequence.Iterator it = changedRows.getRowSequenceIterator()) { do { final long firstKey = it.peekNextKey(); @@ -305,14 +305,14 @@ public long resetWritableChunkToBackingStoreSlice(@NotNull ResettableWritableChu } @Override - protected void fillSparseChunk(@NotNull final WritableChunk destGeneric, @NotNull final RowSequence indices) { - if (indices.size() == 0) { + protected void fillSparseChunk(@NotNull final WritableChunk destGeneric, @NotNull final RowSequence rows) { + if (rows.size() == 0) { destGeneric.setSize(0); return; } final WritableByteChunk dest = destGeneric.asWritableByteChunk(); final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); - indices.forAllRowKeys((final long v) -> { + rows.forAllRowKeys((final long v) -> { if (v >= ctx.capForCurrentBlock) { ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; @@ -324,21 +324,21 @@ protected void fillSparseChunk(@NotNull final WritableChunk dest } @Override - protected void fillSparsePrevChunk(@NotNull final WritableChunk destGeneric, @NotNull final RowSequence indices) { - final long sz = indices.size(); + protected void fillSparsePrevChunk(@NotNull final WritableChunk destGeneric, @NotNull final RowSequence rpws) { + final long sz = rpws.size(); if (sz == 0) { destGeneric.setSize(0); return; } if (prevFlusher == null) { - fillSparseChunk(destGeneric, indices); + fillSparseChunk(destGeneric, rpws); return; } final WritableByteChunk dest = destGeneric.asWritableByteChunk(); final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); - indices.forAllRowKeys((final long v) -> { + rpws.forAllRowKeys((final long v) -> { if (v >= ctx.capForCurrentBlock) { ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; @@ -357,11 +357,11 @@ protected void fillSparsePrevChunk(@NotNull final WritableChunk } @Override - protected void fillSparseChunkUnordered(@NotNull final WritableChunk destGeneric, @NotNull final LongChunk indices) { + protected void fillSparseChunkUnordered(@NotNull final WritableChunk destGeneric, @NotNull final LongChunk rows) { final WritableByteChunk dest = destGeneric.asWritableByteChunk(); - final int sz = indices.size(); + final int sz = rows.size(); for (int ii = 0; ii < sz; ++ii) { - final long fromIndex = indices.get(ii); + final long fromIndex = rows.get(ii); if (fromIndex == RowSequence.NULL_ROW_KEY) { dest.set(ii, NULL_BYTE); continue; @@ -378,11 +378,11 @@ protected void fillSparseChunkUnordered(@NotNull final WritableChunk destGeneric, @NotNull final LongChunk indices) { + protected void fillSparsePrevChunkUnordered(@NotNull final WritableChunk destGeneric, @NotNull final LongChunk rows) { final WritableByteChunk dest = destGeneric.asWritableByteChunk(); - final int sz = indices.size(); + final int sz = rows.size(); for (int ii = 0; ii < sz; ++ii) { - final long fromIndex = indices.get(ii); + final long fromIndex = rows.get(ii); if (fromIndex == RowSequence.NULL_ROW_KEY) { dest.set(ii, NULL_BYTE); continue; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/CharacterArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/CharacterArraySource.java index 14faedfbac4..3883641b616 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/CharacterArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/CharacterArraySource.java @@ -60,23 +60,23 @@ public void ensureCapacity(long capacity, boolean nullFill) { * This version of `prepareForParallelPopulation` will internally call {@link #ensureCapacity(long, boolean)} to * make sure there is room for the incoming values. * - * @param changedIndices indices in the dense table + * @param changedRows row set in the dense table */ @Override - public void prepareForParallelPopulation(RowSet changedIndices) { + public void prepareForParallelPopulation(RowSet changedRows) { final long currentStep = LogicalClock.DEFAULT.currentStep(); if (ensurePreviousClockCycle == currentStep) { throw new IllegalStateException("May not call ensurePrevious twice on one clock cycle!"); } ensurePreviousClockCycle = currentStep; - if (changedIndices.isEmpty()) { + if (changedRows.isEmpty()) { return; } // ensure that this source will have sufficient capacity to store these indices, does not need to be // null-filled as the values will be immediately written - ensureCapacity(changedIndices.lastRowKey() + 1, false); + ensureCapacity(changedRows.lastRowKey() + 1, false); if (prevFlusher != null) { prevFlusher.maybeActivate(); @@ -85,7 +85,7 @@ public void prepareForParallelPopulation(RowSet changedIndices) { return; } - try (final RowSequence.Iterator it = changedIndices.getRowSequenceIterator()) { + try (final RowSequence.Iterator it = changedRows.getRowSequenceIterator()) { do { final long firstKey = it.peekNextKey(); @@ -300,14 +300,14 @@ public long resetWritableChunkToBackingStoreSlice(@NotNull ResettableWritableChu } @Override - protected void fillSparseChunk(@NotNull final WritableChunk destGeneric, @NotNull final RowSequence indices) { - if (indices.size() == 0) { + protected void fillSparseChunk(@NotNull final WritableChunk destGeneric, @NotNull final RowSequence rows) { + if (rows.size() == 0) { destGeneric.setSize(0); return; } final WritableCharChunk dest = destGeneric.asWritableCharChunk(); final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); - indices.forAllRowKeys((final long v) -> { + rows.forAllRowKeys((final long v) -> { if (v >= ctx.capForCurrentBlock) { ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; @@ -319,21 +319,21 @@ protected void fillSparseChunk(@NotNull final WritableChunk dest } @Override - protected void fillSparsePrevChunk(@NotNull final WritableChunk destGeneric, @NotNull final RowSequence indices) { - final long sz = indices.size(); + protected void fillSparsePrevChunk(@NotNull final WritableChunk destGeneric, @NotNull final RowSequence rpws) { + final long sz = rpws.size(); if (sz == 0) { destGeneric.setSize(0); return; } if (prevFlusher == null) { - fillSparseChunk(destGeneric, indices); + fillSparseChunk(destGeneric, rpws); return; } final WritableCharChunk dest = destGeneric.asWritableCharChunk(); final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); - indices.forAllRowKeys((final long v) -> { + rpws.forAllRowKeys((final long v) -> { if (v >= ctx.capForCurrentBlock) { ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; @@ -352,11 +352,11 @@ protected void fillSparsePrevChunk(@NotNull final WritableChunk } @Override - protected void fillSparseChunkUnordered(@NotNull final WritableChunk destGeneric, @NotNull final LongChunk indices) { + protected void fillSparseChunkUnordered(@NotNull final WritableChunk destGeneric, @NotNull final LongChunk rows) { final WritableCharChunk dest = destGeneric.asWritableCharChunk(); - final int sz = indices.size(); + final int sz = rows.size(); for (int ii = 0; ii < sz; ++ii) { - final long fromIndex = indices.get(ii); + final long fromIndex = rows.get(ii); if (fromIndex == RowSequence.NULL_ROW_KEY) { dest.set(ii, NULL_CHAR); continue; @@ -373,11 +373,11 @@ protected void fillSparseChunkUnordered(@NotNull final WritableChunk destGeneric, @NotNull final LongChunk indices) { + protected void fillSparsePrevChunkUnordered(@NotNull final WritableChunk destGeneric, @NotNull final LongChunk rows) { final WritableCharChunk dest = destGeneric.asWritableCharChunk(); - final int sz = indices.size(); + final int sz = rows.size(); for (int ii = 0; ii < sz; ++ii) { - final long fromIndex = indices.get(ii); + final long fromIndex = rows.get(ii); if (fromIndex == RowSequence.NULL_ROW_KEY) { dest.set(ii, NULL_CHAR); continue; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/DoubleArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/DoubleArraySource.java index 48b9b20003f..8a50e241792 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/DoubleArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/DoubleArraySource.java @@ -65,23 +65,23 @@ public void ensureCapacity(long capacity, boolean nullFill) { * This version of `prepareForParallelPopulation` will internally call {@link #ensureCapacity(long, boolean)} to * make sure there is room for the incoming values. * - * @param changedIndices indices in the dense table + * @param changedRows row set in the dense table */ @Override - public void prepareForParallelPopulation(RowSet changedIndices) { + public void prepareForParallelPopulation(RowSet changedRows) { final long currentStep = LogicalClock.DEFAULT.currentStep(); if (ensurePreviousClockCycle == currentStep) { throw new IllegalStateException("May not call ensurePrevious twice on one clock cycle!"); } ensurePreviousClockCycle = currentStep; - if (changedIndices.isEmpty()) { + if (changedRows.isEmpty()) { return; } // ensure that this source will have sufficient capacity to store these indices, does not need to be // null-filled as the values will be immediately written - ensureCapacity(changedIndices.lastRowKey() + 1, false); + ensureCapacity(changedRows.lastRowKey() + 1, false); if (prevFlusher != null) { prevFlusher.maybeActivate(); @@ -90,7 +90,7 @@ public void prepareForParallelPopulation(RowSet changedIndices) { return; } - try (final RowSequence.Iterator it = changedIndices.getRowSequenceIterator()) { + try (final RowSequence.Iterator it = changedRows.getRowSequenceIterator()) { do { final long firstKey = it.peekNextKey(); @@ -305,14 +305,14 @@ public long resetWritableChunkToBackingStoreSlice(@NotNull ResettableWritableChu } @Override - protected void fillSparseChunk(@NotNull final WritableChunk destGeneric, @NotNull final RowSequence indices) { - if (indices.size() == 0) { + protected void fillSparseChunk(@NotNull final WritableChunk destGeneric, @NotNull final RowSequence rows) { + if (rows.size() == 0) { destGeneric.setSize(0); return; } final WritableDoubleChunk dest = destGeneric.asWritableDoubleChunk(); final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); - indices.forAllRowKeys((final long v) -> { + rows.forAllRowKeys((final long v) -> { if (v >= ctx.capForCurrentBlock) { ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; @@ -324,21 +324,21 @@ protected void fillSparseChunk(@NotNull final WritableChunk dest } @Override - protected void fillSparsePrevChunk(@NotNull final WritableChunk destGeneric, @NotNull final RowSequence indices) { - final long sz = indices.size(); + protected void fillSparsePrevChunk(@NotNull final WritableChunk destGeneric, @NotNull final RowSequence rpws) { + final long sz = rpws.size(); if (sz == 0) { destGeneric.setSize(0); return; } if (prevFlusher == null) { - fillSparseChunk(destGeneric, indices); + fillSparseChunk(destGeneric, rpws); return; } final WritableDoubleChunk dest = destGeneric.asWritableDoubleChunk(); final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); - indices.forAllRowKeys((final long v) -> { + rpws.forAllRowKeys((final long v) -> { if (v >= ctx.capForCurrentBlock) { ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; @@ -357,11 +357,11 @@ protected void fillSparsePrevChunk(@NotNull final WritableChunk } @Override - protected void fillSparseChunkUnordered(@NotNull final WritableChunk destGeneric, @NotNull final LongChunk indices) { + protected void fillSparseChunkUnordered(@NotNull final WritableChunk destGeneric, @NotNull final LongChunk rows) { final WritableDoubleChunk dest = destGeneric.asWritableDoubleChunk(); - final int sz = indices.size(); + final int sz = rows.size(); for (int ii = 0; ii < sz; ++ii) { - final long fromIndex = indices.get(ii); + final long fromIndex = rows.get(ii); if (fromIndex == RowSequence.NULL_ROW_KEY) { dest.set(ii, NULL_DOUBLE); continue; @@ -378,11 +378,11 @@ protected void fillSparseChunkUnordered(@NotNull final WritableChunk destGeneric, @NotNull final LongChunk indices) { + protected void fillSparsePrevChunkUnordered(@NotNull final WritableChunk destGeneric, @NotNull final LongChunk rows) { final WritableDoubleChunk dest = destGeneric.asWritableDoubleChunk(); - final int sz = indices.size(); + final int sz = rows.size(); for (int ii = 0; ii < sz; ++ii) { - final long fromIndex = indices.get(ii); + final long fromIndex = rows.get(ii); if (fromIndex == RowSequence.NULL_ROW_KEY) { dest.set(ii, NULL_DOUBLE); continue; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/FloatArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/FloatArraySource.java index 4fc525128cb..1b7e889a2dd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/FloatArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/FloatArraySource.java @@ -65,23 +65,23 @@ public void ensureCapacity(long capacity, boolean nullFill) { * This version of `prepareForParallelPopulation` will internally call {@link #ensureCapacity(long, boolean)} to * make sure there is room for the incoming values. * - * @param changedIndices indices in the dense table + * @param changedRows row set in the dense table */ @Override - public void prepareForParallelPopulation(RowSet changedIndices) { + public void prepareForParallelPopulation(RowSet changedRows) { final long currentStep = LogicalClock.DEFAULT.currentStep(); if (ensurePreviousClockCycle == currentStep) { throw new IllegalStateException("May not call ensurePrevious twice on one clock cycle!"); } ensurePreviousClockCycle = currentStep; - if (changedIndices.isEmpty()) { + if (changedRows.isEmpty()) { return; } // ensure that this source will have sufficient capacity to store these indices, does not need to be // null-filled as the values will be immediately written - ensureCapacity(changedIndices.lastRowKey() + 1, false); + ensureCapacity(changedRows.lastRowKey() + 1, false); if (prevFlusher != null) { prevFlusher.maybeActivate(); @@ -90,7 +90,7 @@ public void prepareForParallelPopulation(RowSet changedIndices) { return; } - try (final RowSequence.Iterator it = changedIndices.getRowSequenceIterator()) { + try (final RowSequence.Iterator it = changedRows.getRowSequenceIterator()) { do { final long firstKey = it.peekNextKey(); @@ -305,14 +305,14 @@ public long resetWritableChunkToBackingStoreSlice(@NotNull ResettableWritableChu } @Override - protected void fillSparseChunk(@NotNull final WritableChunk destGeneric, @NotNull final RowSequence indices) { - if (indices.size() == 0) { + protected void fillSparseChunk(@NotNull final WritableChunk destGeneric, @NotNull final RowSequence rows) { + if (rows.size() == 0) { destGeneric.setSize(0); return; } final WritableFloatChunk dest = destGeneric.asWritableFloatChunk(); final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); - indices.forAllRowKeys((final long v) -> { + rows.forAllRowKeys((final long v) -> { if (v >= ctx.capForCurrentBlock) { ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; @@ -324,21 +324,21 @@ protected void fillSparseChunk(@NotNull final WritableChunk dest } @Override - protected void fillSparsePrevChunk(@NotNull final WritableChunk destGeneric, @NotNull final RowSequence indices) { - final long sz = indices.size(); + protected void fillSparsePrevChunk(@NotNull final WritableChunk destGeneric, @NotNull final RowSequence rpws) { + final long sz = rpws.size(); if (sz == 0) { destGeneric.setSize(0); return; } if (prevFlusher == null) { - fillSparseChunk(destGeneric, indices); + fillSparseChunk(destGeneric, rpws); return; } final WritableFloatChunk dest = destGeneric.asWritableFloatChunk(); final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); - indices.forAllRowKeys((final long v) -> { + rpws.forAllRowKeys((final long v) -> { if (v >= ctx.capForCurrentBlock) { ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; @@ -357,11 +357,11 @@ protected void fillSparsePrevChunk(@NotNull final WritableChunk } @Override - protected void fillSparseChunkUnordered(@NotNull final WritableChunk destGeneric, @NotNull final LongChunk indices) { + protected void fillSparseChunkUnordered(@NotNull final WritableChunk destGeneric, @NotNull final LongChunk rows) { final WritableFloatChunk dest = destGeneric.asWritableFloatChunk(); - final int sz = indices.size(); + final int sz = rows.size(); for (int ii = 0; ii < sz; ++ii) { - final long fromIndex = indices.get(ii); + final long fromIndex = rows.get(ii); if (fromIndex == RowSequence.NULL_ROW_KEY) { dest.set(ii, NULL_FLOAT); continue; @@ -378,11 +378,11 @@ protected void fillSparseChunkUnordered(@NotNull final WritableChunk destGeneric, @NotNull final LongChunk indices) { + protected void fillSparsePrevChunkUnordered(@NotNull final WritableChunk destGeneric, @NotNull final LongChunk rows) { final WritableFloatChunk dest = destGeneric.asWritableFloatChunk(); - final int sz = indices.size(); + final int sz = rows.size(); for (int ii = 0; ii < sz; ++ii) { - final long fromIndex = indices.get(ii); + final long fromIndex = rows.get(ii); if (fromIndex == RowSequence.NULL_ROW_KEY) { dest.set(ii, NULL_FLOAT); continue; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/IntegerArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/IntegerArraySource.java index 9e8ba889a69..226421ebce3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/IntegerArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/IntegerArraySource.java @@ -65,23 +65,23 @@ public void ensureCapacity(long capacity, boolean nullFill) { * This version of `prepareForParallelPopulation` will internally call {@link #ensureCapacity(long, boolean)} to * make sure there is room for the incoming values. * - * @param changedIndices indices in the dense table + * @param changedRows row set in the dense table */ @Override - public void prepareForParallelPopulation(RowSet changedIndices) { + public void prepareForParallelPopulation(RowSet changedRows) { final long currentStep = LogicalClock.DEFAULT.currentStep(); if (ensurePreviousClockCycle == currentStep) { throw new IllegalStateException("May not call ensurePrevious twice on one clock cycle!"); } ensurePreviousClockCycle = currentStep; - if (changedIndices.isEmpty()) { + if (changedRows.isEmpty()) { return; } // ensure that this source will have sufficient capacity to store these indices, does not need to be // null-filled as the values will be immediately written - ensureCapacity(changedIndices.lastRowKey() + 1, false); + ensureCapacity(changedRows.lastRowKey() + 1, false); if (prevFlusher != null) { prevFlusher.maybeActivate(); @@ -90,7 +90,7 @@ public void prepareForParallelPopulation(RowSet changedIndices) { return; } - try (final RowSequence.Iterator it = changedIndices.getRowSequenceIterator()) { + try (final RowSequence.Iterator it = changedRows.getRowSequenceIterator()) { do { final long firstKey = it.peekNextKey(); @@ -305,14 +305,14 @@ public long resetWritableChunkToBackingStoreSlice(@NotNull ResettableWritableChu } @Override - protected void fillSparseChunk(@NotNull final WritableChunk destGeneric, @NotNull final RowSequence indices) { - if (indices.size() == 0) { + protected void fillSparseChunk(@NotNull final WritableChunk destGeneric, @NotNull final RowSequence rows) { + if (rows.size() == 0) { destGeneric.setSize(0); return; } final WritableIntChunk dest = destGeneric.asWritableIntChunk(); final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); - indices.forAllRowKeys((final long v) -> { + rows.forAllRowKeys((final long v) -> { if (v >= ctx.capForCurrentBlock) { ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; @@ -324,21 +324,21 @@ protected void fillSparseChunk(@NotNull final WritableChunk dest } @Override - protected void fillSparsePrevChunk(@NotNull final WritableChunk destGeneric, @NotNull final RowSequence indices) { - final long sz = indices.size(); + protected void fillSparsePrevChunk(@NotNull final WritableChunk destGeneric, @NotNull final RowSequence rpws) { + final long sz = rpws.size(); if (sz == 0) { destGeneric.setSize(0); return; } if (prevFlusher == null) { - fillSparseChunk(destGeneric, indices); + fillSparseChunk(destGeneric, rpws); return; } final WritableIntChunk dest = destGeneric.asWritableIntChunk(); final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); - indices.forAllRowKeys((final long v) -> { + rpws.forAllRowKeys((final long v) -> { if (v >= ctx.capForCurrentBlock) { ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; @@ -357,11 +357,11 @@ protected void fillSparsePrevChunk(@NotNull final WritableChunk } @Override - protected void fillSparseChunkUnordered(@NotNull final WritableChunk destGeneric, @NotNull final LongChunk indices) { + protected void fillSparseChunkUnordered(@NotNull final WritableChunk destGeneric, @NotNull final LongChunk rows) { final WritableIntChunk dest = destGeneric.asWritableIntChunk(); - final int sz = indices.size(); + final int sz = rows.size(); for (int ii = 0; ii < sz; ++ii) { - final long fromIndex = indices.get(ii); + final long fromIndex = rows.get(ii); if (fromIndex == RowSequence.NULL_ROW_KEY) { dest.set(ii, NULL_INT); continue; @@ -378,11 +378,11 @@ protected void fillSparseChunkUnordered(@NotNull final WritableChunk destGeneric, @NotNull final LongChunk indices) { + protected void fillSparsePrevChunkUnordered(@NotNull final WritableChunk destGeneric, @NotNull final LongChunk rows) { final WritableIntChunk dest = destGeneric.asWritableIntChunk(); - final int sz = indices.size(); + final int sz = rows.size(); for (int ii = 0; ii < sz; ++ii) { - final long fromIndex = indices.get(ii); + final long fromIndex = rows.get(ii); if (fromIndex == RowSequence.NULL_ROW_KEY) { dest.set(ii, NULL_INT); continue; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ShortArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ShortArraySource.java index 93cadc083b6..8d1ed871f52 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ShortArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ShortArraySource.java @@ -65,23 +65,23 @@ public void ensureCapacity(long capacity, boolean nullFill) { * This version of `prepareForParallelPopulation` will internally call {@link #ensureCapacity(long, boolean)} to * make sure there is room for the incoming values. * - * @param changedIndices indices in the dense table + * @param changedRows row set in the dense table */ @Override - public void prepareForParallelPopulation(RowSet changedIndices) { + public void prepareForParallelPopulation(RowSet changedRows) { final long currentStep = LogicalClock.DEFAULT.currentStep(); if (ensurePreviousClockCycle == currentStep) { throw new IllegalStateException("May not call ensurePrevious twice on one clock cycle!"); } ensurePreviousClockCycle = currentStep; - if (changedIndices.isEmpty()) { + if (changedRows.isEmpty()) { return; } // ensure that this source will have sufficient capacity to store these indices, does not need to be // null-filled as the values will be immediately written - ensureCapacity(changedIndices.lastRowKey() + 1, false); + ensureCapacity(changedRows.lastRowKey() + 1, false); if (prevFlusher != null) { prevFlusher.maybeActivate(); @@ -90,7 +90,7 @@ public void prepareForParallelPopulation(RowSet changedIndices) { return; } - try (final RowSequence.Iterator it = changedIndices.getRowSequenceIterator()) { + try (final RowSequence.Iterator it = changedRows.getRowSequenceIterator()) { do { final long firstKey = it.peekNextKey(); @@ -305,14 +305,14 @@ public long resetWritableChunkToBackingStoreSlice(@NotNull ResettableWritableChu } @Override - protected void fillSparseChunk(@NotNull final WritableChunk destGeneric, @NotNull final RowSequence indices) { - if (indices.size() == 0) { + protected void fillSparseChunk(@NotNull final WritableChunk destGeneric, @NotNull final RowSequence rows) { + if (rows.size() == 0) { destGeneric.setSize(0); return; } final WritableShortChunk dest = destGeneric.asWritableShortChunk(); final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); - indices.forAllRowKeys((final long v) -> { + rows.forAllRowKeys((final long v) -> { if (v >= ctx.capForCurrentBlock) { ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; @@ -324,21 +324,21 @@ protected void fillSparseChunk(@NotNull final WritableChunk dest } @Override - protected void fillSparsePrevChunk(@NotNull final WritableChunk destGeneric, @NotNull final RowSequence indices) { - final long sz = indices.size(); + protected void fillSparsePrevChunk(@NotNull final WritableChunk destGeneric, @NotNull final RowSequence rpws) { + final long sz = rpws.size(); if (sz == 0) { destGeneric.setSize(0); return; } if (prevFlusher == null) { - fillSparseChunk(destGeneric, indices); + fillSparseChunk(destGeneric, rpws); return; } final WritableShortChunk dest = destGeneric.asWritableShortChunk(); final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); - indices.forAllRowKeys((final long v) -> { + rpws.forAllRowKeys((final long v) -> { if (v >= ctx.capForCurrentBlock) { ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; @@ -357,11 +357,11 @@ protected void fillSparsePrevChunk(@NotNull final WritableChunk } @Override - protected void fillSparseChunkUnordered(@NotNull final WritableChunk destGeneric, @NotNull final LongChunk indices) { + protected void fillSparseChunkUnordered(@NotNull final WritableChunk destGeneric, @NotNull final LongChunk rows) { final WritableShortChunk dest = destGeneric.asWritableShortChunk(); - final int sz = indices.size(); + final int sz = rows.size(); for (int ii = 0; ii < sz; ++ii) { - final long fromIndex = indices.get(ii); + final long fromIndex = rows.get(ii); if (fromIndex == RowSequence.NULL_ROW_KEY) { dest.set(ii, NULL_SHORT); continue; @@ -378,11 +378,11 @@ protected void fillSparseChunkUnordered(@NotNull final WritableChunk destGeneric, @NotNull final LongChunk indices) { + protected void fillSparsePrevChunkUnordered(@NotNull final WritableChunk destGeneric, @NotNull final LongChunk rows) { final WritableShortChunk dest = destGeneric.asWritableShortChunk(); - final int sz = indices.size(); + final int sz = rows.size(); for (int ii = 0; ii < sz; ++ii) { - final long fromIndex = indices.get(ii); + final long fromIndex = rows.get(ii); if (fromIndex == RowSequence.NULL_ROW_KEY) { dest.set(ii, NULL_SHORT); continue; From a02d7266b10cbd774b49e426393a38249865a4f8 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 6 Jan 2023 11:18:14 -0800 Subject: [PATCH 073/123] Cleaned up unnecessary changes to PartitionBy signatures --- .../java/io/deephaven/engine/table/Table.java | 36 ------------------- .../engine/table/impl/QueryTable.java | 13 ++++--- .../engine/table/impl/TableDefaults.java | 13 ------- .../engine/table/impl/UncoalescedTable.java | 8 ++--- .../BucketedPartitionedUpdateByManager.java | 6 ++-- .../engine/table/impl/updateby/UpdateBy.java | 12 +++---- .../updateby/UpdateByOperatorFactory.java | 5 +-- 7 files changed, 21 insertions(+), 72 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 5bb46f0ffc7..3cfcfc47b3b 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 @@ -814,19 +814,6 @@ public static AsOfMatchRule of(ReverseAsOfJoinRule rule) { @ConcurrentMethod PartitionedTable partitionBy(boolean dropKeys, String... keyColumnNames); - /** - * Create a {@link PartitionedTable} from this table, partitioned according to the specified key columns. - *

                - * The underlying partitioned table backing the result contains each row in {@code this} table in exactly one of the - * result's constituent tables. - * - * @param dropKeys Whether to drop key columns in the output constituent tables - * @param keyColumnNames The names of the key columns to partition by - * @return A {@link PartitionedTable} keyed by {@code keyColumnNames} - */ - @ConcurrentMethod - PartitionedTable partitionBy(boolean dropKeys, @NotNull Collection keyColumnNames); - /** * Equivalent to {@code partitionBy(false, keyColumnNames)} *

                @@ -842,29 +829,6 @@ public static AsOfMatchRule of(ReverseAsOfJoinRule rule) { @ConcurrentMethod PartitionedTable partitionBy(String... keyColumnNames); - /** - * Convenience method that performs an {@link #aggBy(Collection, boolean, Table, Collection)} and wraps the result - * in a {@link PartitionedTable}. If {@code aggregations} does not include a {@link io.deephaven.api.agg.Partition - * partition}, one will be added automatically with the default constituent column name and behavior used in - * {@link #partitionBy(String...)}. - * - * @param aggregations The {@link Aggregation aggregations} to apply - * @param preserveEmpty Whether to keep result rows for groups that are initially empty or become empty as a result - * of updates. Each aggregation operator defines its own value for empty groups. - * @param initialGroups A table whose distinct combinations of values for the {@code groupByColumns} should be used - * to create an initial set of aggregation groups. All other columns are ignored. This is useful in - * combination with {@code preserveEmpty == true} to ensure that particular groups appear in the result - * table, or with {@code preserveEmpty == false} to control the encounter order for a collection of groups - * and thus their relative order in the result. Changes to {@code initialGroups} are not expected or handled; - * if {@code initialGroups} is a refreshing table, only its contents at instantiation time will be used. If - * {@code initialGroups == null}, the result will be the same as if a table with no rows was supplied. - * @param keyColumnNames The names of the key columns to aggregate by - * @return A {@link PartitionedTable} keyed by {@code keyColumnNames} - */ - @ConcurrentMethod - PartitionedTable partitionedAggBy(Collection aggregations, boolean preserveEmpty, - Table initialGroups, @NotNull Collection keyColumnNames); - /** * Convenience method that performs an {@link #aggBy} and wraps the result in a {@link PartitionedTable}. If * {@code aggregations} does not include a {@link io.deephaven.api.agg.Partition partition}, one will be added diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java index a98bf6d5705..d9ecd6f1841 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java @@ -468,15 +468,15 @@ public Object[] getRecord(long rowNo, String... columnNames) { } @Override - public PartitionedTable partitionBy(final boolean dropKeys, - @NotNull final Collection columns) { + public PartitionedTable partitionBy(final boolean dropKeys, final String... keyColumnNames) { if (isStream()) { throw streamUnsupported("partitionBy"); } + final List columns = ColumnName.from(keyColumnNames); return memoizeResult(MemoizedOperationKey.partitionBy(dropKeys, columns), () -> { final Table partitioned = aggBy(Partition.of(CONSTITUENT, !dropKeys), columns); final Set keyColumnNamesSet = - columns.stream().map(ColumnName::name).collect(Collectors.toCollection(LinkedHashSet::new)); + Arrays.stream(keyColumnNames).collect(Collectors.toCollection(LinkedHashSet::new)); final TableDefinition constituentDefinition; if (dropKeys) { constituentDefinition = TableDefinition.of(definition.getColumnStream() @@ -491,8 +491,7 @@ public PartitionedTable partitionBy(final boolean dropKeys, @Override public PartitionedTable partitionedAggBy(final Collection aggregations, - final boolean preserveEmpty, @Nullable final Table initialGroups, - @NotNull final Collection columns) { + final boolean preserveEmpty, @Nullable final Table initialGroups, final String... keyColumnNames) { if (isStream()) { throw streamUnsupported("partitionedAggBy"); } @@ -505,9 +504,9 @@ public PartitionedTable partitionedAggBy(final Collection ? aggregations : Stream.concat(aggregations.stream(), Stream.of(partition)).collect(Collectors.toList()); final Table aggregated = - aggBy(aggregationsToUse, preserveEmpty, initialGroups, columns); + aggBy(aggregationsToUse, preserveEmpty, initialGroups, ColumnName.from(keyColumnNames)); final Set keyColumnNamesSet = - columns.stream().map(ColumnName::name).collect(Collectors.toCollection(LinkedHashSet::new)); + Arrays.stream(keyColumnNames).collect(Collectors.toCollection(LinkedHashSet::new)); final TableDefinition constituentDefinition; if (partition.includeGroupByColumns()) { constituentDefinition = definition; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/TableDefaults.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/TableDefaults.java index de032391296..7a92423e5b6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/TableDefaults.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/TableDefaults.java @@ -570,12 +570,6 @@ default Table ungroupAllBut(String... columnsNotToUngroup) { // PartitionBy Operations // ----------------------------------------------------------------------------------------------------------------- - @Override - @ConcurrentMethod - default PartitionedTable partitionBy(boolean dropKeys, String... keyColumnNames) { - return partitionBy(dropKeys, ColumnName.from(keyColumnNames)); - } - @Override @ConcurrentMethod @FinalDefault @@ -583,13 +577,6 @@ default PartitionedTable partitionBy(String... keyColumnNames) { return partitionBy(false, keyColumnNames); } - @Override - @ConcurrentMethod - default PartitionedTable partitionedAggBy(final Collection aggregations, - final boolean preserveEmpty, @Nullable final Table initialGroups, String... keyColumnNames) { - return partitionedAggBy(aggregations, preserveEmpty, initialGroups, ColumnName.from(keyColumnNames)); - } - // ----------------------------------------------------------------------------------------------------------------- // Hierarchical table operations (rollup and tree). // ----------------------------------------------------------------------------------------------------------------- diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UncoalescedTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UncoalescedTable.java index 4d88d63549b..098ba327bdc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UncoalescedTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UncoalescedTable.java @@ -370,15 +370,15 @@ public Table ungroup(boolean nullFill, Collection columnsT @Override @ConcurrentMethod - public PartitionedTable partitionBy(boolean dropKeys, @NotNull Collection columns) { - return coalesce().partitionBy(dropKeys, columns); + public PartitionedTable partitionBy(boolean dropKeys, String... keyColumnNames) { + return coalesce().partitionBy(dropKeys, keyColumnNames); } @Override @ConcurrentMethod public PartitionedTable partitionedAggBy(Collection aggregations, boolean preserveEmpty, - Table initialGroups, @NotNull Collection columns) { - return coalesce().partitionedAggBy(aggregations, preserveEmpty, initialGroups, columns); + Table initialGroups, String... keyColumnNames) { + return coalesce().partitionedAggBy(aggregations, preserveEmpty, initialGroups, keyColumnNames); } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java index 4e7120ad425..61678210a28 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java @@ -61,6 +61,8 @@ protected BucketedPartitionedUpdateByManager( // this table will always have the rowset of the source result = new QueryTable(source.getRowSet(), resultSources); + String[] byColumnNames = ColumnName.names(byColumns).toArray(new String[0]); + final PartitionedTable pt; if (source.isRefreshing()) { // this is a refreshing source, we will need a listener @@ -74,12 +76,12 @@ protected BucketedPartitionedUpdateByManager( op.createInputModifiedColumnSet(source); op.createOutputModifiedColumnSet(result); } - pt = source.partitionedAggBy(List.of(), true, null, byColumns); + pt = source.partitionedAggBy(List.of(), true, null, byColumnNames); // make the source->result transformer transformer = source.newModifiedColumnSetTransformer(result, source.getDefinition().getColumnNamesArray()); } else { - pt = source.partitionedAggBy(List.of(), true, null, byColumns); + pt = source.partitionedAggBy(List.of(), true, null, byColumnNames); } final PartitionedTable transformed = pt.transform(t -> { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java index cb937761f18..baf18784ae7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java @@ -11,7 +11,6 @@ import io.deephaven.base.log.LogOutputAppendable; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ResettableWritableObjectChunk; -import io.deephaven.chunk.WritableLongChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.configuration.Configuration; import io.deephaven.engine.context.ExecutionContext; @@ -150,12 +149,11 @@ private RowSet getInnerKeys(final RowSet outerKeys) { RowSetBuilderRandom builder = RowSetFactory.builderRandom(); final int chunkSize = Math.min(outerKeys.intSize(), REDIRECTION_CHUNK_SIZE); try (final RowSequence.Iterator it = outerKeys.getRowSequenceIterator(); - ChunkSource.FillContext fillContext = rowRedirection.makeFillContext(chunkSize, null); - WritableLongChunk chunk = WritableLongChunk.makeWritableChunk(chunkSize)) { + ChunkSource.GetContext getContext = rowRedirection.makeGetContext(chunkSize)) { while (it.hasMore()) { final RowSequence rs = it.getNextRowSequenceWithLength(chunkSize); - rowRedirection.fillChunk(fillContext, chunk, rs); - builder.addRowKeysChunk(chunk); + Chunk chunk = rowRedirection.getChunk(getContext, rs); + builder.addRowKeysChunk(chunk.asLongChunk()); } } return builder.build(); @@ -471,9 +469,7 @@ private void createCachedColumnSource(int srcIdx, final Runnable resumeAction) { count += PARALLEL_CACHE_CHUNK_SIZE; } } - }, () -> { - resumeAction.run(); - }, + }, resumeAction::run, this::onError); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperatorFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperatorFactory.java index a24cb388bc2..9303ca91830 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperatorFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperatorFactory.java @@ -19,6 +19,7 @@ import io.deephaven.engine.table.impl.util.WritableRowRedirection; import io.deephaven.time.DateTime; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.math.BigDecimal; import java.math.BigInteger; @@ -35,14 +36,14 @@ public class UpdateByOperatorFactory { private final TableDefaults source; private final MatchPair[] groupByColumns; - @NotNull + @Nullable private final WritableRowRedirection rowRedirection; @NotNull private final UpdateByControl control; public UpdateByOperatorFactory(@NotNull final TableDefaults source, @NotNull final MatchPair[] groupByColumns, - @NotNull final WritableRowRedirection rowRedirection, + @Nullable final WritableRowRedirection rowRedirection, @NotNull final UpdateByControl control) { this.source = source; this.groupByColumns = groupByColumns; From 2224a4900693b5a6e9f8557e21980fc3c75fca61 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 6 Jan 2023 11:21:27 -0800 Subject: [PATCH 074/123] spotless applied, whitespace changes --- .../main/java/io/deephaven/engine/table/impl/QueryTable.java | 2 +- .../java/io/deephaven/engine/table/impl/UncoalescedTable.java | 2 +- .../impl/util/InverseWrappedRowSetWritableRowRedirection.java | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java index d9ecd6f1841..9a24f2ec024 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java @@ -491,7 +491,7 @@ public PartitionedTable partitionBy(final boolean dropKeys, final String... keyC @Override public PartitionedTable partitionedAggBy(final Collection aggregations, - final boolean preserveEmpty, @Nullable final Table initialGroups, final String... keyColumnNames) { + final boolean preserveEmpty, @Nullable final Table initialGroups, final String... keyColumnNames) { if (isStream()) { throw streamUnsupported("partitionedAggBy"); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/UncoalescedTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/UncoalescedTable.java index 098ba327bdc..4ce9e0abc91 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/UncoalescedTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/UncoalescedTable.java @@ -377,7 +377,7 @@ public PartitionedTable partitionBy(boolean dropKeys, String... keyColumnNames) @Override @ConcurrentMethod public PartitionedTable partitionedAggBy(Collection aggregations, boolean preserveEmpty, - Table initialGroups, String... keyColumnNames) { + Table initialGroups, String... keyColumnNames) { return coalesce().partitionedAggBy(aggregations, preserveEmpty, initialGroups, keyColumnNames); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/InverseWrappedRowSetWritableRowRedirection.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/InverseWrappedRowSetWritableRowRedirection.java index a0fbe4860c8..6470dea6d36 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/InverseWrappedRowSetWritableRowRedirection.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/InverseWrappedRowSetWritableRowRedirection.java @@ -23,8 +23,8 @@ public class InverseWrappedRowSetWritableRowRedirection implements WritableRowRe /** * This class accepts a {@link RowSet} and attempts to cast to a {@link TrackingRowSet} if {@link #getPrev(long)} or - * {@link #fillPrevChunk(ChunkSource.FillContext, WritableChunk, RowSequence)} is called. Calling these - * functions on a non-tracking RowSet will result in a {@link ClassCastException}. + * {@link #fillPrevChunk(FillContext, WritableChunk, RowSequence)} is called. Calling these functions on + * a non-tracking RowSet will result in a {@link ClassCastException}. * * @param wrappedRowSet the RowSet (or TrackingRowSet) to use as the redirection source */ From 9ac421913408f2dd124d02131b31449a2eef4763 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 6 Jan 2023 12:10:34 -0800 Subject: [PATCH 075/123] more spotless --- .../impl/util/InverseWrappedRowSetWritableRowRedirection.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/InverseWrappedRowSetWritableRowRedirection.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/InverseWrappedRowSetWritableRowRedirection.java index 6470dea6d36..6ab2272dce7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/InverseWrappedRowSetWritableRowRedirection.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/InverseWrappedRowSetWritableRowRedirection.java @@ -23,8 +23,8 @@ public class InverseWrappedRowSetWritableRowRedirection implements WritableRowRe /** * This class accepts a {@link RowSet} and attempts to cast to a {@link TrackingRowSet} if {@link #getPrev(long)} or - * {@link #fillPrevChunk(FillContext, WritableChunk, RowSequence)} is called. Calling these functions on - * a non-tracking RowSet will result in a {@link ClassCastException}. + * {@link #fillPrevChunk(FillContext, WritableChunk, RowSequence)} is called. Calling these functions on a + * non-tracking RowSet will result in a {@link ClassCastException}. * * @param wrappedRowSet the RowSet (or TrackingRowSet) to use as the redirection source */ From c72934cb13661fb52a1face046f194b4d652debe Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 6 Jan 2023 17:18:21 -0800 Subject: [PATCH 076/123] Mid-review changes. --- .../table/impl/sources/ByteArraySource.java | 8 ++++---- .../impl/sources/CharacterArraySource.java | 8 ++++---- .../table/impl/sources/DoubleArraySource.java | 8 ++++---- .../table/impl/sources/FloatArraySource.java | 8 ++++---- .../table/impl/sources/IntegerArraySource.java | 8 ++++---- .../table/impl/sources/ShortArraySource.java | 8 ++++---- .../engine/table/impl/updateby/UpdateBy.java | 17 +++++++---------- .../table/impl/updateby/UpdateByWindow.java | 4 ++-- .../impl/updateby/UpdateByWindowCumulative.java | 8 ++++---- 9 files changed, 37 insertions(+), 40 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ByteArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ByteArraySource.java index 9a33eb65ab5..f141c7ae4fd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ByteArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ByteArraySource.java @@ -324,21 +324,21 @@ protected void fillSparseChunk(@NotNull final WritableChunk dest } @Override - protected void fillSparsePrevChunk(@NotNull final WritableChunk destGeneric, @NotNull final RowSequence rpws) { - final long sz = rpws.size(); + protected void fillSparsePrevChunk(@NotNull final WritableChunk destGeneric, @NotNull final RowSequence rows) { + final long sz = rows.size(); if (sz == 0) { destGeneric.setSize(0); return; } if (prevFlusher == null) { - fillSparseChunk(destGeneric, rpws); + fillSparseChunk(destGeneric, rows); return; } final WritableByteChunk dest = destGeneric.asWritableByteChunk(); final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); - rpws.forAllRowKeys((final long v) -> { + rows.forAllRowKeys((final long v) -> { if (v >= ctx.capForCurrentBlock) { ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/CharacterArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/CharacterArraySource.java index 3883641b616..dfde46a223f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/CharacterArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/CharacterArraySource.java @@ -319,21 +319,21 @@ protected void fillSparseChunk(@NotNull final WritableChunk dest } @Override - protected void fillSparsePrevChunk(@NotNull final WritableChunk destGeneric, @NotNull final RowSequence rpws) { - final long sz = rpws.size(); + protected void fillSparsePrevChunk(@NotNull final WritableChunk destGeneric, @NotNull final RowSequence rows) { + final long sz = rows.size(); if (sz == 0) { destGeneric.setSize(0); return; } if (prevFlusher == null) { - fillSparseChunk(destGeneric, rpws); + fillSparseChunk(destGeneric, rows); return; } final WritableCharChunk dest = destGeneric.asWritableCharChunk(); final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); - rpws.forAllRowKeys((final long v) -> { + rows.forAllRowKeys((final long v) -> { if (v >= ctx.capForCurrentBlock) { ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/DoubleArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/DoubleArraySource.java index 8a50e241792..48d530959df 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/DoubleArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/DoubleArraySource.java @@ -324,21 +324,21 @@ protected void fillSparseChunk(@NotNull final WritableChunk dest } @Override - protected void fillSparsePrevChunk(@NotNull final WritableChunk destGeneric, @NotNull final RowSequence rpws) { - final long sz = rpws.size(); + protected void fillSparsePrevChunk(@NotNull final WritableChunk destGeneric, @NotNull final RowSequence rows) { + final long sz = rows.size(); if (sz == 0) { destGeneric.setSize(0); return; } if (prevFlusher == null) { - fillSparseChunk(destGeneric, rpws); + fillSparseChunk(destGeneric, rows); return; } final WritableDoubleChunk dest = destGeneric.asWritableDoubleChunk(); final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); - rpws.forAllRowKeys((final long v) -> { + rows.forAllRowKeys((final long v) -> { if (v >= ctx.capForCurrentBlock) { ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/FloatArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/FloatArraySource.java index 1b7e889a2dd..3a5ff8b6b5f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/FloatArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/FloatArraySource.java @@ -324,21 +324,21 @@ protected void fillSparseChunk(@NotNull final WritableChunk dest } @Override - protected void fillSparsePrevChunk(@NotNull final WritableChunk destGeneric, @NotNull final RowSequence rpws) { - final long sz = rpws.size(); + protected void fillSparsePrevChunk(@NotNull final WritableChunk destGeneric, @NotNull final RowSequence rows) { + final long sz = rows.size(); if (sz == 0) { destGeneric.setSize(0); return; } if (prevFlusher == null) { - fillSparseChunk(destGeneric, rpws); + fillSparseChunk(destGeneric, rows); return; } final WritableFloatChunk dest = destGeneric.asWritableFloatChunk(); final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); - rpws.forAllRowKeys((final long v) -> { + rows.forAllRowKeys((final long v) -> { if (v >= ctx.capForCurrentBlock) { ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/IntegerArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/IntegerArraySource.java index 226421ebce3..4589ebafd40 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/IntegerArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/IntegerArraySource.java @@ -324,21 +324,21 @@ protected void fillSparseChunk(@NotNull final WritableChunk dest } @Override - protected void fillSparsePrevChunk(@NotNull final WritableChunk destGeneric, @NotNull final RowSequence rpws) { - final long sz = rpws.size(); + protected void fillSparsePrevChunk(@NotNull final WritableChunk destGeneric, @NotNull final RowSequence rows) { + final long sz = rows.size(); if (sz == 0) { destGeneric.setSize(0); return; } if (prevFlusher == null) { - fillSparseChunk(destGeneric, rpws); + fillSparseChunk(destGeneric, rows); return; } final WritableIntChunk dest = destGeneric.asWritableIntChunk(); final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); - rpws.forAllRowKeys((final long v) -> { + rows.forAllRowKeys((final long v) -> { if (v >= ctx.capForCurrentBlock) { ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ShortArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ShortArraySource.java index 8d1ed871f52..aff545b28fb 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ShortArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ShortArraySource.java @@ -324,21 +324,21 @@ protected void fillSparseChunk(@NotNull final WritableChunk dest } @Override - protected void fillSparsePrevChunk(@NotNull final WritableChunk destGeneric, @NotNull final RowSequence rpws) { - final long sz = rpws.size(); + protected void fillSparsePrevChunk(@NotNull final WritableChunk destGeneric, @NotNull final RowSequence rows) { + final long sz = rows.size(); if (sz == 0) { destGeneric.setSize(0); return; } if (prevFlusher == null) { - fillSparseChunk(destGeneric, rpws); + fillSparseChunk(destGeneric, rows); return; } final WritableShortChunk dest = destGeneric.asWritableShortChunk(); final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); - rpws.forAllRowKeys((final long v) -> { + rows.forAllRowKeys((final long v) -> { if (v >= ctx.capForCurrentBlock) { ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java index baf18784ae7..ca15b00ecd8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java @@ -604,11 +604,9 @@ private void cleanUpAndNotify(final Runnable resumeAction) { } // clear the sparse output columns for rows that no longer exist - if (!initialStep && !redirHelper.isRedirected()) { - if (!toClear.isEmpty()) { - for (UpdateByOperator op : operators) { - op.clearOutputRows(toClear); - } + if (!initialStep && !redirHelper.isRedirected() && !toClear.isEmpty()) { + for (UpdateByOperator op : operators) { + op.clearOutputRows(toClear); } } @@ -697,11 +695,10 @@ public void processUpdate() { toClear = redirHelper.getRowsToClear(); // clear them now and let them set their own prev states - if (!initialStep) { - if (!toClear.isEmpty()) { - for (UpdateByOperator op : operators) { - op.clearOutputRows(toClear); - } + if (!initialStep &&!toClear.isEmpty()) { + for (UpdateByOperator op : operators) { + op.clearOutputRows(toClear); + } } } else { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java index c69e086d818..3223c24bbba 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java @@ -29,7 +29,7 @@ public abstract class UpdateByWindow { protected int[] uniqueInputSourceIndices; /** This context will store the necessary info to process a single window for a single bucket */ - public abstract class UpdateByWindowBucketContext implements SafeCloseable { + public class UpdateByWindowBucketContext implements SafeCloseable { /** A reference to the source rowset */ protected final TrackingRowSet sourceRowSet; /** The column source providing the timestamp data for this window */ @@ -317,7 +317,7 @@ protected static int hashCode(boolean windowed, @NotNull String[] inputColumnNam hash = 31 * hash + s.hashCode(); } - hash = 31 * hash + Boolean.hashCode(true); + hash = 31 * hash + Boolean.hashCode(windowed); // treat all cumulative ops with the same input columns as identical, even if they rely on timestamps if (!windowed) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java index 578ee302239..4fd6a4459ba 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java @@ -48,7 +48,7 @@ public UpdateByWindowBucketContext makeWindowContext(final TrackingRowSet source final boolean isInitializeStep) { return new UpdateByWindowBucketContext(sourceRowSet, timestampColumnSource, timestampSsa, chunkSize, - isInitializeStep) {}; + isInitializeStep); } @Override @@ -212,7 +212,7 @@ private static long smallestAffectedKey(@NotNull final RowSet added, @NotNull final RowSet modified, @NotNull final RowSet removed, @NotNull final RowSetShiftData shifted, - @NotNull final RowSet affectedIndex) { + @NotNull final RowSet affectedRowSet) { long smallestModifiedKey = Long.MAX_VALUE; if (removed.isNonempty()) { @@ -229,9 +229,9 @@ private static long smallestAffectedKey(@NotNull final RowSet added, if (shifted.nonempty()) { final long firstModKey = modified.isEmpty() ? Long.MAX_VALUE : modified.firstRowKey(); - boolean modShiftFound = !modified.isEmpty(); + boolean modShiftFound = modified.isNonempty(); boolean affectedFound = false; - try (final RowSequence.Iterator it = affectedIndex.getRowSequenceIterator()) { + try (final RowSequence.Iterator it = affectedRowSet.getRowSequenceIterator()) { for (int shiftIdx = 0; shiftIdx < shifted.size() && (!modShiftFound || !affectedFound); shiftIdx++) { final long shiftStart = shifted.getBeginRange(shiftIdx); From 40cb862502b84c30e7308c5ce674b3509f3fb299 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 10 Jan 2023 08:48:53 -0800 Subject: [PATCH 077/123] Mid-review changes and addressing PR comments. --- .../BucketedPartitionedUpdateByManager.java | 2 +- .../engine/table/impl/updateby/UpdateBy.java | 67 +++---- .../impl/updateby/UpdateByBucketHelper.java | 3 +- .../table/impl/updateby/UpdateByWindow.java | 6 +- .../updateby/UpdateByWindowCumulative.java | 167 ++++++++---------- .../impl/updateby/UpdateByWindowTicks.java | 10 +- .../impl/updateby/UpdateByWindowTime.java | 10 +- .../ema/BasePrimitiveEMAOperator.java | 31 +--- .../updateby/ema/BigDecimalEMAOperator.java | 29 ++- .../updateby/ema/BigIntegerEMAOperator.java | 12 +- .../updateby/ema/BigNumberEMAOperator.java | 30 +--- .../impl/updateby/ema/ByteEMAOperator.java | 20 +-- .../impl/updateby/ema/DoubleEMAOperator.java | 18 +- .../impl/updateby/ema/FloatEMAOperator.java | 18 +- .../impl/updateby/ema/IntEMAOperator.java | 20 +-- .../impl/updateby/ema/LongEMAOperator.java | 20 +-- .../impl/updateby/ema/ShortEMAOperator.java | 20 +-- .../engine/table/impl/updateby/TestEma.java | 112 ++++++------ .../api/updateby/OperationControl.java | 41 ----- 19 files changed, 256 insertions(+), 380 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java index 61678210a28..bc85b22e862 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java @@ -61,7 +61,7 @@ protected BucketedPartitionedUpdateByManager( // this table will always have the rowset of the source result = new QueryTable(source.getRowSet(), resultSources); - String[] byColumnNames = ColumnName.names(byColumns).toArray(new String[0]); + String[] byColumnNames = byColumns.stream().map(ColumnName::name).toArray(String[]::new); final PartitionedTable pt; if (source.isRefreshing()) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java index ca15b00ecd8..f72eb19da22 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java @@ -90,7 +90,8 @@ public abstract class UpdateBy { static class UpdateByRedirectionHelper { @Nullable private final WritableRowRedirection rowRedirection; - private final TrackingWritableRowSet freeRows; + private final WritableRowSet freeRows; + private WritableRowSet toClear; private long maxInnerRowKey; private UpdateByRedirectionHelper(@Nullable final WritableRowRedirection rowRedirection) { @@ -107,18 +108,22 @@ private long requiredCapacity() { return maxInnerRowKey; } - @Nullable - WritableRowRedirection getRowRedirection() { - return rowRedirection; - } - private void processUpdateForRedirection(@NotNull final TableUpdate upstream, final TrackingRowSet sourceRowSet) { assert rowRedirection != null; + + // take this chance to clean up last cycle's rowset + try (final RowSet ignored = toClear) { + } + if (upstream.removed().isNonempty()) { final RowSetBuilderRandom freeBuilder = RowSetFactory.builderRandom(); upstream.removed().forAllRowKeys(key -> freeBuilder.addKey(rowRedirection.remove(key))); - freeRows.insert(freeBuilder.build()); + // store all freed rows as the candidate toClear set + toClear = freeBuilder.build(); + freeRows.insert(toClear); + } else { + toClear = RowSetFactory.empty(); } if (upstream.shifted().nonempty()) { @@ -135,8 +140,12 @@ private void processUpdateForRedirection(@NotNull final TableUpdate upstream, rowRedirection.put(outerKey, innerKey); }); if (freeIt.hasNext()) { - freeRows.removeRange(0, freeIt.nextLong() - 1); + try (final RowSet added = freeRows.subSetByKeyRange(0, freeIt.nextLong() - 1)) { + toClear.remove(added); + freeRows.remove(added); + } } else { + toClear.clear(); freeRows.clear(); } } @@ -162,16 +171,13 @@ private RowSet getInnerKeys(final RowSet outerKeys) { /*** * Compute the inner source keys that need to be cleared. These are rows that were removed this cycle and not * replaced by added rows. These are in the dense key-space and must only be applied to the inner sources of the - * redirected output sources + * redirected output sources. * - * @return the set of rows that should be cleared from the inner (dense) sources + * @return the set of rows that should be cleared from the inner (dense) sources. This {@link RowSet} should be + * closed by the caller. */ WritableRowSet getRowsToClear() { - final WritableRowSet toClear = freeRows.copy(); - try (final RowSet prevKeys = freeRows.copyPrev()) { - toClear.remove(prevKeys); - } - return toClear; + return toClear.copy(); } } @@ -442,10 +448,9 @@ private void createCachedColumnSource(int srcIdx, final Runnable resumeAction) { // holding this reference should protect `rowDirection` and `innerSource` from GC maybeCachedInputSources[srcIdx] = outputSource; - final int rowSetSize = inputRowSet.intSize(); - // how many batches do we need? - final int taskCount = (rowSetSize + PARALLEL_CACHE_BATCH_SIZE - 1) / PARALLEL_CACHE_BATCH_SIZE; + final int taskCount = + Math.toIntExact((inputRowSet.size() + PARALLEL_CACHE_BATCH_SIZE - 1) / PARALLEL_CACHE_BATCH_SIZE); jobScheduler.iterateParallel(ExecutionContext.getContextToRecord(), this, 0, taskCount, @@ -457,16 +462,16 @@ private void createCachedColumnSource(int srcIdx, final Runnable resumeAction) { inputSource.makeGetContext(PARALLEL_CACHE_CHUNK_SIZE)) { // advance to the first key of this block rsIt.advance(inputRowSet.get(idx * PARALLEL_CACHE_BATCH_SIZE)); - int count = 0; - while (rsIt.hasMore() && count < PARALLEL_CACHE_BATCH_SIZE) { - final RowSequence chunkOk = - rsIt.getNextRowSequenceWithLength(PARALLEL_CACHE_CHUNK_SIZE); + int remaining = PARALLEL_CACHE_BATCH_SIZE; + while (rsIt.hasMore() && remaining > 0) { + final RowSequence chunkOk = rsIt + .getNextRowSequenceWithLength(Math.min(remaining, PARALLEL_CACHE_CHUNK_SIZE)); final Chunk values = inputSource.getChunk(gc, chunkOk); outputSource.fillFromChunk(ffc, values, chunkOk); - // increment by the attempted stride, if this is the last block the iterator will + // reduce by the attempted stride, if this is the final block the iterator will // be exhausted and hasMore() will return false - count += PARALLEL_CACHE_CHUNK_SIZE; + remaining -= PARALLEL_CACHE_CHUNK_SIZE; } } }, resumeAction::run, @@ -493,8 +498,8 @@ private void cacheInputSources(final int winIdx, final Runnable resumeAction) { } /** - * Divide the buckets for {@code windows[winIdx]} into sets and process each set in parallel. Calls - * {@code resumeAction} when the work is complete + * Process each bucket in {@code windows[winIdx]} in parallel. Calls {@code resumeAction} when the work is + * complete */ private void processWindowBuckets(int winIdx, final Runnable resumeAction) { if (jobScheduler.threadCount() > 1 && dirtyBuckets.length > 1) { @@ -544,10 +549,12 @@ private void processWindows(final Runnable resumeAction) { windowRowSet.insert(win.getAffectedRows(bucket.windowContexts[winIdx])); } } - try (final RowSet changedRows = redirHelper.isRedirected() + try (final RowSet windowChangedRows = redirHelper.isRedirected() ? redirHelper.getInnerKeys(windowRowSet) - : windowRowSet.copy()) { - win.prepareForParallelPopulation(changedRows); + : null) { + final RowSet rowsToUse = + windowChangedRows == null ? windowRowSet : windowChangedRows; + win.prepareForParallelPopulation(rowsToUse); } } } @@ -695,7 +702,7 @@ public void processUpdate() { toClear = redirHelper.getRowsToClear(); // clear them now and let them set their own prev states - if (!initialStep &&!toClear.isEmpty()) { + if (!initialStep && !toClear.isEmpty()) { for (UpdateByOperator op : operators) { op.clearOutputRows(toClear); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java index 57ce118553a..646e0c6bd3a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java @@ -14,6 +14,7 @@ import io.deephaven.engine.table.impl.InstrumentedTableUpdateListenerAdapter; import io.deephaven.engine.table.impl.QueryTable; import io.deephaven.engine.table.impl.TableUpdateImpl; +import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.sources.ReinterpretUtils; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.util.RowRedirection; @@ -229,7 +230,7 @@ private void fillChunkWithNonNull(LongChunk keysChunk, LongChunk continue; } if (ts < lastTimestamp.longValue()) { - throw (new IllegalStateException( + throw (new TableDataException( "updateBy time-based operators require non-descending timestamp values")); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java index 3223c24bbba..336aa6d8a3c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java @@ -49,8 +49,6 @@ public class UpdateByWindowBucketContext implements SafeCloseable { protected ChunkSource.GetContext[] inputSourceGetContexts; /** A set of chunks used to store working values */ protected Chunk[] inputSourceChunks; - /** An indicator of if each slot has been populated with data or not for this phase. */ - protected boolean[] inputSourceChunkPopulated; /** The rows affected by this update */ protected RowSet affectedRows; /** The rows that will be needed to re-compute `affectedRows` */ @@ -232,7 +230,6 @@ public abstract void computeAffectedRowsAndOperators(final UpdateByWindowBucketC */ public void assignInputSources(final UpdateByWindowBucketContext context, final ColumnSource[] inputSources) { context.inputSources = inputSources; - context.inputSourceChunkPopulated = new boolean[inputSources.length]; context.inputSourceGetContexts = new ChunkSource.GetContext[inputSources.length]; context.inputSourceChunks = new WritableChunk[inputSources.length]; @@ -251,10 +248,9 @@ public void assignInputSources(final UpdateByWindowBucketContext context, final */ protected void prepareValuesChunkForSource(final UpdateByWindowBucketContext context, final int srcIdx, final RowSequence rs) { - if (!context.inputSourceChunkPopulated[srcIdx]) { + if (context.inputSourceChunks[srcIdx] == null) { context.inputSourceChunks[srcIdx] = context.inputSources[srcIdx].getChunk(context.inputSourceGetContexts[srcIdx], rs); - context.inputSourceChunkPopulated[srcIdx] = true; } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java index 4fd6a4459ba..b5b12e01c34 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java @@ -33,7 +33,7 @@ public UpdateByWindowCumulative(UpdateByOperator[] operators, int[][] operatorSo protected void makeOperatorContexts(UpdateByWindowBucketContext context) { // working chunk size need not be larger than affectedRows.size() - context.workingChunkSize = Math.min(context.workingChunkSize, context.affectedRows.intSize()); + context.workingChunkSize = Math.toIntExact(Math.min(context.workingChunkSize, context.affectedRows.size())); // create contexts for the affected operators for (int opIdx : context.dirtyOperatorIndices) { @@ -53,7 +53,6 @@ public UpdateByWindowBucketContext makeWindowContext(final TrackingRowSet source @Override public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, @NotNull TableUpdate upstream) { - // all rows are affected on the initial step if (context.initialStep) { context.affectedRows = context.sourceRowSet.copy(); @@ -99,8 +98,7 @@ public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, return; } - long smallestModifiedKey = smallestAffectedKey(upstream.added(), upstream.modified(), upstream.removed(), - upstream.shifted(), context.sourceRowSet); + long smallestModifiedKey = smallestAffectedKey(upstream, context.sourceRowSet); context.affectedRows = smallestModifiedKey == Long.MAX_VALUE ? RowSetFactory.empty() @@ -114,46 +112,47 @@ public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, public void processRows(UpdateByWindowBucketContext context, final boolean initialStep) { Assert.neqNull(context.inputSources, "assignInputSources() must be called before processRow()"); - // find the key before the first affected row - final long keyBefore; - try (final RowSet.SearchIterator rIt = context.sourceRowSet.reverseIterator()) { - rIt.advance(context.affectedRows.firstRowKey()); - if (rIt.hasNext()) { - keyBefore = rIt.nextLong(); - } else { - keyBefore = NULL_ROW_KEY; + if (initialStep) { + // always at the beginning of the RowSet at creation phase + for (int opIdx : context.dirtyOperatorIndices) { + UpdateByCumulativeOperator cumOp = (UpdateByCumulativeOperator) operators[opIdx]; + cumOp.initializeUpdate(context.opContext[opIdx], NULL_ROW_KEY, NULL_LONG); } - } - - // and preload that data for these operators - for (int opIdx : context.dirtyOperatorIndices) { - UpdateByCumulativeOperator cumOp = (UpdateByCumulativeOperator) operators[opIdx]; - if (cumOp.getTimestampColumnName() == null || keyBefore == NULL_ROW_KEY) { - // this operator doesn't care about timestamps or we know we are at the beginning of the rowset - cumOp.initializeUpdate(context.opContext[opIdx], keyBefore, NULL_LONG); - } else { - // this operator cares about timestamps, so make sure it is starting from a valid value and - // valid timestamp by moving backward until the conditions are met - UpdateByCumulativeOperator.Context cumOpContext = - (UpdateByCumulativeOperator.Context) context.opContext[opIdx]; - long potentialResetTimestamp = context.timestampColumnSource.getLong(keyBefore); - - if (potentialResetTimestamp == NULL_LONG || !cumOpContext.isValueValid(keyBefore)) { - try (final RowSet.SearchIterator rIt = context.sourceRowSet.reverseIterator()) { - if (rIt.advance(keyBefore)) { - while (rIt.hasNext()) { - final long nextKey = rIt.nextLong(); - potentialResetTimestamp = context.timestampColumnSource.getLong(nextKey); - if (potentialResetTimestamp != NULL_LONG && - cumOpContext.isValueValid(nextKey)) { - break; + } else { + // find the key before the first affected row + final long pos = context.sourceRowSet.find(context.affectedRows.firstRowKey()); + final long keyBefore = pos == 0 ? NULL_ROW_KEY : context.sourceRowSet.get(pos - 1); + + // and preload that data for these operators + for (int opIdx : context.dirtyOperatorIndices) { + UpdateByCumulativeOperator cumOp = (UpdateByCumulativeOperator) operators[opIdx]; + if (cumOp.getTimestampColumnName() == null || keyBefore == NULL_ROW_KEY) { + // this operator doesn't care about timestamps or we know we are at the beginning of the rowset + cumOp.initializeUpdate(context.opContext[opIdx], keyBefore, NULL_LONG); + } else { + // this operator cares about timestamps, so make sure it is starting from a valid value and + // valid timestamp by looking backward until the conditions are met + UpdateByCumulativeOperator.Context cumOpContext = + (UpdateByCumulativeOperator.Context) context.opContext[opIdx]; + long potentialResetTimestamp = context.timestampColumnSource.getLong(keyBefore); + + if (potentialResetTimestamp == NULL_LONG || !cumOpContext.isValueValid(keyBefore)) { + try (final RowSet.SearchIterator rIt = context.sourceRowSet.reverseIterator()) { + if (rIt.advance(keyBefore)) { + while (rIt.hasNext()) { + final long nextKey = rIt.nextLong(); + potentialResetTimestamp = context.timestampColumnSource.getLong(nextKey); + if (potentialResetTimestamp != NULL_LONG && + cumOpContext.isValueValid(nextKey)) { + break; + } } } } } + // call the specialized version of `intializeUpdate()` for these operators + cumOp.initializeUpdate(context.opContext[opIdx], keyBefore, potentialResetTimestamp); } - // call the specialized version of `intializeUpdate()` for these operators - cumOp.initializeUpdate(context.opContext[opIdx], keyBefore, potentialResetTimestamp); } } @@ -164,7 +163,7 @@ public void processRows(UpdateByWindowBucketContext context, final boolean initi while (it.hasMore()) { final RowSequence rs = it.getNextRowSequenceWithLength(context.workingChunkSize); final int size = rs.intSize(); - Arrays.fill(context.inputSourceChunkPopulated, false); + Arrays.fill(context.inputSourceChunks, null); // create the timestamp chunk if needed LongChunk tsChunk = context.timestampColumnSource == null ? null @@ -201,72 +200,48 @@ public void processRows(UpdateByWindowBucketContext context, final boolean initi /** * Find the smallest valued key that participated in the upstream {@link TableUpdate}. * - * @param added the added rows - * @param modified the modified rows - * @param removed the removed rows - * @param shifted the shifted rows + * @param upstream the {@link TableUpdate update} from upstream + * @param affectedRowSet the {@link TrackingRowSet rowset} for the current bucket * - * @return the smallest key that participated in any part of the update. + * @return the smallest key that participated in any part of the update. This will be the minimum of the first key + * of each of added, modified and removed (post-shift) rows. */ - private static long smallestAffectedKey(@NotNull final RowSet added, - @NotNull final RowSet modified, - @NotNull final RowSet removed, - @NotNull final RowSetShiftData shifted, - @NotNull final RowSet affectedRowSet) { + private static long smallestAffectedKey(@NotNull TableUpdate upstream, @NotNull TrackingRowSet affectedRowSet) { long smallestModifiedKey = Long.MAX_VALUE; - if (removed.isNonempty()) { - smallestModifiedKey = removed.firstRowKey(); - } + if (upstream.removed().isNonempty()) { + // removed rows aren't represented in the shift data, so choose the row immediately preceding the first + // removed as the removed candidate for smallestAffectedKey + final long pos = affectedRowSet.findPrev(upstream.removed().firstRowKey()); + if (pos == 0) { + // the first row was removed, recompute everything + return affectedRowSet.firstRowKey(); + } - if (added.isNonempty()) { - smallestModifiedKey = Math.min(smallestModifiedKey, added.firstRowKey()); - } + // get the key previous to this one and shift to post-space (if needed) + smallestModifiedKey = affectedRowSet.getPrev(pos - 1); + if (upstream.shifted().nonempty()) { + final RowSetShiftData shifted = upstream.shifted(); + for (int shiftIdx = 0; shiftIdx < shifted.size(); shiftIdx++) { + if (shifted.getBeginRange(shiftIdx) > smallestModifiedKey) { + // no shift applies so we are already in post-shift space + break; + } else if (shifted.getEndRange(shiftIdx) >= smallestModifiedKey) { + // this shift applies, add the delta to get post-shift + smallestModifiedKey += shifted.getShiftDelta(shiftIdx); + break; + } + } + } - if (modified.isNonempty()) { - smallestModifiedKey = Math.min(smallestModifiedKey, modified.firstRowKey()); } - if (shifted.nonempty()) { - final long firstModKey = modified.isEmpty() ? Long.MAX_VALUE : modified.firstRowKey(); - boolean modShiftFound = modified.isNonempty(); - boolean affectedFound = false; - try (final RowSequence.Iterator it = affectedRowSet.getRowSequenceIterator()) { - for (int shiftIdx = 0; shiftIdx < shifted.size() - && (!modShiftFound || !affectedFound); shiftIdx++) { - final long shiftStart = shifted.getBeginRange(shiftIdx); - final long shiftEnd = shifted.getEndRange(shiftIdx); - final long shiftDelta = shifted.getShiftDelta(shiftIdx); - - if (!affectedFound) { - if (it.advance(shiftStart + shiftDelta)) { - final long maybeAffectedKey = it.peekNextKey(); - if (maybeAffectedKey <= shiftEnd + shiftDelta) { - affectedFound = true; - final long keyToCompare = - shiftDelta > 0 ? maybeAffectedKey - shiftDelta : maybeAffectedKey; - smallestModifiedKey = Math.min(smallestModifiedKey, keyToCompare); - } - } else { - affectedFound = true; - } - } + if (upstream.added().isNonempty()) { + smallestModifiedKey = Math.min(smallestModifiedKey, upstream.added().firstRowKey()); + } - if (!modShiftFound) { - if (firstModKey <= (shiftEnd + shiftDelta)) { - modShiftFound = true; - // If the first modified key is in the range we should include it - if (firstModKey >= (shiftStart + shiftDelta)) { - smallestModifiedKey = Math.min(smallestModifiedKey, firstModKey - shiftDelta); - } else { - // Otherwise it's not included in any shifts, and since shifts can't reorder rows - // it is the smallest possible value and we've already accounted for it above. - break; - } - } - } - } - } + if (upstream.modified().isNonempty()) { + smallestModifiedKey = Math.min(smallestModifiedKey, upstream.modified().firstRowKey()); } return smallestModifiedKey; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java index f8943903721..d535624cc72 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java @@ -140,20 +140,20 @@ private void ensureGetContextSize(UpdateByWindowBucketTicksContext ctx, long new "ensureGetContextSize exceeded Integer.MAX_VALUE", size); - // use this to determine which input sources are initialized - Arrays.fill(ctx.inputSourceChunkPopulated, false); + // use this to track which contexts have already resized + boolean[] resized = new boolean[ctx.inputSources.length]; for (int opIdx : ctx.dirtyOperatorIndices) { final int[] sourceIndices = operatorInputSourceSlots[opIdx]; for (int sourceSlot : sourceIndices) { - if (!ctx.inputSourceChunkPopulated[sourceSlot]) { + if (!resized[sourceSlot]) { // close the existing context ctx.inputSourceGetContexts[sourceSlot].close(); // create a new context of the larger size ctx.inputSourceGetContexts[sourceSlot] = ctx.inputSources[sourceSlot].makeGetContext(ctx.currentGetContextSize); - ctx.inputSourceChunkPopulated[sourceSlot] = true; + resized[sourceSlot] = true; } } } @@ -386,7 +386,7 @@ public void processRows(UpdateByWindowBucketContext context, boolean initialStep try (final RowSet chunkInfluencerRs = chunkInfluencerBuilder.build()) { ensureGetContextSize(ctx, chunkInfluencerRs.size()); - Arrays.fill(ctx.inputSourceChunkPopulated, false); + Arrays.fill(ctx.inputSourceChunks, null); for (int opIdx : context.dirtyOperatorIndices) { // prep the chunk array needed by the accumulate call final int[] srcIndices = operatorInputSourceSlots[opIdx]; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java index acb68f4b7c5..bc64f25531d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java @@ -179,20 +179,20 @@ private void ensureGetContextSize(UpdateByWindowBucketTimeContext ctx, long newS "ensureGetContextSize exceeded Integer.MAX_VALUE", size); - // use this to determine which input sources are initialized - Arrays.fill(ctx.inputSourceChunkPopulated, false); + // use this to track which contexts have already resized + boolean[] resized = new boolean[ctx.inputSources.length]; for (int opIdx : ctx.dirtyOperatorIndices) { final int[] sourceIndices = operatorInputSourceSlots[opIdx]; for (int sourceSlot : sourceIndices) { - if (!ctx.inputSourceChunkPopulated[sourceSlot]) { + if (!resized[sourceSlot]) { // close the existing context ctx.inputSourceGetContexts[sourceSlot].close(); // create a new context of the larger size ctx.inputSourceGetContexts[sourceSlot] = ctx.inputSources[sourceSlot].makeGetContext(ctx.currentGetContextSize); - ctx.inputSourceChunkPopulated[sourceSlot] = true; + resized[sourceSlot] = true; } } } @@ -414,7 +414,7 @@ public void processRows(UpdateByWindowBucketContext context, boolean initialStep try (final RowSet chunkInfluencerRs = chunkInfluencerBuilder.build()) { ensureGetContextSize(ctx, chunkInfluencerRs.size()); - Arrays.fill(ctx.inputSourceChunkPopulated, false); + Arrays.fill(ctx.inputSourceChunks, null); for (int opIdx : context.dirtyOperatorIndices) { // prep the chunk array needed by the accumulate call final int[] srcIndices = operatorInputSourceSlots[opIdx]; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java index 3db43d2488b..0b7075a5d57 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java @@ -85,9 +85,8 @@ public void initializeUpdate(@NotNull final UpdateContext updateContext, } void handleBadData(@NotNull final Context ctx, - final boolean isNull, - final boolean isNan, - final boolean isNullTime) { + final boolean isNull, + final boolean isNan) { boolean doReset = false; if (isNull) { if (control.onNullValueOrDefault() == BadDataBehavior.THROW) { @@ -104,32 +103,6 @@ void handleBadData(@NotNull final Context ctx, } } - if (isNullTime) { - if (control.onNullTimeOrDefault() == BadDataBehavior.THROW) { - throw new TableDataException("Encountered null timestamp during EMA processing"); - } - doReset = control.onNullTimeOrDefault() == BadDataBehavior.RESET; - } - - if (doReset) { - ctx.reset(); - } - } - - void handleBadTime(@NotNull final Context ctx, final long dt) { - boolean doReset = false; - if (dt == 0) { - if (control.onZeroDeltaTimeOrDefault() == BadDataBehavior.THROW) { - throw new TableDataException("Encountered zero delta time during EMA processing"); - } - doReset = control.onZeroDeltaTimeOrDefault() == BadDataBehavior.RESET; - } else if (dt < 0) { - if (control.onNegativeDeltaTimeOrDefault() == BadDataBehavior.THROW) { - throw new TableDataException("Encountered negative delta time during EMA processing"); - } - doReset = control.onNegativeDeltaTimeOrDefault() == BadDataBehavior.RESET; - } - if (doReset) { ctx.reset(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java index fb46e41ed15..b4d96ade3e6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java @@ -36,7 +36,7 @@ public void accumulate(RowSequence inputKeys, // read the value from the values chunk final BigDecimal input = objectValueChunk.get(ii); if (input == null) { - handleBadData(this, true, false); + handleBadData(this, true); } else { if (curVal == null) { curVal = input; @@ -56,27 +56,26 @@ public void accumulate(RowSequence inputKeys, final long timestamp = tsChunk.get(ii); final boolean isNull = input == null; final boolean isNullTime = timestamp == NULL_LONG; - if (isNull || isNullTime) { - handleBadData(this, isNull, isNullTime); + if (isNull) { + handleBadData(this, isNull); + } else if (isNullTime) { + // no change to curVal and lastStamp } else { if (curVal == null) { curVal = input; lastStamp = timestamp; + } else { final long dt = timestamp - lastStamp; - if (dt <= 0) { - handleBadTime(this, dt); - } else { - // alpha is dynamic, based on time - BigDecimal alpha = BigDecimal.valueOf(Math.exp(-dt / (double) reverseTimeScaleUnits)); - BigDecimal oneMinusAlpha = - BigDecimal.ONE.subtract(alpha, control.bigValueContextOrDefault()); + // alpha is dynamic, based on time + BigDecimal alpha = BigDecimal.valueOf(Math.exp(-dt / (double) reverseTimeScaleUnits)); + BigDecimal oneMinusAlpha = + BigDecimal.ONE.subtract(alpha, control.bigValueContextOrDefault()); - curVal = curVal.multiply(alpha, control.bigValueContextOrDefault()) - .add(input.multiply(oneMinusAlpha, control.bigValueContextOrDefault()), - control.bigValueContextOrDefault()); - lastStamp = timestamp; - } + curVal = curVal.multiply(alpha, control.bigValueContextOrDefault()) + .add(input.multiply(oneMinusAlpha, control.bigValueContextOrDefault()), + control.bigValueContextOrDefault()); + lastStamp = timestamp; } } outputValues.set(ii, curVal); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java index dd9e62edcef..b50a84a9f8d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java @@ -37,7 +37,7 @@ public void accumulate(RowSequence inputKeys, // read the value from the values chunk final BigInteger input = objectValueChunk.get(ii); if (input == null) { - handleBadData(this, true, false); + handleBadData(this, true); } else { final BigDecimal decimalInput = new BigDecimal(input, control.bigValueContextOrDefault()); if (curVal == null) { @@ -58,8 +58,10 @@ public void accumulate(RowSequence inputKeys, final long timestamp = tsChunk.get(ii); final boolean isNull = input == null; final boolean isNullTime = timestamp == NULL_LONG; - if (isNull || isNullTime) { - handleBadData(this, isNull, isNullTime); + if (isNull) { + handleBadData(this, true); + } else if (isNullTime) { + // no change to curVal and lastStamp } else { final BigDecimal decimalInput = new BigDecimal(input, control.bigValueContextOrDefault()); if (curVal == null) { @@ -67,8 +69,8 @@ public void accumulate(RowSequence inputKeys, lastStamp = timestamp; } else { final long dt = timestamp - lastStamp; - if (dt <= 0) { - handleBadTime(this, dt); + if (dt == 0) { + // preserve curVal and timestamp } else { // alpha is dynamic, based on time BigDecimal alpha = BigDecimal.valueOf(Math.exp(-dt / (double)reverseTimeScaleUnits)); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java index 78df1531506..cbc492ceede 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java @@ -101,9 +101,7 @@ public void initializeUpdate(@NotNull final UpdateContext updateContext, } } - void handleBadData(@NotNull final Context ctx, - final boolean isNull, - final boolean isNullTime) { + void handleBadData(@NotNull final Context ctx, final boolean isNull) { boolean doReset = false; if (isNull) { if (control.onNullValueOrDefault() == BadDataBehavior.THROW) { @@ -112,32 +110,6 @@ void handleBadData(@NotNull final Context ctx, doReset = control.onNullValueOrDefault() == BadDataBehavior.RESET; } - if (isNullTime) { - if (control.onNullTimeOrDefault() == BadDataBehavior.THROW) { - throw new TableDataException("Encountered null timestamp during EMA processing"); - } - doReset = control.onNullTimeOrDefault() == BadDataBehavior.RESET; - } - - if (doReset) { - ctx.reset(); - } - } - - void handleBadTime(@NotNull final Context ctx, final long dt) { - boolean doReset = false; - if (dt == 0) { - if (control.onZeroDeltaTimeOrDefault() == BadDataBehavior.THROW) { - throw new TableDataException("Encountered zero delta time during EMA processing"); - } - doReset = control.onZeroDeltaTimeOrDefault() == BadDataBehavior.RESET; - } else if (dt < 0) { - if (control.onNegativeDeltaTimeOrDefault() == BadDataBehavior.THROW) { - throw new TableDataException("Encountered negative delta time during EMA processing"); - } - doReset = control.onNegativeDeltaTimeOrDefault() == BadDataBehavior.RESET; - } - if (doReset) { ctx.reset(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java index ca7ba210750..b8d28ccd573 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java @@ -46,7 +46,7 @@ public void accumulate(RowSequence inputKeys, final byte input = byteValueChunk.get(ii); if(input == NULL_BYTE) { - handleBadData(this, true, false, false); + handleBadData(this, true, false); } else { if(curVal == NULL_DOUBLE) { curVal = input; @@ -65,22 +65,20 @@ public void accumulate(RowSequence inputKeys, //noinspection ConstantConditions final boolean isNull = input == NULL_BYTE; final boolean isNullTime = timestamp == NULL_LONG; - if(isNull || isNullTime) { - handleBadData(this, isNull, false, isNullTime); + if(isNull) { + handleBadData(this, true, false); + } else if (isNullTime) { + // no change to curVal and lastStamp } else { if(curVal == NULL_DOUBLE) { curVal = input; lastStamp = timestamp; } else { final long dt = timestamp - lastStamp; - if(dt <= 0) { - handleBadTime(this, dt); - } else { - // alpha is dynamic, based on time - final double alpha = Math.exp(-dt / (double)reverseTimeScaleUnits); - curVal = alpha * curVal + ((1 - alpha) * input); - lastStamp = timestamp; - } + // alpha is dynamic, based on time + final double alpha = Math.exp(-dt / (double)reverseTimeScaleUnits); + curVal = alpha * curVal + ((1 - alpha) * input); + lastStamp = timestamp; } } outputValues.set(ii, curVal); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java index 8ce6abfb274..547b3ceb4d6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java @@ -48,7 +48,7 @@ public void accumulate(RowSequence inputKeys, final boolean isNan = Double.isNaN(input); if (isNull || isNan) { - handleBadData(this, isNull, isNan, false); + handleBadData(this, isNull, isNan); } else { if (curVal == NULL_DOUBLE) { curVal = input; @@ -68,8 +68,10 @@ public void accumulate(RowSequence inputKeys, final boolean isNan = Double.isNaN(input); final boolean isNullTime = timestamp == NULL_LONG; // Handle bad data first - if (isNull || isNan || isNullTime) { - handleBadData(this, isNull, isNan, isNullTime); + if (isNull || isNan) { + handleBadData(this, isNull, isNan); + } else if (isNullTime) { + // no change to curVal and lastStamp } else if (curVal == NULL_DOUBLE) { // If the data looks good, and we have a null ema, just accept the current value curVal = input; @@ -82,13 +84,9 @@ public void accumulate(RowSequence inputKeys, lastStamp = timestamp; } else { final long dt = timestamp - lastStamp; - if (dt <= 0) { - handleBadTime(this, dt); - } else if (!currentPoisoned) { - final double alpha = Math.exp(-dt / (double)reverseTimeScaleUnits); - curVal = alpha * curVal + ((1 - alpha) * input); - lastStamp = timestamp; - } + final double alpha = Math.exp(-dt / (double)reverseTimeScaleUnits); + curVal = alpha * curVal + ((1 - alpha) * input); + lastStamp = timestamp; } } outputValues.set(ii, curVal); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java index d92b522a26d..143ad967fbd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java @@ -43,7 +43,7 @@ public void accumulate(RowSequence inputKeys, final boolean isNan = Float.isNaN(input); if (isNull || isNan) { - handleBadData(this, isNull, isNan, false); + handleBadData(this, isNull, isNan); } else { if (curVal == NULL_DOUBLE) { curVal = input; @@ -63,8 +63,10 @@ public void accumulate(RowSequence inputKeys, final boolean isNan = Float.isNaN(input); final boolean isNullTime = timestamp == NULL_LONG; // Handle bad data first - if (isNull || isNan || isNullTime) { - handleBadData(this, isNull, isNan, isNullTime); + if (isNull || isNan) { + handleBadData(this, isNull, isNan); + } else if (isNullTime) { + // no change to curVal and lastStamp } else if (curVal == NULL_DOUBLE) { // If the data looks good, and we have a null ema, just accept the current value curVal = input; @@ -77,13 +79,9 @@ public void accumulate(RowSequence inputKeys, lastStamp = timestamp; } else { final long dt = timestamp - lastStamp; - if (dt <= 0) { - handleBadTime(this, dt); - } else if (!currentPoisoned) { - final double alpha = Math.exp(-dt / (double)reverseTimeScaleUnits); - curVal = alpha * curVal + ((1 - alpha) * input); - lastStamp = timestamp; - } + final double alpha = Math.exp(-dt / (double)reverseTimeScaleUnits); + curVal = alpha * curVal + ((1 - alpha) * input); + lastStamp = timestamp; } } outputValues.set(ii, curVal); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java index 3c374e6e9f7..0c934a49699 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java @@ -46,7 +46,7 @@ public void accumulate(RowSequence inputKeys, final int input = intValueChunk.get(ii); if(input == NULL_INT) { - handleBadData(this, true, false, false); + handleBadData(this, true, false); } else { if(curVal == NULL_DOUBLE) { curVal = input; @@ -65,22 +65,20 @@ public void accumulate(RowSequence inputKeys, //noinspection ConstantConditions final boolean isNull = input == NULL_INT; final boolean isNullTime = timestamp == NULL_LONG; - if(isNull || isNullTime) { - handleBadData(this, isNull, false, isNullTime); + if(isNull) { + handleBadData(this, true, false); + } else if (isNullTime) { + // no change to curVal and lastStamp } else { if(curVal == NULL_DOUBLE) { curVal = input; lastStamp = timestamp; } else { final long dt = timestamp - lastStamp; - if(dt <= 0) { - handleBadTime(this, dt); - } else { - // alpha is dynamic, based on time - final double alpha = Math.exp(-dt / (double)reverseTimeScaleUnits); - curVal = alpha * curVal + ((1 - alpha) * input); - lastStamp = timestamp; - } + // alpha is dynamic, based on time + final double alpha = Math.exp(-dt / (double)reverseTimeScaleUnits); + curVal = alpha * curVal + ((1 - alpha) * input); + lastStamp = timestamp; } } outputValues.set(ii, curVal); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java index a188b4dcfdf..a80cd7be457 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java @@ -46,7 +46,7 @@ public void accumulate(RowSequence inputKeys, final long input = longValueChunk.get(ii); if(input == NULL_LONG) { - handleBadData(this, true, false, false); + handleBadData(this, true, false); } else { if(curVal == NULL_DOUBLE) { curVal = input; @@ -65,22 +65,20 @@ public void accumulate(RowSequence inputKeys, //noinspection ConstantConditions final boolean isNull = input == NULL_LONG; final boolean isNullTime = timestamp == NULL_LONG; - if(isNull || isNullTime) { - handleBadData(this, isNull, false, isNullTime); + if(isNull) { + handleBadData(this, true, false); + } else if (isNullTime) { + // no change to curVal and lastStamp } else { if(curVal == NULL_DOUBLE) { curVal = input; lastStamp = timestamp; } else { final long dt = timestamp - lastStamp; - if(dt <= 0) { - handleBadTime(this, dt); - } else { - // alpha is dynamic, based on time - final double alpha = Math.exp(-dt / (double)reverseTimeScaleUnits); - curVal = alpha * curVal + ((1 - alpha) * input); - lastStamp = timestamp; - } + // alpha is dynamic, based on time + final double alpha = Math.exp(-dt / (double)reverseTimeScaleUnits); + curVal = alpha * curVal + ((1 - alpha) * input); + lastStamp = timestamp; } } outputValues.set(ii, curVal); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java index 781c04d1222..7887780f419 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java @@ -41,7 +41,7 @@ public void accumulate(RowSequence inputKeys, final short input = shortValueChunk.get(ii); if(input == NULL_SHORT) { - handleBadData(this, true, false, false); + handleBadData(this, true, false); } else { if(curVal == NULL_DOUBLE) { curVal = input; @@ -60,22 +60,20 @@ public void accumulate(RowSequence inputKeys, //noinspection ConstantConditions final boolean isNull = input == NULL_SHORT; final boolean isNullTime = timestamp == NULL_LONG; - if(isNull || isNullTime) { - handleBadData(this, isNull, false, isNullTime); + if(isNull) { + handleBadData(this, true, false); + } else if (isNullTime) { + // no change to curVal and lastStamp } else { if(curVal == NULL_DOUBLE) { curVal = input; lastStamp = timestamp; } else { final long dt = timestamp - lastStamp; - if(dt <= 0) { - handleBadTime(this, dt); - } else { - // alpha is dynamic, based on time - final double alpha = Math.exp(-dt / (double)reverseTimeScaleUnits); - curVal = alpha * curVal + ((1 - alpha) * input); - lastStamp = timestamp; - } + // alpha is dynamic, based on time + final double alpha = Math.exp(-dt / (double)reverseTimeScaleUnits); + curVal = alpha * curVal + ((1 - alpha) * input); + lastStamp = timestamp; } } outputValues.set(ii, curVal); diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestEma.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestEma.java index 9ab54d96ca5..e5c34c31739 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestEma.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestEma.java @@ -228,71 +228,88 @@ private void testThrowsInternal(TableDefaults table) { () -> table.updateBy(UpdateByOperation.Ema( OperationControl.builder().build(), "ts", 100)), "Encountered negative delta time during EMA processing"); - - assertThrows(TableDataException.class, - () -> table.updateBy(UpdateByOperation.Ema( - OperationControl.builder() - .onNegativeDeltaTime(BadDataBehavior.SKIP) - .onZeroDeltaTime(BadDataBehavior.THROW).build(), - "ts", 100)), - "Encountered zero delta time during EMA processing"); - - assertThrows(TableDataException.class, - () -> table.updateBy(UpdateByOperation.Ema( - OperationControl.builder() - .onNegativeDeltaTime(BadDataBehavior.SKIP) - .onNullTime(BadDataBehavior.THROW).build(), - "ts", 100)), - "Encountered null timestamp during EMA processing"); } @Test public void testResetBehavior() { + // Value reset + final OperationControl dataResetControl = OperationControl.builder() + .onNullValue(BadDataBehavior.RESET) + .build(); + final ColumnHolder ts = col("ts", convertDateTime("2022-03-11T09:30:00.000 NY"), - convertDateTime("2022-03-11T09:29:00.000 NY"), - convertDateTime("2022-03-11T09:31:00.000 NY"), convertDateTime("2022-03-11T09:31:00.000 NY"), convertDateTime("2022-03-11T09:32:00.000 NY"), - null); + convertDateTime("2022-03-11T09:33:00.000 NY"), + convertDateTime("2022-03-11T09:34:00.000 NY"), + convertDateTime("2022-03-11T09:35:00.000 NY")); Table expected = testTable(RowSetFactory.flat(6).toTracking(), ts, doubleCol("col", 0, NULL_DOUBLE, 2, NULL_DOUBLE, 4, NULL_DOUBLE)); - testResetBehaviorInternal(expected, ts, - byteCol("col", (byte) 0, (byte) 1, (byte) 2, (byte) 3, (byte) 4, (byte) 5)); - testResetBehaviorInternal(expected, ts, - shortCol("col", (short) 0, (short) 1, (short) 2, (short) 3, (short) 4, (short) 5)); - testResetBehaviorInternal(expected, ts, intCol("col", 0, 1, 2, 3, 4, 5)); - testResetBehaviorInternal(expected, ts, longCol("col", 0, 1, 2, 3, 4, 5)); - testResetBehaviorInternal(expected, ts, floatCol("col", 0, 1, 2, 3, 4, 5)); - testResetBehaviorInternal(expected, ts, doubleCol("col", 0, 1, 2, 3, 4, 5)); + TableDefaults input = testTable(RowSetFactory.flat(6).toTracking(), ts, + byteCol("col", (byte) 0, NULL_BYTE, (byte) 2, NULL_BYTE, (byte) 4, NULL_BYTE)); + Table result = input.updateBy(UpdateByOperation.Ema(dataResetControl, "ts", 1_000_000_000)); + assertTableEquals(expected, result); + + input = testTable(RowSetFactory.flat(6).toTracking(), ts, + shortCol("col", (short) 0, NULL_SHORT, (short) 2, NULL_SHORT, (short) 4, NULL_SHORT)); + result = input.updateBy(UpdateByOperation.Ema(dataResetControl, "ts", 1_000_000_000)); + assertTableEquals(expected, result); + + input = testTable(RowSetFactory.flat(6).toTracking(), ts, + intCol("col", 0, NULL_INT, 2, NULL_INT, 4, NULL_INT)); + result = input.updateBy(UpdateByOperation.Ema(dataResetControl, "ts", 1_000_000_000)); + assertTableEquals(expected, result); + + input = testTable(RowSetFactory.flat(6).toTracking(), ts, + longCol("col", 0, NULL_LONG, 2, NULL_LONG, 4, NULL_LONG)); + result = input.updateBy(UpdateByOperation.Ema(dataResetControl, "ts", 1_000_000_000)); + assertTableEquals(expected, result); + + input = testTable(RowSetFactory.flat(6).toTracking(), ts, + floatCol("col", 0, NULL_FLOAT, 2, NULL_FLOAT, 4, NULL_FLOAT)); + result = input.updateBy(UpdateByOperation.Ema(dataResetControl, "ts", 1_000_000_000)); + assertTableEquals(expected, result); + + input = testTable(RowSetFactory.flat(6).toTracking(), ts, + doubleCol("col", 0, NULL_DOUBLE, 2, NULL_DOUBLE, 4, NULL_DOUBLE)); + result = input.updateBy(UpdateByOperation.Ema(dataResetControl, "ts", 1_000_000_000)); + assertTableEquals(expected, result); + + // BigInteger/BigDecimal expected = testTable(RowSetFactory.flat(6).toTracking(), ts, col("col", BigDecimal.valueOf(0), null, BigDecimal.valueOf(2), null, BigDecimal.valueOf(4), null)); - testResetBehaviorInternal(expected, ts, col("col", BigInteger.valueOf(0), - BigInteger.valueOf(1), - BigInteger.valueOf(2), - BigInteger.valueOf(3), - BigInteger.valueOf(4), - BigInteger.valueOf(5))); + input = testTable(RowSetFactory.flat(6).toTracking(), ts, + col("col", BigInteger.valueOf(0), + null, + BigInteger.valueOf(2), + null, + BigInteger.valueOf(4), + null)); + result = input.updateBy(UpdateByOperation.Ema(dataResetControl, "ts", 1_000_000_000)); + assertTableEquals(expected, result); - testResetBehaviorInternal(expected, ts, + input = testTable(RowSetFactory.flat(6).toTracking(), ts, col("col", BigDecimal.valueOf(0), - BigDecimal.valueOf(1), + null, BigDecimal.valueOf(2), - BigDecimal.valueOf(3), + null, BigDecimal.valueOf(4), - BigDecimal.valueOf(5))); + null)); + result = input.updateBy(UpdateByOperation.Ema(dataResetControl, "ts", 1_000_000_000)); + assertTableEquals(expected, result); // Test reset for NaN values final OperationControl resetControl = OperationControl.builder() .onNanValue(BadDataBehavior.RESET) .build(); - TableDefaults input = testTable(RowSetFactory.flat(3).toTracking(), doubleCol("col", 0, Double.NaN, 1)); - Table result = input.updateBy(UpdateByOperation.Ema(resetControl, 100)); + input = testTable(RowSetFactory.flat(3).toTracking(), doubleCol("col", 0, Double.NaN, 1)); + result = input.updateBy(UpdateByOperation.Ema(resetControl, 100)); expected = testTable(RowSetFactory.flat(3).toTracking(), doubleCol("col", 0, NULL_DOUBLE, 1)); assertTableEquals(expected, result); @@ -302,23 +319,10 @@ public void testResetBehavior() { assertTableEquals(expected, result); } - private void testResetBehaviorInternal(Table expected, final ColumnHolder ts, final ColumnHolder col) { - final OperationControl resetControl = OperationControl.builder().onNegativeDeltaTime(BadDataBehavior.RESET) - .onNullTime(BadDataBehavior.RESET) - .onZeroDeltaTime(BadDataBehavior.RESET) - .build(); - - TableDefaults input = testTable(RowSetFactory.flat(6).toTracking(), ts, col); - final Table result = input.updateBy(UpdateByOperation.Ema(resetControl, "ts", 1_000_000_000)); - assertTableEquals(expected, result); - } - @Test public void testPoison() { final OperationControl nanCtl = OperationControl.builder().onNanValue(BadDataBehavior.POISON) .onNullValue(BadDataBehavior.RESET) - .onNullTime(BadDataBehavior.RESET) - .onNegativeDeltaTime(BadDataBehavior.RESET) .build(); Table expected = testTable(RowSetFactory.flat(5).toTracking(), @@ -335,10 +339,10 @@ public void testPoison() { null, convertDateTime("2022-03-11T09:33:00.000 NY"), convertDateTime("2022-03-11T09:34:00.000 NY"), - convertDateTime("2022-03-11T09:33:00.000 NY")); + null); expected = testTable(RowSetFactory.flat(6).toTracking(), ts, - doubleCol("col", 0, Double.NaN, NULL_DOUBLE, Double.NaN, Double.NaN, NULL_DOUBLE)); + doubleCol("col", 0, Double.NaN, Double.NaN, Double.NaN, Double.NaN, Double.NaN)); input = testTable(RowSetFactory.flat(6).toTracking(), ts, doubleCol("col", 0, Double.NaN, 2, Double.NaN, 4, 5)); Table result = input.updateBy(UpdateByOperation.Ema(nanCtl, "ts", 10)); diff --git a/table-api/src/main/java/io/deephaven/api/updateby/OperationControl.java b/table-api/src/main/java/io/deephaven/api/updateby/OperationControl.java index 41ca5fcbd27..b24d3f1056e 100644 --- a/table-api/src/main/java/io/deephaven/api/updateby/OperationControl.java +++ b/table-api/src/main/java/io/deephaven/api/updateby/OperationControl.java @@ -67,38 +67,6 @@ public BadDataBehavior onNanValueOrDefault() { return onNanValue().orElse(BadDataBehavior.SKIP); } - /** - * Get the behavior for when {@code null} timestamps are encountered. Defaults to {@link BadDataBehavior#SKIP SKIP}. - * - * @return the behavior for {@code null} timestamps. - */ - @Value.Derived - public BadDataBehavior onNullTimeOrDefault() { - return onNullTime().orElse(BadDataBehavior.SKIP); - } - - /** - * Get the behavior for when negative sample-to-sample time differences are encountered. Defaults to - * {@link BadDataBehavior#THROW THROW}. - * - * @return the behavior for when dt is negative - */ - @Value.Derived - public BadDataBehavior onNegativeDeltaTimeOrDefault() { - return onNegativeDeltaTime().orElse(BadDataBehavior.THROW); - } - - /** - * Get the behavior for when zero sample-to-sample-time differences are encountered. Defaults to - * {@link BadDataBehavior#SKIP SKIP}. - * - * @return the behavior for when dt is zero - */ - @Value.Derived - public BadDataBehavior onZeroDeltaTimeOrDefault() { - return onZeroDeltaTime().orElse(BadDataBehavior.SKIP); - } - /** * Get the {@link MathContext} to use when processing {@link java.math.BigInteger} and {@link java.math.BigDecimal} * values. Defaults to {@link MathContext#DECIMAL128}. @@ -121,9 +89,6 @@ public final OperationControl materialize() { return builder() .onNullValue(onNullValueOrDefault()) .onNanValue(onNanValueOrDefault()) - .onNullTime(onNullTimeOrDefault()) - .onNegativeDeltaTime(onNegativeDeltaTimeOrDefault()) - .onZeroDeltaTime(onZeroDeltaTimeOrDefault()) .bigValueContext(bigValueContextOrDefault()) .build(); } @@ -133,12 +98,6 @@ public interface Builder { Builder onNanValue(BadDataBehavior badDataBehavior); - Builder onNullTime(BadDataBehavior badDataBehavior); - - Builder onNegativeDeltaTime(BadDataBehavior badDataBehavior); - - Builder onZeroDeltaTime(BadDataBehavior badDataBehavior); - Builder bigValueContext(MathContext context); OperationControl build(); From 2c6b517578c0c39b7d334bc44373b8a835e3862d Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 10 Jan 2023 08:52:06 -0800 Subject: [PATCH 078/123] Spotless whitespace --- .../table/impl/updateby/ema/BasePrimitiveEMAOperator.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java index 0b7075a5d57..0a798bae4a7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java @@ -85,8 +85,8 @@ public void initializeUpdate(@NotNull final UpdateContext updateContext, } void handleBadData(@NotNull final Context ctx, - final boolean isNull, - final boolean isNan) { + final boolean isNull, + final boolean isNan) { boolean doReset = false; if (isNull) { if (control.onNullValueOrDefault() == BadDataBehavior.THROW) { From 0f466e75e9fefbc130aa621691b4af240504b159 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 10 Jan 2023 10:03:34 -0800 Subject: [PATCH 079/123] Compilation failure fixes --- .../deephaven/engine/table/impl/updateby/UpdateBy.java | 4 +--- .../io/deephaven/server/table/ops/UpdateByGrpcImpl.java | 9 --------- 2 files changed, 1 insertion(+), 12 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java index f72eb19da22..5c64e540700 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java @@ -152,9 +152,7 @@ private void processUpdateForRedirection(@NotNull final TableUpdate upstream, } private RowSet getInnerKeys(final RowSet outerKeys) { - if (rowRedirection == null) { - return null; - } + assert (rowRedirection != null); RowSetBuilderRandom builder = RowSetFactory.builderRandom(); final int chunkSize = Math.min(outerKeys.intSize(), REDIRECTION_CHUNK_SIZE); try (final RowSequence.Iterator it = outerKeys.getRowSequenceIterator(); diff --git a/server/src/main/java/io/deephaven/server/table/ops/UpdateByGrpcImpl.java b/server/src/main/java/io/deephaven/server/table/ops/UpdateByGrpcImpl.java index 60af7eab881..bc2a0fb450c 100644 --- a/server/src/main/java/io/deephaven/server/table/ops/UpdateByGrpcImpl.java +++ b/server/src/main/java/io/deephaven/server/table/ops/UpdateByGrpcImpl.java @@ -178,15 +178,6 @@ private static OperationControl adaptEmaOptions(UpdateByEmaOptions options) { if (options.hasOnNanValue()) { builder.onNanValue(adaptBadDataBehavior(options.getOnNanValue())); } - if (options.hasOnNullTime()) { - builder.onNullTime(adaptBadDataBehavior(options.getOnNullTime())); - } - if (options.hasOnNegativeDeltaTime()) { - builder.onNegativeDeltaTime(adaptBadDataBehavior(options.getOnNegativeDeltaTime())); - } - if (options.hasOnZeroDeltaTime()) { - builder.onZeroDeltaTime(adaptBadDataBehavior(options.getOnZeroDeltaTime())); - } if (options.hasBigValueContext()) { builder.bigValueContext(adaptMathContext(options.getBigValueContext())); } From 275450c307cb085dc3539414560ca6521b0cb908 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 10 Jan 2023 10:39:34 -0800 Subject: [PATCH 080/123] More compilation fixes. --- .../table/impl/updateby/TestRollingSum.java | 38 ++++--------------- 1 file changed, 7 insertions(+), 31 deletions(-) diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java index 4d383b87666..f17ca879ec3 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java @@ -11,8 +11,8 @@ import io.deephaven.engine.testutil.EvalNugget; import io.deephaven.engine.testutil.GenerateTableUpdates; import io.deephaven.engine.testutil.TstUtils; -import io.deephaven.engine.testutil.generator.Generator; import io.deephaven.engine.testutil.generator.SortedDateTimeGenerator; +import io.deephaven.engine.testutil.generator.TestDataGenerator; import io.deephaven.engine.updategraph.UpdateGraphProcessor; import io.deephaven.test.types.OutOfBandTest; import io.deephaven.time.DateTime; @@ -90,18 +90,10 @@ public void testStaticZeroKeyFwdRevWindow() { @Test public void testStaticZeroKeyTimed() { final QueryTable t = createTestTable(10000, false, false, false, 0xFFFABBBC, - new String[] {"ts"}, new Generator[] {new SortedDateTimeGenerator( + new String[] {"ts"}, new TestDataGenerator[] {new SortedDateTimeGenerator( convertDateTime("2022-03-09T09:00:00.000 NY"), convertDateTime("2022-03-09T16:30:00.000 NY"))}).t; - final OperationControl skipControl = OperationControl.builder() - .onNullValue(BadDataBehavior.SKIP) - .onNanValue(BadDataBehavior.SKIP).build(); - - final OperationControl resetControl = OperationControl.builder() - .onNullValue(BadDataBehavior.RESET) - .onNanValue(BadDataBehavior.RESET).build(); - Duration prevTime = Duration.ofMinutes(10); Duration postTime = Duration.ZERO; @@ -126,18 +118,10 @@ public void testStaticZeroKeyTimed() { @Test public void testStaticZeroKeyFwdWindowTimed() { final QueryTable t = createTestTable(10000, false, false, false, 0xFFFABBBC, - new String[] {"ts"}, new Generator[] {new SortedDateTimeGenerator( + new String[] {"ts"}, new TestDataGenerator[] {new SortedDateTimeGenerator( convertDateTime("2022-03-09T09:00:00.000 NY"), convertDateTime("2022-03-09T16:30:00.000 NY"))}).t; - final OperationControl skipControl = OperationControl.builder() - .onNullValue(BadDataBehavior.SKIP) - .onNanValue(BadDataBehavior.SKIP).build(); - - final OperationControl resetControl = OperationControl.builder() - .onNullValue(BadDataBehavior.RESET) - .onNanValue(BadDataBehavior.RESET).build(); - Duration prevTime = Duration.ZERO; Duration postTime = Duration.ofMinutes(10); @@ -162,18 +146,10 @@ public void testStaticZeroKeyFwdWindowTimed() { @Test public void testStaticZeroKeyFwdRevWindowTimed() { final QueryTable t = createTestTable(10000, false, false, false, 0xFFFABBBC, - new String[] {"ts"}, new Generator[] {new SortedDateTimeGenerator( + new String[] {"ts"}, new TestDataGenerator[] {new SortedDateTimeGenerator( convertDateTime("2022-03-09T09:00:00.000 NY"), convertDateTime("2022-03-09T16:30:00.000 NY"))}).t; - final OperationControl skipControl = OperationControl.builder() - .onNullValue(BadDataBehavior.SKIP) - .onNanValue(BadDataBehavior.SKIP).build(); - - final OperationControl resetControl = OperationControl.builder() - .onNullValue(BadDataBehavior.RESET) - .onNanValue(BadDataBehavior.RESET).build(); - Duration prevTime = Duration.ofMinutes(10); Duration postTime = Duration.ofMinutes(10); @@ -200,7 +176,7 @@ public void testStaticZeroKeyFwdRevWindowTimed() { // region Bucketed Tests @Test - public void testNullOnBucketChange() throws IOException { + public void testNullOnBucketChange() { final TableDefaults t = testTable(stringCol("Sym", "A", "A", "B", "B"), byteCol("ByteVal", (byte) 1, (byte) 2, NULL_BYTE, (byte) 3), shortCol("ShortVal", (short) 1, (short) 2, NULL_SHORT, (short) 3), @@ -275,7 +251,7 @@ public void testStaticBucketedFwdRevWindowTimed() { private void doTestStaticBucketedTimed(boolean grouped, Duration prevTime, Duration postTime) { final QueryTable t = createTestTable(10000, true, grouped, false, 0xFFFABBBC, - new String[] {"ts"}, new Generator[] {new SortedDateTimeGenerator( + new String[] {"ts"}, new TestDataGenerator[] {new SortedDateTimeGenerator( convertDateTime("2022-03-09T09:00:00.000 NY"), convertDateTime("2022-03-09T16:30:00.000 NY"))}).t; @@ -354,7 +330,7 @@ public void testBucketedAppendOnlyTimed() { private void doTestAppendOnlyTimed(boolean bucketed) { final CreateResult result = createTestTable(10000, bucketed, false, true, 0x31313131, - new String[] {"ts"}, new Generator[] {new SortedDateTimeGenerator( + new String[] {"ts"}, new TestDataGenerator[] {new SortedDateTimeGenerator( convertDateTime("2022-03-09T09:00:00.000 NY"), convertDateTime("2022-03-09T16:30:00.000 NY"))}); final QueryTable t = result.t; From 5af2472098ea9321b5ef2a3efd9fbb92f3060af3 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 10 Jan 2023 12:51:04 -0800 Subject: [PATCH 081/123] Sketchy python changes --- py/server/deephaven/updateby.py | 11 ----------- py/server/tests/test_updateby.py | 6 ------ 2 files changed, 17 deletions(-) diff --git a/py/server/deephaven/updateby.py b/py/server/deephaven/updateby.py index 66abe07cb28..c6d46c694df 100644 --- a/py/server/deephaven/updateby.py +++ b/py/server/deephaven/updateby.py @@ -60,20 +60,12 @@ def j_object(self) -> jpy.JType: def __init__(self, on_null: BadDataBehavior = BadDataBehavior.SKIP, on_nan: BadDataBehavior = BadDataBehavior.SKIP, - on_null_time: BadDataBehavior = BadDataBehavior.SKIP, - on_negative_deltatime: BadDataBehavior = BadDataBehavior.THROW, - on_zero_deltatime: BadDataBehavior = BadDataBehavior.SKIP, big_value_context: MathContext = MathContext.DECIMAL128): """Initializes an OperationControl for use with certain UpdateByOperation, such as EMAs. Args: on_null (BadDataBehavior): the behavior for when null values are encountered, default is SKIP on_nan (BadDataBehavior): the behavior for when NaN values are encountered, default is SKIP - on_null_time (BadDataBehavior): the behavior for when null timestamps are encountered, default is SKIP - on_negative_deltatime (BadDataBehavior): the behavior for when negative sample-to-sample time differences - are encountered, default is THROW - on_zero_deltatime (BadDataBehavior): the behavior for when zero sample-to-sample-time differences are - encountered, default is SKIP big_value_context (MathContext): the context to use when processing arbitrary precision numeric values (Java BigDecimal/BigInteger), default is DECIMAL128. @@ -84,9 +76,6 @@ def __init__(self, on_null: BadDataBehavior = BadDataBehavior.SKIP, j_builder = _JOperationControl.builder() self.j_op_control = (j_builder.onNullValue(on_null.value) .onNanValue(on_nan.value) - .onNullTime(on_null_time.value) - .onNegativeDeltaTime(on_negative_deltatime.value) - .onZeroDeltaTime(on_zero_deltatime.value) .bigValueContext(big_value_context.value).build()) except Exception as e: raise DHError(e, "failed to build an OperationControl object.") from e diff --git a/py/server/tests/test_updateby.py b/py/server/tests/test_updateby.py index 95c0f2670b6..58c1f15a1f3 100644 --- a/py/server/tests/test_updateby.py +++ b/py/server/tests/test_updateby.py @@ -25,10 +25,7 @@ def tearDown(self) -> None: def test_ema(self): op_ctrl = OperationControl(on_null=BadDataBehavior.THROW, - on_null_time=BadDataBehavior.POISON, on_nan=BadDataBehavior.RESET, - on_zero_deltatime=BadDataBehavior.SKIP, - on_negative_deltatime=BadDataBehavior.SKIP, big_value_context=MathContext.UNLIMITED) ema_ops = [ema_tick_decay(time_scale_ticks=100, cols="ema_a = a"), @@ -88,10 +85,7 @@ def test_simple_ops_proxy(self): def test_ema_proxy(self): op_ctrl = OperationControl(on_null=BadDataBehavior.THROW, - on_null_time=BadDataBehavior.POISON, on_nan=BadDataBehavior.RESET, - on_zero_deltatime=BadDataBehavior.SKIP, - on_negative_deltatime=BadDataBehavior.SKIP, big_value_context=MathContext.UNLIMITED) ema_ops = [ema_tick_decay(time_scale_ticks=100, cols="ema_a = a"), From 3733becd004b432bc1cde9d83285ae46385b9467 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 10 Jan 2023 15:00:58 -0800 Subject: [PATCH 082/123] Use BitSet instead of Trove sets --- .../updateby/UpdateByWindowCumulative.java | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java index b5b12e01c34..121f3a5609c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java @@ -15,6 +15,7 @@ import org.jetbrains.annotations.Nullable; import java.util.Arrays; +import java.util.BitSet; import java.util.stream.IntStream; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; @@ -79,19 +80,22 @@ public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, context.isDirty = true; } else { // determine which operators are affected by this update - TIntArrayList dirtyOperatorList = new TIntArrayList(operators.length); - TIntHashSet inputSourcesSet = new TIntHashSet(getUniqueSourceIndices().length); + BitSet dirtyOperators = new BitSet(); + BitSet dirtySourceIndices = new BitSet(); + for (int opIdx = 0; opIdx < operators.length; opIdx++) { UpdateByOperator op = operators[opIdx]; if (upstream.modifiedColumnSet().nonempty() && (op.getInputModifiedColumnSet() == null || upstream.modifiedColumnSet().containsAny(op.getInputModifiedColumnSet()))) { - dirtyOperatorList.add(opIdx); - inputSourcesSet.addAll(operatorInputSourceSlots[opIdx]); - context.isDirty = true; + dirtyOperators.set(opIdx); + Arrays.stream(operatorInputSourceSlots[opIdx]).forEach(srcIdx -> + dirtySourceIndices.set(srcIdx) + ); } } - context.dirtyOperatorIndices = dirtyOperatorList.toArray(); - context.dirtySourceIndices = inputSourcesSet.toArray(); + context.isDirty = !dirtyOperators.isEmpty(); + context.dirtyOperatorIndices = dirtyOperators.stream().toArray(); + context.dirtySourceIndices = dirtySourceIndices.stream().toArray(); } if (!context.isDirty) { From 75e2b142e168b7bd529cbd6ae1d1a33139b8a40b Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 11 Jan 2023 10:14:01 -0800 Subject: [PATCH 083/123] JobScheduler WIP (incomplete) --- .../BucketedPartitionedUpdateByManager.java | 6 +- .../engine/table/impl/updateby/UpdateBy.java | 136 +++++++++--------- .../impl/updateby/UpdateByBucketHelper.java | 4 - .../updateby/UpdateByWindowCumulative.java | 4 +- .../impl/updateby/ZeroKeyUpdateByManager.java | 9 +- .../impl/util/ImmediateJobScheduler.java | 4 +- .../engine/table/impl/util/JobScheduler.java | 135 ++++++++++++----- ...erationInitializationPoolJobScheduler.java | 4 +- .../UpdateGraphProcessorJobScheduler.java | 4 +- 9 files changed, 183 insertions(+), 123 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java index bc85b22e862..0b36163ba7e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java @@ -35,7 +35,6 @@ class BucketedPartitionedUpdateByManager extends UpdateBy { * @param operators the operations to perform * @param windows the unique windows for this UpdateBy * @param inputSources the primitive input sources - * @param operatorInputSourceSlots maps the operators to source indices * @param source the source table * @param resultSources the result sources * @param byColumns the columns to use for the bucket keys @@ -48,15 +47,13 @@ protected BucketedPartitionedUpdateByManager( @NotNull final UpdateByOperator[] operators, @NotNull final UpdateByWindow[] windows, @NotNull final ColumnSource[] inputSources, - @NotNull final int[][] operatorInputSourceSlots, @NotNull final QueryTable source, @NotNull final Map> resultSources, @NotNull final Collection byColumns, @Nullable final String timestampColumnName, @Nullable final WritableRowRedirection rowRedirection, @NotNull final UpdateByControl control) { - super(source, operators, windows, inputSources, operatorInputSourceSlots, timestampColumnName, - rowRedirection, control); + super(source, operators, windows, inputSources, timestampColumnName, rowRedirection, control); // this table will always have the rowset of the source result = new QueryTable(source.getRowSet(), resultSources); @@ -91,7 +88,6 @@ protected BucketedPartitionedUpdateByManager( operators, windows, inputSources, - operatorInputSourceSlots, resultSources, timestampColumnName, rowRedirection, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java index 5c64e540700..3bbf24a3e3c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java @@ -1,7 +1,5 @@ package io.deephaven.engine.table.impl.updateby; -import gnu.trove.list.array.TIntArrayList; -import gnu.trove.map.hash.TIntObjectHashMap; import gnu.trove.map.hash.TObjectIntHashMap; import io.deephaven.UncheckedDeephavenException; import io.deephaven.api.ColumnName; @@ -24,8 +22,11 @@ import io.deephaven.engine.table.impl.sources.sparse.SparseConstants; import io.deephaven.engine.table.impl.util.*; import io.deephaven.engine.updategraph.UpdateGraphProcessor; +import io.deephaven.hash.KeyedObjectHashMap; +import io.deephaven.hash.KeyedObjectKey; import io.deephaven.util.datastructures.linked.IntrusiveDoublyLinkedNode; import io.deephaven.util.datastructures.linked.IntrusiveDoublyLinkedQueue; +import org.apache.commons.lang3.mutable.MutableBoolean; import org.apache.commons.lang3.mutable.MutableInt; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -52,8 +53,6 @@ public abstract class UpdateBy { /** Input sources may be reused by multiple operators, only store and cache unique ones (post-reinterpret) */ protected final ColumnSource[] inputSources; - /** Map operators to input sources, note some operators need more than one input, WAvg e.g. */ - protected final int[][] operatorInputSourceSlots; /** All the operators for this UpdateBy manager */ protected final UpdateByOperator[] operators; /** All the windows for this UpdateBy manager */ @@ -184,7 +183,6 @@ protected UpdateBy( @NotNull final UpdateByOperator[] operators, @NotNull final UpdateByWindow[] windows, @NotNull final ColumnSource[] inputSources, - @NotNull final int[][] operatorInputSourceSlots, @Nullable String timestampColumnName, @Nullable final WritableRowRedirection rowRedirection, @NotNull final UpdateByControl control) { @@ -197,7 +195,6 @@ protected UpdateBy( this.operators = operators; this.windows = windows; this.inputSources = inputSources; - this.operatorInputSourceSlots = operatorInputSourceSlots; this.timestampColumnName = timestampColumnName; this.redirHelper = new UpdateByRedirectionHelper(rowRedirection); this.control = control; @@ -890,80 +887,89 @@ public static Table updateBy(@NotNull final QueryTable source, } } } - } - // the next bit is complicated but the goal is simple. We don't want to have duplicate input column sources, so - // we will store each one only once in inputSources and setup some mapping from the opIdx to the input column. - final ArrayList> inputSourceList = new ArrayList<>(); - final int[][] operatorInputSourceSlotArr = new int[opArr.length][]; - final TObjectIntHashMap> sourceToSlotMap = new TObjectIntHashMap<>(); + // We will divide the operators into similar windows for efficient processing. It will be more useful to store + // indices into the operator in oppArr rather than operator objects. + final KeyedObjectHashMap> windowMap = + new KeyedObjectHashMap<>(new KeyedObjectKey<>() { + @Override + public UpdateByOperator getKey(List updateByOperators) { + return opArr[updateByOperators.get(0)]; + } + + @Override + public int hashKey(UpdateByOperator updateByOperator) { + return UpdateByWindow.hashCodeFromOperator(updateByOperator); + } + @Override + public boolean equalKey(UpdateByOperator updateByOperator, + List updateByOperators) { + return UpdateByWindow.isEquivalentWindow(updateByOperator, opArr[updateByOperators.get(0)]); + } + }); for (int opIdx = 0; opIdx < opArr.length; opIdx++) { - final String[] inputColumnNames = opArr[opIdx].getInputColumnNames(); - // add a new entry for this operator - operatorInputSourceSlotArr[opIdx] = new int[inputColumnNames.length]; - for (int colIdx = 0; colIdx < inputColumnNames.length; colIdx++) { - final ColumnSource input = source.getColumnSource(inputColumnNames[colIdx]); - final int maybeExistingSlot = sourceToSlotMap.get(input); - if (maybeExistingSlot == sourceToSlotMap.getNoEntryValue()) { - int srcIdx = inputSourceList.size(); - // create a new input source and map the operator to it - inputSourceList.add(ReinterpretUtils.maybeConvertToPrimitive(input)); - sourceToSlotMap.put(input, srcIdx); - operatorInputSourceSlotArr[opIdx][colIdx] = srcIdx; - } else { - operatorInputSourceSlotArr[opIdx][colIdx] = maybeExistingSlot; - } + final MutableBoolean created = new MutableBoolean(false); + int finalOpIdx = opIdx; + final List opList = windowMap.putIfAbsent(opArr[opIdx], + (newOpListOp) -> { + final List newOpList = new ArrayList<>(); + newOpList.add(finalOpIdx); + created.setTrue(); + return newOpList; + }); + if (!created.booleanValue()) { + opList.add(finalOpIdx); } } - final ColumnSource[] inputSourceArr = inputSourceList.toArray(ColumnSource.ZERO_LENGTH_COLUMN_SOURCE_ARRAY); - // now we want to divide the operators into similar windows for efficient processing - TIntObjectHashMap windowHashToOperatorIndicesMap = new TIntObjectHashMap<>(); + // make the windows and create unique input sources for all the window operators + final UpdateByWindow[] windowArr = new UpdateByWindow[windowMap.size()]; + final MutableInt winIdx = new MutableInt(0); - for (int opIdx = 0; opIdx < opArr.length; opIdx++) { - int hash = UpdateByWindow.hashCodeFromOperator(opArr[opIdx]); - boolean added = false; - - // rudimentary collision detection and handling - while (!added) { - if (!windowHashToOperatorIndicesMap.containsKey(hash)) { - // does not exist, can add immediately - windowHashToOperatorIndicesMap.put(hash, new TIntArrayList()); - windowHashToOperatorIndicesMap.get(hash).add(opIdx); - added = true; - } else { - final int existingOpIdx = windowHashToOperatorIndicesMap.get(hash).get(0); - if (UpdateByWindow.isEquivalentWindow(opArr[existingOpIdx], opArr[opIdx])) { - // no collision, can add immediately - windowHashToOperatorIndicesMap.get(hash).add(opIdx); - added = true; + final ArrayList> inputSourceList = new ArrayList<>(); + final TObjectIntHashMap> sourceToSlotMap = new TObjectIntHashMap<>(); + + windowMap.forEach((ignored, opList) -> { + // build an array from the operator indices + UpdateByOperator[] windowOps = new UpdateByOperator[opList.size()]; + // local map of operators indices to input source indices + final int[][] windowOpSourceSlots = new int[opList.size()][]; + + // do the mapping from operator input sources to unique input sources + for (int idx = 0; idx < opList.size(); idx++) { + final int opIdx = opList.get(idx); + final UpdateByOperator localOp = opArr[opIdx]; + // store this operator into an array for window creation + windowOps[idx] = localOp; + // iterate over each input column and map this operator to unique source + final String[] inputColumnNames = localOp.getInputColumnNames(); + windowOpSourceSlots[idx] = new int[inputColumnNames.length]; + for (int colIdx = 0; colIdx < inputColumnNames.length; colIdx++) { + final ColumnSource input = source.getColumnSource(inputColumnNames[colIdx]); + final int maybeExistingSlot = sourceToSlotMap.get(input); + if (maybeExistingSlot == sourceToSlotMap.getNoEntryValue()) { + // create a new input source + int srcIdx = inputSourceList.size(); + inputSourceList.add(ReinterpretUtils.maybeConvertToPrimitive(input)); + sourceToSlotMap.put(input, srcIdx); + // map the window operator indices to this new source + windowOpSourceSlots[idx][colIdx] = srcIdx; } else { - // there is a collision, increment hash and try again - hash++; + // map the window indices to this existing source + windowOpSourceSlots[idx][colIdx] = maybeExistingSlot; } } } - } - // store the operators into the windows - final UpdateByWindow[] windowArr = new UpdateByWindow[windowHashToOperatorIndicesMap.size()]; - final MutableInt winIdx = new MutableInt(0); - - windowHashToOperatorIndicesMap.forEachEntry((final int hash, final TIntArrayList opIndices) -> { - final UpdateByOperator[] windowOperators = new UpdateByOperator[opIndices.size()]; - final int[][] windowOperatorSourceSlots = new int[opIndices.size()][]; - for (int ii = 0; ii < opIndices.size(); ii++) { - final int opIdx = opIndices.get(ii); - windowOperators[ii] = opArr[opIdx]; - windowOperatorSourceSlots[ii] = operatorInputSourceSlotArr[opIdx]; - } + // create and store the window for these operators windowArr[winIdx.getAndIncrement()] = - UpdateByWindow.createFromOperatorArray(windowOperators, windowOperatorSourceSlots); - return true; + UpdateByWindow.createFromOperatorArray(windowOps, windowOpSourceSlots); }); + final ColumnSource[] inputSourceArr = inputSourceList.toArray(ColumnSource.ZERO_LENGTH_COLUMN_SOURCE_ARRAY); + final Map> resultSources = new LinkedHashMap<>(source.getColumnSourceMap()); resultSources.putAll(opResultSources); @@ -974,7 +980,6 @@ public static Table updateBy(@NotNull final QueryTable source, opArr, windowArr, inputSourceArr, - operatorInputSourceSlotArr, source, resultSources, timestampColumnName, @@ -1009,7 +1014,6 @@ public static Table updateBy(@NotNull final QueryTable source, opArr, windowArr, inputSourceArr, - operatorInputSourceSlotArr, source, resultSources, byColumns, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java index 646e0c6bd3a..c4f2cd6269e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java @@ -35,7 +35,6 @@ class UpdateByBucketHelper extends IntrusiveDoublyLinkedNode.Impl[] inputSources; // some columns will have multiple inputs, such as time-based and Weighted computations - final int[][] operatorInputSourceSlots; final UpdateByOperator[] operators; final UpdateByWindow[] windows; final QueryTable source; @@ -62,7 +61,6 @@ class UpdateByBucketHelper extends IntrusiveDoublyLinkedNode.Impl[] inputSources, - @NotNull final int[][] operatorInputSourceSlots, @NotNull final Map> resultSources, @Nullable String timestampColumnName, @Nullable final RowRedirection rowRedirection, @@ -84,7 +81,6 @@ protected UpdateByBucketHelper(@NotNull final String description, this.operators = operators; this.windows = windows; this.inputSources = inputSources; - this.operatorInputSourceSlots = operatorInputSourceSlots; this.rowRedirection = rowRedirection; this.control = control; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java index 121f3a5609c..5e8031c2c7a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java @@ -88,9 +88,7 @@ public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, if (upstream.modifiedColumnSet().nonempty() && (op.getInputModifiedColumnSet() == null || upstream.modifiedColumnSet().containsAny(op.getInputModifiedColumnSet()))) { dirtyOperators.set(opIdx); - Arrays.stream(operatorInputSourceSlots[opIdx]).forEach(srcIdx -> - dirtySourceIndices.set(srcIdx) - ); + Arrays.stream(operatorInputSourceSlots[opIdx]).forEach(srcIdx -> dirtySourceIndices.set(srcIdx)); } } context.isDirty = !dirtyOperators.isEmpty(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java index bf56003a439..66384170cfa 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java @@ -27,7 +27,6 @@ public class ZeroKeyUpdateByManager extends UpdateBy { * @param operators the operations to perform * @param windows the unique windows for this UpdateBy * @param inputSources the primitive input sources - * @param operatorInputSourceSlots maps the operators to source indices * @param source the source table * @param resultSources the result sources * @param timestampColumnName the column to use for all time-aware operators @@ -39,14 +38,12 @@ protected ZeroKeyUpdateByManager( @NotNull UpdateByOperator[] operators, @NotNull UpdateByWindow[] windows, @NotNull ColumnSource[] inputSources, - @NotNull int[][] operatorInputSourceSlots, @NotNull QueryTable source, @NotNull final Map> resultSources, @Nullable String timestampColumnName, @Nullable WritableRowRedirection rowRedirection, @NotNull UpdateByControl control) { - super(source, operators, windows, inputSources, operatorInputSourceSlots, timestampColumnName, rowRedirection, - control); + super(source, operators, windows, inputSources, timestampColumnName, rowRedirection, control); if (source.isRefreshing()) { result = new QueryTable(source.getRowSet(), resultSources); @@ -65,7 +62,7 @@ protected ZeroKeyUpdateByManager( // create an updateby bucket instance directly from the source table zeroKeyUpdateBy = new UpdateByBucketHelper(description, source, operators, windows, inputSources, - operatorInputSourceSlots, resultSources, timestampColumnName, rowRedirection, control); + resultSources, timestampColumnName, rowRedirection, control); buckets.offer(zeroKeyUpdateBy); // make the source->result transformer @@ -75,7 +72,7 @@ protected ZeroKeyUpdateByManager( result.addParentReference(zeroKeyUpdateBy); } else { zeroKeyUpdateBy = new UpdateByBucketHelper(description, source, operators, windows, inputSources, - operatorInputSourceSlots, resultSources, timestampColumnName, rowRedirection, control); + resultSources, timestampColumnName, rowRedirection, control); result = zeroKeyUpdateBy.result; buckets.offer(zeroKeyUpdateBy); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/ImmediateJobScheduler.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/ImmediateJobScheduler.java index c0bb0a31c1d..4ae7b9a82ed 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/ImmediateJobScheduler.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/ImmediateJobScheduler.java @@ -3,6 +3,7 @@ import io.deephaven.base.log.LogOutputAppendable; import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.table.impl.perf.BasePerformanceEntry; +import io.deephaven.io.log.impl.LogOutputStringImpl; import io.deephaven.util.SafeCloseable; import io.deephaven.util.process.ProcessEnvironment; @@ -22,7 +23,8 @@ public void submit( } catch (Exception e) { onError.accept(e); } catch (Error e) { - ProcessEnvironment.getGlobalFatalErrorReporter().report("SelectAndView Error", e); + final String logMessage = new LogOutputStringImpl().append(description).append(" Error").toString(); + ProcessEnvironment.getGlobalFatalErrorReporter().report(logMessage, e); throw e; } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java index 5ebef11dd27..7bbe034f877 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java @@ -4,9 +4,10 @@ import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.table.impl.perf.BasePerformanceEntry; import io.deephaven.util.annotations.FinalDefault; +import io.deephaven.util.referencecounting.ReferenceCounted; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; /** @@ -52,7 +53,7 @@ interface IterateAction { } /** - * Helper interface for {@code iterateSerial()} and {@code iterateParallel()}. This provides a functional interface + * Helper interface for {@link #iterateSerial} and {@link #iterateParallel}. This provides a functional interface * with {@code index} indicating which iteration to perform and {@link Runnable resume} providing a mechanism to * inform the scheduler that the current task is complete. When {@code resume} is called, the scheduler will * automatically schedule the next iteration. @@ -65,6 +66,77 @@ interface IterateResumeAction { void run(int index, Runnable resume); } + class ErrorAccounter extends ReferenceCounted implements Consumer, Runnable { + private final AtomicReference exception = new AtomicReference<>(); + private final Consumer finalErrorConsumer; + private final IterateResumeAction action; + private final Runnable completeAction; + private final Runnable resumeAction; + private final AtomicInteger nextIndex; + private final AtomicInteger remaining; + private final int start; + private final int count; + + ErrorAccounter(final int start, final int count, final Consumer finalErrorConsumer, + final IterateResumeAction action, final Runnable completeAction) { + this.start = start; + this.count = count; + this.finalErrorConsumer = finalErrorConsumer; + this.action = action; + this.completeAction = completeAction; + + nextIndex = new AtomicInteger(start); + remaining = new AtomicInteger(count); + + resumeAction = () -> { + // check for completion + if (remaining.decrementAndGet() == 0) { + completeAction.run(); + } + }; + // pre-increment this once so we maintain >=1 under normal conditions + incrementReferenceCount(); + } + + @Override + protected void onReferenceCountAtZero() { + final Exception localException = exception.get(); + if (localException != null) { + finalErrorConsumer.accept(localException); + } + } + + @Override + public void accept(Exception e) { + exception.compareAndSet(null, e); + } + + @Override + public void run() { + while (true) { + try { + if (exception.get() != null) { + + incrementReferenceCount(); + return; + } + int idx = nextIndex.getAndIncrement(); + if (idx < start + count) { + // do the work + action.run(idx, resumeAction); + incrementReferenceCount(); + } else { + // no more work to do + incrementReferenceCount(); + return; + } + } finally { + decrementReferenceCount(); + } + } + } + } + /** * Provides a mechanism to iterate over a range of values in parallel using the {@link JobScheduler} * @@ -78,7 +150,7 @@ interface IterateResumeAction { */ @FinalDefault default void iterateParallel(ExecutionContext executionContext, LogOutputAppendable description, int start, - int count, IterateAction action, Runnable completeAction, final Consumer onError) { + int count, IterateAction action, Runnable completeAction, Consumer onError) { iterateParallel(executionContext, description, start, count, (final int idx, final Runnable resume) -> { action.run(idx); @@ -110,48 +182,39 @@ default void iterateParallel(ExecutionContext executionContext, LogOutputAppenda completeAction.run(); } - final AtomicInteger nextIndex = new AtomicInteger(start); - final AtomicInteger remaining = new AtomicInteger(count); + final ErrorAccounter ea = new ErrorAccounter(start, count, onError, action, completeAction); - final AtomicBoolean cancelRemainingExecution = new AtomicBoolean(false); + // final AtomicBoolean cancelRemainingExecution = new AtomicBoolean(false); - final Consumer localError = exception -> { - // signal only on the first error - if (cancelRemainingExecution.compareAndSet(false, true)) { - onError.accept(exception); - } - }; + // final Consumer localError = exception -> { + // // signal only on the first error + // if (cancelRemainingExecution.compareAndSet(false, true)) { + // onError.accept(exception); + // } + // }; - final Runnable resumeAction = () -> { - // check for completion - if (remaining.decrementAndGet() == 0) { - completeAction.run(); - } - }; + // final Runnable resumeAction = () -> { + // // check for completion + // if (remaining.decrementAndGet() == 0) { + // completeAction.run(); + // } + // }; - final Runnable task = () -> { - // this will run until all tasks have started - while (true) { - if (cancelRemainingExecution.get()) { - return; - } - int idx = nextIndex.getAndIncrement(); - if (idx < start + count) { - // do the work - action.run(idx, resumeAction); - } else { - // no more work to do - return; - } - } - }; + // final Runnable task = () -> { + // // this will run until all tasks have started + // while (true) { + // if (cancelRemainingExecution.get()) { + // return; + // } + // } + // }; // create multiple tasks but not more than one per scheduler thread for (int i = 0; i < Math.min(count, threadCount()); i++) { submit(executionContext, - task, + ea, description, - localError); + ea); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/OperationInitializationPoolJobScheduler.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/OperationInitializationPoolJobScheduler.java index 865ca07290f..2bb865c30b0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/OperationInitializationPoolJobScheduler.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/OperationInitializationPoolJobScheduler.java @@ -5,6 +5,7 @@ import io.deephaven.engine.table.impl.OperationInitializationThreadPool; import io.deephaven.engine.table.impl.perf.BasePerformanceEntry; import io.deephaven.engine.table.impl.util.JobScheduler; +import io.deephaven.io.log.impl.LogOutputStringImpl; import io.deephaven.util.SafeCloseable; import io.deephaven.util.process.ProcessEnvironment; @@ -27,7 +28,8 @@ public void submit( } catch (Exception e) { onError.accept(e); } catch (Error e) { - ProcessEnvironment.getGlobalFatalErrorReporter().report("SelectAndView Error", e); + final String logMessage = new LogOutputStringImpl().append(description).append(" Error").toString(); + ProcessEnvironment.getGlobalFatalErrorReporter().report(logMessage, e); throw e; } finally { basePerformanceEntry.onBaseEntryEnd(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/UpdateGraphProcessorJobScheduler.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/UpdateGraphProcessorJobScheduler.java index ea4d2afaf9e..d85279d6ef3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/UpdateGraphProcessorJobScheduler.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/UpdateGraphProcessorJobScheduler.java @@ -7,6 +7,7 @@ import io.deephaven.engine.table.impl.util.JobScheduler; import io.deephaven.engine.updategraph.AbstractNotification; import io.deephaven.engine.updategraph.UpdateGraphProcessor; +import io.deephaven.io.log.impl.LogOutputStringImpl; import io.deephaven.util.process.ProcessEnvironment; import java.util.function.Consumer; @@ -35,7 +36,8 @@ public void run() { } catch (Exception e) { onError.accept(e); } catch (Error e) { - ProcessEnvironment.getGlobalFatalErrorReporter().report("SelectAndView Error", e); + final String logMessage = new LogOutputStringImpl().append(description).append(" Error").toString(); + ProcessEnvironment.getGlobalFatalErrorReporter().report(logMessage, e); throw e; } finally { baseEntry.onBaseEntryEnd(); From 9b5f295b591c43245854eec374ae8598219fec1f Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Thu, 12 Jan 2023 11:41:08 -0800 Subject: [PATCH 084/123] Rewrite of UpdateByWindowTicks for accuracy and efficiency --- .../engine/table/impl/updateby/UpdateBy.java | 41 ++-- .../table/impl/updateby/UpdateByWindow.java | 22 +- .../impl/updateby/UpdateByWindowTicks.java | 202 ++++++------------ 3 files changed, 89 insertions(+), 176 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java index 3bbf24a3e3c..874414c78c7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java @@ -889,13 +889,12 @@ public static Table updateBy(@NotNull final QueryTable source, } } - // We will divide the operators into similar windows for efficient processing. It will be more useful to store - // indices into the operator in oppArr rather than operator objects. - final KeyedObjectHashMap> windowMap = + // We will divide the operators into similar windows for efficient processing. + final KeyedObjectHashMap> windowMap = new KeyedObjectHashMap<>(new KeyedObjectKey<>() { @Override - public UpdateByOperator getKey(List updateByOperators) { - return opArr[updateByOperators.get(0)]; + public UpdateByOperator getKey(List updateByOperators) { + return updateByOperators.get(0); } @Override @@ -905,33 +904,30 @@ public int hashKey(UpdateByOperator updateByOperator) { @Override public boolean equalKey(UpdateByOperator updateByOperator, - List updateByOperators) { - return UpdateByWindow.isEquivalentWindow(updateByOperator, opArr[updateByOperators.get(0)]); + List updateByOperators) { + return UpdateByWindow.isEquivalentWindow(updateByOperator, updateByOperators.get(0)); } }); - for (int opIdx = 0; opIdx < opArr.length; opIdx++) { + for (UpdateByOperator updateByOperator : opArr) { final MutableBoolean created = new MutableBoolean(false); - int finalOpIdx = opIdx; - final List opList = windowMap.putIfAbsent(opArr[opIdx], + final List opList = windowMap.putIfAbsent(updateByOperator, (newOpListOp) -> { - final List newOpList = new ArrayList<>(); - newOpList.add(finalOpIdx); + final List newOpList = new ArrayList<>(); + newOpList.add(newOpListOp); created.setTrue(); return newOpList; }); if (!created.booleanValue()) { - opList.add(finalOpIdx); + opList.add(updateByOperator); } } // make the windows and create unique input sources for all the window operators - final UpdateByWindow[] windowArr = new UpdateByWindow[windowMap.size()]; - final MutableInt winIdx = new MutableInt(0); final ArrayList> inputSourceList = new ArrayList<>(); final TObjectIntHashMap> sourceToSlotMap = new TObjectIntHashMap<>(); - windowMap.forEach((ignored, opList) -> { + final UpdateByWindow[] windowArr = windowMap.values().stream().map((final List opList) -> { // build an array from the operator indices UpdateByOperator[] windowOps = new UpdateByOperator[opList.size()]; // local map of operators indices to input source indices @@ -939,8 +935,7 @@ public boolean equalKey(UpdateByOperator updateByOperator, // do the mapping from operator input sources to unique input sources for (int idx = 0; idx < opList.size(); idx++) { - final int opIdx = opList.get(idx); - final UpdateByOperator localOp = opArr[opIdx]; + final UpdateByOperator localOp = opList.get(idx); // store this operator into an array for window creation windowOps[idx] = localOp; // iterate over each input column and map this operator to unique source @@ -951,7 +946,7 @@ public boolean equalKey(UpdateByOperator updateByOperator, final int maybeExistingSlot = sourceToSlotMap.get(input); if (maybeExistingSlot == sourceToSlotMap.getNoEntryValue()) { // create a new input source - int srcIdx = inputSourceList.size(); + final int srcIdx = inputSourceList.size(); inputSourceList.add(ReinterpretUtils.maybeConvertToPrimitive(input)); sourceToSlotMap.put(input, srcIdx); // map the window operator indices to this new source @@ -962,12 +957,8 @@ public boolean equalKey(UpdateByOperator updateByOperator, } } } - - // create and store the window for these operators - windowArr[winIdx.getAndIncrement()] = - UpdateByWindow.createFromOperatorArray(windowOps, windowOpSourceSlots); - }); - + return UpdateByWindow.createFromOperatorArray(windowOps, windowOpSourceSlots); + }).toArray(UpdateByWindow[]::new); final ColumnSource[] inputSourceArr = inputSourceList.toArray(ColumnSource.ZERO_LENGTH_COLUMN_SOURCE_ARRAY); final Map> resultSources = new LinkedHashMap<>(source.getColumnSourceMap()); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java index 336aa6d8a3c..e27f12cf6b4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java @@ -10,6 +10,7 @@ import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.util.SafeCloseable; +import io.deephaven.util.SafeCloseableArray; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -79,25 +80,12 @@ public UpdateByWindowBucketContext(final TrackingRowSet sourceRowSet, @Override public void close() { - // these might be the same object, don't close both! - if (influencerRows != null && influencerRows != affectedRows) { - influencerRows.close(); - influencerRows = null; - } - try (final RowSet ignoredRs1 = affectedRows) { - } - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - if (opContext[opIdx] != null) { - opContext[opIdx].close(); - } + try (final SafeCloseable ignoredRs1 = affectedRows == sourceRowSet ? null : affectedRows; + final SafeCloseable ignoredRs2 = influencerRows == affectedRows ? null : influencerRows) { } + SafeCloseableArray.close(opContext); if (inputSources != null) { - for (int srcIdx = 0; srcIdx < inputSources.length; srcIdx++) { - if (inputSourceGetContexts[srcIdx] != null) { - inputSourceGetContexts[srcIdx].close(); - inputSourceGetContexts[srcIdx] = null; - } - } + SafeCloseableArray.close(inputSourceGetContexts); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java index d535624cc72..91e3ddaf56e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java @@ -2,7 +2,7 @@ import gnu.trove.list.array.TIntArrayList; import gnu.trove.set.hash.TIntHashSet; -import io.deephaven.base.ringbuffer.IntRingBuffer; +import io.deephaven.base.ringbuffer.LongRingBuffer; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; @@ -13,10 +13,10 @@ import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; +import io.deephaven.util.SafeCloseable; import io.deephaven.util.datastructures.LongSizedDataStructure; import org.apache.commons.lang3.mutable.MutableLong; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import java.util.Arrays; import java.util.stream.IntStream; @@ -27,48 +27,26 @@ * of `influencer` values to add to the rolling window as the current row changes. */ public class UpdateByWindowTicks extends UpdateByWindow { - private static final int WINDOW_POS_BUFFER_INITIAL_CAPACITY = 512; - protected final long prevUnits; - protected final long fwdUnits; + private final long prevUnits; + private final long fwdUnits; - public class UpdateByWindowBucketTicksContext extends UpdateByWindowBucketContext { + class UpdateByWindowBucketTicksContext extends UpdateByWindowBucketContext { private static final int WINDOW_CHUNK_SIZE = 4096; - - protected final IntRingBuffer currentWindowPositions; - - protected RowSet affectedRowPositions; - protected RowSet influencerPositions; - - protected int nextInfluencerIndex; - protected int nextInfluencerPos; - protected long nextInfluencerKey; - - protected RowSequence.Iterator influencerIt; - protected RowSequence.Iterator influencerPosIt; - protected LongChunk influencerPosChunk; - protected LongChunk influencerKeyChunk; - protected long influencerPosChunkSize; - protected int currentGetContextSize; + private RowSet affectedRowPositions; + private RowSet influencerPositions; + private int currentGetContextSize; public UpdateByWindowBucketTicksContext(final TrackingRowSet sourceRowSet, - @Nullable final ColumnSource timestampColumnSource, - @Nullable final LongSegmentedSortedArray timestampSsa, final int chunkSize, final boolean initialStep) { + final int chunkSize, final boolean initialStep) { super(sourceRowSet, null, null, chunkSize, initialStep); - - currentWindowPositions = new IntRingBuffer(WINDOW_POS_BUFFER_INITIAL_CAPACITY, true); } @Override public void close() { super.close(); - // these might be identical, don't close both! - if (influencerPositions != null && influencerPositions != affectedRowPositions) { - influencerPositions.close(); - } - try (final RowSet ignoredRs1 = affectedRowPositions; - final RowSequence.Iterator ignoreIt1 = influencerIt; - final RowSequence.Iterator ignoreIt2 = influencerPosIt) { - // leveraging try with resources to auto-close + try (final SafeCloseable ignoredRs1 = affectedRowPositions; + final SafeCloseable ignoredRs2 = + influencerPositions == affectedRowPositions ? null : influencerPositions) { } } } @@ -79,7 +57,7 @@ public void close() { this.fwdUnits = fwdUnits; } - protected void makeOperatorContexts(UpdateByWindowBucketContext context) { + private void makeOperatorContexts(UpdateByWindowBucketContext context) { UpdateByWindowBucketTicksContext ctx = (UpdateByWindowBucketTicksContext) context; ctx.workingChunkSize = UpdateByWindowBucketTicksContext.WINDOW_CHUNK_SIZE; @@ -91,20 +69,28 @@ protected void makeOperatorContexts(UpdateByWindowBucketContext context) { } } + @Override public UpdateByWindowBucketContext makeWindowContext(final TrackingRowSet sourceRowSet, final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa, final int chunkSize, final boolean isInitializeStep) { - return new UpdateByWindowBucketTicksContext(sourceRowSet, timestampColumnSource, timestampSsa, - chunkSize, - isInitializeStep); + return new UpdateByWindowBucketTicksContext(sourceRowSet, chunkSize, isInitializeStep); } private static WritableRowSet computeAffectedRowsTicks(final RowSet sourceSet, final RowSet subset, final RowSet invertedSubSet, long revTicks, long fwdTicks) { - // swap fwd/rev to get the influencer windows - return computeInfluencerRowsTicks(sourceSet, subset, invertedSubSet, fwdTicks, revTicks); + // adjust fwd/rev to get the affected windows + + // Potential cases and reasoning: + // 1) rev 1, fwd 0 - this row only influences, affected should also be 1, 0 + // 2) rev 2, fwd 0 - this row and previous influences, affected should be 1, 1 + // 3) rev 10, fwd 0 - this row and previous 9 influeces, affected should be 1, 9 + // 4) rev 0, fwd 10 - next 10 influences, affected should be 11, -1 (looks weird but that is how we would + // exclude the current row) + // 5) rev 10, fwd 50 - affected should be 51, 9 + + return computeInfluencerRowsTicks(sourceSet, subset, invertedSubSet, fwdTicks + 1, revTicks - 1); } private static WritableRowSet computeInfluencerRowsTicks(final RowSet sourceSet, final RowSet subset, @@ -119,7 +105,7 @@ private static WritableRowSet computeInfluencerRowsTicks(final RowSet sourceSet, final MutableLong minPos = new MutableLong(0L); invertedSubSet.forAllRowKeyRanges((s, e) -> { - long sPos = Math.max(s - revTicks, minPos.longValue()); + long sPos = Math.max(s - revTicks + 1, minPos.longValue()); long ePos = Math.min(e + fwdTicks, maxPos); builder.appendRange(sPos, ePos); minPos.setValue(ePos + 1); @@ -141,7 +127,7 @@ private void ensureGetContextSize(UpdateByWindowBucketTicksContext ctx, long new size); // use this to track which contexts have already resized - boolean[] resized = new boolean[ctx.inputSources.length]; + final boolean[] resized = new boolean[ctx.inputSources.length]; for (int opIdx : ctx.dirtyOperatorIndices) { final int[] sourceIndices = operatorInputSourceSlots[opIdx]; @@ -160,37 +146,6 @@ private void ensureGetContextSize(UpdateByWindowBucketTicksContext ctx, long new } } - /*** - * This function takes care of loading/preparing the next set of influencer data, in this case we load the next - * chunk of key and position data and reset the index - */ - private void loadNextInfluencerChunks(UpdateByWindowBucketTicksContext ctx) { - if (!ctx.influencerIt.hasMore()) { - ctx.nextInfluencerPos = Integer.MAX_VALUE; - ctx.nextInfluencerKey = Long.MAX_VALUE; - return; - } - - final RowSequence influencerRs = - ctx.influencerIt.getNextRowSequenceWithLength(UpdateByWindowBucketTicksContext.WINDOW_CHUNK_SIZE); - ctx.influencerKeyChunk = influencerRs.asRowKeyChunk(); - - final RowSequence influencePosRs = - ctx.influencerPosIt.getNextRowSequenceWithLength(UpdateByWindowBucketTicksContext.WINDOW_CHUNK_SIZE); - ctx.influencerPosChunk = influencePosRs.asRowKeyChunk(); - - Assert.eqTrue(influencePosRs.lastRowKey() < Integer.MAX_VALUE, - "updateBy window positions exceeded maximum size"); - - ctx.influencerPosChunkSize = ctx.influencerPosChunk.size(); - - ctx.nextInfluencerIndex = 0; - ctx.nextInfluencerPos = LongSizedDataStructure.intSize( - "updateBy window positions exceeded maximum size", - ctx.influencerPosChunk.get(ctx.nextInfluencerIndex)); - ctx.nextInfluencerKey = ctx.influencerKeyChunk.get(ctx.nextInfluencerIndex); - } - /** * Finding the `affected` and `influencer` rowsets for a windowed operation is complex. We must identify modified * rows (including added rows) and deleted rows and determine which rows are `affected` by the change given the @@ -202,28 +157,43 @@ public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, UpdateByWindowBucketTicksContext ctx = (UpdateByWindowBucketTicksContext) context; + if (upstream.empty()) { + return; + } + // all rows are affected on the initial step if (ctx.initialStep) { - ctx.affectedRows = ctx.sourceRowSet.copy(); - ctx.influencerRows = ctx.affectedRows; - + ctx.affectedRows = ctx.sourceRowSet; // no need to invert, just create a flat rowset ctx.affectedRowPositions = RowSetFactory.flat(ctx.sourceRowSet.size()); - ctx.influencerPositions = RowSetFactory.flat(ctx.sourceRowSet.size()); + + // quick test to see if we will need all rows + if (prevUnits > 0 && fwdUnits >= 0) { + // the current row influences itself, therefore all rows are needed + ctx.influencerRows = ctx.affectedRows; + ctx.influencerPositions = ctx.affectedRowPositions; + } else { + // some rows will be excluded, get the exact set of influencer rows + final long size = ctx.affectedRows.size(); + final long startPos = Math.max(0, 1 - prevUnits); + final long endPos = Math.min(size - 1, size + fwdUnits - 1); + + // subSetByPositionRange() endPos is exclusive + ctx.influencerRows = ctx.affectedRows.subSetByPositionRange(startPos, endPos + 1); + ctx.influencerPositions = ctx.affectedRowPositions.subSetByPositionRange(startPos, endPos + 1); + } // mark all operators as affected by this update context.dirtyOperatorIndices = IntStream.range(0, operators.length).toArray(); context.dirtySourceIndices = getUniqueSourceIndices(); makeOperatorContexts(ctx); - ctx.isDirty = !upstream.empty(); + ctx.isDirty = true; return; } // determine which operators are affected by this update - ctx.isDirty = false; - boolean allAffected = upstream.added().isNonempty() || - upstream.removed().isNonempty(); + boolean allAffected = upstream.added().isNonempty() || upstream.removed().isNonempty(); if (allAffected) { // mark all operators as affected by this update @@ -231,7 +201,7 @@ public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, context.dirtySourceIndices = getUniqueSourceIndices(); context.isDirty = true; } else { - // determine which operators wer affected by this update + // determine which operators were affected by this update TIntArrayList dirtyOperatorList = new TIntArrayList(operators.length); TIntHashSet inputSourcesSet = new TIntHashSet(getUniqueSourceIndices().length); for (int opIdx = 0; opIdx < operators.length; opIdx++) { @@ -251,15 +221,15 @@ public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, return; } - // changed rows are all mods+adds - WritableRowSet changed = upstream.added().union(upstream.modified()); - // need a writable rowset - WritableRowSet tmpAffected; + final WritableRowSet tmpAffected; - // compute the rows affected from these changes - try (final WritableRowSet changedInverted = ctx.sourceRowSet.invert(changed)) { - tmpAffected = computeAffectedRowsTicks(ctx.sourceRowSet, changed, changedInverted, prevUnits, fwdUnits); + // changed rows are all mods+adds + try (final WritableRowSet changed = upstream.added().union(upstream.modified())) { + // compute the rows affected from these changes + try (final WritableRowSet changedInverted = ctx.sourceRowSet.invert(changed)) { + tmpAffected = computeAffectedRowsTicks(ctx.sourceRowSet, changed, changedInverted, prevUnits, fwdUnits); + } } // other rows can be affected by removes @@ -303,19 +273,16 @@ public void processRows(UpdateByWindowBucketContext context, boolean initialStep winOp.initializeUpdate(ctx.opContext[opIdx]); } - ctx.influencerIt = ctx.influencerRows.getRowSequenceIterator(); - ctx.influencerPosIt = ctx.influencerPositions.getRowSequenceIterator(); - try (final RowSequence.Iterator it = ctx.affectedRows.getRowSequenceIterator(); final RowSequence.Iterator posIt = ctx.affectedRowPositions.getRowSequenceIterator(); + final RowSequence.Iterator influencerPosHeadIt = ctx.influencerPositions.getRowSequenceIterator(); + final RowSequence.Iterator influencerPosTailIt = ctx.influencerPositions.getRowSequenceIterator(); + final RowSequence.Iterator influencerKeyIt = ctx.influencerRows.getRowSequenceIterator(); final WritableIntChunk pushChunk = WritableIntChunk.makeWritableChunk(ctx.workingChunkSize); final WritableIntChunk popChunk = WritableIntChunk.makeWritableChunk(ctx.workingChunkSize)) { - // load the first chunk of influencer values (fillWindowTicks() will call in future) - loadNextInfluencerChunks(ctx); - final long sourceRowSetSize = ctx.sourceRowSet.size(); while (it.hasMore()) { @@ -338,48 +305,15 @@ public void processRows(UpdateByWindowBucketContext context, boolean initialStep final long tail = Math.min(sourceRowSetSize - 1, currentPos + fwdUnits); // pop out all values from the current window that are not in the new window - int popCount = 0; - while (!ctx.currentWindowPositions.isEmpty() && ctx.currentWindowPositions.front() < head) { - ctx.currentWindowPositions.remove(); - popCount++; - } + long popCount = influencerPosHeadIt.advanceAndGetPositionDistance(head); - // skip values until they match the window (this can only happen on initial addition of rows - // to the table, because we short-circuited the precise building of the influencer rows for - // efficiency) - while (ctx.nextInfluencerPos < head) { - ctx.nextInfluencerIndex++; - - if (ctx.nextInfluencerIndex < ctx.influencerPosChunkSize) { - ctx.nextInfluencerPos = (int) ctx.influencerPosChunk.get(ctx.nextInfluencerIndex); - ctx.nextInfluencerKey = ctx.influencerKeyChunk.get(ctx.nextInfluencerIndex); - } else { - // try to bring in new data - loadNextInfluencerChunks(ctx); - } - } - - // push matching values - int pushCount = 0; - while (ctx.nextInfluencerPos <= tail) { - ctx.currentWindowPositions.add(ctx.nextInfluencerPos); - pushCount++; - // add this key to the needed set for this chunk - chunkInfluencerBuilder.appendKey(ctx.nextInfluencerKey); - ctx.nextInfluencerIndex++; - - if (ctx.nextInfluencerIndex < ctx.influencerPosChunkSize) { - ctx.nextInfluencerPos = (int) ctx.influencerPosChunk.get(ctx.nextInfluencerIndex); - ctx.nextInfluencerKey = ctx.influencerKeyChunk.get(ctx.nextInfluencerIndex); - } else { - // try to bring in new data - loadNextInfluencerChunks(ctx); - } - } + // push in all values that are in the new window (inclusive of tail) + long pushCount = influencerPosTailIt.advanceAndGetPositionDistance(tail + 1); + chunkInfluencerBuilder.appendRowSequence(influencerKeyIt.getNextRowSequenceWithLength(pushCount)); // write the push and pop counts to the chunks - popChunk.set(ii, popCount); - pushChunk.set(ii, pushCount); + popChunk.set(ii, Math.toIntExact(popCount)); + pushChunk.set(ii, Math.toIntExact(pushCount)); } // execute the operators From 9d57d8574f3a84c5d7fbd6c95f8c1a8501793f58 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Thu, 12 Jan 2023 17:37:39 -0800 Subject: [PATCH 085/123] Bug fixes for forward looking rolling window --- .../engine/table/impl/updateby/UpdateBy.java | 34 ++++---------- .../table/impl/updateby/UpdateByWindow.java | 46 ++++++++----------- .../impl/updateby/UpdateByWindowTicks.java | 23 ++++++---- 3 files changed, 42 insertions(+), 61 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java index 874414c78c7..9917a7fa576 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java @@ -90,7 +90,6 @@ static class UpdateByRedirectionHelper { @Nullable private final WritableRowRedirection rowRedirection; private final WritableRowSet freeRows; - private WritableRowSet toClear; private long maxInnerRowKey; private UpdateByRedirectionHelper(@Nullable final WritableRowRedirection rowRedirection) { @@ -107,13 +106,15 @@ private long requiredCapacity() { return maxInnerRowKey; } - private void processUpdateForRedirection(@NotNull final TableUpdate upstream, + /** + * Process the upstream {@link TableUpdate update} and return the rowset of dense keys that need cleared for + * Object array sources + */ + private WritableRowSet processUpdateForRedirection(@NotNull final TableUpdate upstream, final TrackingRowSet sourceRowSet) { assert rowRedirection != null; - // take this chance to clean up last cycle's rowset - try (final RowSet ignored = toClear) { - } + final WritableRowSet toClear; if (upstream.removed().isNonempty()) { final RowSetBuilderRandom freeBuilder = RowSetFactory.builderRandom(); @@ -148,6 +149,7 @@ private void processUpdateForRedirection(@NotNull final TableUpdate upstream, freeRows.clear(); } } + return toClear; } private RowSet getInnerKeys(final RowSet outerKeys) { @@ -164,18 +166,6 @@ private RowSet getInnerKeys(final RowSet outerKeys) { } return builder.build(); } - - /*** - * Compute the inner source keys that need to be cleared. These are rows that were removed this cycle and not - * replaced by added rows. These are in the dense key-space and must only be applied to the inner sources of the - * redirected output sources. - * - * @return the set of rows that should be cleared from the inner (dense) sources. This {@link RowSet} should be - * closed by the caller. - */ - WritableRowSet getRowsToClear() { - return toClear.copy(); - } } protected UpdateBy( @@ -688,19 +678,15 @@ private TableUpdate computeDownstreamUpdate() { */ public void processUpdate() { if (redirHelper.isRedirected()) { - // this call does all the work needed for redirected output sources, including handling removed rows - redirHelper.processUpdateForRedirection(upstream, source.getRowSet()); + // this call does all the work needed for redirected output sources, returns the set of rows we need + // to clear from our Object array output sources + toClear = redirHelper.processUpdateForRedirection(upstream, source.getRowSet()); changedRows = RowSetFactory.empty(); - // identify which rows we need to clear in our Object columns. These rows will not intersect with rows - // we intend to modify later - toClear = redirHelper.getRowsToClear(); - // clear them now and let them set their own prev states if (!initialStep && !toClear.isEmpty()) { for (UpdateByOperator op : operators) { op.clearOutputRows(toClear); - } } } else { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java index e27f12cf6b4..bff344b2028 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java @@ -17,7 +17,7 @@ import java.util.Arrays; import java.util.Objects; -public abstract class UpdateByWindow { +abstract class UpdateByWindow { @Nullable protected final String timestampColumnName; @@ -30,7 +30,7 @@ public abstract class UpdateByWindow { protected int[] uniqueInputSourceIndices; /** This context will store the necessary info to process a single window for a single bucket */ - public class UpdateByWindowBucketContext implements SafeCloseable { + class UpdateByWindowBucketContext implements SafeCloseable { /** A reference to the source rowset */ protected final TrackingRowSet sourceRowSet; /** The column source providing the timestamp data for this window */ @@ -63,7 +63,7 @@ public class UpdateByWindowBucketContext implements SafeCloseable { /** Indicates which sources are needed to process this window context */ protected int[] dirtySourceIndices; - public UpdateByWindowBucketContext(final TrackingRowSet sourceRowSet, + UpdateByWindowBucketContext(final TrackingRowSet sourceRowSet, @Nullable final ColumnSource timestampColumnSource, @Nullable final LongSegmentedSortedArray timestampSsa, final int chunkSize, @@ -97,7 +97,7 @@ abstract UpdateByWindowBucketContext makeWindowContext(final TrackingRowSet sour final int chunkSize, final boolean isInitializeStep); - protected UpdateByWindow(UpdateByOperator[] operators, int[][] operatorInputSourceSlots, + UpdateByWindow(UpdateByOperator[] operators, int[][] operatorInputSourceSlots, @Nullable String timestampColumnName) { this.operators = operators; this.operatorInputSourceSlots = operatorInputSourceSlots; @@ -112,7 +112,7 @@ protected UpdateByWindow(UpdateByOperator[] operators, int[][] operatorInputSour * * @return a new {@link UpdateByWindow window} from these operators */ - public static UpdateByWindow createFromOperatorArray(final UpdateByOperator[] operators, + static UpdateByWindow createFromOperatorArray(final UpdateByOperator[] operators, final int[][] operatorSourceSlots) { // review operators to extract timestamp column (if one exists) String timestampColumnName = null; @@ -143,22 +143,14 @@ public static UpdateByWindow createFromOperatorArray(final UpdateByOperator[] op } } - /** - * Returns the timestamp column name for this window (or null if no timestamps in use) - */ - @Nullable - public String getTimestampColumnName() { - return timestampColumnName; - } - /** * Returns the operators for this window (a subset of the total operators for this UpdateBy call) */ - public UpdateByOperator[] getOperators() { + UpdateByOperator[] getOperators() { return operators; } - public int[] getUniqueSourceIndices() { + int[] getUniqueSourceIndices() { if (uniqueInputSourceIndices == null) { final TIntHashSet set = new TIntHashSet(); for (int opIdx = 0; opIdx < operators.length; opIdx++) { @@ -174,7 +166,7 @@ public int[] getUniqueSourceIndices() { * * @param srcIdx the index of the input source */ - public boolean isSourceInUse(int srcIdx) { + boolean isSourceInUse(int srcIdx) { // this looks worse than it actually is, windows are defined by their input sources so there will be only // one or two entries in `getUniqueSourceIndices()`. Iterating will be faster than building a lookup table // or a hashset @@ -191,7 +183,7 @@ public boolean isSourceInUse(int srcIdx) { * * @param changes the rowset indicating which rows will be modified or added this cycle */ - public void prepareForParallelPopulation(final RowSet changes) { + void prepareForParallelPopulation(final RowSet changes) { for (UpdateByOperator operator : operators) { operator.prepareForParallelPopulation(changes); } @@ -206,7 +198,7 @@ public void prepareForParallelPopulation(final RowSet changes) { * @param context the window context that will store the results. * @param upstream the update that indicates incoming changes to the data. */ - public abstract void computeAffectedRowsAndOperators(final UpdateByWindowBucketContext context, + abstract void computeAffectedRowsAndOperators(final UpdateByWindowBucketContext context, @NotNull final TableUpdate upstream); /** @@ -216,7 +208,7 @@ public abstract void computeAffectedRowsAndOperators(final UpdateByWindowBucketC * @param context the window context that will store these sources. * @param inputSources the (potentially cached) input sources to use for processing. */ - public void assignInputSources(final UpdateByWindowBucketContext context, final ColumnSource[] inputSources) { + void assignInputSources(final UpdateByWindowBucketContext context, final ColumnSource[] inputSources) { context.inputSources = inputSources; context.inputSourceGetContexts = new ChunkSource.GetContext[inputSources.length]; context.inputSourceChunks = new WritableChunk[inputSources.length]; @@ -248,14 +240,14 @@ protected void prepareValuesChunkForSource(final UpdateByWindowBucketContext con * @param context the window context that will manage the results. * @param initialStep whether this is the creation step of the table. */ - public abstract void processRows(final UpdateByWindowBucketContext context, final boolean initialStep); + abstract void processRows(final UpdateByWindowBucketContext context, final boolean initialStep); /** * Returns `true` if the window for this bucket needs to be processed this cycle. * * @param context the window context that will manage the results. */ - public boolean isWindowDirty(final UpdateByWindowBucketContext context) { + boolean isWindowDirty(final UpdateByWindowBucketContext context) { return context.isDirty; } @@ -264,7 +256,7 @@ public boolean isWindowDirty(final UpdateByWindowBucketContext context) { * * @param context the window context that will manage the results. */ - public int[] getDirtyOperators(final UpdateByWindowBucketContext context) { + int[] getDirtyOperators(final UpdateByWindowBucketContext context) { return context.dirtyOperatorIndices; } @@ -273,7 +265,7 @@ public int[] getDirtyOperators(final UpdateByWindowBucketContext context) { * * @param context the window context that will manage the results. */ - public RowSet getAffectedRows(final UpdateByWindowBucketContext context) { + RowSet getAffectedRows(final UpdateByWindowBucketContext context) { return context.affectedRows; } @@ -282,7 +274,7 @@ public RowSet getAffectedRows(final UpdateByWindowBucketContext context) { * * @param context the window context that will manage the results. */ - public RowSet getInfluencerRows(final UpdateByWindowBucketContext context) { + RowSet getInfluencerRows(final UpdateByWindowBucketContext context) { return context.influencerRows; } @@ -291,7 +283,7 @@ public RowSet getInfluencerRows(final UpdateByWindowBucketContext context) { /** * Returns a hash code to help distinguish between windows on the same UpdateBy call */ - protected static int hashCode(boolean windowed, @NotNull String[] inputColumnNames, + private static int hashCode(boolean windowed, @NotNull String[] inputColumnNames, @Nullable String timestampColumnName, long prevUnits, long fwdUnits) { @@ -318,7 +310,7 @@ protected static int hashCode(boolean windowed, @NotNull String[] inputColumnNam /** * Returns a hash code given a particular operator */ - public static int hashCodeFromOperator(final UpdateByOperator op) { + static int hashCodeFromOperator(final UpdateByOperator op) { return hashCode(op instanceof UpdateByWindowedOperator, op.getInputColumnNames(), op.getTimestampColumnName(), @@ -329,7 +321,7 @@ public static int hashCodeFromOperator(final UpdateByOperator op) { /** * Returns `true` if two operators are compatible and can be executed as part of the same window */ - public static boolean isEquivalentWindow(final UpdateByOperator opA, final UpdateByOperator opB) { + static boolean isEquivalentWindow(final UpdateByOperator opA, final UpdateByOperator opB) { // verify input columns are identical if (!Arrays.equals(opA.getInputColumnNames(), opB.getInputColumnNames())) { return false; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java index 91e3ddaf56e..1ebccd40e07 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java @@ -8,6 +8,7 @@ import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.chunk.util.pools.ChunkPoolConstants; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.ColumnSource; @@ -26,7 +27,7 @@ * maintain a window of data based on row distance rather than timestamps. Window-based operators must maintain a buffer * of `influencer` values to add to the rolling window as the current row changes. */ -public class UpdateByWindowTicks extends UpdateByWindow { +class UpdateByWindowTicks extends UpdateByWindow { private final long prevUnits; private final long fwdUnits; @@ -36,7 +37,7 @@ class UpdateByWindowBucketTicksContext extends UpdateByWindowBucketContext { private RowSet influencerPositions; private int currentGetContextSize; - public UpdateByWindowBucketTicksContext(final TrackingRowSet sourceRowSet, + UpdateByWindowBucketTicksContext(final TrackingRowSet sourceRowSet, final int chunkSize, final boolean initialStep) { super(sourceRowSet, null, null, chunkSize, initialStep); } @@ -70,7 +71,7 @@ private void makeOperatorContexts(UpdateByWindowBucketContext context) { } @Override - public UpdateByWindowBucketContext makeWindowContext(final TrackingRowSet sourceRowSet, + UpdateByWindowBucketContext makeWindowContext(final TrackingRowSet sourceRowSet, final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa, final int chunkSize, @@ -85,7 +86,7 @@ private static WritableRowSet computeAffectedRowsTicks(final RowSet sourceSet, f // Potential cases and reasoning: // 1) rev 1, fwd 0 - this row only influences, affected should also be 1, 0 // 2) rev 2, fwd 0 - this row and previous influences, affected should be 1, 1 - // 3) rev 10, fwd 0 - this row and previous 9 influeces, affected should be 1, 9 + // 3) rev 10, fwd 0 - this row and previous 9 influences, affected should be 1, 9 // 4) rev 0, fwd 10 - next 10 influences, affected should be 11, -1 (looks weird but that is how we would // exclude the current row) // 5) rev 10, fwd 50 - affected should be 51, 9 @@ -95,9 +96,6 @@ private static WritableRowSet computeAffectedRowsTicks(final RowSet sourceSet, f private static WritableRowSet computeInfluencerRowsTicks(final RowSet sourceSet, final RowSet subset, final RowSet invertedSubSet, long revTicks, long fwdTicks) { - if (sourceSet.size() == subset.size()) { - return sourceSet.copy(); - } long maxPos = sourceSet.size() - 1; @@ -122,9 +120,11 @@ private void ensureGetContextSize(UpdateByWindowBucketTicksContext ctx, long new while (size < newSize) { size *= 2; } + + // if size would no longer be poolable, use the exact size for the new contexts ctx.currentGetContextSize = LongSizedDataStructure.intSize( "ensureGetContextSize exceeded Integer.MAX_VALUE", - size); + size >= ChunkPoolConstants.LARGEST_POOLED_CHUNK_CAPACITY ? newSize : size); // use this to track which contexts have already resized final boolean[] resized = new boolean[ctx.inputSources.length]; @@ -153,7 +153,7 @@ private void ensureGetContextSize(UpdateByWindowBucketTicksContext ctx, long new * these values (i.e. that fall within the window and will `influence` this computation). */ @Override - public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, @NotNull TableUpdate upstream) { + void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, @NotNull TableUpdate upstream) { UpdateByWindowBucketTicksContext ctx = (UpdateByWindowBucketTicksContext) context; @@ -247,6 +247,9 @@ public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, } } + // naturally need to compute the newly added rows + tmpAffected.insert(upstream.added()); + ctx.affectedRows = tmpAffected; // now get influencer rows for the affected rows @@ -262,7 +265,7 @@ public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, } @Override - public void processRows(UpdateByWindowBucketContext context, boolean initialStep) { + void processRows(UpdateByWindowBucketContext context, boolean initialStep) { UpdateByWindowBucketTicksContext ctx = (UpdateByWindowBucketTicksContext) context; Assert.neqNull(context.inputSources, "assignInputSources() must be called before processRow()"); From 2b4b7f347755a3e3b4ebf0be1391fe06b161191c Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 13 Jan 2023 11:39:40 -0800 Subject: [PATCH 086/123] Changes to UpdateByWindowTime, plus cleanup --- .../updateby/UpdateByWindowCumulative.java | 18 +- .../impl/updateby/UpdateByWindowTicks.java | 13 +- .../impl/updateby/UpdateByWindowTime.java | 236 +++++++----------- 3 files changed, 111 insertions(+), 156 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java index 5e8031c2c7a..d2fb9e80057 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java @@ -1,7 +1,5 @@ package io.deephaven.engine.table.impl.updateby; -import gnu.trove.list.array.TIntArrayList; -import gnu.trove.set.hash.TIntHashSet; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; @@ -25,14 +23,14 @@ * This is the specialization of {@link UpdateByWindow} that handles `cumulative` operators. These operators do not * maintain a window of data and can be computed from the previous value and the current value. */ -public class UpdateByWindowCumulative extends UpdateByWindow { +class UpdateByWindowCumulative extends UpdateByWindow { - public UpdateByWindowCumulative(UpdateByOperator[] operators, int[][] operatorSourceSlots, + UpdateByWindowCumulative(UpdateByOperator[] operators, int[][] operatorSourceSlots, @Nullable String timestampColumnName) { super(operators, operatorSourceSlots, timestampColumnName); } - protected void makeOperatorContexts(UpdateByWindowBucketContext context) { + private void makeOperatorContexts(UpdateByWindowBucketContext context) { // working chunk size need not be larger than affectedRows.size() context.workingChunkSize = Math.toIntExact(Math.min(context.workingChunkSize, context.affectedRows.size())); @@ -42,7 +40,7 @@ protected void makeOperatorContexts(UpdateByWindowBucketContext context) { } } - public UpdateByWindowBucketContext makeWindowContext(final TrackingRowSet sourceRowSet, + UpdateByWindowBucketContext makeWindowContext(final TrackingRowSet sourceRowSet, final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa, final int chunkSize, @@ -53,7 +51,11 @@ public UpdateByWindowBucketContext makeWindowContext(final TrackingRowSet source } @Override - public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, @NotNull TableUpdate upstream) { + void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, @NotNull TableUpdate upstream) { + if (upstream.empty() || context.sourceRowSet.isEmpty()) { + return; + } + // all rows are affected on the initial step if (context.initialStep) { context.affectedRows = context.sourceRowSet.copy(); @@ -111,7 +113,7 @@ public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, } @Override - public void processRows(UpdateByWindowBucketContext context, final boolean initialStep) { + void processRows(UpdateByWindowBucketContext context, final boolean initialStep) { Assert.neqNull(context.inputSources, "assignInputSources() must be called before processRow()"); if (initialStep) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java index 1ebccd40e07..c63bbdafa5d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java @@ -28,11 +28,11 @@ * of `influencer` values to add to the rolling window as the current row changes. */ class UpdateByWindowTicks extends UpdateByWindow { + private static final int WINDOW_CHUNK_SIZE = 4096; private final long prevUnits; private final long fwdUnits; class UpdateByWindowBucketTicksContext extends UpdateByWindowBucketContext { - private static final int WINDOW_CHUNK_SIZE = 4096; private RowSet affectedRowPositions; private RowSet influencerPositions; private int currentGetContextSize; @@ -61,7 +61,7 @@ public void close() { private void makeOperatorContexts(UpdateByWindowBucketContext context) { UpdateByWindowBucketTicksContext ctx = (UpdateByWindowBucketTicksContext) context; - ctx.workingChunkSize = UpdateByWindowBucketTicksContext.WINDOW_CHUNK_SIZE; + ctx.workingChunkSize = WINDOW_CHUNK_SIZE; ctx.currentGetContextSize = ctx.workingChunkSize; // create contexts for the affected operators @@ -84,8 +84,8 @@ private static WritableRowSet computeAffectedRowsTicks(final RowSet sourceSet, f // adjust fwd/rev to get the affected windows // Potential cases and reasoning: - // 1) rev 1, fwd 0 - this row only influences, affected should also be 1, 0 - // 2) rev 2, fwd 0 - this row and previous influences, affected should be 1, 1 + // 1) rev 1, fwd 0 - this row influences, affected should also be 1, 0 + // 2) rev 2, fwd 0 - this row and previous 1 influences, affected should be 1, 1 // 3) rev 10, fwd 0 - this row and previous 9 influences, affected should be 1, 9 // 4) rev 0, fwd 10 - next 10 influences, affected should be 11, -1 (looks weird but that is how we would // exclude the current row) @@ -96,7 +96,6 @@ private static WritableRowSet computeAffectedRowsTicks(final RowSet sourceSet, f private static WritableRowSet computeInfluencerRowsTicks(final RowSet sourceSet, final RowSet subset, final RowSet invertedSubSet, long revTicks, long fwdTicks) { - long maxPos = sourceSet.size() - 1; final RowSetBuilderSequential builder = RowSetFactory.builderSequential(); @@ -157,7 +156,7 @@ void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, @NotNu UpdateByWindowBucketTicksContext ctx = (UpdateByWindowBucketTicksContext) context; - if (upstream.empty()) { + if (upstream.empty() || context.sourceRowSet.isEmpty()) { return; } @@ -247,7 +246,7 @@ void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, @NotNu } } - // naturally need to compute the newly added rows + // naturally need to compute all newly added rows tmpAffected.insert(upstream.added()); ctx.affectedRows = tmpAffected; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java index bc64f25531d..5bc9f098f2b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java @@ -2,18 +2,18 @@ import gnu.trove.list.array.TIntArrayList; import gnu.trove.set.hash.TIntHashSet; -import io.deephaven.base.ringbuffer.LongRingBuffer; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; +import io.deephaven.engine.table.iterators.LongColumnIterator; +import io.deephaven.util.SafeCloseable; import io.deephaven.util.datastructures.LongSizedDataStructure; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -21,6 +21,7 @@ import java.util.Arrays; import java.util.stream.IntStream; +import static io.deephaven.util.QueryConstants.NULL_INT; import static io.deephaven.util.QueryConstants.NULL_LONG; /** @@ -28,25 +29,13 @@ * maintain a window of data based on a timestamp column rather than row distances. Window-based operators must maintain * a buffer of `influencer` values to add to the rolling window as the current row changes. */ -public class UpdateByWindowTime extends UpdateByWindow { - private static final int WINDOW_TIMESTAMP_BUFFER_INITIAL_CAPACITY = 512; +class UpdateByWindowTime extends UpdateByWindow { + private static final int WINDOW_CHUNK_SIZE = 4096; protected final long prevUnits; protected final long fwdUnits; public class UpdateByWindowBucketTimeContext extends UpdateByWindowBucketContext { - private static final int WINDOW_CHUNK_SIZE = 4096; - protected final ChunkSource.GetContext influencerTimestampContext; - protected final LongRingBuffer currentWindowTimestamps; - - protected int nextInfluencerIndex; - protected long nextInfluencerTimestamp; - protected long nextInfluencerKey; - - protected RowSequence.Iterator influencerIt; - protected LongChunk influencerKeyChunk; - protected LongChunk influencerTimestampChunk; - protected long influencerTimestampChunkSize; protected int currentGetContextSize; public UpdateByWindowBucketTimeContext(final TrackingRowSet sourceRowSet, @@ -55,14 +44,12 @@ public UpdateByWindowBucketTimeContext(final TrackingRowSet sourceRowSet, super(sourceRowSet, timestampColumnSource, timestampSsa, chunkSize, initialStep); influencerTimestampContext = timestampColumnSource.makeGetContext(WINDOW_CHUNK_SIZE); - currentWindowTimestamps = new LongRingBuffer(WINDOW_TIMESTAMP_BUFFER_INITIAL_CAPACITY, true); } @Override public void close() { super.close(); - try (final RowSequence.Iterator ignoreIt1 = influencerIt; - final ChunkSource.GetContext ignoreCtx1 = influencerTimestampContext) { + try (final SafeCloseable ignoreCtx1 = influencerTimestampContext) { // leveraging try with resources to auto-close } } @@ -79,7 +66,7 @@ public void close() { protected void makeOperatorContexts(UpdateByWindowBucketContext context) { UpdateByWindowBucketTimeContext ctx = (UpdateByWindowBucketTimeContext) context; - ctx.workingChunkSize = UpdateByWindowBucketTimeContext.WINDOW_CHUNK_SIZE; + ctx.workingChunkSize = WINDOW_CHUNK_SIZE; ctx.currentGetContextSize = ctx.workingChunkSize; // create contexts for the affected operators @@ -114,15 +101,12 @@ private static WritableRowSet computeAffectedRowsTime(final RowSet sourceSet, fi private static WritableRowSet computeInfluencerRowsTime(final RowSet sourceSet, final RowSet subset, long revNanos, long fwdNanos, final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa, boolean usePrev) { - if (sourceSet.size() == subset.size()) { - return sourceSet.copy(); - } - int chunkSize = (int) Math.min(subset.size(), 4096); try (final RowSequence.Iterator it = subset.getRowSequenceIterator(); final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(chunkSize)) { final RowSetBuilderSequential builder = RowSetFactory.builderSequential(); LongSegmentedSortedArray.Iterator ssaIt = timestampSsa.iterator(false, false); + while (it.hasMore() && ssaIt.hasNext()) { final RowSequence rs = it.getNextRowSequenceWithLength(chunkSize); final int rsSize = rs.intSize(); @@ -132,37 +116,36 @@ private static WritableRowSet computeInfluencerRowsTime(final RowSet sourceSet, : timestampColumnSource.getChunk(context, rs).asLongChunk(); for (int ii = 0; ii < rsSize; ii++) { - // if the timestamp of the row is null, it won't belong to any set and we can ignore it - // completely final long ts = timestamps.get(ii); - if (ts != NULL_LONG) { - // look at every row timestamp, compute the head and tail in nanos - final long head = ts - revNanos; - final long tail = ts + fwdNanos; - - // advance the iterator to the beginning of the window - if (ssaIt.nextValue() < head) { - ssaIt.advanceToBeforeFirst(head); - if (!ssaIt.hasNext()) { - // SSA is exhausted - break; - } - } - - Assert.eqTrue(ssaIt.hasNext() && ssaIt.nextValue() >= head, - "SSA Iterator outside of window"); - - // step through the SSA and collect keys until outside the window - while (ssaIt.hasNext() && ssaIt.nextValue() <= tail) { - builder.appendKey(ssaIt.nextKey()); - ssaIt.next(); - } + if (ts == NULL_LONG) { + // if the timestamp is null, the row won't belong to any set and we can pretend it doesn't exist + continue; + } + // look at every row timestamp, compute the head and tail in nanos + final long head = ts - revNanos; + final long tail = ts + fwdNanos; + // advance the iterator to the beginning of the window + if (ssaIt.nextValue() < head) { + ssaIt.advanceToBeforeFirst(head); if (!ssaIt.hasNext()) { // SSA is exhausted break; } } + Assert.eqTrue(ssaIt.hasNext() && ssaIt.nextValue() >= head, + "SSA Iterator outside of window"); + + // step through the SSA and collect keys until outside the window + while (ssaIt.hasNext() && ssaIt.nextValue() <= tail) { + builder.appendKey(ssaIt.nextKey()); + ssaIt.next(); + } + + if (!ssaIt.hasNext()) { + // SSA is exhausted + break; + } } } return builder.build(); @@ -199,30 +182,6 @@ private void ensureGetContextSize(UpdateByWindowBucketTimeContext ctx, long newS } } - /*** - * This function takes care of loading/preparing the next set of influencer data, in this case we load the next - * chunk of key and position data and reset the index - */ - private void loadNextInfluencerChunks(UpdateByWindowBucketTimeContext ctx) { - if (!ctx.influencerIt.hasMore()) { - ctx.nextInfluencerTimestamp = Long.MAX_VALUE; - ctx.nextInfluencerKey = Long.MAX_VALUE; - return; - } - - final RowSequence influencerRs = - ctx.influencerIt.getNextRowSequenceWithLength(UpdateByWindowBucketTimeContext.WINDOW_CHUNK_SIZE); - ctx.influencerKeyChunk = influencerRs.asRowKeyChunk(); - ctx.influencerTimestampChunk = - ctx.timestampColumnSource.getChunk(ctx.influencerTimestampContext, influencerRs).asLongChunk(); - - ctx.influencerTimestampChunkSize = ctx.influencerTimestampChunk.size(); - - ctx.nextInfluencerIndex = 0; - ctx.nextInfluencerTimestamp = ctx.influencerTimestampChunk.get(ctx.nextInfluencerIndex); - ctx.nextInfluencerKey = ctx.influencerKeyChunk.get(ctx.nextInfluencerIndex); - } - // windowed by time/ticks is more complex to compute: find all the changed rows and the rows that would // be affected by the changes (includes newly added rows) and need to be recomputed. Then include all // the rows that are affected by deletions (if any). After the affected rows have been identified, @@ -231,9 +190,13 @@ private void loadNextInfluencerChunks(UpdateByWindowBucketTimeContext ctx) { public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, @NotNull TableUpdate upstream) { UpdateByWindowBucketTimeContext ctx = (UpdateByWindowBucketTimeContext) context; + if (upstream.empty() || ctx.sourceRowSet.isEmpty()) { + return; + } + // all rows are affected on the initial step if (ctx.initialStep) { - ctx.affectedRows = ctx.sourceRowSet.copy(); + ctx.affectedRows = ctx.sourceRowSet; ctx.influencerRows = ctx.affectedRows; // mark all operators as affected by this update @@ -241,7 +204,7 @@ public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, context.dirtySourceIndices = getUniqueSourceIndices(); makeOperatorContexts(ctx); - ctx.isDirty = !upstream.empty(); + ctx.isDirty = true; return; } @@ -302,6 +265,9 @@ public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, } } + // naturally need to compute all newly added rows + tmpAffected.insert(upstream.added()); + ctx.affectedRows = tmpAffected; // now get influencer rows for the affected rows @@ -311,6 +277,14 @@ public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, makeOperatorContexts(ctx); } + private long advanceToNonNull(LongColumnIterator it) { + long value; + do { + value = it.hasNext() ? it.nextLong() : Long.MAX_VALUE; + } while (value == NULL_LONG); + return value; + } + /*** * This function process the affected rows chunkwise, and will advance the moving window (which is the same for all * operators in this collection). For each row in the dataset the sliding window will adjust and instructions for @@ -330,9 +304,10 @@ public void processRows(UpdateByWindowBucketContext context, boolean initialStep winOp.initializeUpdate(ctx.opContext[opIdx]); } - ctx.influencerIt = ctx.influencerRows.getRowSequenceIterator(); - try (final RowSequence.Iterator it = ctx.affectedRows.getRowSequenceIterator(); + final LongColumnIterator influencerTsHeadIt = new LongColumnIterator(context.timestampColumnSource, context.influencerRows); + final LongColumnIterator influencerTsTailIt = new LongColumnIterator(context.timestampColumnSource, context.influencerRows); + final RowSequence.Iterator influencerKeyIt = ctx.influencerRows.getRowSequenceIterator(); final ChunkSource.GetContext localTimestampContext = ctx.timestampColumnSource.makeGetContext(ctx.workingChunkSize); final WritableIntChunk pushChunk = @@ -340,8 +315,8 @@ public void processRows(UpdateByWindowBucketContext context, boolean initialStep final WritableIntChunk popChunk = WritableIntChunk.makeWritableChunk(ctx.workingChunkSize)) { - // load the first chunk of influencer values (fillWindowTime() will call in future) - loadNextInfluencerChunks(ctx); + long currentHeadTs = advanceToNonNull(influencerTsHeadIt); + long currentTailTs = advanceToNonNull(influencerTsTailIt); while (it.hasMore()) { final RowSequence chunkRs = it.getNextRowSequenceWithLength(ctx.workingChunkSize); @@ -352,88 +327,67 @@ public void processRows(UpdateByWindowBucketContext context, boolean initialStep final LongChunk timestampChunk = ctx.timestampColumnSource.getChunk(localTimestampContext, chunkRs).asLongChunk(); - // we are going to track all the influencer rows that affect this chunk of data - final RowSetBuilderSequential chunkInfluencerBuilder = RowSetFactory.builderSequential(); - // chunk processing + long totalCount = 0; + for (int ii = 0; ii < chunkRsSize; ii++) { - // read the current position + // read the current timestamp final long currentTimestamp = timestampChunk.get(ii); + if (currentTimestamp == NULL_LONG) { + // this signifies that does not belong to a time window + popChunk.set(ii, NULL_INT); + pushChunk.set(ii, NULL_INT); + continue; + } - // compute the head and tail positions (inclusive) + // compute the head and tail timestamps (inclusive) final long head = currentTimestamp - prevUnits; final long tail = currentTimestamp + fwdUnits; // pop out all values from the current window that are not in the new window - int popCount = 0; - while (!ctx.currentWindowTimestamps.isEmpty() && ctx.currentWindowTimestamps.front() < head) { - ctx.currentWindowTimestamps.remove(); + long popCount = 0; + while (currentHeadTs < head) { + currentHeadTs = advanceToNonNull(influencerTsHeadIt); popCount++; } - - // skip values until they match the window (this can only happen on the initial addition of rows - // to the table, because we short-circuited the precise building of the influencer rows for - // efficiency) - while (ctx.nextInfluencerTimestamp < head) { - ctx.nextInfluencerIndex++; - - if (ctx.nextInfluencerIndex < ctx.influencerTimestampChunkSize) { - ctx.nextInfluencerTimestamp = ctx.influencerTimestampChunk.get(ctx.nextInfluencerIndex); - ctx.nextInfluencerKey = ctx.influencerKeyChunk.get(ctx.nextInfluencerIndex); - } else { - // try to bring in new data - loadNextInfluencerChunks(ctx); - } - } - - // push matching values - int pushCount = 0; - while (ctx.nextInfluencerTimestamp <= tail) { - ctx.currentWindowTimestamps.add(ctx.nextInfluencerTimestamp); + // push in all values that are in the new window (inclusive of tail) + long pushCount = 0; + while (currentTailTs <= tail) { + currentTailTs = advanceToNonNull(influencerTsTailIt); pushCount++; - // add this key to the needed set for this chunk - chunkInfluencerBuilder.appendKey(ctx.nextInfluencerKey); - ctx.nextInfluencerIndex++; - - if (ctx.nextInfluencerIndex < ctx.influencerTimestampChunkSize) { - ctx.nextInfluencerTimestamp = ctx.influencerTimestampChunk.get(ctx.nextInfluencerIndex); - ctx.nextInfluencerKey = ctx.influencerKeyChunk.get(ctx.nextInfluencerIndex); - } else { - // try to bring in new data - loadNextInfluencerChunks(ctx); - } } // write the push and pop counts to the chunks - popChunk.set(ii, popCount); - pushChunk.set(ii, pushCount); + popChunk.set(ii, Math.toIntExact(popCount)); + pushChunk.set(ii, Math.toIntExact(pushCount)); + + totalCount += pushCount; } // execute the operators - try (final RowSet chunkInfluencerRs = chunkInfluencerBuilder.build()) { - ensureGetContextSize(ctx, chunkInfluencerRs.size()); - - Arrays.fill(ctx.inputSourceChunks, null); - for (int opIdx : context.dirtyOperatorIndices) { - // prep the chunk array needed by the accumulate call - final int[] srcIndices = operatorInputSourceSlots[opIdx]; - Chunk[] chunkArr = new Chunk[srcIndices.length]; - for (int ii = 0; ii < srcIndices.length; ii++) { - int srcIdx = srcIndices[ii]; - // chunk prep - prepareValuesChunkForSource(ctx, srcIdx, chunkInfluencerRs); - chunkArr[ii] = ctx.inputSourceChunks[srcIdx]; - } - - // make the specialized call for windowed operators - ((UpdateByWindowedOperator.Context) ctx.opContext[opIdx]).accumulate( - chunkRs, - chunkArr, - pushChunk, - popChunk, - chunkRsSize); + final RowSequence chunkInfluencerRs = influencerKeyIt.getNextRowSequenceWithLength(totalCount); + ensureGetContextSize(ctx, chunkInfluencerRs.size()); + + Arrays.fill(ctx.inputSourceChunks, null); + for (int opIdx : context.dirtyOperatorIndices) { + // prep the chunk array needed by the accumulate call + final int[] srcIndices = operatorInputSourceSlots[opIdx]; + Chunk[] chunkArr = new Chunk[srcIndices.length]; + for (int ii = 0; ii < srcIndices.length; ii++) { + int srcIdx = srcIndices[ii]; + // chunk prep + prepareValuesChunkForSource(ctx, srcIdx, chunkInfluencerRs); + chunkArr[ii] = ctx.inputSourceChunks[srcIdx]; } + + // make the specialized call for windowed operators + ((UpdateByWindowedOperator.Context) ctx.opContext[opIdx]).accumulate( + chunkRs, + chunkArr, + pushChunk, + popChunk, + chunkRsSize); } } } From 0214830f4bf6ab52c0c3b80660a3273ec23b809f Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 17 Jan 2023 14:40:39 -0800 Subject: [PATCH 087/123] Large update, JobScheduler changes and tests plus maintenance of validTimestampRowSet for UpdateBy --- .../base/ringbuffer/ByteRingBuffer.java | 57 +- .../base/ringbuffer/CharRingBuffer.java | 57 +- .../base/ringbuffer/DoubleRingBuffer.java | 57 +- .../base/ringbuffer/FloatRingBuffer.java | 57 +- .../base/ringbuffer/IntRingBuffer.java | 57 +- .../base/ringbuffer/LongRingBuffer.java | 57 +- .../base/ringbuffer/ShortRingBuffer.java | 57 +- .../base/ringbuffer/ByteRingBufferTest.java | 75 +++ .../base/ringbuffer/CharRingBufferTest.java | 75 +++ .../base/ringbuffer/DoubleRingBufferTest.java | 75 +++ .../base/ringbuffer/FloatRingBufferTest.java | 75 +++ .../base/ringbuffer/IntRingBufferTest.java | 75 +++ .../base/ringbuffer/LongRingBufferTest.java | 75 +++ .../base/ringbuffer/ShortRingBufferTest.java | 75 +++ .../engine/table/impl/updateby/UpdateBy.java | 71 ++- .../impl/updateby/UpdateByBucketHelper.java | 82 ++- .../table/impl/updateby/UpdateByWindow.java | 5 + .../updateby/UpdateByWindowCumulative.java | 7 +- .../impl/updateby/UpdateByWindowTicks.java | 94 +-- .../impl/updateby/UpdateByWindowTime.java | 82 +-- .../engine/table/impl/util/JobScheduler.java | 160 ++--- .../engine/table/impl/updateby/TestEma.java | 79 ++- .../table/impl/updateby/TestRollingSum.java | 6 +- .../engine/util/TestJobScheduler.java | 547 ++++++++++++++++++ 24 files changed, 1834 insertions(+), 223 deletions(-) create mode 100644 engine/table/src/test/java/io/deephaven/engine/util/TestJobScheduler.java diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java index a5fee8f7c01..6ad69374bbe 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java @@ -22,9 +22,14 @@ public class ByteRingBuffer implements Serializable { protected byte[] storage; protected int head, tail; - private void grow() { + private void grow(int increase) { if (growable) { - byte[] newStorage = new byte[storage.length * 2]; + final int minLength = storage.length + increase; + int newLength = storage.length * 2; + while (newLength < minLength) { + newLength = newLength * 2; + } + byte[] newStorage = new byte[newLength]; if (tail > head) { System.arraycopy(storage, head, newStorage, 0, tail - head); tail = tail - head; @@ -38,6 +43,10 @@ private void grow() { } } + private void grow() { + grow(1); + } + public boolean isFull() { return (tail + 1) % storage.length == head; } @@ -104,6 +113,35 @@ public boolean add(byte e) { return true; } + /** + * Ensure that there is sufficient empty space to store {@code count} items in the buffer. If the buffer is + * {@code growable}, this may result in an internal growth operation. This call should be used in conjunction with + * {@link #addUnsafe(byte)}. + * + * @param count the amount of empty entries in the buffer after this call + * @throws UnsupportedOperationException when {@code growable} is {@code false} and buffer is full + */ + public void ensureRemaining(int count) { + if (remaining() < count) { + if (!growable) { + throw new UnsupportedOperationException("Ring buffer is full and growth is disabled"); + } else { + grow(count); + } + } + } + + /** + * Add values unsafely (will silently overwrite values if the buffer is full). This call should be used in + * conjunction with * {@link #ensureRemaining(int)}. + * + * @param e the value to add to the buffer + */ + public void addUnsafe(byte e) { + storage[tail] = e; + tail = (tail + 1) % storage.length; + } + /** * Add an entry to the ring buffer. If the buffer is full, will overwrite the oldest entry with the new one. * @@ -137,6 +175,21 @@ public boolean offer(byte e) { return true; } + public byte[] remove(int count) { + if (size() < count) { + throw new NoSuchElementException(); + } + final byte[] result = new byte[count]; + if (tail > head || storage.length - head >= count) { + System.arraycopy(storage, head, result, 0, count); + } else { + System.arraycopy(storage, head, result, 0, storage.length - head); + System.arraycopy(storage, 0, result, storage.length - head, count - (storage.length - head)); + } + head = (head + count) % storage.length; + return result; + } + public byte remove() { if (isEmpty()) { throw new NoSuchElementException(); diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java index 8ad9703be20..d92d8085209 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java @@ -17,9 +17,14 @@ public class CharRingBuffer implements Serializable { protected char[] storage; protected int head, tail; - private void grow() { + private void grow(int increase) { if (growable) { - char[] newStorage = new char[storage.length * 2]; + final int minLength = storage.length + increase; + int newLength = storage.length * 2; + while (newLength < minLength) { + newLength = newLength * 2; + } + char[] newStorage = new char[newLength]; if (tail > head) { System.arraycopy(storage, head, newStorage, 0, tail - head); tail = tail - head; @@ -33,6 +38,10 @@ private void grow() { } } + private void grow() { + grow(1); + } + public boolean isFull() { return (tail + 1) % storage.length == head; } @@ -99,6 +108,35 @@ public boolean add(char e) { return true; } + /** + * Ensure that there is sufficient empty space to store {@code count} items in the buffer. If the buffer is + * {@code growable}, this may result in an internal growth operation. This call should be used in conjunction with + * {@link #addUnsafe(char)}. + * + * @param count the amount of empty entries in the buffer after this call + * @throws UnsupportedOperationException when {@code growable} is {@code false} and buffer is full + */ + public void ensureRemaining(int count) { + if (remaining() < count) { + if (!growable) { + throw new UnsupportedOperationException("Ring buffer is full and growth is disabled"); + } else { + grow(count); + } + } + } + + /** + * Add values unsafely (will silently overwrite values if the buffer is full). This call should be used in + * conjunction with * {@link #ensureRemaining(int)}. + * + * @param e the value to add to the buffer + */ + public void addUnsafe(char e) { + storage[tail] = e; + tail = (tail + 1) % storage.length; + } + /** * Add an entry to the ring buffer. If the buffer is full, will overwrite the oldest entry with the new one. * @@ -132,6 +170,21 @@ public boolean offer(char e) { return true; } + public char[] remove(int count) { + if (size() < count) { + throw new NoSuchElementException(); + } + final char[] result = new char[count]; + if (tail > head || storage.length - head >= count) { + System.arraycopy(storage, head, result, 0, count); + } else { + System.arraycopy(storage, head, result, 0, storage.length - head); + System.arraycopy(storage, 0, result, storage.length - head, count - (storage.length - head)); + } + head = (head + count) % storage.length; + return result; + } + public char remove() { if (isEmpty()) { throw new NoSuchElementException(); diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java index 39ea7222ad1..701c0bda002 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java @@ -22,9 +22,14 @@ public class DoubleRingBuffer implements Serializable { protected double[] storage; protected int head, tail; - private void grow() { + private void grow(int increase) { if (growable) { - double[] newStorage = new double[storage.length * 2]; + final int minLength = storage.length + increase; + int newLength = storage.length * 2; + while (newLength < minLength) { + newLength = newLength * 2; + } + double[] newStorage = new double[newLength]; if (tail > head) { System.arraycopy(storage, head, newStorage, 0, tail - head); tail = tail - head; @@ -38,6 +43,10 @@ private void grow() { } } + private void grow() { + grow(1); + } + public boolean isFull() { return (tail + 1) % storage.length == head; } @@ -104,6 +113,35 @@ public boolean add(double e) { return true; } + /** + * Ensure that there is sufficient empty space to store {@code count} items in the buffer. If the buffer is + * {@code growable}, this may result in an internal growth operation. This call should be used in conjunction with + * {@link #addUnsafe(double)}. + * + * @param count the amount of empty entries in the buffer after this call + * @throws UnsupportedOperationException when {@code growable} is {@code false} and buffer is full + */ + public void ensureRemaining(int count) { + if (remaining() < count) { + if (!growable) { + throw new UnsupportedOperationException("Ring buffer is full and growth is disabled"); + } else { + grow(count); + } + } + } + + /** + * Add values unsafely (will silently overwrite values if the buffer is full). This call should be used in + * conjunction with * {@link #ensureRemaining(int)}. + * + * @param e the value to add to the buffer + */ + public void addUnsafe(double e) { + storage[tail] = e; + tail = (tail + 1) % storage.length; + } + /** * Add an entry to the ring buffer. If the buffer is full, will overwrite the oldest entry with the new one. * @@ -137,6 +175,21 @@ public boolean offer(double e) { return true; } + public double[] remove(int count) { + if (size() < count) { + throw new NoSuchElementException(); + } + final double[] result = new double[count]; + if (tail > head || storage.length - head >= count) { + System.arraycopy(storage, head, result, 0, count); + } else { + System.arraycopy(storage, head, result, 0, storage.length - head); + System.arraycopy(storage, 0, result, storage.length - head, count - (storage.length - head)); + } + head = (head + count) % storage.length; + return result; + } + public double remove() { if (isEmpty()) { throw new NoSuchElementException(); diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java index ee301dbf6c5..410c9c1186d 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java @@ -22,9 +22,14 @@ public class FloatRingBuffer implements Serializable { protected float[] storage; protected int head, tail; - private void grow() { + private void grow(int increase) { if (growable) { - float[] newStorage = new float[storage.length * 2]; + final int minLength = storage.length + increase; + int newLength = storage.length * 2; + while (newLength < minLength) { + newLength = newLength * 2; + } + float[] newStorage = new float[newLength]; if (tail > head) { System.arraycopy(storage, head, newStorage, 0, tail - head); tail = tail - head; @@ -38,6 +43,10 @@ private void grow() { } } + private void grow() { + grow(1); + } + public boolean isFull() { return (tail + 1) % storage.length == head; } @@ -104,6 +113,35 @@ public boolean add(float e) { return true; } + /** + * Ensure that there is sufficient empty space to store {@code count} items in the buffer. If the buffer is + * {@code growable}, this may result in an internal growth operation. This call should be used in conjunction with + * {@link #addUnsafe(float)}. + * + * @param count the amount of empty entries in the buffer after this call + * @throws UnsupportedOperationException when {@code growable} is {@code false} and buffer is full + */ + public void ensureRemaining(int count) { + if (remaining() < count) { + if (!growable) { + throw new UnsupportedOperationException("Ring buffer is full and growth is disabled"); + } else { + grow(count); + } + } + } + + /** + * Add values unsafely (will silently overwrite values if the buffer is full). This call should be used in + * conjunction with * {@link #ensureRemaining(int)}. + * + * @param e the value to add to the buffer + */ + public void addUnsafe(float e) { + storage[tail] = e; + tail = (tail + 1) % storage.length; + } + /** * Add an entry to the ring buffer. If the buffer is full, will overwrite the oldest entry with the new one. * @@ -137,6 +175,21 @@ public boolean offer(float e) { return true; } + public float[] remove(int count) { + if (size() < count) { + throw new NoSuchElementException(); + } + final float[] result = new float[count]; + if (tail > head || storage.length - head >= count) { + System.arraycopy(storage, head, result, 0, count); + } else { + System.arraycopy(storage, head, result, 0, storage.length - head); + System.arraycopy(storage, 0, result, storage.length - head, count - (storage.length - head)); + } + head = (head + count) % storage.length; + return result; + } + public float remove() { if (isEmpty()) { throw new NoSuchElementException(); diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java index 7a39bb7b66d..6890526dbff 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java @@ -22,9 +22,14 @@ public class IntRingBuffer implements Serializable { protected int[] storage; protected int head, tail; - private void grow() { + private void grow(int increase) { if (growable) { - int[] newStorage = new int[storage.length * 2]; + final int minLength = storage.length + increase; + int newLength = storage.length * 2; + while (newLength < minLength) { + newLength = newLength * 2; + } + int[] newStorage = new int[newLength]; if (tail > head) { System.arraycopy(storage, head, newStorage, 0, tail - head); tail = tail - head; @@ -38,6 +43,10 @@ private void grow() { } } + private void grow() { + grow(1); + } + public boolean isFull() { return (tail + 1) % storage.length == head; } @@ -104,6 +113,35 @@ public boolean add(int e) { return true; } + /** + * Ensure that there is sufficient empty space to store {@code count} items in the buffer. If the buffer is + * {@code growable}, this may result in an internal growth operation. This call should be used in conjunction with + * {@link #addUnsafe(int)}. + * + * @param count the amount of empty entries in the buffer after this call + * @throws UnsupportedOperationException when {@code growable} is {@code false} and buffer is full + */ + public void ensureRemaining(int count) { + if (remaining() < count) { + if (!growable) { + throw new UnsupportedOperationException("Ring buffer is full and growth is disabled"); + } else { + grow(count); + } + } + } + + /** + * Add values unsafely (will silently overwrite values if the buffer is full). This call should be used in + * conjunction with * {@link #ensureRemaining(int)}. + * + * @param e the value to add to the buffer + */ + public void addUnsafe(int e) { + storage[tail] = e; + tail = (tail + 1) % storage.length; + } + /** * Add an entry to the ring buffer. If the buffer is full, will overwrite the oldest entry with the new one. * @@ -137,6 +175,21 @@ public boolean offer(int e) { return true; } + public int[] remove(int count) { + if (size() < count) { + throw new NoSuchElementException(); + } + final int[] result = new int[count]; + if (tail > head || storage.length - head >= count) { + System.arraycopy(storage, head, result, 0, count); + } else { + System.arraycopy(storage, head, result, 0, storage.length - head); + System.arraycopy(storage, 0, result, storage.length - head, count - (storage.length - head)); + } + head = (head + count) % storage.length; + return result; + } + public int remove() { if (isEmpty()) { throw new NoSuchElementException(); diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java index c3a17ffc1a5..d3b3507cfe4 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java @@ -22,9 +22,14 @@ public class LongRingBuffer implements Serializable { protected long[] storage; protected int head, tail; - private void grow() { + private void grow(int increase) { if (growable) { - long[] newStorage = new long[storage.length * 2]; + final int minLength = storage.length + increase; + int newLength = storage.length * 2; + while (newLength < minLength) { + newLength = newLength * 2; + } + long[] newStorage = new long[newLength]; if (tail > head) { System.arraycopy(storage, head, newStorage, 0, tail - head); tail = tail - head; @@ -38,6 +43,10 @@ private void grow() { } } + private void grow() { + grow(1); + } + public boolean isFull() { return (tail + 1) % storage.length == head; } @@ -104,6 +113,35 @@ public boolean add(long e) { return true; } + /** + * Ensure that there is sufficient empty space to store {@code count} items in the buffer. If the buffer is + * {@code growable}, this may result in an internal growth operation. This call should be used in conjunction with + * {@link #addUnsafe(long)}. + * + * @param count the amount of empty entries in the buffer after this call + * @throws UnsupportedOperationException when {@code growable} is {@code false} and buffer is full + */ + public void ensureRemaining(int count) { + if (remaining() < count) { + if (!growable) { + throw new UnsupportedOperationException("Ring buffer is full and growth is disabled"); + } else { + grow(count); + } + } + } + + /** + * Add values unsafely (will silently overwrite values if the buffer is full). This call should be used in + * conjunction with * {@link #ensureRemaining(int)}. + * + * @param e the value to add to the buffer + */ + public void addUnsafe(long e) { + storage[tail] = e; + tail = (tail + 1) % storage.length; + } + /** * Add an entry to the ring buffer. If the buffer is full, will overwrite the oldest entry with the new one. * @@ -137,6 +175,21 @@ public boolean offer(long e) { return true; } + public long[] remove(int count) { + if (size() < count) { + throw new NoSuchElementException(); + } + final long[] result = new long[count]; + if (tail > head || storage.length - head >= count) { + System.arraycopy(storage, head, result, 0, count); + } else { + System.arraycopy(storage, head, result, 0, storage.length - head); + System.arraycopy(storage, 0, result, storage.length - head, count - (storage.length - head)); + } + head = (head + count) % storage.length; + return result; + } + public long remove() { if (isEmpty()) { throw new NoSuchElementException(); diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java index b6197d7ceeb..f10e28df025 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java @@ -22,9 +22,14 @@ public class ShortRingBuffer implements Serializable { protected short[] storage; protected int head, tail; - private void grow() { + private void grow(int increase) { if (growable) { - short[] newStorage = new short[storage.length * 2]; + final int minLength = storage.length + increase; + int newLength = storage.length * 2; + while (newLength < minLength) { + newLength = newLength * 2; + } + short[] newStorage = new short[newLength]; if (tail > head) { System.arraycopy(storage, head, newStorage, 0, tail - head); tail = tail - head; @@ -38,6 +43,10 @@ private void grow() { } } + private void grow() { + grow(1); + } + public boolean isFull() { return (tail + 1) % storage.length == head; } @@ -104,6 +113,35 @@ public boolean add(short e) { return true; } + /** + * Ensure that there is sufficient empty space to store {@code count} items in the buffer. If the buffer is + * {@code growable}, this may result in an internal growth operation. This call should be used in conjunction with + * {@link #addUnsafe(short)}. + * + * @param count the amount of empty entries in the buffer after this call + * @throws UnsupportedOperationException when {@code growable} is {@code false} and buffer is full + */ + public void ensureRemaining(int count) { + if (remaining() < count) { + if (!growable) { + throw new UnsupportedOperationException("Ring buffer is full and growth is disabled"); + } else { + grow(count); + } + } + } + + /** + * Add values unsafely (will silently overwrite values if the buffer is full). This call should be used in + * conjunction with * {@link #ensureRemaining(int)}. + * + * @param e the value to add to the buffer + */ + public void addUnsafe(short e) { + storage[tail] = e; + tail = (tail + 1) % storage.length; + } + /** * Add an entry to the ring buffer. If the buffer is full, will overwrite the oldest entry with the new one. * @@ -137,6 +175,21 @@ public boolean offer(short e) { return true; } + public short[] remove(int count) { + if (size() < count) { + throw new NoSuchElementException(); + } + final short[] result = new short[count]; + if (tail > head || storage.length - head >= count) { + System.arraycopy(storage, head, result, 0, count); + } else { + System.arraycopy(storage, head, result, 0, storage.length - head); + System.arraycopy(storage, 0, result, storage.length - head, count - (storage.length - head)); + } + head = (head + count) % storage.length; + return result; + } + public short remove() { if (isEmpty()) { throw new NoSuchElementException(); diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/ByteRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/ByteRingBufferTest.java index 89e61661f42..8831f6bcca8 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/ByteRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/ByteRingBufferTest.java @@ -14,6 +14,7 @@ import java.util.NoSuchElementException; +import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertThrows; public class ByteRingBufferTest extends TestCase { @@ -90,6 +91,13 @@ private void assertContents(ByteRingBuffer rb, byte... expectedData) { } } + private void assertArrayEquals(byte[] data, byte... expectedData) { + assertEquals(data.length, expectedData.length); + for (int ii = 0; ii < data.length; ii++) { + assertEquals(data[ii], expectedData[ii]); + } + } + byte A = 'A'; byte B = 'B'; byte C = 'C'; @@ -457,6 +465,73 @@ public void testAddOverwriteAndOffer() { assert (rb.isEmpty()); assert (A == rb.peekBack(A)); + } + + + public void testMultipleRemove() { + ByteRingBuffer rb = new ByteRingBuffer(10, false); + + // this should throw + assertThrows(NoSuchElementException.class, + () -> rb.remove()); + + // this should throw + assertThrows(NoSuchElementException.class, + () -> rb.remove(1)); + + rb.add(A); + rb.add(B); + + byte[] values = rb.remove(2); + assertArrayEquals(values, A, B); + assertEmpty(rb); + + rb.add(C); + rb.add(D); + rb.add(E); + rb.add(F); + + values = rb.remove(2); + assertArrayEquals(values, C, D); + + values = rb.remove(2); + assertArrayEquals(values, E, F); + assertEmpty(rb); + + rb.add(A); + rb.add(B); + rb.add(C); + rb.add(D); + rb.add(E); + rb.add(F); + + values = rb.remove(6); + assertArrayEquals(values, A, B, C, D, E, F); + assertEmpty(rb); + } + public void testAddUnsafe() { + ByteRingBuffer rbNoGrow = new ByteRingBuffer(3, false); + + // this should throw + assertThrows(UnsupportedOperationException.class, + () -> rbNoGrow.ensureRemaining(10)); + + rbNoGrow.ensureRemaining(3); + rbNoGrow.addUnsafe(A); + rbNoGrow.addUnsafe(B); + rbNoGrow.addUnsafe(C); + + assertContents(rbNoGrow, A, B, C); + + ByteRingBuffer rbGrow = new ByteRingBuffer(3, true); + + for (int size = 10; size < 1_000_000; size *= 10) { + rbGrow.ensureRemaining(size); + assert (rbGrow.remaining() >= size); + for (int i = 0; i < size; i++) { + rbGrow.addUnsafe(A); + } + } } } diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java index 3cbf2eb050d..ddce8d8a83f 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java @@ -9,6 +9,7 @@ import java.util.NoSuchElementException; +import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertThrows; public class CharRingBufferTest extends TestCase { @@ -85,6 +86,13 @@ private void assertContents(CharRingBuffer rb, char... expectedData) { } } + private void assertArrayEquals(char[] data, char... expectedData) { + assertEquals(data.length, expectedData.length); + for (int ii = 0; ii < data.length; ii++) { + assertEquals(data[ii], expectedData[ii]); + } + } + char A = 'A'; char B = 'B'; char C = 'C'; @@ -452,6 +460,73 @@ public void testAddOverwriteAndOffer() { assert (rb.isEmpty()); assert (A == rb.peekBack(A)); + } + + + public void testMultipleRemove() { + CharRingBuffer rb = new CharRingBuffer(10, false); + + // this should throw + assertThrows(NoSuchElementException.class, + () -> rb.remove()); + + // this should throw + assertThrows(NoSuchElementException.class, + () -> rb.remove(1)); + + rb.add(A); + rb.add(B); + + char[] values = rb.remove(2); + assertArrayEquals(values, A, B); + assertEmpty(rb); + + rb.add(C); + rb.add(D); + rb.add(E); + rb.add(F); + + values = rb.remove(2); + assertArrayEquals(values, C, D); + + values = rb.remove(2); + assertArrayEquals(values, E, F); + assertEmpty(rb); + + rb.add(A); + rb.add(B); + rb.add(C); + rb.add(D); + rb.add(E); + rb.add(F); + + values = rb.remove(6); + assertArrayEquals(values, A, B, C, D, E, F); + assertEmpty(rb); + } + public void testAddUnsafe() { + CharRingBuffer rbNoGrow = new CharRingBuffer(3, false); + + // this should throw + assertThrows(UnsupportedOperationException.class, + () -> rbNoGrow.ensureRemaining(10)); + + rbNoGrow.ensureRemaining(3); + rbNoGrow.addUnsafe(A); + rbNoGrow.addUnsafe(B); + rbNoGrow.addUnsafe(C); + + assertContents(rbNoGrow, A, B, C); + + CharRingBuffer rbGrow = new CharRingBuffer(3, true); + + for (int size = 10; size < 1_000_000; size *= 10) { + rbGrow.ensureRemaining(size); + assert (rbGrow.remaining() >= size); + for (int i = 0; i < size; i++) { + rbGrow.addUnsafe(A); + } + } } } diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/DoubleRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/DoubleRingBufferTest.java index 9cac78dd348..0773e2ba43c 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/DoubleRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/DoubleRingBufferTest.java @@ -14,6 +14,7 @@ import java.util.NoSuchElementException; +import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertThrows; public class DoubleRingBufferTest extends TestCase { @@ -90,6 +91,13 @@ private void assertContents(DoubleRingBuffer rb, double... expectedData) { } } + private void assertArrayEquals(double[] data, double... expectedData) { + assertEquals(data.length, expectedData.length); + for (int ii = 0; ii < data.length; ii++) { + assertEquals(data[ii], expectedData[ii]); + } + } + double A = 'A'; double B = 'B'; double C = 'C'; @@ -457,6 +465,73 @@ public void testAddOverwriteAndOffer() { assert (rb.isEmpty()); assert (A == rb.peekBack(A)); + } + + + public void testMultipleRemove() { + DoubleRingBuffer rb = new DoubleRingBuffer(10, false); + + // this should throw + assertThrows(NoSuchElementException.class, + () -> rb.remove()); + + // this should throw + assertThrows(NoSuchElementException.class, + () -> rb.remove(1)); + + rb.add(A); + rb.add(B); + + double[] values = rb.remove(2); + assertArrayEquals(values, A, B); + assertEmpty(rb); + + rb.add(C); + rb.add(D); + rb.add(E); + rb.add(F); + + values = rb.remove(2); + assertArrayEquals(values, C, D); + + values = rb.remove(2); + assertArrayEquals(values, E, F); + assertEmpty(rb); + + rb.add(A); + rb.add(B); + rb.add(C); + rb.add(D); + rb.add(E); + rb.add(F); + + values = rb.remove(6); + assertArrayEquals(values, A, B, C, D, E, F); + assertEmpty(rb); + } + public void testAddUnsafe() { + DoubleRingBuffer rbNoGrow = new DoubleRingBuffer(3, false); + + // this should throw + assertThrows(UnsupportedOperationException.class, + () -> rbNoGrow.ensureRemaining(10)); + + rbNoGrow.ensureRemaining(3); + rbNoGrow.addUnsafe(A); + rbNoGrow.addUnsafe(B); + rbNoGrow.addUnsafe(C); + + assertContents(rbNoGrow, A, B, C); + + DoubleRingBuffer rbGrow = new DoubleRingBuffer(3, true); + + for (int size = 10; size < 1_000_000; size *= 10) { + rbGrow.ensureRemaining(size); + assert (rbGrow.remaining() >= size); + for (int i = 0; i < size; i++) { + rbGrow.addUnsafe(A); + } + } } } diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/FloatRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/FloatRingBufferTest.java index c3435fa7792..41c28cb3317 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/FloatRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/FloatRingBufferTest.java @@ -14,6 +14,7 @@ import java.util.NoSuchElementException; +import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertThrows; public class FloatRingBufferTest extends TestCase { @@ -90,6 +91,13 @@ private void assertContents(FloatRingBuffer rb, float... expectedData) { } } + private void assertArrayEquals(float[] data, float... expectedData) { + assertEquals(data.length, expectedData.length); + for (int ii = 0; ii < data.length; ii++) { + assertEquals(data[ii], expectedData[ii]); + } + } + float A = 'A'; float B = 'B'; float C = 'C'; @@ -457,6 +465,73 @@ public void testAddOverwriteAndOffer() { assert (rb.isEmpty()); assert (A == rb.peekBack(A)); + } + + + public void testMultipleRemove() { + FloatRingBuffer rb = new FloatRingBuffer(10, false); + + // this should throw + assertThrows(NoSuchElementException.class, + () -> rb.remove()); + + // this should throw + assertThrows(NoSuchElementException.class, + () -> rb.remove(1)); + + rb.add(A); + rb.add(B); + + float[] values = rb.remove(2); + assertArrayEquals(values, A, B); + assertEmpty(rb); + + rb.add(C); + rb.add(D); + rb.add(E); + rb.add(F); + + values = rb.remove(2); + assertArrayEquals(values, C, D); + + values = rb.remove(2); + assertArrayEquals(values, E, F); + assertEmpty(rb); + + rb.add(A); + rb.add(B); + rb.add(C); + rb.add(D); + rb.add(E); + rb.add(F); + + values = rb.remove(6); + assertArrayEquals(values, A, B, C, D, E, F); + assertEmpty(rb); + } + public void testAddUnsafe() { + FloatRingBuffer rbNoGrow = new FloatRingBuffer(3, false); + + // this should throw + assertThrows(UnsupportedOperationException.class, + () -> rbNoGrow.ensureRemaining(10)); + + rbNoGrow.ensureRemaining(3); + rbNoGrow.addUnsafe(A); + rbNoGrow.addUnsafe(B); + rbNoGrow.addUnsafe(C); + + assertContents(rbNoGrow, A, B, C); + + FloatRingBuffer rbGrow = new FloatRingBuffer(3, true); + + for (int size = 10; size < 1_000_000; size *= 10) { + rbGrow.ensureRemaining(size); + assert (rbGrow.remaining() >= size); + for (int i = 0; i < size; i++) { + rbGrow.addUnsafe(A); + } + } } } diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/IntRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/IntRingBufferTest.java index eed54c9d3a7..f7bec065f34 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/IntRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/IntRingBufferTest.java @@ -14,6 +14,7 @@ import java.util.NoSuchElementException; +import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertThrows; public class IntRingBufferTest extends TestCase { @@ -90,6 +91,13 @@ private void assertContents(IntRingBuffer rb, int... expectedData) { } } + private void assertArrayEquals(int[] data, int... expectedData) { + assertEquals(data.length, expectedData.length); + for (int ii = 0; ii < data.length; ii++) { + assertEquals(data[ii], expectedData[ii]); + } + } + int A = 'A'; int B = 'B'; int C = 'C'; @@ -457,6 +465,73 @@ public void testAddOverwriteAndOffer() { assert (rb.isEmpty()); assert (A == rb.peekBack(A)); + } + + + public void testMultipleRemove() { + IntRingBuffer rb = new IntRingBuffer(10, false); + + // this should throw + assertThrows(NoSuchElementException.class, + () -> rb.remove()); + + // this should throw + assertThrows(NoSuchElementException.class, + () -> rb.remove(1)); + + rb.add(A); + rb.add(B); + + int[] values = rb.remove(2); + assertArrayEquals(values, A, B); + assertEmpty(rb); + + rb.add(C); + rb.add(D); + rb.add(E); + rb.add(F); + + values = rb.remove(2); + assertArrayEquals(values, C, D); + + values = rb.remove(2); + assertArrayEquals(values, E, F); + assertEmpty(rb); + + rb.add(A); + rb.add(B); + rb.add(C); + rb.add(D); + rb.add(E); + rb.add(F); + + values = rb.remove(6); + assertArrayEquals(values, A, B, C, D, E, F); + assertEmpty(rb); + } + public void testAddUnsafe() { + IntRingBuffer rbNoGrow = new IntRingBuffer(3, false); + + // this should throw + assertThrows(UnsupportedOperationException.class, + () -> rbNoGrow.ensureRemaining(10)); + + rbNoGrow.ensureRemaining(3); + rbNoGrow.addUnsafe(A); + rbNoGrow.addUnsafe(B); + rbNoGrow.addUnsafe(C); + + assertContents(rbNoGrow, A, B, C); + + IntRingBuffer rbGrow = new IntRingBuffer(3, true); + + for (int size = 10; size < 1_000_000; size *= 10) { + rbGrow.ensureRemaining(size); + assert (rbGrow.remaining() >= size); + for (int i = 0; i < size; i++) { + rbGrow.addUnsafe(A); + } + } } } diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/LongRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/LongRingBufferTest.java index cc4fd085f38..496e1d84b2b 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/LongRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/LongRingBufferTest.java @@ -14,6 +14,7 @@ import java.util.NoSuchElementException; +import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertThrows; public class LongRingBufferTest extends TestCase { @@ -90,6 +91,13 @@ private void assertContents(LongRingBuffer rb, long... expectedData) { } } + private void assertArrayEquals(long[] data, long... expectedData) { + assertEquals(data.length, expectedData.length); + for (int ii = 0; ii < data.length; ii++) { + assertEquals(data[ii], expectedData[ii]); + } + } + long A = 'A'; long B = 'B'; long C = 'C'; @@ -457,6 +465,73 @@ public void testAddOverwriteAndOffer() { assert (rb.isEmpty()); assert (A == rb.peekBack(A)); + } + + + public void testMultipleRemove() { + LongRingBuffer rb = new LongRingBuffer(10, false); + + // this should throw + assertThrows(NoSuchElementException.class, + () -> rb.remove()); + + // this should throw + assertThrows(NoSuchElementException.class, + () -> rb.remove(1)); + + rb.add(A); + rb.add(B); + + long[] values = rb.remove(2); + assertArrayEquals(values, A, B); + assertEmpty(rb); + + rb.add(C); + rb.add(D); + rb.add(E); + rb.add(F); + + values = rb.remove(2); + assertArrayEquals(values, C, D); + + values = rb.remove(2); + assertArrayEquals(values, E, F); + assertEmpty(rb); + + rb.add(A); + rb.add(B); + rb.add(C); + rb.add(D); + rb.add(E); + rb.add(F); + + values = rb.remove(6); + assertArrayEquals(values, A, B, C, D, E, F); + assertEmpty(rb); + } + public void testAddUnsafe() { + LongRingBuffer rbNoGrow = new LongRingBuffer(3, false); + + // this should throw + assertThrows(UnsupportedOperationException.class, + () -> rbNoGrow.ensureRemaining(10)); + + rbNoGrow.ensureRemaining(3); + rbNoGrow.addUnsafe(A); + rbNoGrow.addUnsafe(B); + rbNoGrow.addUnsafe(C); + + assertContents(rbNoGrow, A, B, C); + + LongRingBuffer rbGrow = new LongRingBuffer(3, true); + + for (int size = 10; size < 1_000_000; size *= 10) { + rbGrow.ensureRemaining(size); + assert (rbGrow.remaining() >= size); + for (int i = 0; i < size; i++) { + rbGrow.addUnsafe(A); + } + } } } diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/ShortRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/ShortRingBufferTest.java index ff12c564d71..b6f0bb0ce35 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/ShortRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/ShortRingBufferTest.java @@ -14,6 +14,7 @@ import java.util.NoSuchElementException; +import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertThrows; public class ShortRingBufferTest extends TestCase { @@ -90,6 +91,13 @@ private void assertContents(ShortRingBuffer rb, short... expectedData) { } } + private void assertArrayEquals(short[] data, short... expectedData) { + assertEquals(data.length, expectedData.length); + for (int ii = 0; ii < data.length; ii++) { + assertEquals(data[ii], expectedData[ii]); + } + } + short A = 'A'; short B = 'B'; short C = 'C'; @@ -457,6 +465,73 @@ public void testAddOverwriteAndOffer() { assert (rb.isEmpty()); assert (A == rb.peekBack(A)); + } + + + public void testMultipleRemove() { + ShortRingBuffer rb = new ShortRingBuffer(10, false); + + // this should throw + assertThrows(NoSuchElementException.class, + () -> rb.remove()); + + // this should throw + assertThrows(NoSuchElementException.class, + () -> rb.remove(1)); + + rb.add(A); + rb.add(B); + + short[] values = rb.remove(2); + assertArrayEquals(values, A, B); + assertEmpty(rb); + + rb.add(C); + rb.add(D); + rb.add(E); + rb.add(F); + + values = rb.remove(2); + assertArrayEquals(values, C, D); + + values = rb.remove(2); + assertArrayEquals(values, E, F); + assertEmpty(rb); + + rb.add(A); + rb.add(B); + rb.add(C); + rb.add(D); + rb.add(E); + rb.add(F); + + values = rb.remove(6); + assertArrayEquals(values, A, B, C, D, E, F); + assertEmpty(rb); + } + public void testAddUnsafe() { + ShortRingBuffer rbNoGrow = new ShortRingBuffer(3, false); + + // this should throw + assertThrows(UnsupportedOperationException.class, + () -> rbNoGrow.ensureRemaining(10)); + + rbNoGrow.ensureRemaining(3); + rbNoGrow.addUnsafe(A); + rbNoGrow.addUnsafe(B); + rbNoGrow.addUnsafe(C); + + assertContents(rbNoGrow, A, B, C); + + ShortRingBuffer rbGrow = new ShortRingBuffer(3, true); + + for (int size = 10; size < 1_000_000; size *= 10) { + rbGrow.ensureRemaining(size); + assert (rbGrow.remaining() >= size); + for (int i = 0; i < size; i++) { + rbGrow.addUnsafe(A); + } + } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java index 9917a7fa576..a9b89229f0b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java @@ -24,10 +24,10 @@ import io.deephaven.engine.updategraph.UpdateGraphProcessor; import io.deephaven.hash.KeyedObjectHashMap; import io.deephaven.hash.KeyedObjectKey; +import io.deephaven.util.SafeCloseable; import io.deephaven.util.datastructures.linked.IntrusiveDoublyLinkedNode; import io.deephaven.util.datastructures.linked.IntrusiveDoublyLinkedQueue; import org.apache.commons.lang3.mutable.MutableBoolean; -import org.apache.commons.lang3.mutable.MutableInt; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -368,9 +368,9 @@ private void computeCachedColumnRowsets(final Runnable resumeAction) { return; } - jobScheduler.iterateParallel(ExecutionContext.getContextToRecord(), this, + jobScheduler.iterateParallel(ExecutionContext.getContextToRecord(), this, JobScheduler.JobContext::new, 0, cacheableSourceIndices.length, - idx -> { + (context, idx) -> { final int srcIdx = cacheableSourceIndices[idx]; for (int winIdx = 0; winIdx < windows.length; winIdx++) { UpdateByWindow win = windows[winIdx]; @@ -437,27 +437,34 @@ private void createCachedColumnSource(int srcIdx, final Runnable resumeAction) { final int taskCount = Math.toIntExact((inputRowSet.size() + PARALLEL_CACHE_BATCH_SIZE - 1) / PARALLEL_CACHE_BATCH_SIZE); - jobScheduler.iterateParallel(ExecutionContext.getContextToRecord(), this, + final class BatchContext extends JobScheduler.JobContext { + final RowSequence.Iterator rsIt = inputRowSet.getRowSequenceIterator(); + final ChunkSink.FillFromContext ffc = + outputSource.makeFillFromContext(PARALLEL_CACHE_CHUNK_SIZE); + final ChunkSource.GetContext gc = + inputSource.makeGetContext(PARALLEL_CACHE_CHUNK_SIZE); + + @Override + public void close() { + SafeCloseable.closeArray(rsIt, ffc, gc); + } + } + + jobScheduler.iterateParallel(ExecutionContext.getContextToRecord(), this, BatchContext::new, 0, taskCount, - idx -> { - try (final RowSequence.Iterator rsIt = inputRowSet.getRowSequenceIterator(); - final ChunkSink.FillFromContext ffc = - outputSource.makeFillFromContext(PARALLEL_CACHE_CHUNK_SIZE); - final ChunkSource.GetContext gc = - inputSource.makeGetContext(PARALLEL_CACHE_CHUNK_SIZE)) { - // advance to the first key of this block - rsIt.advance(inputRowSet.get(idx * PARALLEL_CACHE_BATCH_SIZE)); - int remaining = PARALLEL_CACHE_BATCH_SIZE; - while (rsIt.hasMore() && remaining > 0) { - final RowSequence chunkOk = rsIt - .getNextRowSequenceWithLength(Math.min(remaining, PARALLEL_CACHE_CHUNK_SIZE)); - final Chunk values = inputSource.getChunk(gc, chunkOk); - outputSource.fillFromChunk(ffc, values, chunkOk); - - // reduce by the attempted stride, if this is the final block the iterator will - // be exhausted and hasMore() will return false - remaining -= PARALLEL_CACHE_CHUNK_SIZE; - } + (ctx, idx) -> { + // advance to the first key of this block + ctx.rsIt.advance(inputRowSet.get(idx * PARALLEL_CACHE_BATCH_SIZE)); + int remaining = PARALLEL_CACHE_BATCH_SIZE; + while (ctx.rsIt.hasMore() && remaining > 0) { + final RowSequence chunkOk = ctx.rsIt + .getNextRowSequenceWithLength(Math.min(remaining, PARALLEL_CACHE_CHUNK_SIZE)); + final Chunk values = inputSource.getChunk(ctx.gc, chunkOk); + outputSource.fillFromChunk(ctx.ffc, values, chunkOk); + + // reduce by the attempted stride, if this is the final block the iterator will + // be exhausted and hasMore() will return false + remaining -= PARALLEL_CACHE_CHUNK_SIZE; } }, resumeAction::run, this::onError); @@ -472,8 +479,9 @@ private void cacheInputSources(final int winIdx, final Runnable resumeAction) { final UpdateByWindow win = windows[winIdx]; final int[] uniqueWindowSources = win.getUniqueSourceIndices(); - jobScheduler.iterateParallel(ExecutionContext.getContextToRecord(), this, 0, uniqueWindowSources.length, - (idx, sourceComplete) -> { + jobScheduler.iterateParallel(ExecutionContext.getContextToRecord(), this, JobScheduler.JobContext::new, + 0, uniqueWindowSources.length, + (context, idx, sourceComplete) -> { createCachedColumnSource(uniqueWindowSources[idx], sourceComplete); }, resumeAction, this::onError); } else { @@ -489,8 +497,9 @@ private void cacheInputSources(final int winIdx, final Runnable resumeAction) { private void processWindowBuckets(int winIdx, final Runnable resumeAction) { if (jobScheduler.threadCount() > 1 && dirtyBuckets.length > 1) { // process the buckets in parallel - jobScheduler.iterateParallel(ExecutionContext.getContextToRecord(), this, 0, dirtyBuckets.length, - bucketIdx -> { + jobScheduler.iterateParallel(ExecutionContext.getContextToRecord(), this, JobScheduler.JobContext::new, + 0, dirtyBuckets.length, + (context, bucketIdx) -> { UpdateByBucketHelper bucket = dirtyBuckets[bucketIdx]; bucket.assignInputSources(winIdx, maybeCachedInputSources); bucket.processWindow(winIdx, initialStep); @@ -511,8 +520,9 @@ private void processWindowBuckets(int winIdx, final Runnable resumeAction) { * cached columns before starting the next window. Calls {@code completedAction} when the work is complete */ private void processWindows(final Runnable resumeAction) { - jobScheduler.iterateSerial(ExecutionContext.getContextToRecord(), this, 0, windows.length, - (winIdx, windowComplete) -> { + jobScheduler.iterateSerial(ExecutionContext.getContextToRecord(), this, JobScheduler.JobContext::new, 0, + windows.length, + (context, winIdx, windowComplete) -> { UpdateByWindow win = windows[winIdx]; // this is a chain of calls: cache, then shift, then process the dirty buckets for this window @@ -854,8 +864,7 @@ public static Table updateBy(@NotNull final QueryTable source, String timestampColumnName = null; final Set problems = new LinkedHashSet<>(); final Map> opResultSources = new LinkedHashMap<>(); - for (int opIdx = 0; opIdx < opArr.length; opIdx++) { - final UpdateByOperator op = opArr[opIdx]; + for (final UpdateByOperator op : opArr) { op.getOutputColumns().forEach((name, col) -> { if (opResultSources.putIfAbsent(name, col) != null) { problems.add(name); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java index c4f2cd6269e..b321d0fcfcf 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java @@ -53,6 +53,10 @@ class UpdateByBucketHelper extends IntrusiveDoublyLinkedNode.Impl ssaValues = WritableLongChunk.makeWritableChunk(chunkSize); + final WritableLongChunk ssaKeys = WritableLongChunk.makeWritableChunk(chunkSize); + final WritableLongChunk nullTsKeys = WritableLongChunk.makeWritableChunk(chunkSize)) { + final RowSet restampAdditions = stampModified ? addedAndModified : upstream.added(); final RowSet restampRemovals = stampModified ? removedAndModifiedPreShift : upstream.removed(); // removes if (restampRemovals.isNonempty()) { - final int size = (int) Math.min(restampRemovals.size(), 4096); - try (final RowSequence.Iterator it = restampRemovals.getRowSequenceIterator(); - final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(size); - final WritableLongChunk ssaValues = WritableLongChunk.makeWritableChunk(size); - final WritableLongChunk ssaKeys = WritableLongChunk.makeWritableChunk(size)) { + try (final RowSequence.Iterator it = restampRemovals.getRowSequenceIterator()) { MutableLong lastTimestamp = new MutableLong(NULL_LONG); while (it.hasMore()) { @@ -148,18 +161,35 @@ private void processUpdateForSsa(TableUpdate upstream) { LongChunk keysChunk = chunkRs.asRowKeyChunk(); // push only non-null values/keys into the Ssa - fillChunkWithNonNull(keysChunk, valuesChunk, ssaKeys, ssaValues, lastTimestamp); + nullTimestampCount -= fillChunkWithNonNull(keysChunk, valuesChunk, ssaKeys, ssaValues, + nullTsKeys, lastTimestamp); timestampSsa.remove(ssaValues, ssaKeys); + + // if we have removed all the nulls, we can reset to mirror the source. Otherwise need to + // remove these rows from the non-null set + if (timestampValidRowSet != source.getRowSet()) { + if (nullTimestampCount == 0) { + timestampValidRowSet.close(); + timestampValidRowSet = source.getRowSet().writableCast(); + } else { + timestampValidRowSet.remove(ssaKeys, 0, ssaKeys.size()); + } + } } } } // shifts if (upstream.shifted().nonempty()) { + if (timestampValidRowSet != source.getRowSet()) { + upstream.shifted().apply(timestampValidRowSet.writableCast()); + } + final int size = Math.max( - upstream.modified().intSize() - + Math.max(upstream.added().intSize(), upstream.removed().intSize()), + upstream.modified().intSize() + Math.max(upstream.added().intSize(), + upstream.removed().intSize()), (int) upstream.shifted().getEffectiveSize()); + try (final RowSet fullPrevRowSet = source.getRowSet().copyPrev(); final WritableRowSet previousToShift = fullPrevRowSet.minus(restampRemovals); final ColumnSource.GetContext getContext = timestampColumnSource.makeGetContext(size)) { @@ -184,14 +214,10 @@ private void processUpdateForSsa(TableUpdate upstream) { // adds if (restampAdditions.isNonempty()) { - final int size = (int) Math.min(restampAdditions.size(), 4096); - try (final RowSequence.Iterator it = restampAdditions.getRowSequenceIterator(); - final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(size); - final WritableLongChunk ssaValues = WritableLongChunk.makeWritableChunk(size); - final WritableLongChunk ssaKeys = WritableLongChunk.makeWritableChunk(size)) { + try (final RowSequence.Iterator it = restampAdditions.getRowSequenceIterator()) { MutableLong lastTimestamp = new MutableLong(NULL_LONG); while (it.hasMore()) { - RowSequence chunkRs = it.getNextRowSequenceWithLength(4096); + RowSequence chunkRs = it.getNextRowSequenceWithLength(chunkSize); // get the chunks for values and keys LongChunk valuesChunk = @@ -199,8 +225,19 @@ private void processUpdateForSsa(TableUpdate upstream) { LongChunk keysChunk = chunkRs.asRowKeyChunk(); // push only non-null values/keys into the Ssa - fillChunkWithNonNull(keysChunk, valuesChunk, ssaKeys, ssaValues, lastTimestamp); + nullTimestampCount += fillChunkWithNonNull(keysChunk, valuesChunk, ssaKeys, ssaValues, + nullTsKeys, lastTimestamp); timestampSsa.insert(ssaValues, ssaKeys); + + if (timestampValidRowSet == source.getRowSet()) { + if (nullTimestampCount > 0) { + // make a copy and remove the nulls + timestampValidRowSet = source.getRowSet().copy().toTracking().writableCast(); + timestampValidRowSet.remove(nullTsKeys, 0, nullTsKeys.size()); + } + } else { + timestampValidRowSet.insert(ssaKeys, 0, ssaKeys.size()); + } } } } @@ -210,19 +247,26 @@ private void processUpdateForSsa(TableUpdate upstream) { /** * helper function to fill a LongChunk while skipping values that are NULL_LONG. Used to populate an SSA from a * source containing null values + * + * @return the number of NULL values found in the set */ - private void fillChunkWithNonNull(LongChunk keysChunk, LongChunk valuesChunk, + private int fillChunkWithNonNull(LongChunk keysChunk, LongChunk valuesChunk, WritableLongChunk ssaKeys, WritableLongChunk ssaValues, - MutableLong lastTimestamp) { + WritableLongChunk nullTimestampKeys, MutableLong lastTimestamp) { + int nullCount = 0; + // reset the insertion chunks ssaValues.setSize(0); ssaKeys.setSize(0); + nullTimestampKeys.setSize(0); // add only non-null timestamps to this Ssa for (int i = 0; i < valuesChunk.size(); i++) { long ts = valuesChunk.get(i); if (ts == NULL_LONG) { // null timestamps will not cause problems + nullTimestampKeys.add(keysChunk.get(i)); + nullCount++; continue; } if (ts < lastTimestamp.longValue()) { @@ -236,6 +280,7 @@ private void fillChunkWithNonNull(LongChunk keysChunk, LongChunk // store the current ts for comparison lastTimestamp.setValue(ts); } + return nullCount; } /** @@ -262,6 +307,7 @@ public void prepareForUpdate(final TableUpdate upstream, final boolean initialSt sourceRowSet, timestampColumnSource, timestampSsa, + timestampValidRowSet, control.chunkCapacityOrDefault(), initialStep); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java index bff344b2028..b97ba7df9e8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java @@ -39,6 +39,8 @@ class UpdateByWindowBucketContext implements SafeCloseable { /** The timestamp SSA providing fast lookup for time windows */ @Nullable protected final LongSegmentedSortedArray timestampSsa; + /** This rowset will store row keys where the timestamp is not null (will mirror the SSA contents) */ + protected final TrackingRowSet timestampValidRowSet; /** An array of context objects for each underlying operator */ protected final UpdateByOperator.UpdateContext[] opContext; /** Whether this is the creation phase of this window */ @@ -66,11 +68,13 @@ class UpdateByWindowBucketContext implements SafeCloseable { UpdateByWindowBucketContext(final TrackingRowSet sourceRowSet, @Nullable final ColumnSource timestampColumnSource, @Nullable final LongSegmentedSortedArray timestampSsa, + final TrackingRowSet timestampValidRowSet, final int chunkSize, final boolean initialStep) { this.sourceRowSet = sourceRowSet; this.timestampColumnSource = timestampColumnSource; this.timestampSsa = timestampSsa; + this.timestampValidRowSet = timestampValidRowSet; this.opContext = new UpdateByOperator.UpdateContext[operators.length]; @@ -94,6 +98,7 @@ public void close() { abstract UpdateByWindowBucketContext makeWindowContext(final TrackingRowSet sourceRowSet, final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa, + final TrackingRowSet timestampValidRowSet, final int chunkSize, final boolean isInitializeStep); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java index d2fb9e80057..8e9f2b9d04d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java @@ -40,14 +40,15 @@ private void makeOperatorContexts(UpdateByWindowBucketContext context) { } } + @Override UpdateByWindowBucketContext makeWindowContext(final TrackingRowSet sourceRowSet, final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa, + final TrackingRowSet timestampValidRowSet, final int chunkSize, final boolean isInitializeStep) { - return new UpdateByWindowBucketContext(sourceRowSet, timestampColumnSource, timestampSsa, - chunkSize, - isInitializeStep); + return new UpdateByWindowBucketContext(sourceRowSet, timestampColumnSource, timestampSsa, timestampValidRowSet, + chunkSize, isInitializeStep); } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java index c63bbdafa5d..bcbfee3f292 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java @@ -2,7 +2,6 @@ import gnu.trove.list.array.TIntArrayList; import gnu.trove.set.hash.TIntHashSet; -import io.deephaven.base.ringbuffer.LongRingBuffer; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; @@ -28,18 +27,20 @@ * of `influencer` values to add to the rolling window as the current row changes. */ class UpdateByWindowTicks extends UpdateByWindow { + /** growth rate after the contexts have exceeded the poolable chunk size */ + public static final double CONTEXT_GROWTH_PERCENTAGE = 0.25; private static final int WINDOW_CHUNK_SIZE = 4096; private final long prevUnits; private final long fwdUnits; - class UpdateByWindowBucketTicksContext extends UpdateByWindowBucketContext { + class UpdateByWindowTicksBucketContext extends UpdateByWindowBucketContext { private RowSet affectedRowPositions; private RowSet influencerPositions; private int currentGetContextSize; - UpdateByWindowBucketTicksContext(final TrackingRowSet sourceRowSet, + UpdateByWindowTicksBucketContext(final TrackingRowSet sourceRowSet, final int chunkSize, final boolean initialStep) { - super(sourceRowSet, null, null, chunkSize, initialStep); + super(sourceRowSet, null, null, null, chunkSize, initialStep); } @Override @@ -59,7 +60,7 @@ public void close() { } private void makeOperatorContexts(UpdateByWindowBucketContext context) { - UpdateByWindowBucketTicksContext ctx = (UpdateByWindowBucketTicksContext) context; + UpdateByWindowTicksBucketContext ctx = (UpdateByWindowTicksBucketContext) context; ctx.workingChunkSize = WINDOW_CHUNK_SIZE; ctx.currentGetContextSize = ctx.workingChunkSize; @@ -74,13 +75,14 @@ private void makeOperatorContexts(UpdateByWindowBucketContext context) { UpdateByWindowBucketContext makeWindowContext(final TrackingRowSet sourceRowSet, final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa, + final TrackingRowSet timestampValidRowSet, final int chunkSize, final boolean isInitializeStep) { - return new UpdateByWindowBucketTicksContext(sourceRowSet, chunkSize, isInitializeStep); + return new UpdateByWindowTicksBucketContext(sourceRowSet, chunkSize, isInitializeStep); } - private static WritableRowSet computeAffectedRowsTicks(final RowSet sourceSet, final RowSet subset, - final RowSet invertedSubSet, long revTicks, long fwdTicks) { + private static WritableRowSet computeAffectedRowsTicks(final RowSet sourceSet, final RowSet invertedSubSet, + long revTicks, long fwdTicks) { // adjust fwd/rev to get the affected windows // Potential cases and reasoning: @@ -91,11 +93,11 @@ private static WritableRowSet computeAffectedRowsTicks(final RowSet sourceSet, f // exclude the current row) // 5) rev 10, fwd 50 - affected should be 51, 9 - return computeInfluencerRowsTicks(sourceSet, subset, invertedSubSet, fwdTicks + 1, revTicks - 1); + return computeInfluencerRowsTicks(sourceSet, invertedSubSet, fwdTicks + 1, revTicks - 1); } - private static WritableRowSet computeInfluencerRowsTicks(final RowSet sourceSet, final RowSet subset, - final RowSet invertedSubSet, long revTicks, long fwdTicks) { + private static WritableRowSet computeInfluencerRowsTicks(final RowSet sourceSet, final RowSet invertedSubSet, + long revTicks, long fwdTicks) { long maxPos = sourceSet.size() - 1; final RowSetBuilderSequential builder = RowSetFactory.builderSequential(); @@ -113,17 +115,19 @@ private static WritableRowSet computeInfluencerRowsTicks(final RowSet sourceSet, } } - private void ensureGetContextSize(UpdateByWindowBucketTicksContext ctx, long newSize) { + private void ensureGetContextSize(UpdateByWindowTicksBucketContext ctx, long newSize) { if (ctx.currentGetContextSize < newSize) { long size = ctx.currentGetContextSize; while (size < newSize) { size *= 2; } - // if size would no longer be poolable, use the exact size for the new contexts + // if size would no longer be poolable, use percentage growth for the new contexts ctx.currentGetContextSize = LongSizedDataStructure.intSize( "ensureGetContextSize exceeded Integer.MAX_VALUE", - size >= ChunkPoolConstants.LARGEST_POOLED_CHUNK_CAPACITY ? newSize : size); + size >= ChunkPoolConstants.LARGEST_POOLED_CHUNK_CAPACITY + ? (long) (newSize * (1.0 + CONTEXT_GROWTH_PERCENTAGE)) + : size); // use this to track which contexts have already resized final boolean[] resized = new boolean[ctx.inputSources.length]; @@ -154,7 +158,7 @@ private void ensureGetContextSize(UpdateByWindowBucketTicksContext ctx, long new @Override void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, @NotNull TableUpdate upstream) { - UpdateByWindowBucketTicksContext ctx = (UpdateByWindowBucketTicksContext) context; + UpdateByWindowTicksBucketContext ctx = (UpdateByWindowTicksBucketContext) context; if (upstream.empty() || context.sourceRowSet.isEmpty()) { return; @@ -227,7 +231,7 @@ void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, @NotNu try (final WritableRowSet changed = upstream.added().union(upstream.modified())) { // compute the rows affected from these changes try (final WritableRowSet changedInverted = ctx.sourceRowSet.invert(changed)) { - tmpAffected = computeAffectedRowsTicks(ctx.sourceRowSet, changed, changedInverted, prevUnits, fwdUnits); + tmpAffected = computeAffectedRowsTicks(ctx.sourceRowSet, changedInverted, prevUnits, fwdUnits); } } @@ -236,7 +240,7 @@ void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, @NotNu try (final RowSet prev = ctx.sourceRowSet.copyPrev(); final RowSet removedPositions = prev.invert(upstream.removed()); final WritableRowSet affectedByRemoves = - computeAffectedRowsTicks(prev, upstream.removed(), removedPositions, prevUnits, + computeAffectedRowsTicks(prev, removedPositions, prevUnits, fwdUnits)) { // apply shifts to get back to pos-shift space upstream.shifted().apply(affectedByRemoves); @@ -255,7 +259,7 @@ void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, @NotNu // generate position data rowsets for efficiently computed position offsets ctx.affectedRowPositions = ctx.sourceRowSet.invert(ctx.affectedRows); - ctx.influencerRows = computeInfluencerRowsTicks(ctx.sourceRowSet, ctx.affectedRows, ctx.affectedRowPositions, + ctx.influencerRows = computeInfluencerRowsTicks(ctx.sourceRowSet, ctx.affectedRowPositions, prevUnits, fwdUnits); ctx.influencerPositions = ctx.sourceRowSet.invert(ctx.influencerRows); @@ -265,7 +269,7 @@ void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, @NotNu @Override void processRows(UpdateByWindowBucketContext context, boolean initialStep) { - UpdateByWindowBucketTicksContext ctx = (UpdateByWindowBucketTicksContext) context; + UpdateByWindowTicksBucketContext ctx = (UpdateByWindowTicksBucketContext) context; Assert.neqNull(context.inputSources, "assignInputSources() must be called before processRow()"); @@ -287,6 +291,9 @@ void processRows(UpdateByWindowBucketContext context, boolean initialStep) { final long sourceRowSetSize = ctx.sourceRowSet.size(); + // chunk processing + long totalCount = 0; + while (it.hasMore()) { final RowSequence chunkRs = it.getNextRowSequenceWithLength(ctx.workingChunkSize); final RowSequence chunkPosRs = posIt.getNextRowSequenceWithLength(ctx.workingChunkSize); @@ -294,9 +301,6 @@ void processRows(UpdateByWindowBucketContext context, boolean initialStep) { final LongChunk posChunk = chunkPosRs.asRowKeyChunk(); - // we are going to track all the influencer rows that affect this chunk of data - final RowSetBuilderSequential chunkInfluencerBuilder = RowSetFactory.builderSequential(); - // chunk processing for (int ii = 0; ii < chunkRsSize; ii++) { // read the current position @@ -311,37 +315,37 @@ void processRows(UpdateByWindowBucketContext context, boolean initialStep) { // push in all values that are in the new window (inclusive of tail) long pushCount = influencerPosTailIt.advanceAndGetPositionDistance(tail + 1); - chunkInfluencerBuilder.appendRowSequence(influencerKeyIt.getNextRowSequenceWithLength(pushCount)); // write the push and pop counts to the chunks popChunk.set(ii, Math.toIntExact(popCount)); pushChunk.set(ii, Math.toIntExact(pushCount)); + + totalCount += pushCount; } // execute the operators - try (final RowSet chunkInfluencerRs = chunkInfluencerBuilder.build()) { - ensureGetContextSize(ctx, chunkInfluencerRs.size()); - - Arrays.fill(ctx.inputSourceChunks, null); - for (int opIdx : context.dirtyOperatorIndices) { - // prep the chunk array needed by the accumulate call - final int[] srcIndices = operatorInputSourceSlots[opIdx]; - Chunk[] chunkArr = new Chunk[srcIndices.length]; - for (int ii = 0; ii < srcIndices.length; ii++) { - int srcIdx = srcIndices[ii]; - // chunk prep - prepareValuesChunkForSource(ctx, srcIdx, chunkInfluencerRs); - chunkArr[ii] = ctx.inputSourceChunks[srcIdx]; - } - - // make the specialized call for windowed operators - ((UpdateByWindowedOperator.Context) ctx.opContext[opIdx]).accumulate( - chunkRs, - chunkArr, - pushChunk, - popChunk, - chunkRsSize); + final RowSequence chunkInfluencerRs = influencerKeyIt.getNextRowSequenceWithLength(totalCount); + ensureGetContextSize(ctx, chunkInfluencerRs.size()); + + Arrays.fill(ctx.inputSourceChunks, null); + for (int opIdx : context.dirtyOperatorIndices) { + // prep the chunk array needed by the accumulate call + final int[] srcIndices = operatorInputSourceSlots[opIdx]; + Chunk[] chunkArr = new Chunk[srcIndices.length]; + for (int ii = 0; ii < srcIndices.length; ii++) { + int srcIdx = srcIndices[ii]; + // chunk prep + prepareValuesChunkForSource(ctx, srcIdx, chunkInfluencerRs); + chunkArr[ii] = ctx.inputSourceChunks[srcIdx]; } + + // make the specialized call for windowed operators + ((UpdateByWindowedOperator.Context) ctx.opContext[opIdx]).accumulate( + chunkRs, + chunkArr, + pushChunk, + popChunk, + chunkRsSize); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java index 5bc9f098f2b..2fad697ad84 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java @@ -7,6 +7,7 @@ import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.chunk.util.pools.ChunkPoolConstants; import io.deephaven.engine.rowset.*; import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.ColumnSource; @@ -30,18 +31,23 @@ * a buffer of `influencer` values to add to the rolling window as the current row changes. */ class UpdateByWindowTime extends UpdateByWindow { + /** growth rate after the contexts have exceeded the poolable chunk size */ + public static final double CONTEXT_GROWTH_PERCENTAGE = 0.25; private static final int WINDOW_CHUNK_SIZE = 4096; protected final long prevUnits; protected final long fwdUnits; - public class UpdateByWindowBucketTimeContext extends UpdateByWindowBucketContext { + public class UpdateByWindowTimeBucketContext extends UpdateByWindowBucketContext { protected final ChunkSource.GetContext influencerTimestampContext; protected int currentGetContextSize; - public UpdateByWindowBucketTimeContext(final TrackingRowSet sourceRowSet, + public UpdateByWindowTimeBucketContext(final TrackingRowSet sourceRowSet, @NotNull final ColumnSource timestampColumnSource, - @Nullable final LongSegmentedSortedArray timestampSsa, final int chunkSize, final boolean initialStep) { - super(sourceRowSet, timestampColumnSource, timestampSsa, chunkSize, initialStep); + @Nullable final LongSegmentedSortedArray timestampSsa, + final TrackingRowSet timestampValidRowSet, + final int chunkSize, + final boolean initialStep) { + super(sourceRowSet, timestampColumnSource, timestampSsa, timestampValidRowSet, chunkSize, initialStep); influencerTimestampContext = timestampColumnSource.makeGetContext(WINDOW_CHUNK_SIZE); } @@ -64,7 +70,7 @@ public void close() { } protected void makeOperatorContexts(UpdateByWindowBucketContext context) { - UpdateByWindowBucketTimeContext ctx = (UpdateByWindowBucketTimeContext) context; + UpdateByWindowTimeBucketContext ctx = (UpdateByWindowTimeBucketContext) context; ctx.workingChunkSize = WINDOW_CHUNK_SIZE; ctx.currentGetContextSize = ctx.workingChunkSize; @@ -75,13 +81,15 @@ protected void makeOperatorContexts(UpdateByWindowBucketContext context) { } } + @Override public UpdateByWindowBucketContext makeWindowContext(final TrackingRowSet sourceRowSet, final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa, + final TrackingRowSet timestampValidRowSet, final int chunkSize, final boolean isInitializeStep) { - return new UpdateByWindowBucketTimeContext(sourceRowSet, timestampColumnSource, timestampSsa, chunkSize, - isInitializeStep); + return new UpdateByWindowTimeBucketContext(sourceRowSet, timestampColumnSource, timestampSsa, + timestampValidRowSet, chunkSize, isInitializeStep); } /** @@ -152,15 +160,19 @@ private static WritableRowSet computeInfluencerRowsTime(final RowSet sourceSet, } } - private void ensureGetContextSize(UpdateByWindowBucketTimeContext ctx, long newSize) { + private void ensureGetContextSize(UpdateByWindowTimeBucketContext ctx, long newSize) { if (ctx.currentGetContextSize < newSize) { long size = ctx.currentGetContextSize; while (size < newSize) { size *= 2; } + + // if size would no longer be poolable, use percentage growth for the new contexts ctx.currentGetContextSize = LongSizedDataStructure.intSize( "ensureGetContextSize exceeded Integer.MAX_VALUE", - size); + size >= ChunkPoolConstants.LARGEST_POOLED_CHUNK_CAPACITY + ? (long) (newSize * (1.0 + CONTEXT_GROWTH_PERCENTAGE)) + : size); // use this to track which contexts have already resized boolean[] resized = new boolean[ctx.inputSources.length]; @@ -188,7 +200,7 @@ private void ensureGetContextSize(UpdateByWindowBucketTimeContext ctx, long newS // determine which rows will be needed to compute new values for the affected rows (influencer rows) @Override public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, @NotNull TableUpdate upstream) { - UpdateByWindowBucketTimeContext ctx = (UpdateByWindowBucketTimeContext) context; + UpdateByWindowTimeBucketContext ctx = (UpdateByWindowTimeBucketContext) context; if (upstream.empty() || ctx.sourceRowSet.isEmpty()) { return; @@ -197,7 +209,11 @@ public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, // all rows are affected on the initial step if (ctx.initialStep) { ctx.affectedRows = ctx.sourceRowSet; - ctx.influencerRows = ctx.affectedRows; + // only non-null timestamps are actual influencers. Note that this is fast rather than precise. This set + // of rows may contain rows not needed for computation. E.g. when fwd and rev are both non-zero, we could + // eliminate rows at either the beginning or end of the set, but not convinced ROI > 0 and these rows are + // properly skipped by window creation + ctx.influencerRows = ctx.timestampValidRowSet.copy(); // mark all operators as affected by this update context.dirtyOperatorIndices = IntStream.range(0, operators.length).toArray(); @@ -239,16 +255,18 @@ public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, return; } - // changed rows are all mods+adds - WritableRowSet changed = upstream.added().union(upstream.modified()); + final WritableRowSet tmpAffected; - // need a writable rowset - WritableRowSet tmpAffected = computeAffectedRowsTime(ctx.sourceRowSet, changed, prevUnits, fwdUnits, - ctx.timestampColumnSource, ctx.timestampSsa, false); + // changed rows are all mods+adds + try (WritableRowSet changed = upstream.added().union(upstream.modified())) { + // need a writable rowset + tmpAffected = computeAffectedRowsTime(ctx.timestampValidRowSet, changed, prevUnits, fwdUnits, + ctx.timestampColumnSource, ctx.timestampSsa, false); + } // other rows can be affected by removes or mods if (upstream.removed().isNonempty()) { - try (final RowSet prev = ctx.sourceRowSet.copyPrev(); + try (final RowSet prev = ctx.timestampValidRowSet.copyPrev(); final WritableRowSet affectedByRemoves = computeAffectedRowsTime(prev, upstream.removed(), prevUnits, fwdUnits, ctx.timestampColumnSource, ctx.timestampSsa, true); @@ -257,8 +275,8 @@ public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, ctx.timestampColumnSource, ctx.timestampSsa, true)) { // we used the SSA (post-shift) to get these keys, no need to shift // retain only the rows that still exist in the sourceRowSet - affectedByRemoves.retain(ctx.sourceRowSet); - affectedByModifies.retain(ctx.sourceRowSet); + affectedByRemoves.retain(ctx.timestampValidRowSet); + affectedByModifies.retain(ctx.timestampValidRowSet); tmpAffected.insert(affectedByRemoves); tmpAffected.insert(affectedByModifies); @@ -271,18 +289,14 @@ public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, ctx.affectedRows = tmpAffected; // now get influencer rows for the affected rows - ctx.influencerRows = computeInfluencerRowsTime(ctx.sourceRowSet, ctx.affectedRows, prevUnits, fwdUnits, + ctx.influencerRows = computeInfluencerRowsTime(ctx.timestampValidRowSet, ctx.affectedRows, prevUnits, fwdUnits, ctx.timestampColumnSource, ctx.timestampSsa, false); makeOperatorContexts(ctx); } - private long advanceToNonNull(LongColumnIterator it) { - long value; - do { - value = it.hasNext() ? it.nextLong() : Long.MAX_VALUE; - } while (value == NULL_LONG); - return value; + private long nextLongOrMax(LongColumnIterator it) { + return it.hasNext() ? it.nextLong() : Long.MAX_VALUE; } /*** @@ -296,7 +310,7 @@ private long advanceToNonNull(LongColumnIterator it) { */ @Override public void processRows(UpdateByWindowBucketContext context, boolean initialStep) { - UpdateByWindowBucketTimeContext ctx = (UpdateByWindowBucketTimeContext) context; + UpdateByWindowTimeBucketContext ctx = (UpdateByWindowTimeBucketContext) context; for (int opIdx : context.dirtyOperatorIndices) { UpdateByWindowedOperator winOp = (UpdateByWindowedOperator) operators[opIdx]; @@ -305,8 +319,10 @@ public void processRows(UpdateByWindowBucketContext context, boolean initialStep } try (final RowSequence.Iterator it = ctx.affectedRows.getRowSequenceIterator(); - final LongColumnIterator influencerTsHeadIt = new LongColumnIterator(context.timestampColumnSource, context.influencerRows); - final LongColumnIterator influencerTsTailIt = new LongColumnIterator(context.timestampColumnSource, context.influencerRows); + final LongColumnIterator influencerTsHeadIt = + new LongColumnIterator(context.timestampColumnSource, context.influencerRows); + final LongColumnIterator influencerTsTailIt = + new LongColumnIterator(context.timestampColumnSource, context.influencerRows); final RowSequence.Iterator influencerKeyIt = ctx.influencerRows.getRowSequenceIterator(); final ChunkSource.GetContext localTimestampContext = ctx.timestampColumnSource.makeGetContext(ctx.workingChunkSize); @@ -315,8 +331,8 @@ public void processRows(UpdateByWindowBucketContext context, boolean initialStep final WritableIntChunk popChunk = WritableIntChunk.makeWritableChunk(ctx.workingChunkSize)) { - long currentHeadTs = advanceToNonNull(influencerTsHeadIt); - long currentTailTs = advanceToNonNull(influencerTsTailIt); + long currentHeadTs = nextLongOrMax(influencerTsHeadIt); + long currentTailTs = nextLongOrMax(influencerTsTailIt); while (it.hasMore()) { final RowSequence chunkRs = it.getNextRowSequenceWithLength(ctx.workingChunkSize); @@ -347,14 +363,14 @@ public void processRows(UpdateByWindowBucketContext context, boolean initialStep // pop out all values from the current window that are not in the new window long popCount = 0; while (currentHeadTs < head) { - currentHeadTs = advanceToNonNull(influencerTsHeadIt); + currentHeadTs = nextLongOrMax(influencerTsHeadIt); popCount++; } // push in all values that are in the new window (inclusive of tail) long pushCount = 0; while (currentTailTs <= tail) { - currentTailTs = advanceToNonNull(influencerTsTailIt); + currentTailTs = nextLongOrMax(influencerTsTailIt); pushCount++; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java index 7bbe034f877..a5e8ebedd0e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java @@ -2,6 +2,7 @@ import io.deephaven.base.log.LogOutputAppendable; import io.deephaven.engine.context.ExecutionContext; +import io.deephaven.engine.table.Context; import io.deephaven.engine.table.impl.perf.BasePerformanceEntry; import io.deephaven.util.annotations.FinalDefault; import io.deephaven.util.referencecounting.ReferenceCounted; @@ -9,6 +10,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; +import java.util.function.Supplier; /** * An interface for submitting jobs to be executed. Submitted jobs may be executed on the current thread, or in separate @@ -16,6 +18,14 @@ * thread for inclusion in overall task metrics. */ public interface JobScheduler { + + /** + * A default context for the scheduled job actions. Override this to provide reusable resources for the serial and + * parallel iterate actions. + */ + class JobContext implements Context { + } + /** * Cause runnable to be executed. * @@ -48,8 +58,8 @@ void submit( * schedule the next iteration. */ @FunctionalInterface - interface IterateAction { - void run(int index); + interface IterateAction { + void run(CONTEXT_TYPE taskThreadContext, int index); } /** @@ -62,28 +72,31 @@ interface IterateAction { * will not block the scheduler, but the {@code completeAction} {@link Runnable} will never be called. */ @FunctionalInterface - interface IterateResumeAction { - void run(int index, Runnable resume); + interface IterateResumeAction { + void run(CONTEXT_TYPE taskThreadContext, int index, Runnable resume); } - class ErrorAccounter extends ReferenceCounted implements Consumer, Runnable { - private final AtomicReference exception = new AtomicReference<>(); + class ErrorAccounter extends ReferenceCounted + implements Consumer, Runnable { + private final Supplier taskThreadContextFactory; + private final int start; + private final int count; private final Consumer finalErrorConsumer; - private final IterateResumeAction action; - private final Runnable completeAction; - private final Runnable resumeAction; + private final IterateResumeAction action; private final AtomicInteger nextIndex; private final AtomicInteger remaining; - private final int start; - private final int count; + private final Runnable resumeAction; - ErrorAccounter(final int start, final int count, final Consumer finalErrorConsumer, - final IterateResumeAction action, final Runnable completeAction) { + private final AtomicReference exception = new AtomicReference<>(); + + ErrorAccounter(final Supplier taskThreadContextFactory, + final int start, final int count, final Consumer finalErrorConsumer, + final IterateResumeAction action, final Runnable completeAction) { + this.taskThreadContextFactory = taskThreadContextFactory; this.start = start; this.count = count; this.finalErrorConsumer = finalErrorConsumer; this.action = action; - this.completeAction = completeAction; nextIndex = new AtomicInteger(start); remaining = new AtomicInteger(count); @@ -109,29 +122,30 @@ protected void onReferenceCountAtZero() { @Override public void accept(Exception e) { exception.compareAndSet(null, e); + decrementReferenceCount(); } @Override public void run() { - while (true) { - try { + try (final CONTEXT_TYPE taskThreadContext = taskThreadContextFactory.get()) { + while (true) { if (exception.get() != null) { - - incrementReferenceCount(); return; } - int idx = nextIndex.getAndIncrement(); - if (idx < start + count) { - // do the work - action.run(idx, resumeAction); - incrementReferenceCount(); - } else { - // no more work to do - incrementReferenceCount(); + final int idx = nextIndex.getAndIncrement(); + if (idx >= start + count) { + return; + } + if (!tryIncrementReferenceCount()) { + // We raced with the exception consumer return; } - } finally { - decrementReferenceCount(); + try { + // do the work + action.run(taskThreadContext, idx, resumeAction); + } finally { + decrementReferenceCount(); + } } } } @@ -142,6 +156,7 @@ public void run() { * * @param executionContext the execution context for this task * @param description the description to use for logging + * @param taskThreadContextFactory the factory that supplies {@link JobContext contexts} for the tasks * @param start the integer value from which to start iterating * @param count the number of times this task should be called * @param action the task to perform, the current iteration index is provided as a parameter @@ -149,11 +164,18 @@ public void run() { * @param onError error handler for the scheduler to use while iterating */ @FinalDefault - default void iterateParallel(ExecutionContext executionContext, LogOutputAppendable description, int start, - int count, IterateAction action, Runnable completeAction, Consumer onError) { - iterateParallel(executionContext, description, start, count, - (final int idx, final Runnable resume) -> { - action.run(idx); + default void iterateParallel( + ExecutionContext executionContext, + LogOutputAppendable description, + Supplier taskThreadContextFactory, + int start, + int count, + IterateAction action, + Runnable completeAction, + Consumer onError) { + iterateParallel(executionContext, description, taskThreadContextFactory, start, count, + (final CONTEXT_TYPE taskThreadContext, final int idx, final Runnable resume) -> { + action.run(taskThreadContext, idx); resume.run(); }, completeAction, onError); @@ -167,6 +189,7 @@ default void iterateParallel(ExecutionContext executionContext, LogOutputAppenda * * @param executionContext the execution context for this task * @param description the description to use for logging + * @param taskThreadContextFactory the factory that supplies {@link JobContext contexts} for the tasks * @param start the integer value from which to start iterating * @param count the number of times this task should be called * @param action the task to perform, the current iteration index and a resume Runnable are parameters @@ -174,40 +197,23 @@ default void iterateParallel(ExecutionContext executionContext, LogOutputAppenda * @param onError error handler for the scheduler to use while iterating */ @FinalDefault - default void iterateParallel(ExecutionContext executionContext, LogOutputAppendable description, int start, - int count, IterateResumeAction action, Runnable completeAction, Consumer onError) { + default void iterateParallel( + ExecutionContext executionContext, + LogOutputAppendable description, + Supplier taskThreadContextFactory, + int start, + int count, + IterateResumeAction action, + Runnable completeAction, + Consumer onError) { if (count == 0) { // no work to do completeAction.run(); } - final ErrorAccounter ea = new ErrorAccounter(start, count, onError, action, completeAction); - - // final AtomicBoolean cancelRemainingExecution = new AtomicBoolean(false); - - // final Consumer localError = exception -> { - // // signal only on the first error - // if (cancelRemainingExecution.compareAndSet(false, true)) { - // onError.accept(exception); - // } - // }; - - // final Runnable resumeAction = () -> { - // // check for completion - // if (remaining.decrementAndGet() == 0) { - // completeAction.run(); - // } - // }; - - // final Runnable task = () -> { - // // this will run until all tasks have started - // while (true) { - // if (cancelRemainingExecution.get()) { - // return; - // } - // } - // }; + final ErrorAccounter ea = + new ErrorAccounter<>(taskThreadContextFactory, start, count, onError, action, completeAction); // create multiple tasks but not more than one per scheduler thread for (int i = 0; i < Math.min(count, threadCount()); i++) { @@ -226,6 +232,7 @@ default void iterateParallel(ExecutionContext executionContext, LogOutputAppenda * * @param executionContext the execution context for this task * @param description the description to use for logging + * @param taskThreadContextFactory the factory that supplies {@link JobContext contexts} for the tasks * @param start the integer value from which to start iterating * @param count the number of times this task should be called * @param action the task to perform, the current iteration index and a resume Runnable are parameters @@ -233,14 +240,28 @@ default void iterateParallel(ExecutionContext executionContext, LogOutputAppenda * @param onError error handler for the scheduler to use while iterating */ @FinalDefault - default void iterateSerial(ExecutionContext executionContext, LogOutputAppendable description, int start, - int count, IterateResumeAction action, Runnable completeAction, Consumer onError) { + default void iterateSerial(ExecutionContext executionContext, + LogOutputAppendable description, + Supplier taskThreadContextFactory, + int start, + int count, + IterateResumeAction action, + Runnable completeAction, + Consumer onError) { if (count == 0) { // no work to do completeAction.run(); } + // create a single execution context for all iterations + final CONTEXT_TYPE taskThreadContext = taskThreadContextFactory.get(); + + final Consumer localError = exception -> { + taskThreadContext.close(); + onError.accept(exception); + }; + // no lambda, need the `this` reference to re-execute final Runnable resumeAction = new Runnable() { int nextIndex = start + 1; @@ -250,6 +271,7 @@ default void iterateSerial(ExecutionContext executionContext, LogOutputAppendabl public void run() { // check for completion if (--remaining == 0) { + taskThreadContext.close(); completeAction.run(); } else { @@ -257,13 +279,11 @@ public void run() { submit(executionContext, () -> { int idx = nextIndex++; - if (idx < start + count) { - // do the work - action.run(idx, this); - } + // do the work + action.run(taskThreadContext, idx, this); }, description, - onError); + localError); } } @@ -271,8 +291,8 @@ public void run() { // create a single task submit(executionContext, - () -> action.run(start, resumeAction), + () -> action.run(taskThreadContext, start, resumeAction), description, - onError); + localError); } } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestEma.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestEma.java index 1bd3dfc4ce6..0d5abc40282 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestEma.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestEma.java @@ -1,18 +1,22 @@ package io.deephaven.engine.table.impl.updateby; +import io.deephaven.api.updateby.BadDataBehavior; import io.deephaven.api.updateby.OperationControl; import io.deephaven.api.updateby.UpdateByOperation; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.ObjectChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.context.QueryScope; +import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; -import io.deephaven.api.updateby.BadDataBehavior; import io.deephaven.engine.table.Table; -import io.deephaven.engine.testutil.EvalNugget; import io.deephaven.engine.table.impl.QueryTable; import io.deephaven.engine.table.impl.TableDefaults; import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.util.ColumnHolder; -import io.deephaven.engine.context.QueryScope; -import io.deephaven.engine.testutil.generator.TestDataGenerator; +import io.deephaven.engine.testutil.EvalNugget; import io.deephaven.engine.testutil.generator.SortedDateTimeGenerator; +import io.deephaven.engine.testutil.generator.TestDataGenerator; import io.deephaven.engine.updategraph.UpdateGraphProcessor; import io.deephaven.engine.util.TableDiff; import io.deephaven.engine.util.string.StringUtils; @@ -20,6 +24,7 @@ import io.deephaven.numerics.movingaverages.ByEma; import io.deephaven.numerics.movingaverages.ByEmaSimple; import io.deephaven.test.types.OutOfBandTest; +import io.deephaven.time.DateTime; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -29,8 +34,8 @@ import java.util.concurrent.TimeUnit; import static io.deephaven.engine.testutil.GenerateTableUpdates.generateAppends; -import static io.deephaven.engine.testutil.testcase.RefreshingTableTestCase.simulateShiftAwareStep; import static io.deephaven.engine.testutil.TstUtils.*; +import static io.deephaven.engine.testutil.testcase.RefreshingTableTestCase.simulateShiftAwareStep; import static io.deephaven.engine.util.TableTools.*; import static io.deephaven.time.DateTimeUtils.MINUTE; import static io.deephaven.time.DateTimeUtils.convertDateTime; @@ -353,6 +358,70 @@ public void testPoison() { assertTableEquals(expected, input.updateBy(UpdateByOperation.Ema(nanCtl, "ts", 10))); } + /** + * This is a hacky, inefficient way to force nulls into the timestamps while maintaining sorted-ness otherwise + */ + private class SortedIntGeneratorWithNulls extends SortedDateTimeGenerator { + final double nullFrac; + + public SortedIntGeneratorWithNulls(DateTime minTime, DateTime maxTime, double nullFrac) { + super(minTime, maxTime); + this.nullFrac = nullFrac; + } + + @Override + public Chunk populateChunk(RowSet toAdd, Random random) { + Chunk retChunk = super.populateChunk(toAdd, random); + if (nullFrac == 0.0) { + return retChunk; + } + ObjectChunk srcChunk = retChunk.asObjectChunk(); + Object[] dateArr = new Object[srcChunk.size()]; + srcChunk.copyToArray(0, dateArr, 0, dateArr.length); + + // force some entries to null + for (int ii = 0; ii < srcChunk.size(); ii++) { + if (random.nextDouble() < nullFrac) { + dateArr[ii] = null; + } + } + return ObjectChunk.chunkWrap(dateArr); + } + } + + @Test + public void testNullTimestamps() { + final CreateResult timeResult = createTestTable(100, true, false, true, 0x31313131, + new String[] {"ts"}, new TestDataGenerator[] {new SortedIntGeneratorWithNulls( + convertDateTime("2022-03-09T09:00:00.000 NY"), + convertDateTime("2022-03-09T16:30:00.000 NY"), + 0.25)}); + + final OperationControl skipControl = OperationControl.builder() + .onNullValue(BadDataBehavior.SKIP) + .onNanValue(BadDataBehavior.SKIP).build(); + + final EvalNugget[] timeNuggets = new EvalNugget[] { + new EvalNugget() { + @Override + protected Table e() { + TableDefaults base = timeResult.t; + // short timescale to make sure we trigger all the transition behavior + return base.updateBy(UpdateByOperation.Ema(skipControl, "ts", 2 * MINUTE)); + } + } + }; + final Random billy = new Random(0xB177B177); + for (int ii = 0; ii < 100; ii++) { + try { + simulateShiftAwareStep(10, billy, timeResult.t, timeResult.infos, timeNuggets); + } catch (Throwable t) { + System.out.println("Crapped out on step " + ii); + throw t; + } + } + } + // endregion // region Live Tests diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java index f17ca879ec3..b700004b7eb 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java @@ -1,13 +1,12 @@ package io.deephaven.engine.table.impl.updateby; -import io.deephaven.api.updateby.BadDataBehavior; -import io.deephaven.api.updateby.OperationControl; import io.deephaven.api.updateby.UpdateByControl; import io.deephaven.api.updateby.UpdateByOperation; import io.deephaven.engine.table.ColumnDefinition; import io.deephaven.engine.table.PartitionedTable; import io.deephaven.engine.table.Table; -import io.deephaven.engine.table.impl.*; +import io.deephaven.engine.table.impl.QueryTable; +import io.deephaven.engine.table.impl.TableDefaults; import io.deephaven.engine.testutil.EvalNugget; import io.deephaven.engine.testutil.GenerateTableUpdates; import io.deephaven.engine.testutil.TstUtils; @@ -20,7 +19,6 @@ import org.junit.Test; import org.junit.experimental.categories.Category; -import java.io.IOException; import java.math.BigDecimal; import java.math.BigInteger; import java.time.Duration; diff --git a/engine/table/src/test/java/io/deephaven/engine/util/TestJobScheduler.java b/engine/table/src/test/java/io/deephaven/engine/util/TestJobScheduler.java new file mode 100644 index 00000000000..0161c10bf43 --- /dev/null +++ b/engine/table/src/test/java/io/deephaven/engine/util/TestJobScheduler.java @@ -0,0 +1,547 @@ +package io.deephaven.engine.util; + +import io.deephaven.UncheckedDeephavenException; +import io.deephaven.base.verify.Assert; +import io.deephaven.base.verify.AssertionFailure; +import io.deephaven.engine.context.ExecutionContext; +import io.deephaven.engine.exceptions.CancellationException; +import io.deephaven.engine.table.impl.util.JobScheduler; +import io.deephaven.engine.table.impl.util.UpdateGraphProcessorJobScheduler; +import io.deephaven.engine.updategraph.UpdateGraphProcessor; +import io.deephaven.test.types.OutOfBandTest; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicInteger; + +@Category(OutOfBandTest.class) +public final class TestJobScheduler { + + @Test + public void TestParallel() { + final CompletableFuture waitForResult = new CompletableFuture<>(); + + UpdateGraphProcessor.DEFAULT.enableUnitTestMode(); + UpdateGraphProcessor.DEFAULT.resetForUnitTests(false, true, 0, 4, 10, 5); + UpdateGraphProcessor.DEFAULT.runWithinUnitTestCycle(() -> { + final boolean[] completed = new boolean[100]; + + final JobScheduler scheduler = new UpdateGraphProcessorJobScheduler(); + scheduler.iterateParallel( + ExecutionContext.getContext(), + null, + JobScheduler.JobContext::new, + 0, + 50, + (context, idx) -> { + completed[idx] = true; + }, + () -> { + // verify the set for the first 50 + for (int ii = 0; ii < 50; ii++) { + Assert.eqTrue(completed[ii], "completed[" + ii + "]"); + } + for (int ii = 50; ii < completed.length; ii++) { + Assert.eqFalse(completed[ii], "completed[" + ii + "]"); + } + waitForResult.complete(null); + }, + exception -> { + waitForResult.completeExceptionally(new AssertionFailure("unexpected error")); + }); + }); + + try { + // need to wait until this future is complete + waitForResult.get(); + } catch (InterruptedException e) { + throw new CancellationException("interrupted while processing test"); + } catch (ExecutionException e) { + if (e.getCause() instanceof RuntimeException) { + throw (RuntimeException) e.getCause(); + } else { + // rethrow the error + throw new UncheckedDeephavenException("failure while processing test", e.getCause()); + } + } + } + + @Test + public void TestParallelWithResume() { + final CompletableFuture waitForResult = new CompletableFuture<>(); + + UpdateGraphProcessor.DEFAULT.enableUnitTestMode(); + UpdateGraphProcessor.DEFAULT.resetForUnitTests(false, true, 0, 4, 10, 5); + UpdateGraphProcessor.DEFAULT.runWithinUnitTestCycle(() -> { + final boolean[] completed = new boolean[100]; + + final JobScheduler scheduler = new UpdateGraphProcessorJobScheduler(); + scheduler.iterateParallel( + ExecutionContext.getContext(), + null, + JobScheduler.JobContext::new, + 0, + 50, + (context, idx, resume) -> { + completed[idx] = true; + resume.run(); + }, + () -> { + // verify the set for the first 50 + for (int ii = 0; ii < 50; ii++) { + Assert.eqTrue(completed[ii], "completed[" + ii + "]"); + } + for (int ii = 50; ii < completed.length; ii++) { + Assert.eqFalse(completed[ii], "completed[" + ii + "]"); + } + waitForResult.complete(null); + }, + exception -> { + waitForResult.completeExceptionally(new AssertionFailure("unexpected error")); + }); + }); + + try { + // need to wait until this future is complete + waitForResult.get(); + } catch (InterruptedException e) { + throw new CancellationException("interrupted while processing test"); + } catch (ExecutionException e) { + if (e.getCause() instanceof RuntimeException) { + throw (RuntimeException) e.getCause(); + } else { + // rethrow the error + throw new UncheckedDeephavenException("failure while processing test", e.getCause()); + } + } + } + + @Test + public void TestParallelWithContext() { + final CompletableFuture waitForResult = new CompletableFuture<>(); + final AtomicInteger openCount = new AtomicInteger(0); + + UpdateGraphProcessor.DEFAULT.enableUnitTestMode(); + UpdateGraphProcessor.DEFAULT.resetForUnitTests(false, true, 0, 4, 10, 5); + UpdateGraphProcessor.DEFAULT.runWithinUnitTestCycle(() -> { + + class TestJobContext extends JobScheduler.JobContext { + TestJobContext() { + openCount.incrementAndGet(); + } + + @Override + public void close() { + openCount.decrementAndGet(); + } + } + + final boolean[] completed = new boolean[100]; + + final JobScheduler scheduler = new UpdateGraphProcessorJobScheduler(); + scheduler.iterateParallel( + ExecutionContext.getContext(), + null, + TestJobContext::new, + 0, + 50, + (context, idx, resume) -> { + // verify the type is correct + Assert.instanceOf(context, "context", TestJobContext.class); + + completed[idx] = true; + resume.run(); + }, + () -> { + // verify true for the first 50 + for (int ii = 0; ii < 50; ii++) { + Assert.eqTrue(completed[ii], "completed[" + ii + "]"); + } + // verify false for the next 50 + for (int ii = 50; ii < completed.length; ii++) { + Assert.eqFalse(completed[ii], "completed[" + ii + "]"); + } + waitForResult.complete(null); + }, + exception -> { + waitForResult.completeExceptionally(new AssertionFailure("unexpected error")); + }); + }); + + try { + // need to wait until this future is complete + waitForResult.get(); + // make sure all the contexts were closed + Assert.eqZero(openCount.get(), "openCount"); + } catch (InterruptedException e) { + throw new CancellationException("interrupted while processing test"); + } catch (ExecutionException e) { + if (e.getCause() instanceof RuntimeException) { + throw (RuntimeException) e.getCause(); + } else { + // rethrow the error + throw new UncheckedDeephavenException("failure while processing test", e.getCause()); + } + } + } + + @Test + public void TestSerialWithResume() { + final CompletableFuture waitForResult = new CompletableFuture<>(); + + UpdateGraphProcessor.DEFAULT.enableUnitTestMode(); + UpdateGraphProcessor.DEFAULT.resetForUnitTests(false, true, 0, 4, 10, 5); + UpdateGraphProcessor.DEFAULT.runWithinUnitTestCycle(() -> { + final boolean[] completed = new boolean[100]; + + final JobScheduler scheduler = new UpdateGraphProcessorJobScheduler(); + scheduler.iterateSerial( + ExecutionContext.getContext(), + null, + JobScheduler.JobContext::new, + 0, + 50, + (context, idx, resume) -> { + completed[idx] = true; + resume.run(); + }, + () -> { + // verify true for the first 50 + for (int ii = 0; ii < 50; ii++) { + Assert.eqTrue(completed[ii], "completed[" + ii + "]"); + } + // verify false for the next 50 + for (int ii = 50; ii < completed.length; ii++) { + Assert.eqFalse(completed[ii], "completed[" + ii + "]"); + } + waitForResult.complete(null); + }, + exception -> { + waitForResult.completeExceptionally(new AssertionFailure("unexpected error")); + }); + }); + + try { + // need to wait until this future is complete + waitForResult.get(); + } catch (InterruptedException e) { + throw new CancellationException("interrupted while processing test"); + } catch (ExecutionException e) { + if (e.getCause() instanceof RuntimeException) { + throw (RuntimeException) e.getCause(); + } else { + // rethrow the error + throw new UncheckedDeephavenException("failure while processing test", e.getCause()); + } + } + } + + @Test + public void TestSerialWithContext() { + final CompletableFuture waitForResult = new CompletableFuture<>(); + final AtomicInteger openCount = new AtomicInteger(0); + + UpdateGraphProcessor.DEFAULT.enableUnitTestMode(); + UpdateGraphProcessor.DEFAULT.resetForUnitTests(false, true, 0, 4, 10, 5); + UpdateGraphProcessor.DEFAULT.runWithinUnitTestCycle(() -> { + + class TestJobContext extends JobScheduler.JobContext { + TestJobContext() { + openCount.incrementAndGet(); + } + + @Override + public void close() { + openCount.decrementAndGet(); + } + } + + final boolean[] completed = new boolean[100]; + + final JobScheduler scheduler = new UpdateGraphProcessorJobScheduler(); + scheduler.iterateSerial( + ExecutionContext.getContext(), + null, + TestJobContext::new, + 0, + 50, + (context, idx, resume) -> { + // verify the type is correct + Assert.instanceOf(context, "context", TestJobContext.class); + + completed[idx] = true; + resume.run(); + }, + () -> { + // verify true for the first 50 + for (int ii = 0; ii < 50; ii++) { + Assert.eqTrue(completed[ii], "completed[" + ii + "]"); + } + // verify false for the next 50 + for (int ii = 50; ii < completed.length; ii++) { + Assert.eqFalse(completed[ii], "completed[" + ii + "]"); + } + waitForResult.complete(null); + }, + exception -> { + waitForResult.completeExceptionally(new AssertionFailure("unexpected error")); + }); + }); + + try { + // need to wait until this future is complete + waitForResult.get(); + // make sure all the contexts were closed + Assert.eqZero(openCount.get(), "openCount"); + } catch (InterruptedException e) { + throw new CancellationException("interrupted while processing test"); + } catch (ExecutionException e) { + if (e.getCause() instanceof RuntimeException) { + throw (RuntimeException) e.getCause(); + } else { + // rethrow the error + throw new UncheckedDeephavenException("failure while processing test", e.getCause()); + } + } + } + + @Test + public void TestSerialEmpty() { + final CompletableFuture waitForResult = new CompletableFuture<>(); + + UpdateGraphProcessor.DEFAULT.enableUnitTestMode(); + UpdateGraphProcessor.DEFAULT.resetForUnitTests(false, true, 0, 4, 10, 5); + UpdateGraphProcessor.DEFAULT.runWithinUnitTestCycle(() -> { + + final JobScheduler scheduler = new UpdateGraphProcessorJobScheduler(); + scheduler.iterateSerial( + ExecutionContext.getContext(), + null, + JobScheduler.JobContext::new, + 0, + 0, + (context, idx, resume) -> { + // nop + }, + () -> { + waitForResult.complete(null); + }, + exception -> { + waitForResult.completeExceptionally(new AssertionFailure("unexpected error")); + }); + }); + + try { + // need to wait until this future is complete + waitForResult.get(); + } catch (InterruptedException e) { + throw new CancellationException("interrupted while processing test"); + } catch (ExecutionException e) { + if (e.getCause() instanceof RuntimeException) { + throw (RuntimeException) e.getCause(); + } else { + // rethrow the error + throw new UncheckedDeephavenException("failure while processing test", e.getCause()); + } + } + } + + @Test + public void TestParallelEmpty() { + final CompletableFuture waitForResult = new CompletableFuture<>(); + + UpdateGraphProcessor.DEFAULT.enableUnitTestMode(); + UpdateGraphProcessor.DEFAULT.resetForUnitTests(false, true, 0, 4, 10, 5); + UpdateGraphProcessor.DEFAULT.runWithinUnitTestCycle(() -> { + + final JobScheduler scheduler = new UpdateGraphProcessorJobScheduler(); + scheduler.iterateParallel( + ExecutionContext.getContext(), + null, + JobScheduler.JobContext::new, + 0, + 0, + (context, idx, resume) -> { + // nop + }, + () -> { + waitForResult.complete(null); + }, + exception -> { + waitForResult.completeExceptionally(new AssertionFailure("unexpected error")); + }); + }); + + try { + // need to wait until this future is complete + waitForResult.get(); + } catch (InterruptedException e) { + throw new CancellationException("interrupted while processing test"); + } catch (ExecutionException e) { + if (e.getCause() instanceof RuntimeException) { + throw (RuntimeException) e.getCause(); + } else { + // rethrow the error + throw new UncheckedDeephavenException("failure while processing test", e.getCause()); + } + } + } + + @Test + public void TestParallelError() { + final CompletableFuture waitForResult = new CompletableFuture<>(); + final AtomicInteger openCount = new AtomicInteger(0); + + UpdateGraphProcessor.DEFAULT.enableUnitTestMode(); + UpdateGraphProcessor.DEFAULT.resetForUnitTests(false, true, 0, 4, 10, 5); + UpdateGraphProcessor.DEFAULT.runWithinUnitTestCycle(() -> { + final boolean[] completed = new boolean[100]; + + class TestJobContext extends JobScheduler.JobContext { + TestJobContext() { + openCount.incrementAndGet(); + } + + @Override + public void close() { + openCount.decrementAndGet(); + } + } + + final JobScheduler scheduler = new UpdateGraphProcessorJobScheduler(); + scheduler.iterateParallel( + ExecutionContext.getContext(), + null, + TestJobContext::new, + 0, + 50, + (context, idx) -> { + // verify the type is correct + Assert.instanceOf(context, "context", TestJobContext.class); + + completed[idx] = true; + // throw after this is set to make verification easy + if (idx == 10) { + throw new IndexOutOfBoundsException("test error"); + } + }, + () -> { + // if this is called, we failed the test + waitForResult + .completeExceptionally(new AssertionFailure("IndexOutOfBoundsException not thrown")); + }, + exception -> { + if (!(exception instanceof IndexOutOfBoundsException)) { + waitForResult.completeExceptionally( + new AssertionFailure("IndexOutOfBoundsException not thrown")); + } + + // assert that the job was terminated before all tasks were executed (one is still false) + for (int ii = 0; ii < 50; ii++) { + if (!completed[ii]) { + waitForResult.complete(null); + return; + } + } + waitForResult.completeExceptionally(new AssertionFailure("Tasks not terminated")); + }); + }); + + try { + // need to wait until this future is complete + waitForResult.get(); + // make sure all the contexts were closed + Assert.eqZero(openCount.get(), "openCount"); + } catch (InterruptedException e) { + throw new CancellationException("interrupted while processing test"); + } catch (ExecutionException e) { + if (e.getCause() instanceof RuntimeException) { + throw (RuntimeException) e.getCause(); + } else { + // rethrow the error + throw new UncheckedDeephavenException("failure while processing test", e.getCause()); + } + } + } + + @Test + public void TestSerialError() { + final CompletableFuture waitForResult = new CompletableFuture<>(); + final AtomicInteger openCount = new AtomicInteger(0); + + UpdateGraphProcessor.DEFAULT.enableUnitTestMode(); + UpdateGraphProcessor.DEFAULT.resetForUnitTests(false, true, 0, 4, 10, 5); + UpdateGraphProcessor.DEFAULT.runWithinUnitTestCycle(() -> { + final boolean[] completed = new boolean[100]; + + class TestJobContext extends JobScheduler.JobContext { + TestJobContext() { + openCount.incrementAndGet(); + } + + @Override + public void close() { + openCount.decrementAndGet(); + } + } + + final JobScheduler scheduler = new UpdateGraphProcessorJobScheduler(); + scheduler.iterateSerial( + ExecutionContext.getContext(), + null, + TestJobContext::new, + 0, + 50, + (context, idx, resume) -> { + // verify the type is correct + Assert.instanceOf(context, "context", TestJobContext.class); + + completed[idx] = true; + + // throw after this is set to make verification easy + if (idx == 10) { + throw new IndexOutOfBoundsException("test error"); + } + resume.run(); + }, + () -> { + // if this is called, we failed the test + waitForResult + .completeExceptionally(new AssertionFailure("IndexOutOfBoundsException not thrown")); + }, + exception -> { + if (!(exception instanceof IndexOutOfBoundsException)) { + waitForResult.completeExceptionally( + new AssertionFailure("IndexOutOfBoundsException not thrown")); + } + + // assert that the job was terminated before all tasks were executed (one is still false) + for (int ii = 0; ii < 50; ii++) { + if (!completed[ii]) { + waitForResult.complete(null); + return; + } + } + waitForResult.completeExceptionally(new AssertionFailure("Tasks not terminated")); + }); + }); + + try { + // need to wait until this future is complete + waitForResult.get(); + // make sure all the contexts were closed + Assert.eqZero(openCount.get(), "openCount"); + } catch (InterruptedException e) { + throw new CancellationException("interrupted while processing test"); + } catch (ExecutionException e) { + if (e.getCause() instanceof RuntimeException) { + throw (RuntimeException) e.getCause(); + } else { + // rethrow the error + throw new UncheckedDeephavenException("failure while processing test", e.getCause()); + } + } + } +} From bed69d8001a16c93487c9291e5d4272e92344bb7 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 18 Jan 2023 18:51:41 -0800 Subject: [PATCH 088/123] Far too large of commit, apologies --- .../base/ringbuffer/ByteRingBuffer.java | 30 +- .../base/ringbuffer/CharRingBuffer.java | 30 +- .../base/ringbuffer/DoubleRingBuffer.java | 30 +- .../base/ringbuffer/FloatRingBuffer.java | 30 +- .../base/ringbuffer/IntRingBuffer.java | 30 +- .../base/ringbuffer/LongRingBuffer.java | 30 +- .../base/ringbuffer/ShortRingBuffer.java | 30 +- .../base/ringbuffer/ByteRingBufferTest.java | 12 +- .../base/ringbuffer/CharRingBufferTest.java | 12 +- .../base/ringbuffer/DoubleRingBufferTest.java | 12 +- .../base/ringbuffer/FloatRingBufferTest.java | 12 +- .../base/ringbuffer/IntRingBufferTest.java | 12 +- .../base/ringbuffer/LongRingBufferTest.java | 12 +- .../base/ringbuffer/ShortRingBufferTest.java | 12 +- .../updateby/UpdateByCumulativeOperator.java | 2 +- .../table/impl/updateby/UpdateByOperator.java | 9 +- .../impl/updateby/UpdateByWindowTicks.java | 63 +- .../impl/updateby/UpdateByWindowTime.java | 72 +- .../updateby/ema/BigDecimalEMAOperator.java | 2 +- .../updateby/ema/BigIntegerEMAOperator.java | 2 +- .../impl/updateby/ema/ByteEMAOperator.java | 2 +- .../impl/updateby/ema/DoubleEMAOperator.java | 2 +- .../impl/updateby/ema/FloatEMAOperator.java | 2 +- .../impl/updateby/ema/IntEMAOperator.java | 2 +- .../impl/updateby/ema/LongEMAOperator.java | 2 +- .../impl/updateby/ema/ShortEMAOperator.java | 2 +- .../updateby/fill/BooleanFillByOperator.java | 5 +- .../updateby/fill/ByteFillByOperator.java | 5 +- .../updateby/fill/CharFillByOperator.java | 5 +- .../updateby/fill/DoubleFillByOperator.java | 5 +- .../updateby/fill/FloatFillByOperator.java | 5 +- .../impl/updateby/fill/IntFillByOperator.java | 5 +- .../updateby/fill/LongFillByOperator.java | 5 +- .../updateby/fill/ObjectFillByOperator.java | 5 +- .../updateby/fill/ShortFillByOperator.java | 5 +- .../internal/BaseByteUpdateByOperator.java | 2 +- .../internal/BaseCharUpdateByOperator.java | 2 +- .../internal/BaseDoubleUpdateByOperator.java | 2 +- .../internal/BaseFloatUpdateByOperator.java | 2 +- .../internal/BaseIntUpdateByOperator.java | 2 +- .../internal/BaseLongUpdateByOperator.java | 2 +- .../internal/BaseObjectBinaryOperator.java | 16 +- .../internal/BaseObjectUpdateByOperator.java | 2 +- .../internal/BaseShortUpdateByOperator.java | 2 +- .../BaseWindowedByteUpdateByOperator.java | 19 +- .../BaseWindowedCharUpdateByOperator.java | 19 +- .../BaseWindowedDoubleUpdateByOperator.java | 21 +- .../BaseWindowedFloatUpdateByOperator.java | 21 +- .../BaseWindowedIntUpdateByOperator.java | 19 +- .../BaseWindowedLongUpdateByOperator.java | 19 +- .../BaseWindowedObjectUpdateByOperator.java | 19 +- .../BaseWindowedShortUpdateByOperator.java | 19 +- .../internal/PairwiseDoubleRingBuffer.java | 104 ++- .../internal/PairwiseFloatRingBuffer.java | 104 ++- .../minmax/ByteCumMinMaxOperator.java | 5 +- .../minmax/DoubleCumMinMaxOperator.java | 5 +- .../minmax/FloatCumMinMaxOperator.java | 5 +- .../updateby/minmax/IntCumMinMaxOperator.java | 5 +- .../minmax/LongCumMinMaxOperator.java | 5 +- .../minmax/ShortCumMinMaxOperator.java | 5 +- .../prod/BigDecimalCumProdOperator.java | 5 +- .../prod/BigIntegerCumProdOperator.java | 5 +- .../updateby/prod/ByteCumProdOperator.java | 5 +- .../updateby/prod/DoubleCumProdOperator.java | 5 +- .../updateby/prod/FloatCumProdOperator.java | 5 +- .../updateby/prod/IntCumProdOperator.java | 5 +- .../updateby/prod/LongCumProdOperator.java | 5 +- .../updateby/prod/ShortCumProdOperator.java | 5 +- .../BigDecimalRollingSumOperator.java | 41 +- .../BigIntegerRollingSumOperator.java | 40 +- .../rollingsum/ByteRollingSumOperator.java | 45 +- .../rollingsum/DoubleRollingSumOperator.java | 28 +- .../rollingsum/FloatRollingSumOperator.java | 28 +- .../rollingsum/IntRollingSumOperator.java | 45 +- .../rollingsum/LongRollingSumOperator.java | 45 +- .../rollingsum/ShortRollingSumOperator.java | 45 +- .../sum/BigDecimalCumSumOperator.java | 5 +- .../sum/BigIntegerCumSumOperator.java | 5 +- .../impl/updateby/sum/ByteCumSumOperator.java | 5 +- .../updateby/sum/DoubleCumSumOperator.java | 5 +- .../updateby/sum/FloatCumSumOperator.java | 5 +- .../impl/updateby/sum/IntCumSumOperator.java | 5 +- .../impl/updateby/sum/LongCumSumOperator.java | 5 +- .../updateby/sum/ShortCumSumOperator.java | 5 +- .../table/impl/updateby/TestRollingSum.java | 679 ++++++++++++++++-- .../impl/updateby/TestUpdateByGeneral.java | 53 +- .../PairwiseDoubleRingBufferTest.java | 59 ++ .../internal/PairwiseFloatRingBufferTest.java | 59 ++ 88 files changed, 1716 insertions(+), 458 deletions(-) diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java index 6ad69374bbe..3267e2864a3 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java @@ -24,20 +24,22 @@ public class ByteRingBuffer implements Serializable { private void grow(int increase) { if (growable) { - final int minLength = storage.length + increase; + final int size = size(); + + final int minLength = size + increase + 1; int newLength = storage.length * 2; while (newLength < minLength) { newLength = newLength * 2; } byte[] newStorage = new byte[newLength]; - if (tail > head) { - System.arraycopy(storage, head, newStorage, 0, tail - head); - tail = tail - head; + if (tail >= head) { + System.arraycopy(storage, head, newStorage, 0, size); } else { - System.arraycopy(storage, head, newStorage, 0, storage.length - head); - System.arraycopy(storage, 0, newStorage, storage.length - head, tail); - tail += storage.length - head; + final int firstCopyLen = storage.length - head; + System.arraycopy(storage, head, newStorage, 0, firstCopyLen); + System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); } + tail = size; head = 0; storage = newStorage; } @@ -180,11 +182,13 @@ public byte[] remove(int count) { throw new NoSuchElementException(); } final byte[] result = new byte[count]; - if (tail > head || storage.length - head >= count) { + final int firstCopyLen = storage.length - head; + + if (tail >= head || firstCopyLen >= count) { System.arraycopy(storage, head, result, 0, count); } else { - System.arraycopy(storage, head, result, 0, storage.length - head); - System.arraycopy(storage, 0, result, storage.length - head, count - (storage.length - head)); + System.arraycopy(storage, head, result, 0, firstCopyLen); + System.arraycopy(storage, 0, result, firstCopyLen, count - firstCopyLen); } head = (head + count) % storage.length; return result; @@ -199,6 +203,12 @@ public byte remove() { return e; } + public byte removeUnsafe() { + byte e = storage[head]; + head = (head + 1) % storage.length; + return e; + } + public byte poll(byte onEmpty) { if (isEmpty()) { return onEmpty; diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java index d92d8085209..fd559a01219 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java @@ -19,20 +19,22 @@ public class CharRingBuffer implements Serializable { private void grow(int increase) { if (growable) { - final int minLength = storage.length + increase; + final int size = size(); + + final int minLength = size + increase + 1; int newLength = storage.length * 2; while (newLength < minLength) { newLength = newLength * 2; } char[] newStorage = new char[newLength]; - if (tail > head) { - System.arraycopy(storage, head, newStorage, 0, tail - head); - tail = tail - head; + if (tail >= head) { + System.arraycopy(storage, head, newStorage, 0, size); } else { - System.arraycopy(storage, head, newStorage, 0, storage.length - head); - System.arraycopy(storage, 0, newStorage, storage.length - head, tail); - tail += storage.length - head; + final int firstCopyLen = storage.length - head; + System.arraycopy(storage, head, newStorage, 0, firstCopyLen); + System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); } + tail = size; head = 0; storage = newStorage; } @@ -175,11 +177,13 @@ public char[] remove(int count) { throw new NoSuchElementException(); } final char[] result = new char[count]; - if (tail > head || storage.length - head >= count) { + final int firstCopyLen = storage.length - head; + + if (tail >= head || firstCopyLen >= count) { System.arraycopy(storage, head, result, 0, count); } else { - System.arraycopy(storage, head, result, 0, storage.length - head); - System.arraycopy(storage, 0, result, storage.length - head, count - (storage.length - head)); + System.arraycopy(storage, head, result, 0, firstCopyLen); + System.arraycopy(storage, 0, result, firstCopyLen, count - firstCopyLen); } head = (head + count) % storage.length; return result; @@ -194,6 +198,12 @@ public char remove() { return e; } + public char removeUnsafe() { + char e = storage[head]; + head = (head + 1) % storage.length; + return e; + } + public char poll(char onEmpty) { if (isEmpty()) { return onEmpty; diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java index 701c0bda002..998b0c81e28 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java @@ -24,20 +24,22 @@ public class DoubleRingBuffer implements Serializable { private void grow(int increase) { if (growable) { - final int minLength = storage.length + increase; + final int size = size(); + + final int minLength = size + increase + 1; int newLength = storage.length * 2; while (newLength < minLength) { newLength = newLength * 2; } double[] newStorage = new double[newLength]; - if (tail > head) { - System.arraycopy(storage, head, newStorage, 0, tail - head); - tail = tail - head; + if (tail >= head) { + System.arraycopy(storage, head, newStorage, 0, size); } else { - System.arraycopy(storage, head, newStorage, 0, storage.length - head); - System.arraycopy(storage, 0, newStorage, storage.length - head, tail); - tail += storage.length - head; + final int firstCopyLen = storage.length - head; + System.arraycopy(storage, head, newStorage, 0, firstCopyLen); + System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); } + tail = size; head = 0; storage = newStorage; } @@ -180,11 +182,13 @@ public double[] remove(int count) { throw new NoSuchElementException(); } final double[] result = new double[count]; - if (tail > head || storage.length - head >= count) { + final int firstCopyLen = storage.length - head; + + if (tail >= head || firstCopyLen >= count) { System.arraycopy(storage, head, result, 0, count); } else { - System.arraycopy(storage, head, result, 0, storage.length - head); - System.arraycopy(storage, 0, result, storage.length - head, count - (storage.length - head)); + System.arraycopy(storage, head, result, 0, firstCopyLen); + System.arraycopy(storage, 0, result, firstCopyLen, count - firstCopyLen); } head = (head + count) % storage.length; return result; @@ -199,6 +203,12 @@ public double remove() { return e; } + public double removeUnsafe() { + double e = storage[head]; + head = (head + 1) % storage.length; + return e; + } + public double poll(double onEmpty) { if (isEmpty()) { return onEmpty; diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java index 410c9c1186d..3fcc3ece33c 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java @@ -24,20 +24,22 @@ public class FloatRingBuffer implements Serializable { private void grow(int increase) { if (growable) { - final int minLength = storage.length + increase; + final int size = size(); + + final int minLength = size + increase + 1; int newLength = storage.length * 2; while (newLength < minLength) { newLength = newLength * 2; } float[] newStorage = new float[newLength]; - if (tail > head) { - System.arraycopy(storage, head, newStorage, 0, tail - head); - tail = tail - head; + if (tail >= head) { + System.arraycopy(storage, head, newStorage, 0, size); } else { - System.arraycopy(storage, head, newStorage, 0, storage.length - head); - System.arraycopy(storage, 0, newStorage, storage.length - head, tail); - tail += storage.length - head; + final int firstCopyLen = storage.length - head; + System.arraycopy(storage, head, newStorage, 0, firstCopyLen); + System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); } + tail = size; head = 0; storage = newStorage; } @@ -180,11 +182,13 @@ public float[] remove(int count) { throw new NoSuchElementException(); } final float[] result = new float[count]; - if (tail > head || storage.length - head >= count) { + final int firstCopyLen = storage.length - head; + + if (tail >= head || firstCopyLen >= count) { System.arraycopy(storage, head, result, 0, count); } else { - System.arraycopy(storage, head, result, 0, storage.length - head); - System.arraycopy(storage, 0, result, storage.length - head, count - (storage.length - head)); + System.arraycopy(storage, head, result, 0, firstCopyLen); + System.arraycopy(storage, 0, result, firstCopyLen, count - firstCopyLen); } head = (head + count) % storage.length; return result; @@ -199,6 +203,12 @@ public float remove() { return e; } + public float removeUnsafe() { + float e = storage[head]; + head = (head + 1) % storage.length; + return e; + } + public float poll(float onEmpty) { if (isEmpty()) { return onEmpty; diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java index 6890526dbff..85431bca67e 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java @@ -24,20 +24,22 @@ public class IntRingBuffer implements Serializable { private void grow(int increase) { if (growable) { - final int minLength = storage.length + increase; + final int size = size(); + + final int minLength = size + increase + 1; int newLength = storage.length * 2; while (newLength < minLength) { newLength = newLength * 2; } int[] newStorage = new int[newLength]; - if (tail > head) { - System.arraycopy(storage, head, newStorage, 0, tail - head); - tail = tail - head; + if (tail >= head) { + System.arraycopy(storage, head, newStorage, 0, size); } else { - System.arraycopy(storage, head, newStorage, 0, storage.length - head); - System.arraycopy(storage, 0, newStorage, storage.length - head, tail); - tail += storage.length - head; + final int firstCopyLen = storage.length - head; + System.arraycopy(storage, head, newStorage, 0, firstCopyLen); + System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); } + tail = size; head = 0; storage = newStorage; } @@ -180,11 +182,13 @@ public int[] remove(int count) { throw new NoSuchElementException(); } final int[] result = new int[count]; - if (tail > head || storage.length - head >= count) { + final int firstCopyLen = storage.length - head; + + if (tail >= head || firstCopyLen >= count) { System.arraycopy(storage, head, result, 0, count); } else { - System.arraycopy(storage, head, result, 0, storage.length - head); - System.arraycopy(storage, 0, result, storage.length - head, count - (storage.length - head)); + System.arraycopy(storage, head, result, 0, firstCopyLen); + System.arraycopy(storage, 0, result, firstCopyLen, count - firstCopyLen); } head = (head + count) % storage.length; return result; @@ -199,6 +203,12 @@ public int remove() { return e; } + public int removeUnsafe() { + int e = storage[head]; + head = (head + 1) % storage.length; + return e; + } + public int poll(int onEmpty) { if (isEmpty()) { return onEmpty; diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java index d3b3507cfe4..f5f097a5901 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java @@ -24,20 +24,22 @@ public class LongRingBuffer implements Serializable { private void grow(int increase) { if (growable) { - final int minLength = storage.length + increase; + final int size = size(); + + final int minLength = size + increase + 1; int newLength = storage.length * 2; while (newLength < minLength) { newLength = newLength * 2; } long[] newStorage = new long[newLength]; - if (tail > head) { - System.arraycopy(storage, head, newStorage, 0, tail - head); - tail = tail - head; + if (tail >= head) { + System.arraycopy(storage, head, newStorage, 0, size); } else { - System.arraycopy(storage, head, newStorage, 0, storage.length - head); - System.arraycopy(storage, 0, newStorage, storage.length - head, tail); - tail += storage.length - head; + final int firstCopyLen = storage.length - head; + System.arraycopy(storage, head, newStorage, 0, firstCopyLen); + System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); } + tail = size; head = 0; storage = newStorage; } @@ -180,11 +182,13 @@ public long[] remove(int count) { throw new NoSuchElementException(); } final long[] result = new long[count]; - if (tail > head || storage.length - head >= count) { + final int firstCopyLen = storage.length - head; + + if (tail >= head || firstCopyLen >= count) { System.arraycopy(storage, head, result, 0, count); } else { - System.arraycopy(storage, head, result, 0, storage.length - head); - System.arraycopy(storage, 0, result, storage.length - head, count - (storage.length - head)); + System.arraycopy(storage, head, result, 0, firstCopyLen); + System.arraycopy(storage, 0, result, firstCopyLen, count - firstCopyLen); } head = (head + count) % storage.length; return result; @@ -199,6 +203,12 @@ public long remove() { return e; } + public long removeUnsafe() { + long e = storage[head]; + head = (head + 1) % storage.length; + return e; + } + public long poll(long onEmpty) { if (isEmpty()) { return onEmpty; diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java index f10e28df025..ec870a0350f 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java @@ -24,20 +24,22 @@ public class ShortRingBuffer implements Serializable { private void grow(int increase) { if (growable) { - final int minLength = storage.length + increase; + final int size = size(); + + final int minLength = size + increase + 1; int newLength = storage.length * 2; while (newLength < minLength) { newLength = newLength * 2; } short[] newStorage = new short[newLength]; - if (tail > head) { - System.arraycopy(storage, head, newStorage, 0, tail - head); - tail = tail - head; + if (tail >= head) { + System.arraycopy(storage, head, newStorage, 0, size); } else { - System.arraycopy(storage, head, newStorage, 0, storage.length - head); - System.arraycopy(storage, 0, newStorage, storage.length - head, tail); - tail += storage.length - head; + final int firstCopyLen = storage.length - head; + System.arraycopy(storage, head, newStorage, 0, firstCopyLen); + System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); } + tail = size; head = 0; storage = newStorage; } @@ -180,11 +182,13 @@ public short[] remove(int count) { throw new NoSuchElementException(); } final short[] result = new short[count]; - if (tail > head || storage.length - head >= count) { + final int firstCopyLen = storage.length - head; + + if (tail >= head || firstCopyLen >= count) { System.arraycopy(storage, head, result, 0, count); } else { - System.arraycopy(storage, head, result, 0, storage.length - head); - System.arraycopy(storage, 0, result, storage.length - head, count - (storage.length - head)); + System.arraycopy(storage, head, result, 0, firstCopyLen); + System.arraycopy(storage, 0, result, firstCopyLen, count - firstCopyLen); } head = (head + count) % storage.length; return result; @@ -199,6 +203,12 @@ public short remove() { return e; } + public short removeUnsafe() { + short e = storage[head]; + head = (head + 1) % storage.length; + return e; + } + public short poll(short onEmpty) { if (isEmpty()) { return onEmpty; diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/ByteRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/ByteRingBufferTest.java index 8831f6bcca8..f37bc31a0af 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/ByteRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/ByteRingBufferTest.java @@ -510,7 +510,7 @@ public void testMultipleRemove() { assertEmpty(rb); } - public void testAddUnsafe() { + public void testAddRemoveUnsafe() { ByteRingBuffer rbNoGrow = new ByteRingBuffer(3, false); // this should throw @@ -524,6 +524,16 @@ public void testAddUnsafe() { assertContents(rbNoGrow, A, B, C); + assertEquals(rbNoGrow.removeUnsafe(), A); + assertContents(rbNoGrow, B, C); + + assertEquals(rbNoGrow.removeUnsafe(), B); + assertContents(rbNoGrow, C); + + assertEquals(rbNoGrow.removeUnsafe(), C); + assertEmpty(rbNoGrow); + + ByteRingBuffer rbGrow = new ByteRingBuffer(3, true); for (int size = 10; size < 1_000_000; size *= 10) { diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java index ddce8d8a83f..b753d49dbe2 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java @@ -505,7 +505,7 @@ public void testMultipleRemove() { assertEmpty(rb); } - public void testAddUnsafe() { + public void testAddRemoveUnsafe() { CharRingBuffer rbNoGrow = new CharRingBuffer(3, false); // this should throw @@ -519,6 +519,16 @@ public void testAddUnsafe() { assertContents(rbNoGrow, A, B, C); + assertEquals(rbNoGrow.removeUnsafe(), A); + assertContents(rbNoGrow, B, C); + + assertEquals(rbNoGrow.removeUnsafe(), B); + assertContents(rbNoGrow, C); + + assertEquals(rbNoGrow.removeUnsafe(), C); + assertEmpty(rbNoGrow); + + CharRingBuffer rbGrow = new CharRingBuffer(3, true); for (int size = 10; size < 1_000_000; size *= 10) { diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/DoubleRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/DoubleRingBufferTest.java index 0773e2ba43c..145c897c43a 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/DoubleRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/DoubleRingBufferTest.java @@ -510,7 +510,7 @@ public void testMultipleRemove() { assertEmpty(rb); } - public void testAddUnsafe() { + public void testAddRemoveUnsafe() { DoubleRingBuffer rbNoGrow = new DoubleRingBuffer(3, false); // this should throw @@ -524,6 +524,16 @@ public void testAddUnsafe() { assertContents(rbNoGrow, A, B, C); + assertEquals(rbNoGrow.removeUnsafe(), A); + assertContents(rbNoGrow, B, C); + + assertEquals(rbNoGrow.removeUnsafe(), B); + assertContents(rbNoGrow, C); + + assertEquals(rbNoGrow.removeUnsafe(), C); + assertEmpty(rbNoGrow); + + DoubleRingBuffer rbGrow = new DoubleRingBuffer(3, true); for (int size = 10; size < 1_000_000; size *= 10) { diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/FloatRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/FloatRingBufferTest.java index 41c28cb3317..74e336732cf 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/FloatRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/FloatRingBufferTest.java @@ -510,7 +510,7 @@ public void testMultipleRemove() { assertEmpty(rb); } - public void testAddUnsafe() { + public void testAddRemoveUnsafe() { FloatRingBuffer rbNoGrow = new FloatRingBuffer(3, false); // this should throw @@ -524,6 +524,16 @@ public void testAddUnsafe() { assertContents(rbNoGrow, A, B, C); + assertEquals(rbNoGrow.removeUnsafe(), A); + assertContents(rbNoGrow, B, C); + + assertEquals(rbNoGrow.removeUnsafe(), B); + assertContents(rbNoGrow, C); + + assertEquals(rbNoGrow.removeUnsafe(), C); + assertEmpty(rbNoGrow); + + FloatRingBuffer rbGrow = new FloatRingBuffer(3, true); for (int size = 10; size < 1_000_000; size *= 10) { diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/IntRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/IntRingBufferTest.java index f7bec065f34..7a6461009cd 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/IntRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/IntRingBufferTest.java @@ -510,7 +510,7 @@ public void testMultipleRemove() { assertEmpty(rb); } - public void testAddUnsafe() { + public void testAddRemoveUnsafe() { IntRingBuffer rbNoGrow = new IntRingBuffer(3, false); // this should throw @@ -524,6 +524,16 @@ public void testAddUnsafe() { assertContents(rbNoGrow, A, B, C); + assertEquals(rbNoGrow.removeUnsafe(), A); + assertContents(rbNoGrow, B, C); + + assertEquals(rbNoGrow.removeUnsafe(), B); + assertContents(rbNoGrow, C); + + assertEquals(rbNoGrow.removeUnsafe(), C); + assertEmpty(rbNoGrow); + + IntRingBuffer rbGrow = new IntRingBuffer(3, true); for (int size = 10; size < 1_000_000; size *= 10) { diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/LongRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/LongRingBufferTest.java index 496e1d84b2b..48ab6a2dbca 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/LongRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/LongRingBufferTest.java @@ -510,7 +510,7 @@ public void testMultipleRemove() { assertEmpty(rb); } - public void testAddUnsafe() { + public void testAddRemoveUnsafe() { LongRingBuffer rbNoGrow = new LongRingBuffer(3, false); // this should throw @@ -524,6 +524,16 @@ public void testAddUnsafe() { assertContents(rbNoGrow, A, B, C); + assertEquals(rbNoGrow.removeUnsafe(), A); + assertContents(rbNoGrow, B, C); + + assertEquals(rbNoGrow.removeUnsafe(), B); + assertContents(rbNoGrow, C); + + assertEquals(rbNoGrow.removeUnsafe(), C); + assertEmpty(rbNoGrow); + + LongRingBuffer rbGrow = new LongRingBuffer(3, true); for (int size = 10; size < 1_000_000; size *= 10) { diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/ShortRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/ShortRingBufferTest.java index b6f0bb0ce35..fc64b402e94 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/ShortRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/ShortRingBufferTest.java @@ -510,7 +510,7 @@ public void testMultipleRemove() { assertEmpty(rb); } - public void testAddUnsafe() { + public void testAddRemoveUnsafe() { ShortRingBuffer rbNoGrow = new ShortRingBuffer(3, false); // this should throw @@ -524,6 +524,16 @@ public void testAddUnsafe() { assertContents(rbNoGrow, A, B, C); + assertEquals(rbNoGrow.removeUnsafe(), A); + assertContents(rbNoGrow, B, C); + + assertEquals(rbNoGrow.removeUnsafe(), B); + assertContents(rbNoGrow, C); + + assertEquals(rbNoGrow.removeUnsafe(), C); + assertEmpty(rbNoGrow); + + ShortRingBuffer rbGrow = new ShortRingBuffer(3, true); for (int size = 10; size < 1_000_000; size *= 10) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByCumulativeOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByCumulativeOperator.java index 54e414a4785..65dfa15871b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByCumulativeOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByCumulativeOperator.java @@ -30,7 +30,7 @@ public boolean isValueValid(long atKey) { public void close() {} @FinalDefault - public void pop() { + public void pop(int count) { throw new UnsupportedOperationException("Cumulative operators should never call pop()"); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperator.java index d9a6a0868c2..ac71bb79c14 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperator.java @@ -70,14 +70,17 @@ public interface UpdateContext extends SafeCloseable { * @param pos the index in the associated chunk where this value can be found. Depending on the usage, might be * a values chunk (for cumulative operators) or an influencer values chunk (for windowed). It is the task * of the operator to pull the data from the chunk and use it properly + * @param count the number of items to push from the chunk */ - void push(long key, int pos); + void push(long key, int pos, int count); /** - * Remove a value from the operators current data set. This is only valid for windowed operators since + * Remove a value from the operators current data set. This is only valid for windowed operators as * cumulative operators only append values + * + * @param count the number of items to pop from the data set */ - void pop(); + void pop(int count); /** * Write the current value for this row to the output chunk diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java index bcbfee3f292..2c0fe4bce37 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java @@ -104,10 +104,17 @@ private static WritableRowSet computeInfluencerRowsTicks(final RowSet sourceSet, final MutableLong minPos = new MutableLong(0L); invertedSubSet.forAllRowKeyRanges((s, e) -> { - long sPos = Math.max(s - revTicks + 1, minPos.longValue()); - long ePos = Math.min(e + fwdTicks, maxPos); - builder.appendRange(sPos, ePos); - minPos.setValue(ePos + 1); + long head = s - revTicks + 1; + long tail = e + fwdTicks; + + if (tail < minPos.longValue() || head > maxPos) { + // ignore this range + return; + } + head = Math.max(head, minPos.longValue()); + tail = Math.min(tail, maxPos); + builder.appendRange(head, tail); + minPos.setValue(tail + 1); }); try (final RowSet positions = builder.build()) { @@ -225,34 +232,44 @@ void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, @NotNu } // need a writable rowset - final WritableRowSet tmpAffected; + final WritableRowSet tmpAffected = RowSetFactory.empty(); - // changed rows are all mods+adds - try (final WritableRowSet changed = upstream.added().union(upstream.modified())) { + if (upstream.modified().isNonempty()) { // compute the rows affected from these changes - try (final WritableRowSet changedInverted = ctx.sourceRowSet.invert(changed)) { - tmpAffected = computeAffectedRowsTicks(ctx.sourceRowSet, changedInverted, prevUnits, fwdUnits); + try (final WritableRowSet modifiedInverted = ctx.sourceRowSet.invert(upstream.modified()); + final RowSet modifiedAffected = computeAffectedRowsTicks(ctx.sourceRowSet, modifiedInverted, prevUnits, fwdUnits)) { + tmpAffected.insert(modifiedAffected); + } + } + + if (upstream.added().isNonempty()) { + // add the new rows and any cascading changes from inserting rows + final long prev = Math.max(1, prevUnits); + final long fwd = Math.max(0, fwdUnits); + + try (final RowSet addedInverted = ctx.sourceRowSet.invert(upstream.added()); + final RowSet addedAffected = computeAffectedRowsTicks(ctx.sourceRowSet, addedInverted, prev, fwd)) { + tmpAffected.insert(addedAffected); } } - // other rows can be affected by removes if (upstream.removed().isNonempty()) { - try (final RowSet prev = ctx.sourceRowSet.copyPrev(); - final RowSet removedPositions = prev.invert(upstream.removed()); - final WritableRowSet affectedByRemoves = - computeAffectedRowsTicks(prev, removedPositions, prevUnits, - fwdUnits)) { + // add the cascading changes from removing rows + final long prev = Math.max(0, prevUnits); + final long fwd = Math.max(0, fwdUnits); + + try (final RowSet prevRows = ctx.sourceRowSet.copyPrev(); + final RowSet removedInverted = prevRows.invert(upstream.removed()); + final WritableRowSet removedAffected = + computeAffectedRowsTicks(prevRows, removedInverted, prev, fwd)) { // apply shifts to get back to pos-shift space - upstream.shifted().apply(affectedByRemoves); + upstream.shifted().apply(removedAffected); // retain only the rows that still exist in the sourceRowSet - affectedByRemoves.retain(ctx.sourceRowSet); - tmpAffected.insert(affectedByRemoves); + removedAffected.retain(ctx.sourceRowSet); + tmpAffected.insert(removedAffected); } } - // naturally need to compute all newly added rows - tmpAffected.insert(upstream.added()); - ctx.affectedRows = tmpAffected; // now get influencer rows for the affected rows @@ -292,8 +309,6 @@ void processRows(UpdateByWindowBucketContext context, boolean initialStep) { final long sourceRowSetSize = ctx.sourceRowSet.size(); // chunk processing - long totalCount = 0; - while (it.hasMore()) { final RowSequence chunkRs = it.getNextRowSequenceWithLength(ctx.workingChunkSize); final RowSequence chunkPosRs = posIt.getNextRowSequenceWithLength(ctx.workingChunkSize); @@ -302,6 +317,8 @@ void processRows(UpdateByWindowBucketContext context, boolean initialStep) { final LongChunk posChunk = chunkPosRs.asRowKeyChunk(); // chunk processing + long totalCount = 0; + for (int ii = 0; ii < chunkRsSize; ii++) { // read the current position final long currentPos = posChunk.get(ii); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java index 2fad697ad84..8b5385e6f16 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java @@ -209,11 +209,9 @@ public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, // all rows are affected on the initial step if (ctx.initialStep) { ctx.affectedRows = ctx.sourceRowSet; - // only non-null timestamps are actual influencers. Note that this is fast rather than precise. This set - // of rows may contain rows not needed for computation. E.g. when fwd and rev are both non-zero, we could - // eliminate rows at either the beginning or end of the set, but not convinced ROI > 0 and these rows are - // properly skipped by window creation - ctx.influencerRows = ctx.timestampValidRowSet.copy(); + + ctx.influencerRows = computeInfluencerRowsTime(ctx.timestampValidRowSet, ctx.affectedRows, prevUnits, fwdUnits, + ctx.timestampColumnSource, ctx.timestampSsa, false); // mark all operators as affected by this update context.dirtyOperatorIndices = IntStream.range(0, operators.length).toArray(); @@ -255,38 +253,48 @@ public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, return; } - final WritableRowSet tmpAffected; + try (final RowSet prevRows = upstream.modified().isNonempty() || upstream.removed().isNonempty() ? + ctx.timestampValidRowSet.copyPrev() : + null) { - // changed rows are all mods+adds - try (WritableRowSet changed = upstream.added().union(upstream.modified())) { - // need a writable rowset - tmpAffected = computeAffectedRowsTime(ctx.timestampValidRowSet, changed, prevUnits, fwdUnits, - ctx.timestampColumnSource, ctx.timestampSsa, false); - } + final WritableRowSet tmpAffected = RowSetFactory.empty(); - // other rows can be affected by removes or mods - if (upstream.removed().isNonempty()) { - try (final RowSet prev = ctx.timestampValidRowSet.copyPrev(); - final WritableRowSet affectedByRemoves = - computeAffectedRowsTime(prev, upstream.removed(), prevUnits, fwdUnits, - ctx.timestampColumnSource, ctx.timestampSsa, true); - final WritableRowSet affectedByModifies = - computeAffectedRowsTime(prev, upstream.getModifiedPreShift(), prevUnits, fwdUnits, - ctx.timestampColumnSource, ctx.timestampSsa, true)) { - // we used the SSA (post-shift) to get these keys, no need to shift - // retain only the rows that still exist in the sourceRowSet - affectedByRemoves.retain(ctx.timestampValidRowSet); - affectedByModifies.retain(ctx.timestampValidRowSet); - - tmpAffected.insert(affectedByRemoves); - tmpAffected.insert(affectedByModifies); + if (upstream.modified().isNonempty()) { + // modified timestamps will affect the current and previous values + try (final RowSet modifiedAffected = computeAffectedRowsTime(ctx.timestampValidRowSet, upstream.modified(), prevUnits, fwdUnits, ctx.timestampColumnSource, ctx.timestampSsa, false)) { + tmpAffected.insert(modifiedAffected); + } + try (final WritableRowSet modifiedAffectedPrev = computeAffectedRowsTime(prevRows, upstream.getModifiedPreShift(), prevUnits, fwdUnits, ctx.timestampColumnSource, ctx.timestampSsa, true)) { + // we used the SSA (post-shift) to get these keys, no need to shift + // retain only the rows that still exist in the sourceRowSet + modifiedAffectedPrev.retain(ctx.timestampValidRowSet); + tmpAffected.insert(modifiedAffectedPrev); + } + // naturally need to compute all modified rows + tmpAffected.insert(upstream.modified()); } - } - // naturally need to compute all newly added rows - tmpAffected.insert(upstream.added()); + if (upstream.added().isNonempty()) { + try (final RowSet addedAffected = computeAffectedRowsTime(ctx.timestampValidRowSet, upstream.added(), prevUnits, fwdUnits, ctx.timestampColumnSource, ctx.timestampSsa, false)) { + tmpAffected.insert(addedAffected); + } + // naturally need to compute all new rows + tmpAffected.insert(upstream.added()); + } - ctx.affectedRows = tmpAffected; + // other rows can be affected by removes or mods + if (upstream.removed().isNonempty()) { + try (final WritableRowSet removedAffected = computeAffectedRowsTime(prevRows, upstream.removed(), prevUnits, fwdUnits, ctx.timestampColumnSource, ctx.timestampSsa, true)) { + // we used the SSA (post-shift) to get these keys, no need to shift + // retain only the rows that still exist in the sourceRowSet + removedAffected.retain(ctx.timestampValidRowSet); + + tmpAffected.insert(removedAffected); + } + } + + ctx.affectedRows = tmpAffected; + } // now get influencer rows for the affected rows ctx.influencerRows = computeInfluencerRowsTime(ctx.timestampValidRowSet, ctx.affectedRows, prevUnits, fwdUnits, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java index b4d96ade3e6..9ef1b54f133 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java @@ -87,7 +87,7 @@ public void accumulate(RowSequence inputKeys, } @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { throw new IllegalStateException("EMAOperator#push() is not used"); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java index b50a84a9f8d..daf7356affe 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java @@ -92,7 +92,7 @@ public void accumulate(RowSequence inputKeys, } @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { throw new IllegalStateException("EMAOperator#push() is not used"); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java index b8d28ccd573..9e9481138af 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java @@ -100,7 +100,7 @@ public boolean isValueValid(long atKey) { } @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { throw new IllegalStateException("EMAOperator#push() is not used"); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java index 547b3ceb4d6..ed229e91a8d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java @@ -116,7 +116,7 @@ public boolean isValueValid(long atKey) { @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { throw new IllegalStateException("EMAOperator#push() is not used"); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java index 143ad967fbd..c4e84ab80d9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java @@ -111,7 +111,7 @@ public boolean isValueValid(long atKey) { @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { throw new IllegalStateException("EMAOperator#push() is not used"); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java index 0c934a49699..c4419e545ad 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java @@ -100,7 +100,7 @@ public boolean isValueValid(long atKey) { } @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { throw new IllegalStateException("EMAOperator#push() is not used"); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java index a80cd7be457..ed6cf3b078b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java @@ -100,7 +100,7 @@ public boolean isValueValid(long atKey) { } @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { throw new IllegalStateException("EMAOperator#push() is not used"); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java index 7887780f419..ca2a0d275ed 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java @@ -95,7 +95,7 @@ public boolean isValueValid(long atKey) { } @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { throw new IllegalStateException("EMAOperator#push() is not used"); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java index 29f9de7e77e..6d7fd883044 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java @@ -12,6 +12,7 @@ import io.deephaven.engine.table.impl.sources.BooleanSparseArraySource; import io.deephaven.engine.table.WritableColumnSource; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.ByteChunk; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; @@ -40,7 +41,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { + Assert.eq(count, "push count", 1); + byte currentVal = booleanValueChunk.get(pos); if(currentVal != NULL_BOOLEAN_AS_BYTE) { curVal = currentVal; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java index e686da6e843..c4bb080867e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java @@ -5,6 +5,7 @@ */ package io.deephaven.engine.table.impl.updateby.fill; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.ByteChunk; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; @@ -33,7 +34,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { + Assert.eq(count, "push count", 1); + byte currentVal = byteValueChunk.get(pos); if(currentVal != NULL_BYTE) { curVal = currentVal; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java index 1671b930172..19c80152d8a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java @@ -1,5 +1,6 @@ package io.deephaven.engine.table.impl.updateby.fill; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.CharChunk; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; @@ -28,7 +29,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { + Assert.eq(count, "push count", 1); + char currentVal = charValueChunk.get(pos); if(currentVal != NULL_CHAR) { curVal = currentVal; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java index e0dec94756d..fb5003dc073 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java @@ -5,6 +5,7 @@ */ package io.deephaven.engine.table.impl.updateby.fill; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.DoubleChunk; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; @@ -33,7 +34,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { + Assert.eq(count, "push count", 1); + double currentVal = doubleValueChunk.get(pos); if(currentVal != NULL_DOUBLE) { curVal = currentVal; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java index cdac08e5e5e..81d58b706b9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java @@ -5,6 +5,7 @@ */ package io.deephaven.engine.table.impl.updateby.fill; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.FloatChunk; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; @@ -33,7 +34,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { + Assert.eq(count, "push count", 1); + float currentVal = floatValueChunk.get(pos); if(currentVal != NULL_FLOAT) { curVal = currentVal; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java index 3fc9b4e79f4..0f7aa85801a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java @@ -5,6 +5,7 @@ */ package io.deephaven.engine.table.impl.updateby.fill; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; @@ -33,7 +34,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { + Assert.eq(count, "push count", 1); + int currentVal = intValueChunk.get(pos); if(currentVal != NULL_INT) { curVal = currentVal; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java index e61c3ccb7fd..d32da8790d3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java @@ -12,6 +12,7 @@ import java.time.Instant; import io.deephaven.engine.table.impl.sources.ReinterpretUtils; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; @@ -41,7 +42,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { + Assert.eq(count, "push count", 1); + long currentVal = longValueChunk.get(pos); if(currentVal != NULL_LONG) { curVal = currentVal; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java index bb51017fff3..b52ed0f786a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java @@ -7,6 +7,7 @@ import io.deephaven.engine.table.impl.util.ChunkUtils; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; @@ -34,7 +35,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { + Assert.eq(count, "push count", 1); + T currentVal = ObjectValueChunk.get(pos); if(currentVal != null) { curVal = currentVal; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java index ec911cb650f..6a455136756 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java @@ -5,6 +5,7 @@ */ package io.deephaven.engine.table.impl.updateby.fill; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.ShortChunk; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; @@ -33,7 +34,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { + Assert.eq(count, "push count", 1); + short currentVal = shortValueChunk.get(pos); if(currentVal != NULL_SHORT) { curVal = currentVal; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java index d6165638624..c6e182304f9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java @@ -60,7 +60,7 @@ public void accumulate(RowSequence inputKeys, // chunk processing for (int ii = 0; ii < len; ii++) { - push(NULL_ROW_KEY, ii); + push(NULL_ROW_KEY, ii, 1); writeToOutputChunk(ii); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java index 7ffbbaf1d8c..8fedb2a0794 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java @@ -49,7 +49,7 @@ public void accumulate(RowSequence inputKeys, // chunk processing for (int ii = 0; ii < len; ii++) { - push(NULL_ROW_KEY, ii); + push(NULL_ROW_KEY, ii, 1); writeToOutputChunk(ii); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java index e3a77c7c66f..755985fff4e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java @@ -56,7 +56,7 @@ public void accumulate(RowSequence inputKeys, // chunk processing for (int ii = 0; ii < len; ii++) { - push(NULL_ROW_KEY, ii); + push(NULL_ROW_KEY, ii, 1); writeToOutputChunk(ii); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java index f0b555920f9..febd72f9bc9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java @@ -51,7 +51,7 @@ public void accumulate(RowSequence inputKeys, // chunk processing for (int ii = 0; ii < len; ii++) { - push(NULL_ROW_KEY, ii); + push(NULL_ROW_KEY, ii, 1); writeToOutputChunk(ii); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java index 3b6f6d739cf..5f56bce323f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java @@ -54,7 +54,7 @@ public void accumulate(RowSequence inputKeys, // chunk processing for (int ii = 0; ii < len; ii++) { - push(NULL_ROW_KEY, ii); + push(NULL_ROW_KEY, ii, 1); writeToOutputChunk(ii); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java index 6fa2f0d2939..21e1f2bba8b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java @@ -54,7 +54,7 @@ public void accumulate(RowSequence inputKeys, // chunk processing for (int ii = 0; ii < len; ii++) { - push(NULL_ROW_KEY, ii); + push(NULL_ROW_KEY, ii, 1); writeToOutputChunk(ii); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java index 59e57e13721..b59e0ba4696 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java @@ -22,13 +22,15 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { - // read the value from the values chunk - final T currentVal = objectValueChunk.get(pos); - if(curVal == null) { - curVal = currentVal; - } else if(currentVal != null) { - curVal = doOperation(curVal, currentVal); + public void push(long key, int pos, int count) { + for (int ii = 0; ii < count; ii++) { + // read the value from the values chunk + final T currentVal = objectValueChunk.get(pos + ii); + if (curVal == null) { + curVal = currentVal; + } else if (currentVal != null) { + curVal = doOperation(curVal, currentVal); + } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java index 910b3e8e9ef..5442ac84457 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java @@ -57,7 +57,7 @@ public void accumulate(RowSequence inputKeys, // chunk processing for (int ii = 0; ii < len; ii++) { - push(NULL_ROW_KEY, ii); + push(NULL_ROW_KEY, ii, 1); writeToOutputChunk(ii); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java index 07315a95827..7ba9576463c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java @@ -54,7 +54,7 @@ public void accumulate(RowSequence inputKeys, // chunk processing for (int ii = 0; ii < len; ii++) { - push(NULL_ROW_KEY, ii); + push(NULL_ROW_KEY, ii, 1); writeToOutputChunk(ii); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java index c83a077a624..301a7c5a899 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java @@ -65,16 +65,21 @@ public void accumulate(RowSequence inputKeys, final int pushCount = pushChunk.get(ii); final int popCount = popChunk.get(ii); + if (pushCount == NULL_INT) { + writeNullToOutputChunk(ii); + continue; + } + // pop for this row - for (int count = 0; count < popCount; count++) { - pop(); + if (popCount > 0) { + pop(popCount); } // push for this row - for (int count = 0; count < pushCount; count++) { - push(NULL_ROW_KEY, pushIndex + count); + if (pushCount > 0) { + push(NULL_ROW_KEY, pushIndex, pushCount); + pushIndex += pushCount; } - pushIndex += pushCount; // write the results to the output chunk writeToOutputChunk(ii); @@ -95,6 +100,10 @@ public void writeToOutputChunk(int outIdx) { outputValues.set(outIdx, curVal); } + void writeNullToOutputChunk(int outIdx) { + outputValues.set(outIdx, NULL_BYTE); + } + @Override public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java index 8bf6b9a0606..b98d515d662 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java @@ -54,16 +54,21 @@ public void accumulate(RowSequence inputKeys, final int pushCount = pushChunk.get(ii); final int popCount = popChunk.get(ii); + if (pushCount == NULL_INT) { + writeNullToOutputChunk(ii); + continue; + } + // pop for this row - for (int count = 0; count < popCount; count++) { - pop(); + if (popCount > 0) { + pop(popCount); } // push for this row - for (int count = 0; count < pushCount; count++) { - push(NULL_ROW_KEY, pushIndex + count); + if (pushCount > 0) { + push(NULL_ROW_KEY, pushIndex, pushCount); + pushIndex += pushCount; } - pushIndex += pushCount; // write the results to the output chunk writeToOutputChunk(ii); @@ -84,6 +89,10 @@ public void writeToOutputChunk(int outIdx) { outputValues.set(outIdx, curVal); } + void writeNullToOutputChunk(int outIdx) { + outputValues.set(outIdx, NULL_CHAR); + } + @Override public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java index 6b94780d9a5..708c9ce3518 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java @@ -26,7 +26,7 @@ import java.util.Map; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; -import static io.deephaven.util.QueryConstants.NULL_DOUBLE; +import static io.deephaven.util.QueryConstants.*; public abstract class BaseWindowedDoubleUpdateByOperator extends UpdateByWindowedOperator { protected final WritableColumnSource outputSource; @@ -61,16 +61,21 @@ public void accumulate(RowSequence inputKeys, final int pushCount = pushChunk.get(ii); final int popCount = popChunk.get(ii); + if (pushCount == NULL_INT) { + writeNullToOutputChunk(ii); + continue; + } + // pop for this row - for (int count = 0; count < popCount; count++) { - pop(); + if (popCount > 0) { + pop(popCount); } // push for this row - for (int count = 0; count < pushCount; count++) { - push(NULL_ROW_KEY, pushIndex + count); + if (pushCount > 0) { + push(NULL_ROW_KEY, pushIndex, pushCount); + pushIndex += pushCount; } - pushIndex += pushCount; // write the results to the output chunk writeToOutputChunk(ii); @@ -91,6 +96,10 @@ public void writeToOutputChunk(int outIdx) { outputValues.set(outIdx, curVal); } + void writeNullToOutputChunk(int outIdx) { + outputValues.set(outIdx, NULL_DOUBLE); + } + @Override public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java index cf9c8064dc1..f0d32c0b944 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java @@ -21,7 +21,7 @@ import java.util.Map; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; -import static io.deephaven.util.QueryConstants.NULL_FLOAT; +import static io.deephaven.util.QueryConstants.*; public abstract class BaseWindowedFloatUpdateByOperator extends UpdateByWindowedOperator { protected final WritableColumnSource outputSource; @@ -56,16 +56,21 @@ public void accumulate(RowSequence inputKeys, final int pushCount = pushChunk.get(ii); final int popCount = popChunk.get(ii); + if (pushCount == NULL_INT) { + writeNullToOutputChunk(ii); + continue; + } + // pop for this row - for (int count = 0; count < popCount; count++) { - pop(); + if (popCount > 0) { + pop(popCount); } // push for this row - for (int count = 0; count < pushCount; count++) { - push(NULL_ROW_KEY, pushIndex + count); + if (pushCount > 0) { + push(NULL_ROW_KEY, pushIndex, pushCount); + pushIndex += pushCount; } - pushIndex += pushCount; // write the results to the output chunk writeToOutputChunk(ii); @@ -86,6 +91,10 @@ public void writeToOutputChunk(int outIdx) { outputValues.set(outIdx, curVal); } + void writeNullToOutputChunk(int outIdx) { + outputValues.set(outIdx, NULL_FLOAT); + } + @Override public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java index c918ea17b00..fb5e9c96270 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java @@ -59,16 +59,21 @@ public void accumulate(RowSequence inputKeys, final int pushCount = pushChunk.get(ii); final int popCount = popChunk.get(ii); + if (pushCount == NULL_INT) { + writeNullToOutputChunk(ii); + continue; + } + // pop for this row - for (int count = 0; count < popCount; count++) { - pop(); + if (popCount > 0) { + pop(popCount); } // push for this row - for (int count = 0; count < pushCount; count++) { - push(NULL_ROW_KEY, pushIndex + count); + if (pushCount > 0) { + push(NULL_ROW_KEY, pushIndex, pushCount); + pushIndex += pushCount; } - pushIndex += pushCount; // write the results to the output chunk writeToOutputChunk(ii); @@ -89,6 +94,10 @@ public void writeToOutputChunk(int outIdx) { outputValues.set(outIdx, curVal); } + void writeNullToOutputChunk(int outIdx) { + outputValues.set(outIdx, NULL_INT); + } + @Override public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java index 56b4a992600..2f1d34863f2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java @@ -59,16 +59,21 @@ public void accumulate(RowSequence inputKeys, final int pushCount = pushChunk.get(ii); final int popCount = popChunk.get(ii); + if (pushCount == NULL_INT) { + writeNullToOutputChunk(ii); + continue; + } + // pop for this row - for (int count = 0; count < popCount; count++) { - pop(); + if (popCount > 0) { + pop(popCount); } // push for this row - for (int count = 0; count < pushCount; count++) { - push(NULL_ROW_KEY, pushIndex + count); + if (pushCount > 0) { + push(NULL_ROW_KEY, pushIndex, pushCount); + pushIndex += pushCount; } - pushIndex += pushCount; // write the results to the output chunk writeToOutputChunk(ii); @@ -89,6 +94,10 @@ public void writeToOutputChunk(int outIdx) { outputValues.set(outIdx, curVal); } + void writeNullToOutputChunk(int outIdx) { + outputValues.set(outIdx, NULL_LONG); + } + @Override public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java index 55cfc68836c..5883d1d8256 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java @@ -62,16 +62,21 @@ public void accumulate(RowSequence inputKeys, final int pushCount = pushChunk.get(ii); final int popCount = popChunk.get(ii); + if (pushCount == NULL_INT) { + writeNullToOutputChunk(ii); + continue; + } + // pop for this row - for (int count = 0; count < popCount; count++) { - pop(); + if (popCount > 0) { + pop(popCount); } // push for this row - for (int count = 0; count < pushCount; count++) { - push(NULL_ROW_KEY, pushIndex + count); + if (pushCount > 0) { + push(NULL_ROW_KEY, pushIndex, pushCount); + pushIndex += pushCount; } - pushIndex += pushCount; // write the results to the output chunk writeToOutputChunk(ii); @@ -92,6 +97,10 @@ public void writeToOutputChunk(int outIdx) { outputValues.set(outIdx, curVal); } + void writeNullToOutputChunk(int outIdx) { + outputValues.set(outIdx, null); + } + @Override public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java index 48a287d439c..4156d242975 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java @@ -59,16 +59,21 @@ public void accumulate(RowSequence inputKeys, final int pushCount = pushChunk.get(ii); final int popCount = popChunk.get(ii); + if (pushCount == NULL_INT) { + writeNullToOutputChunk(ii); + continue; + } + // pop for this row - for (int count = 0; count < popCount; count++) { - pop(); + if (popCount > 0) { + pop(popCount); } // push for this row - for (int count = 0; count < pushCount; count++) { - push(NULL_ROW_KEY, pushIndex + count); + if (pushCount > 0) { + push(NULL_ROW_KEY, pushIndex, pushCount); + pushIndex += pushCount; } - pushIndex += pushCount; // write the results to the output chunk writeToOutputChunk(ii); @@ -89,6 +94,10 @@ public void writeToOutputChunk(int outIdx) { outputValues.set(outIdx, curVal); } + void writeNullToOutputChunk(int outIdx) { + outputValues.set(outIdx, NULL_SHORT); + } + @Override public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java index cf6de3cec26..486d736a840 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java @@ -146,13 +146,19 @@ public double evaluate() { return storageChunk.get(1); } - private void grow() { + private void grow(int increase) { int oldCapacity = capacity; int oldChunkSize = chunkSize; - // double the current capacity - capacity *= 2; - chunkSize = capacity * 2; + int size = size(); + + final int minLength = size + increase; + + // double the current capacity until there is sufficient space for the increase + while (capacity <= minLength) { + capacity *= 2; + chunkSize = capacity * 2; + } // transfer to the new chunk WritableDoubleChunk oldChunk = storageChunk; @@ -162,14 +168,16 @@ private void grow() { storageChunk.fillWithValue(0, capacity, emptyVal); // move the data to the new chunk, note that we store the ring data in the second half of the array - if (tail > head) { - storageChunk.copyFromTypedChunk(oldChunk, head, capacity, tail - head); - tail = capacity + tail - head; + + if (tail >= head) { + storageChunk.copyFromTypedChunk(oldChunk, head, capacity, size); } else { - storageChunk.copyFromTypedChunk(oldChunk, head, capacity, oldChunkSize - head); - storageChunk.copyFromTypedChunk(oldChunk, oldCapacity, oldChunkSize - head + capacity, tail - oldCapacity); - tail = capacity + oldCapacity - 1; + final int firstCopyLen = oldChunkSize - head; + storageChunk.copyFromTypedChunk(oldChunk, head, capacity, firstCopyLen); + storageChunk.copyFromTypedChunk(oldChunk, oldCapacity, capacity + firstCopyLen , size - firstCopyLen); } + tail = capacity + size; + // fill the unused storage with the empty value storageChunk.fillWithValue(tail, chunkSize - tail, emptyVal); @@ -184,6 +192,10 @@ private void grow() { allDirty = true; } + private void grow() { + grow(1); + } + public void push(double val) { if (isFull()) { grow(); @@ -198,6 +210,31 @@ public void push(double val) { tail = ((tail + 1) % capacity) + capacity; } + public void pushUnsafe(double val) { + // add the new data + storageChunk.set(tail, val); + if (!allDirty) { + dirtyIndices.add(tail); + } + + // move the tail + tail = ((tail + 1) % capacity) + capacity; + } + + /** + * Ensure that there is sufficient empty space to store {@code count} items in the buffer. If the buffer is + * {@code growable}, this may result in an internal growth operation. This call should be used in conjunction with + * {@link #pushUnsafe(double)}. + * + * @param count the amount of empty entries in the buffer after this call + * @throws UnsupportedOperationException when {@code growable} is {@code false} and buffer is full + */ + public void ensureRemaining(int count) { + if (remaining() < count) { + grow(count); + } + } + public void pushEmptyValue() { push(emptyVal); } @@ -217,6 +254,53 @@ public double pop() { return val; } + public double popUnsafe() { + double val = storageChunk.get(head); + storageChunk.set(head, emptyVal); + if (!allDirty) { + dirtyIndices.add(head); + } + + // move the head + head = ((head + 1) % capacity) + capacity; + return val; + } + + public double[] pop(int count) { + if (size() < count) { + throw new NoSuchElementException(); + } + final double[] result = new double[count]; + final int firstCopyLen = chunkSize - head; + + if (tail > head || firstCopyLen >= count) { + storageChunk.copyToArray(head, result, 0, count); + storageChunk.fillWithValue(head, count, emptyVal); + if (!allDirty) { + for (int ii = 0; ii < count; ii++) { + dirtyIndices.add(head + ii); + } + } + } else { + storageChunk.copyToArray(head, result, 0, firstCopyLen); + storageChunk.fillWithValue(head, firstCopyLen, emptyVal); + storageChunk.copyToArray(capacity, result, firstCopyLen, count - firstCopyLen); + storageChunk.fillWithValue(capacity, count - firstCopyLen, emptyVal); + if (!allDirty) { + for (int ii = 0; ii < firstCopyLen; ii++) { + dirtyIndices.add(head + ii); + } + for (int ii = 0; ii < count - firstCopyLen; ii++) { + dirtyIndices.add(capacity + ii); + } + } + } + + // move the head + head = ((head + count) % capacity) + capacity; + return result; + } + public boolean isFull() { return ((tail + 1) % capacity) + capacity == head; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java index 2cd2e19ee3b..2bd2fbdb391 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java @@ -141,13 +141,19 @@ public float evaluate() { return storageChunk.get(1); } - private void grow() { + private void grow(int increase) { int oldCapacity = capacity; int oldChunkSize = chunkSize; - // double the current capacity - capacity *= 2; - chunkSize = capacity * 2; + int size = size(); + + final int minLength = size + increase; + + // double the current capacity until there is sufficient space for the increase + while (capacity <= minLength) { + capacity *= 2; + chunkSize = capacity * 2; + } // transfer to the new chunk WritableFloatChunk oldChunk = storageChunk; @@ -157,14 +163,16 @@ private void grow() { storageChunk.fillWithValue(0, capacity, emptyVal); // move the data to the new chunk, note that we store the ring data in the second half of the array - if (tail > head) { - storageChunk.copyFromTypedChunk(oldChunk, head, capacity, tail - head); - tail = capacity + tail - head; + + if (tail >= head) { + storageChunk.copyFromTypedChunk(oldChunk, head, capacity, size); } else { - storageChunk.copyFromTypedChunk(oldChunk, head, capacity, oldChunkSize - head); - storageChunk.copyFromTypedChunk(oldChunk, oldCapacity, oldChunkSize - head + capacity, tail - oldCapacity); - tail = capacity + oldCapacity - 1; + final int firstCopyLen = oldChunkSize - head; + storageChunk.copyFromTypedChunk(oldChunk, head, capacity, firstCopyLen); + storageChunk.copyFromTypedChunk(oldChunk, oldCapacity, capacity + firstCopyLen , size - firstCopyLen); } + tail = capacity + size; + // fill the unused storage with the empty value storageChunk.fillWithValue(tail, chunkSize - tail, emptyVal); @@ -179,6 +187,10 @@ private void grow() { allDirty = true; } + private void grow() { + grow(1); + } + public void push(float val) { if (isFull()) { grow(); @@ -193,6 +205,31 @@ public void push(float val) { tail = ((tail + 1) % capacity) + capacity; } + public void pushUnsafe(float val) { + // add the new data + storageChunk.set(tail, val); + if (!allDirty) { + dirtyIndices.add(tail); + } + + // move the tail + tail = ((tail + 1) % capacity) + capacity; + } + + /** + * Ensure that there is sufficient empty space to store {@code count} items in the buffer. If the buffer is + * {@code growable}, this may result in an internal growth operation. This call should be used in conjunction with + * {@link #pushUnsafe(float)}. + * + * @param count the amount of empty entries in the buffer after this call + * @throws UnsupportedOperationException when {@code growable} is {@code false} and buffer is full + */ + public void ensureRemaining(int count) { + if (remaining() < count) { + grow(count); + } + } + public void pushEmptyValue() { push(emptyVal); } @@ -212,6 +249,53 @@ public float pop() { return val; } + public float popUnsafe() { + float val = storageChunk.get(head); + storageChunk.set(head, emptyVal); + if (!allDirty) { + dirtyIndices.add(head); + } + + // move the head + head = ((head + 1) % capacity) + capacity; + return val; + } + + public float[] pop(int count) { + if (size() < count) { + throw new NoSuchElementException(); + } + final float[] result = new float[count]; + final int firstCopyLen = chunkSize - head; + + if (tail > head || firstCopyLen >= count) { + storageChunk.copyToArray(head, result, 0, count); + storageChunk.fillWithValue(head, count, emptyVal); + if (!allDirty) { + for (int ii = 0; ii < count; ii++) { + dirtyIndices.add(head + ii); + } + } + } else { + storageChunk.copyToArray(head, result, 0, firstCopyLen); + storageChunk.fillWithValue(head, firstCopyLen, emptyVal); + storageChunk.copyToArray(capacity, result, firstCopyLen, count - firstCopyLen); + storageChunk.fillWithValue(capacity, count - firstCopyLen, emptyVal); + if (!allDirty) { + for (int ii = 0; ii < firstCopyLen; ii++) { + dirtyIndices.add(head + ii); + } + for (int ii = 0; ii < count - firstCopyLen; ii++) { + dirtyIndices.add(capacity + ii); + } + } + } + + // move the head + head = ((head + count) % capacity) + capacity; + return result; + } + public boolean isFull() { return ((tail + 1) % capacity) + capacity == head; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java index 2899a5396ef..e475066c626 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java @@ -5,6 +5,7 @@ */ package io.deephaven.engine.table.impl.updateby.minmax; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ByteChunk; import io.deephaven.chunk.attributes.Values; @@ -35,7 +36,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { + Assert.eq(count, "push count", 1); + // read the value from the values chunk final byte currentVal = byteValueChunk.get(pos); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java index 7b866e41c02..97616a166de 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java @@ -5,6 +5,7 @@ */ package io.deephaven.engine.table.impl.updateby.minmax; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.DoubleChunk; import io.deephaven.chunk.attributes.Values; @@ -35,7 +36,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { + Assert.eq(count, "push count", 1); + // read the value from the values chunk final double currentVal = doubleValueChunk.get(pos); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java index 21b523cd9b3..6a0077e6e28 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java @@ -1,5 +1,6 @@ package io.deephaven.engine.table.impl.updateby.minmax; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.FloatChunk; import io.deephaven.chunk.attributes.Values; @@ -30,7 +31,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { + Assert.eq(count, "push count", 1); + // read the value from the values chunk final float currentVal = floatValueChunk.get(pos); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java index 54ead8a9d9c..c7f16cc62ed 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java @@ -5,6 +5,7 @@ */ package io.deephaven.engine.table.impl.updateby.minmax; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.attributes.Values; @@ -35,7 +36,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { + Assert.eq(count, "push count", 1); + // read the value from the values chunk final int currentVal = intValueChunk.get(pos); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java index ac848323665..503ab7cefc9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java @@ -12,6 +12,7 @@ import java.time.Instant; import io.deephaven.engine.table.impl.sources.ReinterpretUtils; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; @@ -43,7 +44,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { + Assert.eq(count, "push count", 1); + // read the value from the values chunk final long currentVal = longValueChunk.get(pos); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java index 3b955581750..05c169c60f4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java @@ -1,5 +1,6 @@ package io.deephaven.engine.table.impl.updateby.minmax; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ShortChunk; import io.deephaven.chunk.attributes.Values; @@ -30,7 +31,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { + Assert.eq(count, "push count", 1); + // read the value from the values chunk final short currentVal = shortValueChunk.get(pos); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java index ac9607aa0a2..24ed54f4ac0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java @@ -1,5 +1,6 @@ package io.deephaven.engine.table.impl.updateby.prod; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.attributes.Values; @@ -29,7 +30,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { + Assert.eq(count, "push count", 1); + // read the value from the values chunk final BigDecimal currentVal = objectValueChunk.get(pos); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java index 21a5a737b5b..1a773e0f776 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java @@ -1,5 +1,6 @@ package io.deephaven.engine.table.impl.updateby.prod; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.attributes.Values; @@ -25,7 +26,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { + Assert.eq(count, "push count", 1); + // read the value from the values chunk final BigInteger currentVal = objectValueChunk.get(pos); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java index 0bdcc4d502b..a0033df0b62 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java @@ -5,6 +5,7 @@ */ package io.deephaven.engine.table.impl.updateby.prod; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ByteChunk; import io.deephaven.chunk.attributes.Values; @@ -34,7 +35,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { + Assert.eq(count, "push count", 1); + // read the value from the values chunk final byte currentVal = byteValueChunk.get(pos); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java index c179a412b28..ebc603c882a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java @@ -5,6 +5,7 @@ */ package io.deephaven.engine.table.impl.updateby.prod; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.DoubleChunk; import io.deephaven.chunk.attributes.Values; @@ -33,7 +34,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { + Assert.eq(count, "push count", 1); + // read the value from the values chunk final double currentVal = doubleValueChunk.get(pos); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java index 537f608c553..5efd40cb4da 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java @@ -1,5 +1,6 @@ package io.deephaven.engine.table.impl.updateby.prod; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.FloatChunk; import io.deephaven.chunk.attributes.Values; @@ -28,7 +29,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { + Assert.eq(count, "push count", 1); + // read the value from the values chunk final float currentVal = floatValueChunk.get(pos); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java index 5b1edb348f8..2a0ebab5a26 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java @@ -5,6 +5,7 @@ */ package io.deephaven.engine.table.impl.updateby.prod; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.attributes.Values; @@ -34,7 +35,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { + Assert.eq(count, "push count", 1); + // read the value from the values chunk final int currentVal = intValueChunk.get(pos); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java index a546256f7bb..07caa35394f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java @@ -5,6 +5,7 @@ */ package io.deephaven.engine.table.impl.updateby.prod; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; @@ -34,7 +35,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { + Assert.eq(count, "push count", 1); + // read the value from the values chunk final long currentVal = longValueChunk.get(pos); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java index fe47c108379..feceae27c15 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java @@ -1,5 +1,6 @@ package io.deephaven.engine.table.impl.updateby.prod; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ShortChunk; import io.deephaven.chunk.attributes.Values; @@ -29,7 +30,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { + Assert.eq(count, "push count", 1); + // read the value from the values chunk final short currentVal = shortValueChunk.get(pos); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java index f60ded6b5a1..016ea10ac86 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java @@ -42,31 +42,36 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { - BigDecimal val = objectInfluencerValuesChunk.get(pos); - objectWindowValues.add(val); - - // increase the running sum - if (val != null) { - if (curVal == null) { - curVal = val; + public void push(long key, int pos, int count) { + for (int ii = 0; ii < count; ii++) { + BigDecimal val = objectInfluencerValuesChunk.get(pos + ii); + objectWindowValues.add(val); + + // increase the running sum + if (val != null) { + if (curVal == null) { + curVal = val; + } else { + curVal = curVal.add(val, mathContext); + } } else { - curVal = curVal.add(val, mathContext); + nullCount++; } - } else { - nullCount++; } } @Override - public void pop() { - BigDecimal val = objectWindowValues.remove(); + public void pop(int count) { + for (int ii = 0; ii < count; ii++) { + BigDecimal val = objectWindowValues.remove(); - // reduce the running sum - if (val != null) { - curVal = curVal.subtract(val, mathContext); - } else { - nullCount--; + // reduce the running sum + if (val != null) { + curVal = curVal.subtract(val, mathContext); + } else { + nullCount--; + + } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java index eb4663bfcca..d0d32926a61 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java @@ -40,31 +40,35 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { - BigInteger val = objectInfluencerValuesChunk.get(pos); - objectWindowValues.add(val); - - // increase the running sum - if (val != null) { - if (curVal == null) { - curVal = val; + public void push(long key, int pos, int count) { + for (int ii = 0; ii < count; ii++) { + BigInteger val = objectInfluencerValuesChunk.get(pos + ii); + objectWindowValues.add(val); + + // increase the running sum + if (val != null) { + if (curVal == null) { + curVal = val; + } else { + curVal = curVal.add(val); + } } else { - curVal = curVal.add(val); + nullCount++; } - } else { - nullCount++; } } @Override - public void pop() { - BigInteger val = objectWindowValues.remove(); + public void pop(int count) { + for (int ii = 0; ii < count; ii++) { + BigInteger val = objectWindowValues.remove(); - // reduce the running sum - if (val != null) { - curVal = curVal.subtract(val); - } else { - nullCount--; + // reduce the running sum + if (val != null) { + curVal = curVal.subtract(val); + } else { + nullCount--; + } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java index 00f165eea5d..936c510993b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java @@ -7,6 +7,7 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.base.ringbuffer.ByteRingBuffer; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ByteChunk; import io.deephaven.chunk.attributes.Values; @@ -47,31 +48,39 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { - byte val = byteInfluencerValuesChunk.get(pos); - byteWindowValues.add(val); - - // increase the running sum - if (val != NULL_BYTE) { - if (curVal == NULL_LONG) { - curVal = val; + public void push(long key, int pos, int count) { + byteWindowValues.ensureRemaining(count); + + for (int ii = 0; ii < count; ii++) { + byte val = byteInfluencerValuesChunk.get(pos + ii); + byteWindowValues.addUnsafe(val); + + // increase the running sum + if (val != NULL_BYTE) { + if (curVal == NULL_LONG) { + curVal = val; + } else { + curVal += val; + } } else { - curVal += val; + nullCount++; } - } else { - nullCount++; } } @Override - public void pop() { - byte val = byteWindowValues.remove(); + public void pop(int count) { + Assert.geq(byteWindowValues.size(), "byteWindowValues.size()", count); - // reduce the running sum - if (val != NULL_BYTE) { - curVal -= val; - } else { - nullCount--; + for (int ii = 0; ii < count; ii++) { + byte val = byteWindowValues.removeUnsafe(); + + // reduce the running sum + if (val != NULL_BYTE) { + curVal -= val; + } else { + nullCount--; + } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java index 54181987157..5474d1e8261 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java @@ -6,6 +6,7 @@ package io.deephaven.engine.table.impl.updateby.rollingsum; import io.deephaven.api.updateby.OperationControl; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.DoubleChunk; import io.deephaven.chunk.attributes.Values; @@ -21,6 +22,7 @@ import java.util.Map; import static io.deephaven.util.QueryConstants.NULL_DOUBLE; +import static io.deephaven.util.QueryConstants.NULL_SHORT; public class DoubleRollingSumOperator extends BaseWindowedDoubleUpdateByOperator { private static final int PAIRWISE_BUFFER_INITIAL_SIZE = 64; @@ -55,21 +57,29 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { - double val = doubleInfluencerValuesChunk.get(pos); + public void push(long key, int pos, int count) { + doublePairwiseSum.ensureRemaining(count); - doublePairwiseSum.push(val); - if (val == NULL_DOUBLE) { - nullCount++; + for (int ii = 0; ii < count; ii++) { + double val = doubleInfluencerValuesChunk.get(pos + ii); + doublePairwiseSum.pushUnsafe(val); + + if (val == NULL_DOUBLE) { + nullCount++; + } } } @Override - public void pop() { - double val = doublePairwiseSum.pop(); + public void pop(int count) { + Assert.geq(doublePairwiseSum.size(), "shortWindowValues.size()", count); - if (val == NULL_DOUBLE) { - nullCount--; + for (int ii = 0; ii < count; ii++) { + double val = doublePairwiseSum.popUnsafe(); + + if (val == NULL_DOUBLE) { + nullCount--; + } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java index a8237b6c840..0ed74b9af37 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java @@ -1,6 +1,7 @@ package io.deephaven.engine.table.impl.updateby.rollingsum; import io.deephaven.api.updateby.OperationControl; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.FloatChunk; import io.deephaven.chunk.attributes.Values; @@ -16,6 +17,7 @@ import java.util.Map; import static io.deephaven.util.QueryConstants.NULL_FLOAT; +import static io.deephaven.util.QueryConstants.NULL_SHORT; public class FloatRollingSumOperator extends BaseWindowedFloatUpdateByOperator { private static final int PAIRWISE_BUFFER_INITIAL_SIZE = 64; @@ -50,21 +52,29 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { - float val = floatInfluencerValuesChunk.get(pos); + public void push(long key, int pos, int count) { + floatPairwiseSum.ensureRemaining(count); - floatPairwiseSum.push(val); - if (val == NULL_FLOAT) { - nullCount++; + for (int ii = 0; ii < count; ii++) { + float val = floatInfluencerValuesChunk.get(pos + ii); + floatPairwiseSum.pushUnsafe(val); + + if (val == NULL_FLOAT) { + nullCount++; + } } } @Override - public void pop() { - float val = floatPairwiseSum.pop(); + public void pop(int count) { + Assert.geq(floatPairwiseSum.size(), "shortWindowValues.size()", count); - if (val == NULL_FLOAT) { - nullCount--; + for (int ii = 0; ii < count; ii++) { + float val = floatPairwiseSum.popUnsafe(); + + if (val == NULL_FLOAT) { + nullCount--; + } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java index 159352884ca..c0bb7d97648 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java @@ -7,6 +7,7 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.base.ringbuffer.IntRingBuffer; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.attributes.Values; @@ -46,31 +47,39 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { - int val = intInfluencerValuesChunk.get(pos); - intWindowValues.add(val); - - // increase the running sum - if (val != NULL_INT) { - if (curVal == NULL_LONG) { - curVal = val; + public void push(long key, int pos, int count) { + intWindowValues.ensureRemaining(count); + + for (int ii = 0; ii < count; ii++) { + int val = intInfluencerValuesChunk.get(pos + ii); + intWindowValues.addUnsafe(val); + + // increase the running sum + if (val != NULL_INT) { + if (curVal == NULL_LONG) { + curVal = val; + } else { + curVal += val; + } } else { - curVal += val; + nullCount++; } - } else { - nullCount++; } } @Override - public void pop() { - int val = intWindowValues.remove(); + public void pop(int count) { + Assert.geq(intWindowValues.size(), "intWindowValues.size()", count); - // reduce the running sum - if (val != NULL_INT) { - curVal -= val; - } else { - nullCount--; + for (int ii = 0; ii < count; ii++) { + int val = intWindowValues.removeUnsafe(); + + // reduce the running sum + if (val != NULL_INT) { + curVal -= val; + } else { + nullCount--; + } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java index 5abe8b76cc5..a1f5e89cd44 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java @@ -7,6 +7,7 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.base.ringbuffer.LongRingBuffer; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; @@ -46,31 +47,39 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { - long val = longInfluencerValuesChunk.get(pos); - longWindowValues.add(val); - - // increase the running sum - if (val != NULL_LONG) { - if (curVal == NULL_LONG) { - curVal = val; + public void push(long key, int pos, int count) { + longWindowValues.ensureRemaining(count); + + for (int ii = 0; ii < count; ii++) { + long val = longInfluencerValuesChunk.get(pos + ii); + longWindowValues.addUnsafe(val); + + // increase the running sum + if (val != NULL_LONG) { + if (curVal == NULL_LONG) { + curVal = val; + } else { + curVal += val; + } } else { - curVal += val; + nullCount++; } - } else { - nullCount++; } } @Override - public void pop() { - long val = longWindowValues.remove(); + public void pop(int count) { + Assert.geq(longWindowValues.size(), "longWindowValues.size()", count); - // reduce the running sum - if (val != NULL_LONG) { - curVal -= val; - } else { - nullCount--; + for (int ii = 0; ii < count; ii++) { + long val = longWindowValues.removeUnsafe(); + + // reduce the running sum + if (val != NULL_LONG) { + curVal -= val; + } else { + nullCount--; + } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java index e70796f1eb6..c6407781d1e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java @@ -2,6 +2,7 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.base.ringbuffer.ShortRingBuffer; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ShortChunk; import io.deephaven.chunk.attributes.Values; @@ -41,31 +42,39 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { - short val = shortInfluencerValuesChunk.get(pos); - shortWindowValues.add(val); - - // increase the running sum - if (val != NULL_SHORT) { - if (curVal == NULL_LONG) { - curVal = val; + public void push(long key, int pos, int count) { + shortWindowValues.ensureRemaining(count); + + for (int ii = 0; ii < count; ii++) { + short val = shortInfluencerValuesChunk.get(pos + ii); + shortWindowValues.addUnsafe(val); + + // increase the running sum + if (val != NULL_SHORT) { + if (curVal == NULL_LONG) { + curVal = val; + } else { + curVal += val; + } } else { - curVal += val; + nullCount++; } - } else { - nullCount++; } } @Override - public void pop() { - short val = shortWindowValues.remove(); + public void pop(int count) { + Assert.geq(shortWindowValues.size(), "shortWindowValues.size()", count); - // reduce the running sum - if (val != NULL_SHORT) { - curVal -= val; - } else { - nullCount--; + for (int ii = 0; ii < count; ii++) { + short val = shortWindowValues.removeUnsafe(); + + // reduce the running sum + if (val != NULL_SHORT) { + curVal -= val; + } else { + nullCount--; + } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java index c435b0639ae..3d27c0728d2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java @@ -1,5 +1,6 @@ package io.deephaven.engine.table.impl.updateby.sum; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.attributes.Values; @@ -30,7 +31,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { + Assert.eq(count, "push count", 1); + // read the value from the values chunk final BigDecimal currentVal = objectValueChunk.get(pos); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java index 05c154f19df..bc463efd278 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java @@ -1,5 +1,6 @@ package io.deephaven.engine.table.impl.updateby.sum; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.attributes.Values; @@ -25,7 +26,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { + Assert.eq(count, "push count", 1); + // read the value from the values chunk final BigInteger currentVal = objectValueChunk.get(pos); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java index b5f2141c925..f7c94aba0d9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java @@ -5,6 +5,7 @@ */ package io.deephaven.engine.table.impl.updateby.sum; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ByteChunk; import io.deephaven.chunk.attributes.Values; @@ -35,7 +36,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { + Assert.eq(count, "push count", 1); + // read the value from the values chunk final byte currentVal = byteValueChunk.get(pos); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java index 03df794c6cc..3fdd0dece62 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java @@ -5,6 +5,7 @@ */ package io.deephaven.engine.table.impl.updateby.sum; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.DoubleChunk; import io.deephaven.chunk.attributes.Values; @@ -33,7 +34,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { + Assert.eq(count, "push count", 1); + // read the value from the values chunk final double currentVal = doubleValueChunk.get(pos); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java index 58efcc76409..8893d3eeb9d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java @@ -1,5 +1,6 @@ package io.deephaven.engine.table.impl.updateby.sum; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.FloatChunk; import io.deephaven.chunk.attributes.Values; @@ -28,7 +29,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { + Assert.eq(count, "push count", 1); + // read the value from the values chunk final float currentVal = floatValueChunk.get(pos); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java index b2770ba525f..c2d55c041fa 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java @@ -5,6 +5,7 @@ */ package io.deephaven.engine.table.impl.updateby.sum; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.attributes.Values; @@ -34,7 +35,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { + Assert.eq(count, "push count", 1); + // read the value from the values chunk final int currentVal = intValueChunk.get(pos); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java index 7dadcb6e1f6..300b0f52eef 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java @@ -5,6 +5,7 @@ */ package io.deephaven.engine.table.impl.updateby.sum; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; @@ -34,7 +35,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { + Assert.eq(count, "push count", 1); + // read the value from the values chunk final long currentVal = longValueChunk.get(pos); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java index d9275ea207a..a0967d15637 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java @@ -1,5 +1,6 @@ package io.deephaven.engine.table.impl.updateby.sum; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ShortChunk; import io.deephaven.chunk.attributes.Values; @@ -29,7 +30,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { } @Override - public void push(long key, int pos) { + public void push(long key, int pos, int count) { + Assert.eq(count, "push count", 1); + // read the value from the values chunk final short currentVal = shortValueChunk.get(pos); diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java index b700004b7eb..ae1dc15e601 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java @@ -38,15 +38,15 @@ @Category(OutOfBandTest.class) public class TestRollingSum extends BaseUpdateByTest { - // region Zero Key Tests + // region Static Zero Key Tests @Test - public void testStaticZeroKey() { + public void testStaticZeroKeyRev() { final QueryTable t = createTestTable(10000, false, false, false, 0x31313131).t; t.setRefreshing(false); - int prevTicks = 100; - int postTicks = 0; + final int prevTicks = 100; + final int postTicks = 0; final Table summed = t.updateBy(UpdateByOperation.RollingSum(prevTicks, postTicks)); for (String col : t.getDefinition().getColumnNamesArray()) { @@ -56,12 +56,42 @@ public void testStaticZeroKey() { } @Test - public void testStaticZeroKeyFwdWindow() { + public void testStaticZeroKeyRevExclusive() { final QueryTable t = createTestTable(10000, false, false, false, 0x31313131).t; t.setRefreshing(false); - int prevTicks = 0; - int postTicks = 100; + final int prevTicks = 100; + final int postTicks = -50; + + final Table summed = t.updateBy(UpdateByOperation.RollingSum(prevTicks, postTicks)); + for (String col : t.getDefinition().getColumnNamesArray()) { + assertWithRollingSumTicks(t.getColumn(col).getDirect(), summed.getColumn(col).getDirect(), + summed.getColumn(col).getType(), prevTicks, postTicks); + } + } + + @Test + public void testStaticZeroKeyFwd() { + final QueryTable t = createTestTable(10000, false, false, false, 0x31313131).t; + t.setRefreshing(false); + + final int prevTicks = 0; + final int postTicks = 100; + + final Table summed = t.updateBy(UpdateByOperation.RollingSum(prevTicks, postTicks)); + for (String col : t.getDefinition().getColumnNamesArray()) { + assertWithRollingSumTicks(t.getColumn(col).getDirect(), summed.getColumn(col).getDirect(), + summed.getColumn(col).getType(), prevTicks, postTicks); + } + } + + @Test + public void testStaticZeroKeyFwdExclusive() { + final QueryTable t = createTestTable(10000, false, false, false, 0x31313131).t; + t.setRefreshing(false); + + final int prevTicks = -50; + final int postTicks = 100; final Table summed = t.updateBy(UpdateByOperation.RollingSum(prevTicks, postTicks)); for (String col : t.getDefinition().getColumnNamesArray()) { @@ -75,8 +105,8 @@ public void testStaticZeroKeyFwdRevWindow() { final QueryTable t = createTestTable(10000, false, false, false, 0x31313131).t; t.setRefreshing(false); - int prevTicks = 100; - int postTicks = 100; + final int prevTicks = 100; + final int postTicks = 100; final Table summed = t.updateBy(UpdateByOperation.RollingSum(prevTicks, postTicks)); for (String col : t.getDefinition().getColumnNamesArray()) { @@ -86,14 +116,42 @@ public void testStaticZeroKeyFwdRevWindow() { } @Test - public void testStaticZeroKeyTimed() { + public void testStaticZeroKeyTimedRev() { + final QueryTable t = createTestTable(10000, false, false, false, 0xFFFABBBC, + new String[] {"ts"}, new TestDataGenerator[] {new SortedDateTimeGenerator( + convertDateTime("2022-03-09T09:00:00.000 NY"), + convertDateTime("2022-03-09T16:30:00.000 NY"))}).t; + + final Duration prevTime = Duration.ofMinutes(10); + final Duration postTime = Duration.ZERO; + + final Table summed = + t.updateBy(UpdateByOperation.RollingSum("ts", prevTime, postTime, "byteCol", "shortCol", "intCol", + "longCol", "floatCol", + "doubleCol", "boolCol", "bigIntCol", "bigDecimalCol")); + + + final DateTime[] ts = (DateTime[]) t.getColumn("ts").getDirect(); + final long[] timestamps = new long[t.intSize()]; + for (int i = 0; i < t.intSize(); i++) { + timestamps[i] = ts[i].getNanos(); + } + + for (String col : t.getDefinition().getColumnNamesArray()) { + assertWithRollingSumTime(t.getColumn(col).getDirect(), summed.getColumn(col).getDirect(), timestamps, + summed.getColumn(col).getType(), prevTime.toNanos(), postTime.toNanos()); + } + } + + @Test + public void testStaticZeroKeyTimedRevExclusive() { final QueryTable t = createTestTable(10000, false, false, false, 0xFFFABBBC, new String[] {"ts"}, new TestDataGenerator[] {new SortedDateTimeGenerator( convertDateTime("2022-03-09T09:00:00.000 NY"), convertDateTime("2022-03-09T16:30:00.000 NY"))}).t; - Duration prevTime = Duration.ofMinutes(10); - Duration postTime = Duration.ZERO; + final Duration prevTime = Duration.ofMinutes(10); + final Duration postTime = Duration.ofMinutes(-5); final Table summed = t.updateBy(UpdateByOperation.RollingSum("ts", prevTime, postTime, "byteCol", "shortCol", "intCol", @@ -101,8 +159,8 @@ public void testStaticZeroKeyTimed() { "doubleCol", "boolCol", "bigIntCol", "bigDecimalCol")); - DateTime[] ts = (DateTime[]) t.getColumn("ts").getDirect(); - long[] timestamps = new long[t.intSize()]; + final DateTime[] ts = (DateTime[]) t.getColumn("ts").getDirect(); + final long[] timestamps = new long[t.intSize()]; for (int i = 0; i < t.intSize(); i++) { timestamps[i] = ts[i].getNanos(); } @@ -114,14 +172,14 @@ public void testStaticZeroKeyTimed() { } @Test - public void testStaticZeroKeyFwdWindowTimed() { + public void testStaticZeroKeyTimedFwd() { final QueryTable t = createTestTable(10000, false, false, false, 0xFFFABBBC, new String[] {"ts"}, new TestDataGenerator[] {new SortedDateTimeGenerator( convertDateTime("2022-03-09T09:00:00.000 NY"), convertDateTime("2022-03-09T16:30:00.000 NY"))}).t; - Duration prevTime = Duration.ZERO; - Duration postTime = Duration.ofMinutes(10); + final Duration prevTime = Duration.ZERO; + final Duration postTime = Duration.ofMinutes(10); final Table summed = t.updateBy(UpdateByOperation.RollingSum("ts", prevTime, postTime, "byteCol", "shortCol", "intCol", @@ -129,8 +187,8 @@ public void testStaticZeroKeyFwdWindowTimed() { "doubleCol", "boolCol", "bigIntCol", "bigDecimalCol")); - DateTime[] ts = (DateTime[]) t.getColumn("ts").getDirect(); - long[] timestamps = new long[t.intSize()]; + final DateTime[] ts = (DateTime[]) t.getColumn("ts").getDirect(); + final long[] timestamps = new long[t.intSize()]; for (int i = 0; i < t.intSize(); i++) { timestamps[i] = ts[i].getNanos(); } @@ -142,14 +200,14 @@ public void testStaticZeroKeyFwdWindowTimed() { } @Test - public void testStaticZeroKeyFwdRevWindowTimed() { + public void testStaticZeroKeyTimedFwdExclusive() { final QueryTable t = createTestTable(10000, false, false, false, 0xFFFABBBC, new String[] {"ts"}, new TestDataGenerator[] {new SortedDateTimeGenerator( convertDateTime("2022-03-09T09:00:00.000 NY"), convertDateTime("2022-03-09T16:30:00.000 NY"))}).t; - Duration prevTime = Duration.ofMinutes(10); - Duration postTime = Duration.ofMinutes(10); + final Duration prevTime = Duration.ofMinutes(-5); + final Duration postTime = Duration.ofMinutes(10); final Table summed = t.updateBy(UpdateByOperation.RollingSum("ts", prevTime, postTime, "byteCol", "shortCol", "intCol", @@ -157,8 +215,36 @@ public void testStaticZeroKeyFwdRevWindowTimed() { "doubleCol", "boolCol", "bigIntCol", "bigDecimalCol")); - DateTime[] ts = (DateTime[]) t.getColumn("ts").getDirect(); - long[] timestamps = new long[t.intSize()]; + final DateTime[] ts = (DateTime[]) t.getColumn("ts").getDirect(); + final long[] timestamps = new long[t.intSize()]; + for (int i = 0; i < t.intSize(); i++) { + timestamps[i] = ts[i].getNanos(); + } + + for (String col : t.getDefinition().getColumnNamesArray()) { + assertWithRollingSumTime(t.getColumn(col).getDirect(), summed.getColumn(col).getDirect(), timestamps, + summed.getColumn(col).getType(), prevTime.toNanos(), postTime.toNanos()); + } + } + + @Test + public void testStaticZeroKeyTimedFwdRev() { + final QueryTable t = createTestTable(10000, false, false, false, 0xFFFABBBC, + new String[] {"ts"}, new TestDataGenerator[] {new SortedDateTimeGenerator( + convertDateTime("2022-03-09T09:00:00.000 NY"), + convertDateTime("2022-03-09T16:30:00.000 NY"))}).t; + + final Duration prevTime = Duration.ofMinutes(10); + final Duration postTime = Duration.ofMinutes(10); + + final Table summed = + t.updateBy(UpdateByOperation.RollingSum("ts", prevTime, postTime, "byteCol", "shortCol", "intCol", + "longCol", "floatCol", + "doubleCol", "boolCol", "bigIntCol", "bigDecimalCol")); + + + final DateTime[] ts = (DateTime[]) t.getColumn("ts").getDirect(); + final long[] timestamps = new long[t.intSize()]; for (int i = 0; i < t.intSize(); i++) { timestamps[i] = ts[i].getNanos(); } @@ -171,7 +257,7 @@ public void testStaticZeroKeyFwdRevWindowTimed() { // endregion - // region Bucketed Tests + // region Static Bucketed Tests @Test public void testNullOnBucketChange() { @@ -197,21 +283,43 @@ public void testNullOnBucketChange() { } @Test - public void testStaticBucketed() { - doTestStaticBucketed(false); + public void testStaticBucketedRev() { + final int prevTicks = 100; + final int postTicks = 0; + doTestStaticBucketed(false, prevTicks, postTicks); + } + + @Test + public void testStaticBucketedRevExclusive() { + final int prevTicks = 100; + final int postTicks = -50; + doTestStaticBucketed(false, prevTicks, postTicks); + } + + @Test + public void testStaticBucketedFwd() { + final int prevTicks = 0; + final int postTicks = 100; + doTestStaticBucketed(false, prevTicks, postTicks); + } + + @Test + public void testStaticBucketedFwdExclusive() { + final int prevTicks = -50; + final int postTicks = 100; + doTestStaticBucketed(false, prevTicks, postTicks); } @Test public void testStaticGroupedBucketed() { - doTestStaticBucketed(true); + final int prevTicks = 100; + final int postTicks = 0; + doTestStaticBucketed(true, prevTicks, postTicks); } - private void doTestStaticBucketed(boolean grouped) { + private void doTestStaticBucketed(boolean grouped, int prevTicks, int postTicks) { final QueryTable t = createTestTable(100000, true, grouped, false, 0x31313131).t; - int prevTicks = 100; - int postTicks = 10; - final Table summed = t.updateBy(UpdateByOperation.RollingSum(prevTicks, postTicks, "byteCol", "shortCol", "intCol", "longCol", "floatCol", @@ -221,7 +329,7 @@ private void doTestStaticBucketed(boolean grouped) { final PartitionedTable preOp = t.partitionBy("Sym"); final PartitionedTable postOp = summed.partitionBy("Sym"); - String[] columns = t.getDefinition().getColumnStream().map(ColumnDefinition::getName).toArray(String[]::new); + final String[] columns = t.getDefinition().getColumnStream().map(ColumnDefinition::getName).toArray(String[]::new); preOp.partitionedTransform(postOp, (source, actual) -> { Arrays.stream(columns).forEach(col -> { @@ -233,18 +341,38 @@ private void doTestStaticBucketed(boolean grouped) { } @Test - public void testStaticBucketedTimed() { - doTestStaticBucketedTimed(false, Duration.ofMinutes(10), Duration.ZERO); + public void testStaticBucketedTimedRev() { + final Duration prevTime = Duration.ofMinutes(10); + final Duration postTime = Duration.ofMinutes(0); + doTestStaticBucketedTimed(false, prevTime, postTime); } @Test - public void testStaticBucketedFwdWindowTimed() { - doTestStaticBucketedTimed(false, Duration.ZERO, Duration.ofMinutes(10)); + public void testStaticBucketedTimedRevExclusive() { + final Duration prevTime = Duration.ofMinutes(10); + final Duration postTime = Duration.ofMinutes(-5); + doTestStaticBucketedTimed(false, prevTime, postTime); + } + + @Test + public void testStaticBucketedTimedFwd() { + final Duration prevTime = Duration.ofMinutes(0); + final Duration postTime = Duration.ofMinutes(10); + doTestStaticBucketedTimed(false, prevTime, postTime); + } + + @Test + public void testStaticBucketedTimedFwdExclusive() { + final Duration prevTime = Duration.ofMinutes(-5); + final Duration postTime = Duration.ofMinutes(10); + doTestStaticBucketedTimed(false, prevTime, postTime); } @Test public void testStaticBucketedFwdRevWindowTimed() { - doTestStaticBucketedTimed(false, Duration.ofMinutes(10), Duration.ofMinutes(10)); + final Duration prevTime = Duration.ofMinutes(5); + final Duration postTime = Duration.ofMinutes(5); + doTestStaticBucketedTimed(false, prevTime, postTime); } private void doTestStaticBucketedTimed(boolean grouped, Duration prevTime, Duration postTime) { @@ -284,16 +412,69 @@ private void doTestStaticBucketedTimed(boolean grouped, Duration prevTime, Durat // region Live Tests @Test - public void testZeroKeyAppendOnly() { - doTestAppendOnly(false); + public void testZeroKeyAppendOnlyRev() { + final int prevTicks = 100; + final int postTicks = 0; + doTestAppendOnly(false, prevTicks, postTicks); + } + + @Test + public void testZeroKeyAppendOnlyRevExclusive() { + final int prevTicks = 100; + final int postTicks = -50; + doTestAppendOnly(false, prevTicks, postTicks); } @Test - public void testBucketedAppendOnly() { - doTestAppendOnly(true); + public void testZeroKeyAppendOnlyFwd() { + final int prevTicks = 0; + final int postTicks = 100; + doTestAppendOnly(false, prevTicks, postTicks); } - private void doTestAppendOnly(boolean bucketed) { + @Test + public void testZeroKeyAppendOnlyFwdExclusive() { + final int prevTicks = -50; + final int postTicks = 100; + doTestAppendOnly(false, prevTicks, postTicks); + } + + @Test + public void testZeroKeyAppendOnlyFwdRev() { + final int prevTicks = 50; + final int postTicks = 50; + doTestAppendOnly(false, prevTicks, postTicks); + } + + @Test + public void testBucketedAppendOnlyRev() { + final int prevTicks = 100; + final int postTicks = 0; + doTestAppendOnly(true, prevTicks, postTicks); + } + + @Test + public void testBucketedAppendOnlyRevExclusive() { + final int prevTicks = 100; + final int postTicks = -50; + doTestAppendOnly(true, prevTicks, postTicks); + } + + @Test + public void testBucketedAppendOnlyFwd() { + final int prevTicks = 0; + final int postTicks = 100; + doTestAppendOnly(true, prevTicks, postTicks); + } + + @Test + public void testBucketedAppendOnlyFwdExclusive() { + final int prevTicks = -50; + final int postTicks = 100; + doTestAppendOnly(true, prevTicks, postTicks); + } + + private void doTestAppendOnly(boolean bucketed, int prevTicks, int postTicks) { final CreateResult result = createTestTable(10000, bucketed, false, true, 0x31313131); final QueryTable t = result.t; t.setAttribute(Table.ADD_ONLY_TABLE_ATTRIBUTE, Boolean.TRUE); @@ -302,7 +483,7 @@ private void doTestAppendOnly(boolean bucketed) { new EvalNugget() { @Override protected Table e() { - return bucketed ? t.updateBy(UpdateByOperation.RollingSum(100), "Sym") + return bucketed ? t.updateBy(UpdateByOperation.RollingSum(prevTicks, postTicks), "Sym") : t.updateBy(UpdateByOperation.RollingSum(100)); } } @@ -316,17 +497,41 @@ protected Table e() { } @Test - public void testZeroKeyAppendOnlyTimed() { - doTestAppendOnlyTimed(false); + public void testZeroKeyAppendOnlyTimedRev() { + final Duration prevTime = Duration.ofMinutes(10); + final Duration postTime = Duration.ofMinutes(0); + doTestAppendOnlyTimed(false, prevTime, postTime); } @Test - public void testBucketedAppendOnlyTimed() { - doTestAppendOnlyTimed(true); + public void testZeroKeyAppendOnlyTimedRevExclusive() { + final Duration prevTime = Duration.ofMinutes(10); + final Duration postTime = Duration.ofMinutes(-5); + doTestAppendOnlyTimed(false, prevTime, postTime); } + @Test + public void testZeroKeyAppendOnlyTimedFwd() { + final Duration prevTime = Duration.ofMinutes(0); + final Duration postTime = Duration.ofMinutes(10); + doTestAppendOnlyTimed(false, prevTime, postTime); + } + + @Test + public void testZeroKeyAppendOnlyTimedFwdExclusive() { + final Duration prevTime = Duration.ofMinutes(-5); + final Duration postTime = Duration.ofMinutes(10); + doTestAppendOnlyTimed(false, prevTime, postTime); + } + + @Test + public void testZeroKeyAppendOnlyTimedFwdRev() { + final Duration prevTime = Duration.ofMinutes(5); + final Duration postTime = Duration.ofMinutes(5); + doTestAppendOnlyTimed(false, prevTime, postTime); + } - private void doTestAppendOnlyTimed(boolean bucketed) { + private void doTestAppendOnlyTimed(boolean bucketed, Duration prevTime, Duration postTime) { final CreateResult result = createTestTable(10000, bucketed, false, true, 0x31313131, new String[] {"ts"}, new TestDataGenerator[] {new SortedDateTimeGenerator( convertDateTime("2022-03-09T09:00:00.000 NY"), @@ -336,9 +541,6 @@ private void doTestAppendOnlyTimed(boolean bucketed) { t.setAttribute(Table.ADD_ONLY_TABLE_ATTRIBUTE, Boolean.TRUE); - Duration prevTime = Duration.ofMinutes(10); - Duration postTime = Duration.ZERO; - final EvalNugget[] nuggets = new EvalNugget[] { new EvalNugget() { @Override @@ -357,17 +559,78 @@ protected Table e() { } @Test - public void testZeroKeyGeneralTicking() { + public void testBucketedAppendOnlyTimedRev() { + final Duration prevTime = Duration.ofMinutes(10); + final Duration postTime = Duration.ofMinutes(0); + doTestAppendOnlyTimed(true, prevTime, postTime); + } + + @Test + public void testBucketedAppendOnlyTimedRevExclusive() { + final Duration prevTime = Duration.ofMinutes(10); + final Duration postTime = Duration.ofMinutes(-5); + doTestAppendOnlyTimed(true, prevTime, postTime); + } + + @Test + public void testBucketedAppendOnlyTimedFwd() { + final Duration prevTime = Duration.ofMinutes(0); + final Duration postTime = Duration.ofMinutes(10); + doTestAppendOnlyTimed(true, prevTime, postTime); + } + + @Test + public void testBucketedAppendOnlyTimedFwdExclusive() { + final Duration prevTime = Duration.ofMinutes(-5); + final Duration postTime = Duration.ofMinutes(10); + doTestAppendOnlyTimed(true, prevTime, postTime); + } + + @Test + public void testBucketedAppendOnlyTimedFwdRev() { + final Duration prevTime = Duration.ofMinutes(5); + final Duration postTime = Duration.ofMinutes(5); + doTestAppendOnlyTimed(true, prevTime, postTime); + } + + @Test + public void testZeroKeyGeneralTickingRev() { + final CreateResult result = createTestTable(10000, false, false, true, 0x31313131); + final QueryTable t = result.t; + + final long prevTicks = 100; + final long fwdTicks = 0; + + final EvalNugget[] nuggets = new EvalNugget[] { + new EvalNugget() { + @Override + protected Table e() { + return t.updateBy(UpdateByOperation.RollingSum(prevTicks, fwdTicks)); + } + } + }; + + final Random billy = new Random(0xB177B177); + for (int ii = 0; ii < 100; ii++) { + UpdateGraphProcessor.DEFAULT.runWithinUnitTestCycle( + () -> GenerateTableUpdates.generateTableUpdates(100, billy, t, result.infos)); + TstUtils.validate("Table - step " + ii, nuggets); + } + } + + @Test + public void testZeroKeyGeneralTickingRevExclusive() { final CreateResult result = createTestTable(10000, false, false, true, 0x31313131); final QueryTable t = result.t; final long prevTicks = 100; + final long fwdTicks = -50; final EvalNugget[] nuggets = new EvalNugget[] { new EvalNugget() { @Override protected Table e() { - return t.updateBy(UpdateByOperation.RollingSum(prevTicks)); + return t.updateBy(UpdateByOperation.RollingSum(prevTicks, fwdTicks)); } } }; @@ -381,7 +644,7 @@ protected Table e() { } @Test - public void testZeroKeyGeneralTickingFwdWindow() { + public void testZeroKeyGeneralTickingFwd() { final CreateResult result = createTestTable(10000, false, false, true, 0x31313131); final QueryTable t = result.t; @@ -403,7 +666,116 @@ protected Table e() { } @Test - public void testBucketedGeneralTicking() { + public void testZeroKeyGeneralTickingFwdExclusive() { + final CreateResult result = createTestTable(10000, false, false, true, 0x31313131); + final QueryTable t = result.t; + + final EvalNugget[] nuggets = new EvalNugget[] { + new EvalNugget() { + @Override + protected Table e() { + return t.updateBy(UpdateByOperation.RollingSum(-50, 100)); + } + } + }; + + final Random billy = new Random(0xB177B177); + for (int ii = 0; ii < 100; ii++) { + UpdateGraphProcessor.DEFAULT.runWithinUnitTestCycle( + () -> GenerateTableUpdates.generateTableUpdates(100, billy, t, result.infos)); + TstUtils.validate("Table - step " + ii, nuggets); + } + } + + @Test + public void testBucketedGeneralTickingRev() { + final int prevTicks = 100; + final int postTicks = 0; + + final CreateResult result = createTestTable(10000, true, false, true, 0x31313131); + final QueryTable t = result.t; + + final EvalNugget[] nuggets = new EvalNugget[] { + new EvalNugget() { + @Override + protected Table e() { + return t.updateBy(UpdateByOperation.RollingSum(prevTicks, postTicks), "Sym"); + } + } + }; + + final Random billy = new Random(0xB177B177); + for (int ii = 0; ii < 100; ii++) { + try { + simulateShiftAwareStep(100, billy, t, result.infos, nuggets); + } catch (Throwable ex) { + System.out.println("Crapped out on step " + ii); + throw ex; + } + } + } + + @Test + public void testBucketedGeneralTickingRevExclusive() { + final int prevTicks = 100; + final int postTicks = -50; + + final CreateResult result = createTestTable(10000, true, false, true, 0x31313131); + final QueryTable t = result.t; + + final EvalNugget[] nuggets = new EvalNugget[] { + new EvalNugget() { + @Override + protected Table e() { + return t.updateBy(UpdateByOperation.RollingSum(prevTicks, postTicks), "Sym"); + } + } + }; + + final Random billy = new Random(0xB177B177); + for (int ii = 0; ii < 100; ii++) { + try { + simulateShiftAwareStep(100, billy, t, result.infos, nuggets); + } catch (Throwable ex) { + System.out.println("Crapped out on step " + ii); + throw ex; + } + } + } + + @Test + public void testBucketedGeneralTickingFwd() { + final int prevTicks = 0; + final int postTicks = 100; + + final CreateResult result = createTestTable(10000, true, false, true, 0x31313131); + final QueryTable t = result.t; + + final EvalNugget[] nuggets = new EvalNugget[] { + new EvalNugget() { + @Override + protected Table e() { + return t.updateBy(UpdateByOperation.RollingSum(prevTicks, postTicks), "Sym"); + } + } + }; + + final Random billy = new Random(0xB177B177); + for (int ii = 0; ii < 100; ii++) { + try { + simulateShiftAwareStep(100, billy, t, result.infos, nuggets); + } catch (Throwable ex) { + System.out.println("Crapped out on step " + ii); + throw ex; + } + } + } + + @Test + public void testBucketedGeneralTickingFwdExclusive() { + final int prevTicks = -50; + final int postTicks = 100; + final CreateResult result = createTestTable(10000, true, false, true, 0x31313131); final QueryTable t = result.t; @@ -411,7 +783,200 @@ public void testBucketedGeneralTicking() { new EvalNugget() { @Override protected Table e() { - return t.updateBy(UpdateByOperation.RollingSum(100), "Sym"); + return t.updateBy(UpdateByOperation.RollingSum(prevTicks, postTicks), "Sym"); + } + } + }; + + final Random billy = new Random(0xB177B177); + for (int ii = 0; ii < 100; ii++) { + try { + simulateShiftAwareStep(100, billy, t, result.infos, nuggets); + } catch (Throwable ex) { + System.out.println("Crapped out on step " + ii); + throw ex; + } + } + } + + @Test + public void testBucketedGeneralTickingFwdRev() { + final int prevTicks = 50; + final int postTicks = 50; + + final CreateResult result = createTestTable(10000, true, false, true, 0x31313131); + final QueryTable t = result.t; + + final EvalNugget[] nuggets = new EvalNugget[] { + new EvalNugget() { + @Override + protected Table e() { + return t.updateBy(UpdateByOperation.RollingSum(prevTicks, postTicks), "Sym"); + } + } + }; + + final Random billy = new Random(0xB177B177); + for (int ii = 0; ii < 100; ii++) { + try { + simulateShiftAwareStep(100, billy, t, result.infos, nuggets); + } catch (Throwable ex) { + System.out.println("Crapped out on step " + ii); + throw ex; + } + } + } + + @Test + public void testBucketedGeneralTickingTimedRev() { + final Duration prevTime = Duration.ofMinutes(10); + final Duration postTime = Duration.ofMinutes(0); + + final CreateResult result = createTestTable(10000, true, false, true, 0x31313131, + new String[] {"ts"}, new TestDataGenerator[] {new SortedDateTimeGenerator( + convertDateTime("2022-03-09T09:00:00.000 NY"), + convertDateTime("2022-03-09T16:30:00.000 NY"))}); + + + final QueryTable t = result.t; + + final EvalNugget[] nuggets = new EvalNugget[] { + new EvalNugget() { + @Override + protected Table e() { + return t.updateBy(UpdateByOperation.RollingSum("ts", prevTime, postTime), "Sym"); + } + } + }; + + final Random billy = new Random(0xB177B177); + for (int ii = 0; ii < 100; ii++) { + try { + simulateShiftAwareStep(100, billy, t, result.infos, nuggets); + } catch (Throwable ex) { + System.out.println("Crapped out on step " + ii); + throw ex; + } + } + } + + @Test + public void testBucketedGeneralTickingTimedRevExclusive() { + final Duration prevTime = Duration.ofMinutes(10); + final Duration postTime = Duration.ofMinutes(-5); + + final CreateResult result = createTestTable(100, true, false, true, 0x31313131, + new String[] {"ts"}, new TestDataGenerator[] {new SortedDateTimeGenerator( + convertDateTime("2022-03-09T09:00:00.000 NY"), + convertDateTime("2022-03-09T16:30:00.000 NY"))}); + + + final QueryTable t = result.t; + + final EvalNugget[] nuggets = new EvalNugget[] { + new EvalNugget() { + @Override + protected Table e() { + return t.updateBy(UpdateByOperation.RollingSum("ts", prevTime, postTime), "Sym"); + } + } + }; + + final Random billy = new Random(0xB177B177); + for (int ii = 0; ii < 100; ii++) { + try { + simulateShiftAwareStep(10, billy, t, result.infos, nuggets); + } catch (Throwable ex) { + System.out.println("Crapped out on step " + ii); + throw ex; + } + } + } + + @Test + public void testBucketedGeneralTickingTimedFwd() { + final Duration prevTime = Duration.ofMinutes(0); + final Duration postTime = Duration.ofMinutes(10); + + final CreateResult result = createTestTable(10000, true, false, true, 0x31313131, + new String[] {"ts"}, new TestDataGenerator[] {new SortedDateTimeGenerator( + convertDateTime("2022-03-09T09:00:00.000 NY"), + convertDateTime("2022-03-09T16:30:00.000 NY"))}); + + + final QueryTable t = result.t; + + final EvalNugget[] nuggets = new EvalNugget[] { + new EvalNugget() { + @Override + protected Table e() { + return t.updateBy(UpdateByOperation.RollingSum("ts", prevTime, postTime), "Sym"); + } + } + }; + + final Random billy = new Random(0xB177B177); + for (int ii = 0; ii < 100; ii++) { + try { + simulateShiftAwareStep(100, billy, t, result.infos, nuggets); + } catch (Throwable ex) { + System.out.println("Crapped out on step " + ii); + throw ex; + } + } + } + + @Test + public void testBucketedGeneralTickingTimedFwdExclusive() { + final Duration prevTime = Duration.ofMinutes(-5); + final Duration postTime = Duration.ofMinutes(10); + + final CreateResult result = createTestTable(10000, true, false, true, 0x31313131, + new String[] {"ts"}, new TestDataGenerator[] {new SortedDateTimeGenerator( + convertDateTime("2022-03-09T09:00:00.000 NY"), + convertDateTime("2022-03-09T16:30:00.000 NY"))}); + + + final QueryTable t = result.t; + + final EvalNugget[] nuggets = new EvalNugget[] { + new EvalNugget() { + @Override + protected Table e() { + return t.updateBy(UpdateByOperation.RollingSum("ts", prevTime, postTime), "Sym"); + } + } + }; + + final Random billy = new Random(0xB177B177); + for (int ii = 0; ii < 100; ii++) { + try { + simulateShiftAwareStep(100, billy, t, result.infos, nuggets); + } catch (Throwable ex) { + System.out.println("Crapped out on step " + ii); + throw ex; + } + } + } + + @Test + public void testBucketedGeneralTickingTimedFwdRev() { + final Duration prevTime = Duration.ofMinutes(5); + final Duration postTime = Duration.ofMinutes(5); + + final CreateResult result = createTestTable(10000, true, false, true, 0x31313131, + new String[] {"ts"}, new TestDataGenerator[] {new SortedDateTimeGenerator( + convertDateTime("2022-03-09T09:00:00.000 NY"), + convertDateTime("2022-03-09T16:30:00.000 NY"))}); + + + final QueryTable t = result.t; + + final EvalNugget[] nuggets = new EvalNugget[] { + new EvalNugget() { + @Override + protected Table e() { + return t.updateBy(UpdateByOperation.RollingSum("ts", prevTime, postTime), "Sym"); } } }; diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java index fc356665f11..4b199b82754 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java @@ -89,29 +89,36 @@ protected Table e() { final String[] columnNamesArray = base.getDefinition().getColumnNamesArray(); final Collection clauses = List.of( - UpdateByOperation.Fill(), - UpdateByOperation.RollingSum(100, 0, - makeOpColNames(columnNamesArray, "_rollsumticksrev", "Sym", "ts", "boolCol")), - UpdateByOperation.RollingSum("ts", Duration.ofMinutes(15), Duration.ofMinutes(0), - makeOpColNames(columnNamesArray, "_rollsumtimerev", "Sym", "ts", "boolCol")), - UpdateByOperation.RollingSum(0, 100, - makeOpColNames(columnNamesArray, "_rollsumticksfwd", "Sym", "ts", "boolCol")), - UpdateByOperation.RollingSum("ts", Duration.ofMinutes(0), Duration.ofMinutes(15), - makeOpColNames(columnNamesArray, "_rollsumtimefwd", "Sym", "ts", "boolCol")), - UpdateByOperation.RollingSum(50, 50, - makeOpColNames(columnNamesArray, "_rollsumticksfwdrev", "Sym", "ts", - "boolCol")), - UpdateByOperation.RollingSum("ts", Duration.ofMinutes(5), Duration.ofMinutes(5), - makeOpColNames(columnNamesArray, "_rollsumtimebothfwdrev", "Sym", "ts", - "boolCol")), - - UpdateByOperation.Ema(skipControl, "ts", 10 * MINUTE, - makeOpColNames(columnNamesArray, "_ema", "Sym", "ts", "boolCol")), - UpdateByOperation.CumSum(makeOpColNames(columnNamesArray, "_sum", "Sym", "ts")), - UpdateByOperation.CumMin(makeOpColNames(columnNamesArray, "_min", "boolCol")), - UpdateByOperation.CumMax(makeOpColNames(columnNamesArray, "_max", "boolCol")), - UpdateByOperation - .CumProd(makeOpColNames(columnNamesArray, "_prod", "Sym", "ts", "boolCol"))); + UpdateByOperation.RollingSum(-50, 100, + makeOpColNames(columnNamesArray, "_rollsumticksfwdex", "Sym", "ts", "boolCol"))); + +// UpdateByOperation.Fill(), +// UpdateByOperation.RollingSum(100, 0, +// makeOpColNames(columnNamesArray, "_rollsumticksrev", "Sym", "ts", "boolCol")), +// UpdateByOperation.RollingSum("ts", Duration.ofMinutes(15), Duration.ofMinutes(0), +// makeOpColNames(columnNamesArray, "_rollsumtimerev", "Sym", "ts", "boolCol")), +// UpdateByOperation.RollingSum(0, 100, +// makeOpColNames(columnNamesArray, "_rollsumticksfwd", "Sym", "ts", "boolCol")), +// UpdateByOperation.RollingSum(-50, 100, +// makeOpColNames(columnNamesArray, "_rollsumticksfwdex", "Sym", "ts", "boolCol")), +// UpdateByOperation.RollingSum("ts", Duration.ofMinutes(0), Duration.ofMinutes(15), +// makeOpColNames(columnNamesArray, "_rollsumtimefwd", "Sym", "ts", "boolCol")), +// UpdateByOperation.RollingSum("ts", Duration.ofMinutes(-10), Duration.ofMinutes(15), +// makeOpColNames(columnNamesArray, "_rollsumtimefwdex", "Sym", "ts", "boolCol")), +// UpdateByOperation.RollingSum(50, 50, +// makeOpColNames(columnNamesArray, "_rollsumticksfwdrev", "Sym", "ts", +// "boolCol")), +// UpdateByOperation.RollingSum("ts", Duration.ofMinutes(5), Duration.ofMinutes(5), +// makeOpColNames(columnNamesArray, "_rollsumtimebothfwdrev", "Sym", "ts", +// "boolCol")), +// +// UpdateByOperation.Ema(skipControl, "ts", 10 * MINUTE, +// makeOpColNames(columnNamesArray, "_ema", "Sym", "ts", "boolCol")), +// UpdateByOperation.CumSum(makeOpColNames(columnNamesArray, "_sum", "Sym", "ts")), +// UpdateByOperation.CumMin(makeOpColNames(columnNamesArray, "_min", "boolCol")), +// UpdateByOperation.CumMax(makeOpColNames(columnNamesArray, "_max", "boolCol")), +// UpdateByOperation +// .CumProd(makeOpColNames(columnNamesArray, "_prod", "Sym", "ts", "boolCol"))); final UpdateByControl control = UpdateByControl.builder().useRedirection(redirected).build(); return bucketed ? base.updateBy(control, clauses, ColumnName.from("Sym")) diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBufferTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBufferTest.java index 513ab148150..de5004221ac 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBufferTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBufferTest.java @@ -359,4 +359,63 @@ public void testEvaluationEdgeCase() { assertEquals((double)99, rb.evaluate()); // last value added is max } } + + public void testPushPopUnsafe() { + try (final PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(3, -Double.MAX_VALUE, Double::max)) { + // move the head and tail off zero + rb.ensureRemaining(500); + for (double i = 0; i < 500; i++) { + rb.pushUnsafe(i); + } + for (double i = 0; i < 500; i++) { + assertEquals(rb.popUnsafe(), i); + } + + // do it again with an offset + rb.ensureRemaining(500); + for (double i = 0; i < 500; i++) { + rb.pushUnsafe(i + (double)1000); + } + for (double i = 0; i < 500; i++) { + assertEquals(rb.popUnsafe(), i + (double)1000); + } + + for (double i = 0; i < 500; i++) { + rb.pushUnsafe(i + (double)1000); + } + rb.clear(); + + for (double i = 0; i < 100; i++) { + rb.push(i); + } + assertEquals((double)99, rb.evaluate()); // last value added is max + } + } + + private double sum1toN(double n) { + return ((double)n * (double)(n+1) / (double)2); + } + + public void testPopMultiple() { + try (final PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(3, (double)0, Double::sum)) { + + for (int step = 0; step < 10; step++) { + rb.ensureRemaining(500); + for (double i = 0; i < 500; i++) { + rb.pushUnsafe(i); + } + + if (step % 2 == 0) { + rb.evaluate(); + } + + double[] values = rb.pop(500); + for (double i = 0; i < 500; i++) { + assertEquals(values[(int) i], i); + } + assertEmpty(rb); + } + } + } + } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBufferTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBufferTest.java index e5bfb6d2264..a7098783182 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBufferTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBufferTest.java @@ -354,4 +354,63 @@ public void testEvaluationEdgeCase() { assertEquals((float)99, rb.evaluate()); // last value added is max } } + + public void testPushPopUnsafe() { + try (final PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(3, -Float.MAX_VALUE, Float::max)) { + // move the head and tail off zero + rb.ensureRemaining(500); + for (float i = 0; i < 500; i++) { + rb.pushUnsafe(i); + } + for (float i = 0; i < 500; i++) { + assertEquals(rb.popUnsafe(), i); + } + + // do it again with an offset + rb.ensureRemaining(500); + for (float i = 0; i < 500; i++) { + rb.pushUnsafe(i + (float)1000); + } + for (float i = 0; i < 500; i++) { + assertEquals(rb.popUnsafe(), i + (float)1000); + } + + for (float i = 0; i < 500; i++) { + rb.pushUnsafe(i + (float)1000); + } + rb.clear(); + + for (float i = 0; i < 100; i++) { + rb.push(i); + } + assertEquals((float)99, rb.evaluate()); // last value added is max + } + } + + private float sum1toN(float n) { + return ((float)n * (float)(n+1) / (float)2); + } + + public void testPopMultiple() { + try (final PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(3, (float)0, Float::sum)) { + + for (int step = 0; step < 10; step++) { + rb.ensureRemaining(500); + for (float i = 0; i < 500; i++) { + rb.pushUnsafe(i); + } + + if (step % 2 == 0) { + rb.evaluate(); + } + + float[] values = rb.pop(500); + for (float i = 0; i < 500; i++) { + assertEquals(values[(int) i], i); + } + assertEmpty(rb); + } + } + } + } From d7d59c054c2f18247927fb12a44f525c11e1af02 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Thu, 19 Jan 2023 13:07:09 -0800 Subject: [PATCH 089/123] Address PR comments --- .../engine/rowset/RowSetShiftData.java | 19 +++ .../updateby/UpdateByCumulativeOperator.java | 39 ++---- .../table/impl/updateby/UpdateByOperator.java | 69 +++++----- .../updateby/UpdateByOperatorFactory.java | 42 +++--- .../table/impl/updateby/UpdateByWindow.java | 2 +- .../updateby/UpdateByWindowCumulative.java | 30 ++--- .../impl/updateby/UpdateByWindowTicks.java | 30 +++-- .../impl/updateby/UpdateByWindowTime.java | 124 +++++++++--------- .../updateby/UpdateByWindowedOperator.java | 28 ++-- .../ema/BasePrimitiveEMAOperator.java | 4 +- .../updateby/ema/BigDecimalEMAOperator.java | 10 +- .../updateby/ema/BigIntegerEMAOperator.java | 10 +- .../updateby/ema/BigNumberEMAOperator.java | 4 +- .../impl/updateby/ema/ByteEMAOperator.java | 10 +- .../impl/updateby/ema/DoubleEMAOperator.java | 10 +- .../impl/updateby/ema/FloatEMAOperator.java | 10 +- .../impl/updateby/ema/IntEMAOperator.java | 10 +- .../impl/updateby/ema/LongEMAOperator.java | 10 +- .../impl/updateby/ema/ShortEMAOperator.java | 10 +- .../updateby/fill/BooleanFillByOperator.java | 8 +- .../updateby/fill/ByteFillByOperator.java | 8 +- .../updateby/fill/CharFillByOperator.java | 8 +- .../updateby/fill/DoubleFillByOperator.java | 8 +- .../updateby/fill/FloatFillByOperator.java | 8 +- .../impl/updateby/fill/IntFillByOperator.java | 8 +- .../updateby/fill/LongFillByOperator.java | 8 +- .../updateby/fill/ObjectFillByOperator.java | 8 +- .../updateby/fill/ShortFillByOperator.java | 8 +- .../internal/BaseByteUpdateByOperator.java | 9 +- .../internal/BaseCharUpdateByOperator.java | 9 +- .../internal/BaseDoubleUpdateByOperator.java | 47 ++++--- .../internal/BaseFloatUpdateByOperator.java | 50 +++---- .../internal/BaseIntUpdateByOperator.java | 9 +- .../internal/BaseLongUpdateByOperator.java | 9 +- .../internal/BaseObjectBinaryOperator.java | 8 +- .../internal/BaseObjectUpdateByOperator.java | 11 +- .../internal/BaseShortUpdateByOperator.java | 9 +- .../BaseWindowedByteUpdateByOperator.java | 14 +- .../BaseWindowedCharUpdateByOperator.java | 14 +- .../BaseWindowedDoubleUpdateByOperator.java | 28 ++-- .../BaseWindowedFloatUpdateByOperator.java | 31 +++-- .../BaseWindowedIntUpdateByOperator.java | 14 +- .../BaseWindowedLongUpdateByOperator.java | 14 +- .../BaseWindowedObjectUpdateByOperator.java | 16 +-- .../BaseWindowedShortUpdateByOperator.java | 14 +- .../minmax/ByteCumMinMaxOperator.java | 8 +- .../minmax/DoubleCumMinMaxOperator.java | 8 +- .../minmax/FloatCumMinMaxOperator.java | 8 +- .../updateby/minmax/IntCumMinMaxOperator.java | 8 +- .../minmax/LongCumMinMaxOperator.java | 8 +- .../minmax/ShortCumMinMaxOperator.java | 8 +- .../prod/BigDecimalCumProdOperator.java | 8 +- .../prod/BigIntegerCumProdOperator.java | 8 +- .../updateby/prod/ByteCumProdOperator.java | 8 +- .../updateby/prod/DoubleCumProdOperator.java | 8 +- .../updateby/prod/FloatCumProdOperator.java | 8 +- .../updateby/prod/IntCumProdOperator.java | 8 +- .../updateby/prod/LongCumProdOperator.java | 8 +- .../updateby/prod/ShortCumProdOperator.java | 8 +- .../BigDecimalRollingSumOperator.java | 15 +-- .../BigIntegerRollingSumOperator.java | 15 +-- .../rollingsum/ByteRollingSumOperator.java | 15 +-- .../rollingsum/DoubleRollingSumOperator.java | 15 +-- .../rollingsum/FloatRollingSumOperator.java | 15 +-- .../rollingsum/IntRollingSumOperator.java | 15 +-- .../rollingsum/LongRollingSumOperator.java | 15 +-- .../rollingsum/ShortRollingSumOperator.java | 15 +-- .../sum/BigDecimalCumSumOperator.java | 8 +- .../sum/BigIntegerCumSumOperator.java | 8 +- .../impl/updateby/sum/ByteCumSumOperator.java | 8 +- .../updateby/sum/DoubleCumSumOperator.java | 8 +- .../updateby/sum/FloatCumSumOperator.java | 8 +- .../impl/updateby/sum/IntCumSumOperator.java | 8 +- .../impl/updateby/sum/LongCumSumOperator.java | 8 +- .../updateby/sum/ShortCumSumOperator.java | 8 +- .../table/impl/updateby/TestRollingSum.java | 3 +- .../impl/updateby/TestUpdateByGeneral.java | 55 ++++---- .../replicators/ReplicateUpdateBy.java | 12 +- 78 files changed, 585 insertions(+), 630 deletions(-) diff --git a/engine/rowset/src/main/java/io/deephaven/engine/rowset/RowSetShiftData.java b/engine/rowset/src/main/java/io/deephaven/engine/rowset/RowSetShiftData.java index 291fd5d24c1..ad8659623c0 100644 --- a/engine/rowset/src/main/java/io/deephaven/engine/rowset/RowSetShiftData.java +++ b/engine/rowset/src/main/java/io/deephaven/engine/rowset/RowSetShiftData.java @@ -320,6 +320,25 @@ public boolean apply(final WritableRowSet rowSet) { } } + /** + * Apply all shifts to {@code keyToShift}. Moves the single from pre-shift keyspace to post-shift keyspace. + * + * @param keyToShift The single row key to shift + * @return the key in post-shift space + */ + public long apply(final long keyToShift) { + for (int shiftIdx = 0; shiftIdx < size(); shiftIdx++) { + if (getBeginRange(shiftIdx) > keyToShift) { + // no shift applies so we are already in post-shift space + return keyToShift; + } else if (getEndRange(shiftIdx) >= keyToShift) { + // this shift applies, add the delta to get post-shift + return keyToShift + getShiftDelta(shiftIdx); + } + } + return keyToShift; + } + /** * Apply a shift to the provided rowSet. Moves rowSet from pre-shift keyspace to post-shift keyspace. * diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByCumulativeOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByCumulativeOperator.java index 65dfa15871b..97f235f0e2a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByCumulativeOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByCumulativeOperator.java @@ -11,14 +11,13 @@ import javax.annotation.Nullable; -import static io.deephaven.util.QueryConstants.NULL_LONG; - public abstract class UpdateByCumulativeOperator extends UpdateByOperator { public abstract static class Context implements UpdateContext { - public long curTimestamp; + /** Holds the chunks of input data for use by the accumulate call */ + public final Chunk[] chunkArr; - protected Context() { - curTimestamp = NULL_LONG; + public Context(int chunkCount) { + chunkArr = new Chunk[chunkCount]; } public boolean isValueValid(long atKey) { @@ -41,41 +40,25 @@ public abstract void accumulate(RowSequence inputKeys, } /** - * An operator that computes a cumulative operation from a column + * An operator that computes a cumulative operation from a column. The operation may be time or ticks aware (e.g. + * EMA) and timestamp column name and time units (ticks or nanoseconds) may optionally be provided * * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this operation * @param rowRedirection the row redirection context to use for the operation */ - public UpdateByCumulativeOperator(@NotNull final MatchPair pair, - @NotNull final String[] affectingColumns, - @Nullable final RowRedirection rowRedirection) { - super(pair, affectingColumns, null, null, 0L, 0L, rowRedirection); - } - - /** - * An operator that computes a cumulative operation from a column while providing an optional timestamp column name - * and a - * - * @param pair the {@link MatchPair} that defines the input/output for this operation - * @param affectingColumns the names of the columns that affect this operation - * @param rowRedirection the row redirection context to use for the operation - */ - public UpdateByCumulativeOperator(@NotNull final MatchPair pair, + protected UpdateByCumulativeOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @Nullable final RowRedirection rowRedirection, @Nullable final String timestampColumnName, - final long reverseTimeScaleUnits) { - super(pair, affectingColumns, null, timestampColumnName, reverseTimeScaleUnits, 0L, rowRedirection); + final long reverseWindowScaleUnits) { + super(pair, affectingColumns, timestampColumnName, reverseWindowScaleUnits, 0L, rowRedirection); } /** * Initialize the bucket context for this cumulative operator */ - abstract public void initializeUpdate(@NotNull final UpdateContext context, final long firstUnmodifiedKey, - long firstUnmodifiedTimestamp); - - @Override - public void finishUpdate(@NotNull final UpdateContext context) {} + public void initializeUpdate(@NotNull final UpdateContext context, final long firstUnmodifiedKey, + long firstUnmodifiedTimestamp) {} } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperator.java index ac71bb79c14..809bb2b1e63 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperator.java @@ -1,6 +1,5 @@ package io.deephaven.engine.table.impl.updateby; -import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.LongChunk; @@ -33,26 +32,24 @@ *

              */ public abstract class UpdateByOperator { - public static UpdateByOperator[] ZERO_LENGTH_OP_ARRAY = new UpdateByOperator[0]; + protected static UpdateByOperator[] ZERO_LENGTH_OP_ARRAY = new UpdateByOperator[0]; protected final MatchPair pair; protected final String[] affectingColumns; protected final RowRedirection rowRedirection; - // these will be used by the timestamp-aware operators (EMA for example) - protected final OperationControl control; - protected final long reverseTimeScaleUnits; - protected final long forwardTimeScaleUnits; + protected final long reverseWindowScaleUnits; + protected final long forwardWindowScaleUnits; protected final String timestampColumnName; /** * The input modifiedColumnSet for this operator */ - protected ModifiedColumnSet inputModifiedColumnSet; + ModifiedColumnSet inputModifiedColumnSet; /** * The output modifiedColumnSet for this operator */ - protected ModifiedColumnSet outputModifiedColumnSet; + ModifiedColumnSet outputModifiedColumnSet; /** * A context item for use with updateBy operators @@ -64,7 +61,7 @@ public interface UpdateContext extends SafeCloseable { void setTimestampChunk(@NotNull LongChunk valuesChunk); /** - * Add a value to the operators current data set + * Add values to the operators current data set * * @param key the row key associated with the value * @param pos the index in the associated chunk where this value can be found. Depending on the usage, might be @@ -75,8 +72,8 @@ public interface UpdateContext extends SafeCloseable { void push(long key, int pos, int count); /** - * Remove a value from the operators current data set. This is only valid for windowed operators as - * cumulative operators only append values + * Remove values from the operators current data set. This is only valid for windowed operators as cumulative + * operators only append values * * @param count the number of items to pop from the data set */ @@ -102,18 +99,16 @@ public interface UpdateContext extends SafeCloseable { protected UpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final OperationControl control, @Nullable final String timestampColumnName, - final long reverseTimeScaleUnits, - final long forwardTimeScaleUnits, + final long reverseWindowScaleUnits, + final long forwardWindowScaleUnits, @Nullable final RowRedirection rowRedirection) { this.pair = pair; this.affectingColumns = affectingColumns; this.rowRedirection = rowRedirection; this.timestampColumnName = timestampColumnName; - this.control = control; - this.reverseTimeScaleUnits = reverseTimeScaleUnits; - this.forwardTimeScaleUnits = forwardTimeScaleUnits; + this.reverseWindowScaleUnits = reverseWindowScaleUnits; + this.forwardWindowScaleUnits = forwardWindowScaleUnits; } /** @@ -122,7 +117,7 @@ protected UpdateByOperator(@NotNull final MatchPair pair, * @return the names of the input column */ @NotNull - public String[] getInputColumnNames() { + protected String[] getInputColumnNames() { return new String[] {pair.rightColumn}; } @@ -132,22 +127,22 @@ public String[] getInputColumnNames() { * @return the name of the timestamp column */ @Nullable - public String getTimestampColumnName() { + protected String getTimestampColumnName() { return timestampColumnName; } /** * Get the value of the backward-looking window (might be nanoseconds or ticks). */ - public long getPrevWindowUnits() { - return reverseTimeScaleUnits; + protected long getPrevWindowUnits() { + return reverseWindowScaleUnits; } /** * Get the value of the forward-looking window (might be nanoseconds or ticks). */ - public long getFwdWindowUnits() { - return forwardTimeScaleUnits; + protected long getFwdWindowUnits() { + return forwardWindowScaleUnits; } /** @@ -156,7 +151,7 @@ public long getFwdWindowUnits() { * @return an array of column names that affect this operator. */ @NotNull - public String[] getAffectingColumnNames() { + protected String[] getAffectingColumnNames() { return affectingColumns; } @@ -166,7 +161,7 @@ public String[] getAffectingColumnNames() { * @return the output column names. */ @NotNull - public String[] getOutputColumnNames() { + protected String[] getOutputColumnNames() { return new String[] {pair.leftColumn}; } @@ -176,21 +171,22 @@ public String[] getOutputColumnNames() { * @return a map of output column name to output column source */ @NotNull - public abstract Map> getOutputColumns(); + protected abstract Map> getOutputColumns(); /** * Indicate that the operation should start tracking previous values for ticking updates. */ - public abstract void startTrackingPrev(); + protected abstract void startTrackingPrev(); /** * Make an {@link UpdateContext} suitable for use with updates. * * @param chunkSize The expected size of chunks that will be provided during the update, + * @param chunkCount The number of chunks that will be provided during the update, * @return a new context */ @NotNull - public abstract UpdateContext makeUpdateContext(final int chunkSize); + public abstract UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount); /** * Perform any bookkeeping required at the end of a single part of the update. This is always preceded with a call @@ -198,49 +194,48 @@ public String[] getOutputColumnNames() { * * @param context the context object */ - public abstract void finishUpdate(@NotNull final UpdateContext context); + protected void finishUpdate(@NotNull final UpdateContext context) {} /** * Apply a shift to the operation. */ - public abstract void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long delta); + protected abstract void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long delta); /** * Prepare this operator output column for parallel updated. */ - public abstract void prepareForParallelPopulation(final RowSet changedRows); + protected abstract void prepareForParallelPopulation(final RowSet changedRows); /** * Create the modified column set for the input columns of this operator. */ - public void createInputModifiedColumnSet(@NotNull final QueryTable source) { + protected void createInputModifiedColumnSet(@NotNull final QueryTable source) { inputModifiedColumnSet = source.newModifiedColumnSet(getAffectingColumnNames()); } /** * Create the modified column set for the output columns from this operator. */ - public void createOutputModifiedColumnSet(@NotNull final QueryTable result) { + protected void createOutputModifiedColumnSet(@NotNull final QueryTable result) { outputModifiedColumnSet = result.newModifiedColumnSet(getOutputColumnNames()); } /** * Return the modified column set for the input columns of this operator. */ - public ModifiedColumnSet getInputModifiedColumnSet() { + protected ModifiedColumnSet getInputModifiedColumnSet() { return inputModifiedColumnSet; } /** * Return the modified column set for the output columns from this operator. */ - public ModifiedColumnSet getOutputModifiedColumnSet() { + protected ModifiedColumnSet getOutputModifiedColumnSet() { return outputModifiedColumnSet; } /** * Clear the output rows by setting value to NULL. Dense sources will apply removes to the inner source. */ - public abstract void clearOutputRows(RowSet toClear); - + protected abstract void clearOutputRows(RowSet toClear); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperatorFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperatorFactory.java index 9303ca91830..2fd99dd11a9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperatorFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperatorFactory.java @@ -370,41 +370,31 @@ private UpdateByOperator makeRollingSumOperator(@NotNull final MatchPair pair, final long fwdTimeScaleUnits = rs.fwdTimeScale().timescaleUnits(); if (csType == Boolean.class || csType == boolean.class) { - return new ByteRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), - rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, - rowRedirection, NULL_BOOLEAN_AS_BYTE); + return new ByteRollingSumOperator(pair, affectingColumns, rs.prevTimeScale().timestampCol(), + prevTimeScaleUnits, fwdTimeScaleUnits, rowRedirection, NULL_BOOLEAN_AS_BYTE); } else if (csType == byte.class || csType == Byte.class) { - return new ByteRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), - rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, - rowRedirection, NULL_BYTE); + return new ByteRollingSumOperator(pair, affectingColumns, rs.prevTimeScale().timestampCol(), + prevTimeScaleUnits, fwdTimeScaleUnits, rowRedirection, NULL_BYTE); } else if (csType == short.class || csType == Short.class) { - return new ShortRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), - rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, - rowRedirection); + return new ShortRollingSumOperator(pair, affectingColumns, rs.prevTimeScale().timestampCol(), + prevTimeScaleUnits, fwdTimeScaleUnits, rowRedirection); } else if (csType == int.class || csType == Integer.class) { - return new IntRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), - rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, - rowRedirection); + return new IntRollingSumOperator(pair, affectingColumns, rs.prevTimeScale().timestampCol(), + prevTimeScaleUnits, fwdTimeScaleUnits, rowRedirection); } else if (csType == long.class || csType == Long.class) { - return new LongRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), - rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, - rowRedirection); + return new LongRollingSumOperator(pair, affectingColumns, rs.prevTimeScale().timestampCol(), + prevTimeScaleUnits, fwdTimeScaleUnits, rowRedirection); } else if (csType == float.class || csType == Float.class) { - return new FloatRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), - rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, - rowRedirection); + return new FloatRollingSumOperator(pair, affectingColumns, rs.prevTimeScale().timestampCol(), + prevTimeScaleUnits, fwdTimeScaleUnits, rowRedirection); } else if (csType == double.class || csType == Double.class) { - return new DoubleRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), - rs.prevTimeScale().timestampCol(), + return new DoubleRollingSumOperator(pair, affectingColumns, rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, rowRedirection); } else if (csType == BigDecimal.class) { - return new BigDecimalRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), - rs.prevTimeScale().timestampCol(), - prevTimeScaleUnits, fwdTimeScaleUnits, rowRedirection, - control.mathContextOrDefault()); + return new BigDecimalRollingSumOperator(pair, affectingColumns, rs.prevTimeScale().timestampCol(), + prevTimeScaleUnits, fwdTimeScaleUnits, rowRedirection, control.mathContextOrDefault()); } else if (csType == BigInteger.class) { - return new BigIntegerRollingSumOperator(pair, affectingColumns, rs.controlOrDefault(), - rs.prevTimeScale().timestampCol(), + return new BigIntegerRollingSumOperator(pair, affectingColumns, rs.prevTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, rowRedirection); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java index b97ba7df9e8..51f663087fd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java @@ -155,7 +155,7 @@ UpdateByOperator[] getOperators() { return operators; } - int[] getUniqueSourceIndices() { + public int[] getUniqueSourceIndices() { if (uniqueInputSourceIndices == null) { final TIntHashSet set = new TIntHashSet(); for (int opIdx = 0; opIdx < operators.length; opIdx++) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java index 8e9f2b9d04d..4902b39b3d1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java @@ -1,7 +1,6 @@ package io.deephaven.engine.table.impl.updateby; import io.deephaven.base.verify.Assert; -import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.*; @@ -36,7 +35,8 @@ private void makeOperatorContexts(UpdateByWindowBucketContext context) { // create contexts for the affected operators for (int opIdx : context.dirtyOperatorIndices) { - context.opContext[opIdx] = operators[opIdx].makeUpdateContext(context.workingChunkSize); + context.opContext[opIdx] = operators[opIdx].makeUpdateContext(context.workingChunkSize, + operatorInputSourceSlots[opIdx].length); } } @@ -175,20 +175,21 @@ void processRows(UpdateByWindowBucketContext context, final boolean initialStep) : context.timestampColumnSource.getChunk(tsGetCtx, rs).asLongChunk(); for (int opIdx : context.dirtyOperatorIndices) { + UpdateByCumulativeOperator.Context opCtx = + (UpdateByCumulativeOperator.Context) context.opContext[opIdx]; // prep the chunk array needed by the accumulate call final int[] srcIndices = operatorInputSourceSlots[opIdx]; - Chunk[] chunkArr = new Chunk[srcIndices.length]; for (int ii = 0; ii < srcIndices.length; ii++) { int srcIdx = srcIndices[ii]; // chunk prep prepareValuesChunkForSource(context, srcIdx, rs); - chunkArr[ii] = context.inputSourceChunks[srcIdx]; + opCtx.chunkArr[ii] = context.inputSourceChunks[srcIdx]; } // make the specialized call for cumulative operators ((UpdateByCumulativeOperator.Context) context.opContext[opIdx]).accumulate( rs, - chunkArr, + opCtx.chunkArr, tsChunk, size); } @@ -223,22 +224,17 @@ private static long smallestAffectedKey(@NotNull TableUpdate upstream, @NotNull return affectedRowSet.firstRowKey(); } - // get the key previous to this one and shift to post-space (if needed) + // get the key previous to this one and shift to post-space smallestModifiedKey = affectedRowSet.getPrev(pos - 1); if (upstream.shifted().nonempty()) { - final RowSetShiftData shifted = upstream.shifted(); - for (int shiftIdx = 0; shiftIdx < shifted.size(); shiftIdx++) { - if (shifted.getBeginRange(shiftIdx) > smallestModifiedKey) { - // no shift applies so we are already in post-shift space - break; - } else if (shifted.getEndRange(shiftIdx) >= smallestModifiedKey) { - // this shift applies, add the delta to get post-shift - smallestModifiedKey += shifted.getShiftDelta(shiftIdx); - break; - } - } + smallestModifiedKey = upstream.shifted().apply(smallestModifiedKey); } + // tighten this up more by advancing one key in the post-shift space. This leaves us with first key + // following the first remove + if (smallestModifiedKey < affectedRowSet.lastRowKey()) { + smallestModifiedKey = affectedRowSet.get(affectedRowSet.find(smallestModifiedKey) + 1); + } } if (upstream.added().isNonempty()) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java index 2c0fe4bce37..50ee477d639 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java @@ -3,7 +3,6 @@ import gnu.trove.list.array.TIntArrayList; import gnu.trove.set.hash.TIntHashSet; import io.deephaven.base.verify.Assert; -import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.attributes.Values; @@ -57,6 +56,14 @@ public void close() { super(operators, operatorSourceSlots, null); this.prevUnits = prevUnits; this.fwdUnits = fwdUnits; + + // We would like to use jdk.internal.util.ArraysSupport.MAX_ARRAY_LENGTH, but it is not exported + final int MAX_ARRAY_SIZE = Integer.MAX_VALUE - 8; + + if (prevUnits + fwdUnits > MAX_ARRAY_SIZE) { + throw (new IllegalArgumentException( + "UpdateBy window size may not exceed MAX_ARRAY_SIZE (" + MAX_ARRAY_SIZE + ")")); + } } private void makeOperatorContexts(UpdateByWindowBucketContext context) { @@ -67,7 +74,8 @@ private void makeOperatorContexts(UpdateByWindowBucketContext context) { // create contexts for the affected operators for (int opIdx : context.dirtyOperatorIndices) { - context.opContext[opIdx] = operators[opIdx].makeUpdateContext(context.workingChunkSize); + context.opContext[opIdx] = operators[opIdx].makeUpdateContext(context.workingChunkSize, + operatorInputSourceSlots[opIdx].length); } } @@ -237,7 +245,8 @@ void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, @NotNu if (upstream.modified().isNonempty()) { // compute the rows affected from these changes try (final WritableRowSet modifiedInverted = ctx.sourceRowSet.invert(upstream.modified()); - final RowSet modifiedAffected = computeAffectedRowsTicks(ctx.sourceRowSet, modifiedInverted, prevUnits, fwdUnits)) { + final RowSet modifiedAffected = + computeAffectedRowsTicks(ctx.sourceRowSet, modifiedInverted, prevUnits, fwdUnits)) { tmpAffected.insert(modifiedAffected); } } @@ -248,7 +257,7 @@ void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, @NotNu final long fwd = Math.max(0, fwdUnits); try (final RowSet addedInverted = ctx.sourceRowSet.invert(upstream.added()); - final RowSet addedAffected = computeAffectedRowsTicks(ctx.sourceRowSet, addedInverted, prev, fwd)) { + final RowSet addedAffected = computeAffectedRowsTicks(ctx.sourceRowSet, addedInverted, prev, fwd)) { tmpAffected.insert(addedAffected); } } @@ -259,9 +268,9 @@ void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, @NotNu final long fwd = Math.max(0, fwdUnits); try (final RowSet prevRows = ctx.sourceRowSet.copyPrev(); - final RowSet removedInverted = prevRows.invert(upstream.removed()); - final WritableRowSet removedAffected = - computeAffectedRowsTicks(prevRows, removedInverted, prev, fwd)) { + final RowSet removedInverted = prevRows.invert(upstream.removed()); + final WritableRowSet removedAffected = + computeAffectedRowsTicks(prevRows, removedInverted, prev, fwd)) { // apply shifts to get back to pos-shift space upstream.shifted().apply(removedAffected); // retain only the rows that still exist in the sourceRowSet @@ -346,20 +355,21 @@ void processRows(UpdateByWindowBucketContext context, boolean initialStep) { Arrays.fill(ctx.inputSourceChunks, null); for (int opIdx : context.dirtyOperatorIndices) { + UpdateByWindowedOperator.Context opCtx = + (UpdateByWindowedOperator.Context) context.opContext[opIdx]; // prep the chunk array needed by the accumulate call final int[] srcIndices = operatorInputSourceSlots[opIdx]; - Chunk[] chunkArr = new Chunk[srcIndices.length]; for (int ii = 0; ii < srcIndices.length; ii++) { int srcIdx = srcIndices[ii]; // chunk prep prepareValuesChunkForSource(ctx, srcIdx, chunkInfluencerRs); - chunkArr[ii] = ctx.inputSourceChunks[srcIdx]; + opCtx.chunkArr[ii] = ctx.inputSourceChunks[srcIdx]; } // make the specialized call for windowed operators ((UpdateByWindowedOperator.Context) ctx.opContext[opIdx]).accumulate( chunkRs, - chunkArr, + opCtx.chunkArr, pushChunk, popChunk, chunkRsSize); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java index 8b5385e6f16..6fcc6bbba78 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java @@ -3,7 +3,6 @@ import gnu.trove.list.array.TIntArrayList; import gnu.trove.set.hash.TIntHashSet; import io.deephaven.base.verify.Assert; -import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.attributes.Values; @@ -39,6 +38,7 @@ class UpdateByWindowTime extends UpdateByWindow { public class UpdateByWindowTimeBucketContext extends UpdateByWindowBucketContext { protected final ChunkSource.GetContext influencerTimestampContext; + final ChunkSource.GetContext timestampColumnGetContext; protected int currentGetContextSize; public UpdateByWindowTimeBucketContext(final TrackingRowSet sourceRowSet, @@ -50,12 +50,14 @@ public UpdateByWindowTimeBucketContext(final TrackingRowSet sourceRowSet, super(sourceRowSet, timestampColumnSource, timestampSsa, timestampValidRowSet, chunkSize, initialStep); influencerTimestampContext = timestampColumnSource.makeGetContext(WINDOW_CHUNK_SIZE); + timestampColumnGetContext = timestampColumnSource.makeGetContext(WINDOW_CHUNK_SIZE); } @Override public void close() { super.close(); - try (final SafeCloseable ignoreCtx1 = influencerTimestampContext) { + try (final SafeCloseable ignoreCtx1 = influencerTimestampContext; + final SafeCloseable ignoreCtx2 = timestampColumnGetContext) { // leveraging try with resources to auto-close } } @@ -77,7 +79,8 @@ protected void makeOperatorContexts(UpdateByWindowBucketContext context) { // create contexts for the affected operators for (int opIdx : context.dirtyOperatorIndices) { - context.opContext[opIdx] = operators[opIdx].makeUpdateContext(context.workingChunkSize); + context.opContext[opIdx] = operators[opIdx].makeUpdateContext(context.workingChunkSize, + operatorInputSourceSlots[opIdx].length); } } @@ -98,30 +101,26 @@ public UpdateByWindowBucketContext makeWindowContext(final TrackingRowSet source * window parameters. After these rows have been identified, must determine which rows will be needed to recompute * these values (i.e. that fall within the window and will `influence` this computation). */ - private static WritableRowSet computeAffectedRowsTime(final RowSet sourceSet, final RowSet subset, long revNanos, - long fwdNanos, final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa, - boolean usePrev) { + private static WritableRowSet computeAffectedRowsTime(final UpdateByWindowTimeBucketContext ctx, + final RowSet subset, long revNanos, long fwdNanos, boolean usePrev) { // swap fwd/rev to get the affected windows - return computeInfluencerRowsTime(sourceSet, subset, fwdNanos, revNanos, timestampColumnSource, timestampSsa, - usePrev); + return computeInfluencerRowsTime(ctx, subset, fwdNanos, revNanos, usePrev); } - private static WritableRowSet computeInfluencerRowsTime(final RowSet sourceSet, final RowSet subset, long revNanos, - long fwdNanos, final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa, - boolean usePrev) { - int chunkSize = (int) Math.min(subset.size(), 4096); - try (final RowSequence.Iterator it = subset.getRowSequenceIterator(); - final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(chunkSize)) { + private static WritableRowSet computeInfluencerRowsTime(final UpdateByWindowTimeBucketContext ctx, + final RowSet subset, + long revNanos, long fwdNanos, boolean usePrev) { + try (final RowSequence.Iterator it = subset.getRowSequenceIterator()) { final RowSetBuilderSequential builder = RowSetFactory.builderSequential(); - LongSegmentedSortedArray.Iterator ssaIt = timestampSsa.iterator(false, false); + LongSegmentedSortedArray.Iterator ssaIt = ctx.timestampSsa.iterator(false, false); while (it.hasMore() && ssaIt.hasNext()) { - final RowSequence rs = it.getNextRowSequenceWithLength(chunkSize); + final RowSequence rs = it.getNextRowSequenceWithLength(WINDOW_CHUNK_SIZE); final int rsSize = rs.intSize(); LongChunk timestamps = usePrev - ? timestampColumnSource.getPrevChunk(context, rs).asLongChunk() - : timestampColumnSource.getChunk(context, rs).asLongChunk(); + ? ctx.timestampColumnSource.getPrevChunk(ctx.timestampColumnGetContext, rs).asLongChunk() + : ctx.timestampColumnSource.getChunk(ctx.timestampColumnGetContext, rs).asLongChunk(); for (int ii = 0; ii < rsSize; ii++) { final long ts = timestamps.get(ii); @@ -194,10 +193,12 @@ private void ensureGetContextSize(UpdateByWindowTimeBucketContext ctx, long newS } } - // windowed by time/ticks is more complex to compute: find all the changed rows and the rows that would - // be affected by the changes (includes newly added rows) and need to be recomputed. Then include all - // the rows that are affected by deletions (if any). After the affected rows have been identified, - // determine which rows will be needed to compute new values for the affected rows (influencer rows) + /** + * Windowed by time/ticks is complex to compute: must find all the changed rows and rows that would be affected by + * the changes (includes newly added rows) and need to be recomputed. Then include all the rows that are affected by + * deletions (if any). After the affected rows have been identified, determine which rows will be needed to compute + * new values for the affected rows (influencer rows) + */ @Override public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, @NotNull TableUpdate upstream) { UpdateByWindowTimeBucketContext ctx = (UpdateByWindowTimeBucketContext) context; @@ -210,8 +211,7 @@ public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, if (ctx.initialStep) { ctx.affectedRows = ctx.sourceRowSet; - ctx.influencerRows = computeInfluencerRowsTime(ctx.timestampValidRowSet, ctx.affectedRows, prevUnits, fwdUnits, - ctx.timestampColumnSource, ctx.timestampSsa, false); + ctx.influencerRows = computeInfluencerRowsTime(ctx, ctx.affectedRows, prevUnits, fwdUnits, false); // mark all operators as affected by this update context.dirtyOperatorIndices = IntStream.range(0, operators.length).toArray(); @@ -253,52 +253,51 @@ public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, return; } - try (final RowSet prevRows = upstream.modified().isNonempty() || upstream.removed().isNonempty() ? - ctx.timestampValidRowSet.copyPrev() : - null) { - final WritableRowSet tmpAffected = RowSetFactory.empty(); + final WritableRowSet tmpAffected = RowSetFactory.empty(); - if (upstream.modified().isNonempty()) { - // modified timestamps will affect the current and previous values - try (final RowSet modifiedAffected = computeAffectedRowsTime(ctx.timestampValidRowSet, upstream.modified(), prevUnits, fwdUnits, ctx.timestampColumnSource, ctx.timestampSsa, false)) { - tmpAffected.insert(modifiedAffected); - } - try (final WritableRowSet modifiedAffectedPrev = computeAffectedRowsTime(prevRows, upstream.getModifiedPreShift(), prevUnits, fwdUnits, ctx.timestampColumnSource, ctx.timestampSsa, true)) { - // we used the SSA (post-shift) to get these keys, no need to shift - // retain only the rows that still exist in the sourceRowSet - modifiedAffectedPrev.retain(ctx.timestampValidRowSet); - tmpAffected.insert(modifiedAffectedPrev); - } - // naturally need to compute all modified rows - tmpAffected.insert(upstream.modified()); + if (upstream.modified().isNonempty()) { + // modified timestamps will affect the current and previous values + try (final RowSet modifiedAffected = + computeAffectedRowsTime(ctx, upstream.modified(), prevUnits, fwdUnits, false)) { + tmpAffected.insert(modifiedAffected); + } + try (final WritableRowSet modifiedAffectedPrev = + computeAffectedRowsTime(ctx, upstream.getModifiedPreShift(), prevUnits, fwdUnits, true)) { + // we used the SSA (post-shift) to get these keys, no need to shift + // retain only the rows that still exist in the sourceRowSet + modifiedAffectedPrev.retain(ctx.timestampValidRowSet); + tmpAffected.insert(modifiedAffectedPrev); } + // naturally need to compute all modified rows + tmpAffected.insert(upstream.modified()); + } - if (upstream.added().isNonempty()) { - try (final RowSet addedAffected = computeAffectedRowsTime(ctx.timestampValidRowSet, upstream.added(), prevUnits, fwdUnits, ctx.timestampColumnSource, ctx.timestampSsa, false)) { - tmpAffected.insert(addedAffected); - } - // naturally need to compute all new rows - tmpAffected.insert(upstream.added()); + if (upstream.added().isNonempty()) { + try (final RowSet addedAffected = + computeAffectedRowsTime(ctx, upstream.added(), prevUnits, fwdUnits, false)) { + tmpAffected.insert(addedAffected); } + // naturally need to compute all new rows + tmpAffected.insert(upstream.added()); + } - // other rows can be affected by removes or mods - if (upstream.removed().isNonempty()) { - try (final WritableRowSet removedAffected = computeAffectedRowsTime(prevRows, upstream.removed(), prevUnits, fwdUnits, ctx.timestampColumnSource, ctx.timestampSsa, true)) { - // we used the SSA (post-shift) to get these keys, no need to shift - // retain only the rows that still exist in the sourceRowSet - removedAffected.retain(ctx.timestampValidRowSet); + // other rows can be affected by removes or mods + if (upstream.removed().isNonempty()) { + try (final WritableRowSet removedAffected = + computeAffectedRowsTime(ctx, upstream.removed(), prevUnits, fwdUnits, true)) { + // we used the SSA (post-shift) to get these keys, no need to shift + // retain only the rows that still exist in the sourceRowSet + removedAffected.retain(ctx.timestampValidRowSet); - tmpAffected.insert(removedAffected); - } + tmpAffected.insert(removedAffected); } - - ctx.affectedRows = tmpAffected; } + ctx.affectedRows = tmpAffected; + // now get influencer rows for the affected rows - ctx.influencerRows = computeInfluencerRowsTime(ctx.timestampValidRowSet, ctx.affectedRows, prevUnits, fwdUnits, - ctx.timestampColumnSource, ctx.timestampSsa, false); + ctx.influencerRows = computeInfluencerRowsTime(ctx, ctx.affectedRows, prevUnits, fwdUnits, false); makeOperatorContexts(ctx); } @@ -395,20 +394,21 @@ public void processRows(UpdateByWindowBucketContext context, boolean initialStep Arrays.fill(ctx.inputSourceChunks, null); for (int opIdx : context.dirtyOperatorIndices) { + UpdateByWindowedOperator.Context opCtx = + (UpdateByWindowedOperator.Context) context.opContext[opIdx]; // prep the chunk array needed by the accumulate call final int[] srcIndices = operatorInputSourceSlots[opIdx]; - Chunk[] chunkArr = new Chunk[srcIndices.length]; for (int ii = 0; ii < srcIndices.length; ii++) { int srcIdx = srcIndices[ii]; // chunk prep prepareValuesChunkForSource(ctx, srcIdx, chunkInfluencerRs); - chunkArr[ii] = ctx.inputSourceChunks[srcIdx]; + opCtx.chunkArr[ii] = ctx.inputSourceChunks[srcIdx]; } // make the specialized call for windowed operators ((UpdateByWindowedOperator.Context) ctx.opContext[opIdx]).accumulate( chunkRs, - chunkArr, + opCtx.chunkArr, pushChunk, popChunk, chunkRsSize); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowedOperator.java index 113a8360622..a61aa82de21 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowedOperator.java @@ -1,6 +1,5 @@ package io.deephaven.engine.table.impl.updateby; -import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.attributes.Values; @@ -13,7 +12,13 @@ public abstract class UpdateByWindowedOperator extends UpdateByOperator { public abstract static class Context implements UpdateContext { - public int nullCount = 0; + /** Holds the chunks of input data for use by the accumulate call */ + protected final Chunk[] chunkArr; + protected int nullCount = 0; + + public Context(int chunkCount) { + chunkArr = new Chunk[chunkCount]; + } @Override public void close() {} @@ -30,28 +35,23 @@ public abstract void accumulate(RowSequence inputKeys, * * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this operation - * @param control the control parameters for operation * @param timestampColumnName the optional time stamp column for windowing (uses ticks if not provided) - * @param reverseTimeScaleUnits the time (us) or ticks to extend the window backwards - * @param forwardTimeScaleUnits the time (us) or ticks to extend the window forwards + * @param reverseWindowScaleUnits the time (us) or ticks to extend the window backwards + * @param forwardWindowScaleUnits the time (us) or ticks to extend the window forwards * @param rowRedirection the row redirection to use for the operator output columns */ - public UpdateByWindowedOperator(@NotNull final MatchPair pair, + protected UpdateByWindowedOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final OperationControl control, @Nullable final String timestampColumnName, - final long reverseTimeScaleUnits, - final long forwardTimeScaleUnits, + final long reverseWindowScaleUnits, + final long forwardWindowScaleUnits, @Nullable final RowRedirection rowRedirection) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, + super(pair, affectingColumns, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, rowRedirection); } /** * Initialize the bucket context for this windowed operator */ - public abstract void initializeUpdate(@NotNull final UpdateContext context); - - @Override - public void finishUpdate(@NotNull final UpdateContext context) {} + public void initializeUpdate(@NotNull final UpdateContext context) {} } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java index 0a798bae4a7..4ea698c7092 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java @@ -30,8 +30,8 @@ public void setTimestampChunk(@NotNull final LongChunk valuesC timestampValueChunk = valuesChunk; } - Context(final int chunkSize) { - super(chunkSize); + Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java index 9ef1b54f133..132e89ebcb0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java @@ -17,8 +17,8 @@ public class BigDecimalEMAOperator extends BigNumberEMAOperator { public class Context extends BigNumberEMAOperator.Context { - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -68,7 +68,7 @@ public void accumulate(RowSequence inputKeys, } else { final long dt = timestamp - lastStamp; // alpha is dynamic, based on time - BigDecimal alpha = BigDecimal.valueOf(Math.exp(-dt / (double) reverseTimeScaleUnits)); + BigDecimal alpha = BigDecimal.valueOf(Math.exp(-dt / (double) reverseWindowScaleUnits)); BigDecimal oneMinusAlpha = BigDecimal.ONE.subtract(alpha, control.bigValueContextOrDefault()); @@ -116,7 +116,7 @@ public BigDecimalEMAOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java index daf7356affe..8c17fdd89d4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java @@ -18,8 +18,8 @@ public class BigIntegerEMAOperator extends BigNumberEMAOperator { public class Context extends BigNumberEMAOperator.Context { - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -73,7 +73,7 @@ public void accumulate(RowSequence inputKeys, // preserve curVal and timestamp } else { // alpha is dynamic, based on time - BigDecimal alpha = BigDecimal.valueOf(Math.exp(-dt / (double)reverseTimeScaleUnits)); + BigDecimal alpha = BigDecimal.valueOf(Math.exp(-dt / (double) reverseWindowScaleUnits)); BigDecimal oneMinusAlpha = BigDecimal.ONE.subtract(alpha, control.bigValueContextOrDefault()); curVal = curVal.multiply(alpha, control.bigValueContextOrDefault()) @@ -121,7 +121,7 @@ public BigIntegerEMAOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java index cbc492ceede..e2669b46fbf 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java @@ -32,8 +32,8 @@ public abstract class Context extends BaseObjectUpdateByOperator.Con long lastStamp = NULL_LONG; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java index 9e9481138af..2698860cc27 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java @@ -26,8 +26,8 @@ protected class Context extends BasePrimitiveEMAOperator.Context { public ByteChunk byteValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -76,7 +76,7 @@ public void accumulate(RowSequence inputKeys, } else { final long dt = timestamp - lastStamp; // alpha is dynamic, based on time - final double alpha = Math.exp(-dt / (double)reverseTimeScaleUnits); + final double alpha = Math.exp(-dt / (double) reverseWindowScaleUnits); curVal = alpha * curVal + ((1 - alpha) * input); lastStamp = timestamp; } @@ -132,7 +132,7 @@ public ByteEMAOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java index ed229e91a8d..edf7de855fe 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java @@ -26,8 +26,8 @@ public class DoubleEMAOperator extends BasePrimitiveEMAOperator { protected class Context extends BasePrimitiveEMAOperator.Context { public DoubleChunk doubleValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -84,7 +84,7 @@ public void accumulate(RowSequence inputKeys, lastStamp = timestamp; } else { final long dt = timestamp - lastStamp; - final double alpha = Math.exp(-dt / (double)reverseTimeScaleUnits); + final double alpha = Math.exp(-dt / (double) reverseWindowScaleUnits); curVal = alpha * curVal + ((1 - alpha) * input); lastStamp = timestamp; } @@ -148,7 +148,7 @@ public DoubleEMAOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java index c4e84ab80d9..350e09db701 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java @@ -21,8 +21,8 @@ public class FloatEMAOperator extends BasePrimitiveEMAOperator { protected class Context extends BasePrimitiveEMAOperator.Context { public FloatChunk floatValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -79,7 +79,7 @@ public void accumulate(RowSequence inputKeys, lastStamp = timestamp; } else { final long dt = timestamp - lastStamp; - final double alpha = Math.exp(-dt / (double)reverseTimeScaleUnits); + final double alpha = Math.exp(-dt / (double) reverseWindowScaleUnits); curVal = alpha * curVal + ((1 - alpha) * input); lastStamp = timestamp; } @@ -143,7 +143,7 @@ public FloatEMAOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java index c4419e545ad..82e0e953521 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java @@ -26,8 +26,8 @@ protected class Context extends BasePrimitiveEMAOperator.Context { public IntChunk intValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -76,7 +76,7 @@ public void accumulate(RowSequence inputKeys, } else { final long dt = timestamp - lastStamp; // alpha is dynamic, based on time - final double alpha = Math.exp(-dt / (double)reverseTimeScaleUnits); + final double alpha = Math.exp(-dt / (double) reverseWindowScaleUnits); curVal = alpha * curVal + ((1 - alpha) * input); lastStamp = timestamp; } @@ -132,7 +132,7 @@ public IntEMAOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java index ed6cf3b078b..6d936d3f7d4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java @@ -26,8 +26,8 @@ protected class Context extends BasePrimitiveEMAOperator.Context { public LongChunk longValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -76,7 +76,7 @@ public void accumulate(RowSequence inputKeys, } else { final long dt = timestamp - lastStamp; // alpha is dynamic, based on time - final double alpha = Math.exp(-dt / (double)reverseTimeScaleUnits); + final double alpha = Math.exp(-dt / (double) reverseWindowScaleUnits); curVal = alpha * curVal + ((1 - alpha) * input); lastStamp = timestamp; } @@ -132,7 +132,7 @@ public LongEMAOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java index ca2a0d275ed..e50a8734b63 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java @@ -21,8 +21,8 @@ protected class Context extends BasePrimitiveEMAOperator.Context { public ShortChunk shortValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -71,7 +71,7 @@ public void accumulate(RowSequence inputKeys, } else { final long dt = timestamp - lastStamp; // alpha is dynamic, based on time - final double alpha = Math.exp(-dt / (double)reverseTimeScaleUnits); + final double alpha = Math.exp(-dt / (double) reverseWindowScaleUnits); curVal = alpha * curVal + ((1 - alpha) * input); lastStamp = timestamp; } @@ -127,7 +127,7 @@ public ShortEMAOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java index 6d7fd883044..9f760ee5a0e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java @@ -31,8 +31,8 @@ public class BooleanFillByOperator extends BaseByteUpdateByOperator { protected class Context extends BaseByteUpdateByOperator.Context { public ByteChunk booleanValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -63,8 +63,8 @@ public BooleanFillByOperator(@NotNull final MatchPair fillPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } // region extra-methods diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java index c4bb080867e..813986ae2f4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java @@ -24,8 +24,8 @@ public class ByteFillByOperator extends BaseByteUpdateByOperator { protected class Context extends BaseByteUpdateByOperator.Context { public ByteChunk byteValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -56,8 +56,8 @@ public ByteFillByOperator(@NotNull final MatchPair fillPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } // region extra-methods diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java index 19c80152d8a..d3c15a0eef1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java @@ -19,8 +19,8 @@ public class CharFillByOperator extends BaseCharUpdateByOperator { protected class Context extends BaseCharUpdateByOperator.Context { public CharChunk charValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -51,8 +51,8 @@ public CharFillByOperator(@NotNull final MatchPair fillPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } // region extra-methods diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java index fb5003dc073..5362f37ef41 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java @@ -24,8 +24,8 @@ public class DoubleFillByOperator extends BaseDoubleUpdateByOperator { protected class Context extends BaseDoubleUpdateByOperator.Context { public DoubleChunk doubleValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -56,8 +56,8 @@ public DoubleFillByOperator(@NotNull final MatchPair fillPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } // region extra-methods diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java index 81d58b706b9..f29e82a5a00 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java @@ -24,8 +24,8 @@ public class FloatFillByOperator extends BaseFloatUpdateByOperator { protected class Context extends BaseFloatUpdateByOperator.Context { public FloatChunk floatValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -56,8 +56,8 @@ public FloatFillByOperator(@NotNull final MatchPair fillPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } // region extra-methods diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java index 0f7aa85801a..82ecd5a1117 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java @@ -24,8 +24,8 @@ public class IntFillByOperator extends BaseIntUpdateByOperator { protected class Context extends BaseIntUpdateByOperator.Context { public IntChunk intValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -56,8 +56,8 @@ public IntFillByOperator(@NotNull final MatchPair fillPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } // region extra-methods diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java index d32da8790d3..f273d01660b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java @@ -32,8 +32,8 @@ public class LongFillByOperator extends BaseLongUpdateByOperator { protected class Context extends BaseLongUpdateByOperator.Context { public LongChunk longValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -66,8 +66,8 @@ public LongFillByOperator(@NotNull final MatchPair fillPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } // region extra-methods diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java index b52ed0f786a..39484c130cd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java @@ -25,8 +25,8 @@ public class ObjectFillByOperator extends BaseObjectUpdateByOperator { protected class Context extends BaseObjectUpdateByOperator.Context { public ObjectChunk ObjectValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -58,8 +58,8 @@ public ObjectFillByOperator(@NotNull final MatchPair fillPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } // region extra-methods diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java index 6a455136756..9aaada7e18b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java @@ -24,8 +24,8 @@ public class ShortFillByOperator extends BaseShortUpdateByOperator { protected class Context extends BaseShortUpdateByOperator.Context { public ShortChunk shortValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -56,8 +56,8 @@ public ShortFillByOperator(@NotNull final MatchPair fillPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } // region extra-methods diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java index c6e182304f9..faa5985adb6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java @@ -19,6 +19,7 @@ import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.updateby.UpdateByCumulativeOperator; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -43,8 +44,8 @@ protected abstract class Context extends UpdateByCumulativeOperator.Context { public byte curVal = NULL_BYTE; - protected Context(final int chunkSize) { - super(); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkCount); this.outputFillContext = outputSource.makeFillFromContext(chunkSize); this.outputValues = WritableByteChunk.makeWritableChunk(chunkSize); } @@ -123,8 +124,8 @@ public BaseByteUpdateByOperator(@NotNull final MatchPair pair, * @param rowRedirection the {@link RowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is - * measured in ticks, otherwise it is measured in nanoseconds. + * @param timeScaleUnits the smoothing window for the operator. If no {@code timestampColumnName} is provided, this + * is measured in ticks, otherwise it is measured in nanoseconds. */ public BaseByteUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java index 8fedb2a0794..ce28fae2498 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java @@ -9,6 +9,7 @@ import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.updateby.UpdateByCumulativeOperator; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -32,8 +33,8 @@ protected abstract class Context extends UpdateByCumulativeOperator.Context { public char curVal = NULL_CHAR; - protected Context(final int chunkSize) { - super(); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkCount); this.outputFillContext = outputSource.makeFillFromContext(chunkSize); this.outputValues = WritableCharChunk.makeWritableChunk(chunkSize); } @@ -112,8 +113,8 @@ public BaseCharUpdateByOperator(@NotNull final MatchPair pair, * @param rowRedirection the {@link RowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is - * measured in ticks, otherwise it is measured in nanoseconds. + * @param timeScaleUnits the smoothing window for the operator. If no {@code timestampColumnName} is provided, this + * is measured in ticks, otherwise it is measured in nanoseconds. */ public BaseCharUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java index 755985fff4e..47549ea01c5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java @@ -1,6 +1,6 @@ /* * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit BaseFloatUpdateByOperator and regenerate + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit BaseCharUpdateByOperator and regenerate * --------------------------------------------------------------------------------------------------------------------- */ package io.deephaven.engine.table.impl.updateby.internal; @@ -12,10 +12,9 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.sources.DoubleArraySource; -import io.deephaven.engine.table.impl.sources.DoubleSparseArraySource; -import io.deephaven.engine.table.impl.sources.WritableRedirectedColumnSource; +import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.updateby.UpdateByCumulativeOperator; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -24,7 +23,7 @@ import java.util.Map; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; -import static io.deephaven.util.QueryConstants.NULL_DOUBLE; +import static io.deephaven.util.QueryConstants.*; public abstract class BaseDoubleUpdateByOperator extends UpdateByCumulativeOperator { protected final WritableColumnSource outputSource; @@ -39,8 +38,8 @@ protected abstract class Context extends UpdateByCumulativeOperator.Context { public double curVal = NULL_DOUBLE; - protected Context(final int chunkSize) { - super(); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkCount); this.outputFillContext = outputSource.makeFillFromContext(chunkSize); this.outputValues = WritableDoubleChunk.makeWritableChunk(chunkSize); } @@ -93,20 +92,19 @@ public void reset() { } } - /** * Construct a base operator for operations that produce double outputs. * - * @param pair the {@link MatchPair} that defines the input/output for this operation + * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. * @param rowRedirection the {@link RowRedirection} for the output column */ public BaseDoubleUpdateByOperator(@NotNull final MatchPair pair, - @NotNull final String[] affectingColumns, - @Nullable final RowRedirection rowRedirection - // region extra-constructor-args - // endregion extra-constructor-args + @NotNull final String[] affectingColumns, + @Nullable final RowRedirection rowRedirection + // region extra-constructor-args + // endregion extra-constructor-args ) { this(pair, affectingColumns, rowRedirection, null, 0); } @@ -114,23 +112,23 @@ public BaseDoubleUpdateByOperator(@NotNull final MatchPair pair, /** * Construct a base operator for operations that produce double outputs. * - * @param pair the {@link MatchPair} that defines the input/output for this operation + * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. * @param rowRedirection the {@link RowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is - * measured in ticks, otherwise it is measured in nanoseconds. + * @param timeScaleUnits the smoothing window for the operator. If no {@code timestampColumnName} is provided, this + * is measured in ticks, otherwise it is measured in nanoseconds. */ public BaseDoubleUpdateByOperator(@NotNull final MatchPair pair, - @NotNull final String[] affectingColumns, - @Nullable final RowRedirection rowRedirection, - @Nullable final String timestampColumnName, - final long timeScaleUnits - // region extra-constructor-args - // endregion extra-constructor-args - ) { + @NotNull final String[] affectingColumns, + @Nullable final RowRedirection rowRedirection, + @Nullable final String timestampColumnName, + final long timeScaleUnits + // region extra-constructor-args + // endregion extra-constructor-args + ) { super(pair, affectingColumns, rowRedirection, timestampColumnName, timeScaleUnits); if(rowRedirection != null) { // region create-dense @@ -148,6 +146,7 @@ public BaseDoubleUpdateByOperator(@NotNull final MatchPair pair, // endregion constructor } + // region extra-methods // endregion extra-methods @@ -175,7 +174,7 @@ public void startTrackingPrev() { public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long delta) { ((DoubleSparseArraySource)outputSource).shift(subRowSetToShift, delta); } - // endregion + // endregion Shifts @Override public void prepareForParallelPopulation(final RowSet changedRows) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java index febd72f9bc9..d6a43f6904a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java @@ -1,3 +1,8 @@ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit BaseCharUpdateByOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ package io.deephaven.engine.table.impl.updateby.internal; import io.deephaven.chunk.Chunk; @@ -7,10 +12,9 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.sources.FloatArraySource; -import io.deephaven.engine.table.impl.sources.FloatSparseArraySource; -import io.deephaven.engine.table.impl.sources.WritableRedirectedColumnSource; +import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.updateby.UpdateByCumulativeOperator; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -19,7 +23,7 @@ import java.util.Map; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; -import static io.deephaven.util.QueryConstants.NULL_FLOAT; +import static io.deephaven.util.QueryConstants.*; public abstract class BaseFloatUpdateByOperator extends UpdateByCumulativeOperator { protected final WritableColumnSource outputSource; @@ -34,8 +38,8 @@ protected abstract class Context extends UpdateByCumulativeOperator.Context { public float curVal = NULL_FLOAT; - protected Context(final int chunkSize) { - super(); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkCount); this.outputFillContext = outputSource.makeFillFromContext(chunkSize); this.outputValues = WritableFloatChunk.makeWritableChunk(chunkSize); } @@ -88,20 +92,19 @@ public void reset() { } } - /** * Construct a base operator for operations that produce float outputs. * - * @param pair the {@link MatchPair} that defines the input/output for this operation + * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. * @param rowRedirection the {@link RowRedirection} for the output column */ public BaseFloatUpdateByOperator(@NotNull final MatchPair pair, - @NotNull final String[] affectingColumns, - @Nullable final RowRedirection rowRedirection - // region extra-constructor-args - // endregion extra-constructor-args + @NotNull final String[] affectingColumns, + @Nullable final RowRedirection rowRedirection + // region extra-constructor-args + // endregion extra-constructor-args ) { this(pair, affectingColumns, rowRedirection, null, 0); } @@ -109,23 +112,23 @@ public BaseFloatUpdateByOperator(@NotNull final MatchPair pair, /** * Construct a base operator for operations that produce float outputs. * - * @param pair the {@link MatchPair} that defines the input/output for this operation + * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns a list of all columns (including the input column from the pair) that affects the result * of this operator. * @param rowRedirection the {@link RowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is - * measured in ticks, otherwise it is measured in nanoseconds. + * @param timeScaleUnits the smoothing window for the operator. If no {@code timestampColumnName} is provided, this + * is measured in ticks, otherwise it is measured in nanoseconds. */ public BaseFloatUpdateByOperator(@NotNull final MatchPair pair, - @NotNull final String[] affectingColumns, - @Nullable final RowRedirection rowRedirection, - @Nullable final String timestampColumnName, - final long timeScaleUnits - // region extra-constructor-args - // endregion extra-constructor-args - ) { + @NotNull final String[] affectingColumns, + @Nullable final RowRedirection rowRedirection, + @Nullable final String timestampColumnName, + final long timeScaleUnits + // region extra-constructor-args + // endregion extra-constructor-args + ) { super(pair, affectingColumns, rowRedirection, timestampColumnName, timeScaleUnits); if(rowRedirection != null) { // region create-dense @@ -143,6 +146,7 @@ public BaseFloatUpdateByOperator(@NotNull final MatchPair pair, // endregion constructor } + // region extra-methods // endregion extra-methods @@ -170,7 +174,7 @@ public void startTrackingPrev() { public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long delta) { ((FloatSparseArraySource)outputSource).shift(subRowSetToShift, delta); } - // endregion + // endregion Shifts @Override public void prepareForParallelPopulation(final RowSet changedRows) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java index 5f56bce323f..80abcd981c8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java @@ -14,6 +14,7 @@ import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.updateby.UpdateByCumulativeOperator; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -37,8 +38,8 @@ protected abstract class Context extends UpdateByCumulativeOperator.Context { public int curVal = NULL_INT; - protected Context(final int chunkSize) { - super(); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkCount); this.outputFillContext = outputSource.makeFillFromContext(chunkSize); this.outputValues = WritableIntChunk.makeWritableChunk(chunkSize); } @@ -117,8 +118,8 @@ public BaseIntUpdateByOperator(@NotNull final MatchPair pair, * @param rowRedirection the {@link RowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is - * measured in ticks, otherwise it is measured in nanoseconds. + * @param timeScaleUnits the smoothing window for the operator. If no {@code timestampColumnName} is provided, this + * is measured in ticks, otherwise it is measured in nanoseconds. */ public BaseIntUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java index 21e1f2bba8b..af578394c60 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java @@ -14,6 +14,7 @@ import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.updateby.UpdateByCumulativeOperator; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -37,8 +38,8 @@ protected abstract class Context extends UpdateByCumulativeOperator.Context { public long curVal = NULL_LONG; - protected Context(final int chunkSize) { - super(); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkCount); this.outputFillContext = outputSource.makeFillFromContext(chunkSize); this.outputValues = WritableLongChunk.makeWritableChunk(chunkSize); } @@ -117,8 +118,8 @@ public BaseLongUpdateByOperator(@NotNull final MatchPair pair, * @param rowRedirection the {@link RowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is - * measured in ticks, otherwise it is measured in nanoseconds. + * @param timeScaleUnits the smoothing window for the operator. If no {@code timestampColumnName} is provided, this + * is measured in ticks, otherwise it is measured in nanoseconds. */ public BaseLongUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java index b59e0ba4696..62d38714e26 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java @@ -12,8 +12,8 @@ public abstract class BaseObjectBinaryOperator extends BaseObjectUpdateByOper protected class Context extends BaseObjectUpdateByOperator.Context { public ObjectChunk objectValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -51,7 +51,7 @@ public BaseObjectBinaryOperator(@NotNull final Class type, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java index 5442ac84457..40296f875c0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java @@ -16,6 +16,7 @@ import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.updateby.UpdateByCumulativeOperator; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -40,8 +41,8 @@ protected abstract class Context extends UpdateByCumulativeOperator.Context { public T curVal = null; - protected Context(final int chunkSize) { - super(); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkCount); this.outputFillContext = outputSource.makeFillFromContext(chunkSize); this.outputValues = WritableObjectChunk.makeWritableChunk(chunkSize); } @@ -121,8 +122,8 @@ public BaseObjectUpdateByOperator(@NotNull final MatchPair pair, * @param rowRedirection the {@link RowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is - * measured in ticks, otherwise it is measured in nanoseconds. + * @param timeScaleUnits the smoothing window for the operator. If no {@code timestampColumnName} is provided, this + * is measured in ticks, otherwise it is measured in nanoseconds. */ public BaseObjectUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @@ -136,7 +137,7 @@ public BaseObjectUpdateByOperator(@NotNull final MatchPair pair, super(pair, affectingColumns, rowRedirection, timestampColumnName, timeScaleUnits); if(rowRedirection != null) { // region create-dense - this.maybeInnerSource = new ObjectArraySource(colType); + this.maybeInnerSource = new ObjectArraySource<>(colType); // endregion create-dense this.outputSource = new WritableRedirectedColumnSource<>(rowRedirection, maybeInnerSource, 0); } else { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java index 7ba9576463c..015696565b8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java @@ -14,6 +14,7 @@ import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.updateby.UpdateByCumulativeOperator; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -37,8 +38,8 @@ protected abstract class Context extends UpdateByCumulativeOperator.Context { public short curVal = NULL_SHORT; - protected Context(final int chunkSize) { - super(); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkCount); this.outputFillContext = outputSource.makeFillFromContext(chunkSize); this.outputValues = WritableShortChunk.makeWritableChunk(chunkSize); } @@ -117,8 +118,8 @@ public BaseShortUpdateByOperator(@NotNull final MatchPair pair, * @param rowRedirection the {@link RowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is - * measured in ticks, otherwise it is measured in nanoseconds. + * @param timeScaleUnits the smoothing window for the operator. If no {@code timestampColumnName} is provided, this + * is measured in ticks, otherwise it is measured in nanoseconds. */ public BaseShortUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java index 301a7c5a899..e038b79241e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java @@ -45,7 +45,8 @@ protected abstract class Context extends UpdateByWindowedOperator.Context { public byte curVal = NULL_BYTE; - protected Context(final int chunkSize) { + protected Context(final int chunkSize, final int chunkCount) { + super(chunkCount); this.outputFillContext = outputSource.makeFillFromContext(chunkSize); this.outputValues = WritableByteChunk.makeWritableChunk(chunkSize); } @@ -125,15 +126,14 @@ public void reset() { public BaseWindowedByteUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final OperationControl control, @Nullable final String timestampColumnName, - final long reverseTimeScaleUnits, - final long forwardTimeScaleUnits, + final long reverseWindowScaleUnits, + final long forwardWindowScaleUnits, @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); + super(pair, affectingColumns, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, rowRedirection); if (rowRedirection != null) { // region create-dense this.maybeInnerSource = makeDenseSource(); @@ -166,10 +166,6 @@ protected WritableColumnSource makeDenseSource() { } // endregion extra-methods - @Override - public void initializeUpdate(@NotNull UpdateContext context) { - } - @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java index b98d515d662..84a791eeec3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java @@ -34,7 +34,8 @@ protected abstract class Context extends UpdateByWindowedOperator.Context { public char curVal = NULL_CHAR; - protected Context(final int chunkSize) { + protected Context(final int chunkSize, final int chunkCount) { + super(chunkCount); this.outputFillContext = outputSource.makeFillFromContext(chunkSize); this.outputValues = WritableCharChunk.makeWritableChunk(chunkSize); } @@ -114,15 +115,14 @@ public void reset() { public BaseWindowedCharUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final OperationControl control, @Nullable final String timestampColumnName, - final long reverseTimeScaleUnits, - final long forwardTimeScaleUnits, + final long reverseWindowScaleUnits, + final long forwardWindowScaleUnits, @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); + super(pair, affectingColumns, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, rowRedirection); if (rowRedirection != null) { // region create-dense this.maybeInnerSource = new CharacterArraySource(); @@ -142,10 +142,6 @@ public BaseWindowedCharUpdateByOperator(@NotNull final MatchPair pair, // region extra-methods // endregion extra-methods - @Override - public void initializeUpdate(@NotNull UpdateContext context) { - } - @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java index 708c9ce3518..14d782af741 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java @@ -1,6 +1,6 @@ /* * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit BaseWindowedFloatUpdateByOperator and regenerate + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit BaseWindowedCharUpdateByOperator and regenerate * --------------------------------------------------------------------------------------------------------------------- */ package io.deephaven.engine.table.impl.updateby.internal; @@ -14,9 +14,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.sources.DoubleArraySource; -import io.deephaven.engine.table.impl.sources.DoubleSparseArraySource; -import io.deephaven.engine.table.impl.sources.WritableRedirectedColumnSource; +import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.updateby.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -32,8 +30,6 @@ public abstract class BaseWindowedDoubleUpdateByOperator extends UpdateByWindowe protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; - public double curVal = NULL_DOUBLE; - // region extra-fields // endregion extra-fields @@ -41,7 +37,10 @@ protected abstract class Context extends UpdateByWindowedOperator.Context { public final ChunkSink.FillFromContext outputFillContext; public final WritableDoubleChunk outputValues; - protected Context(final int chunkSize) { + public double curVal = NULL_DOUBLE; + + protected Context(final int chunkSize, final int chunkCount) { + super(chunkCount); this.outputFillContext = outputSource.makeFillFromContext(chunkSize); this.outputValues = WritableDoubleChunk.makeWritableChunk(chunkSize); } @@ -121,16 +120,15 @@ public void reset() { public BaseWindowedDoubleUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final OperationControl control, @Nullable final String timestampColumnName, - final long reverseTimeScaleUnits, - final long forwardTimeScaleUnits, + final long reverseWindowScaleUnits, + final long forwardWindowScaleUnits, @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args - ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); - if(rowRedirection != null) { + ) { + super(pair, affectingColumns, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, rowRedirection); + if (rowRedirection != null) { // region create-dense this.maybeInnerSource = new DoubleArraySource(); // endregion create-dense @@ -149,10 +147,6 @@ public BaseWindowedDoubleUpdateByOperator(@NotNull final MatchPair pair, // region extra-methods // endregion extra-methods - @Override - public void initializeUpdate(@NotNull UpdateContext context) { - } - @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java index f0d32c0b944..3e2f9ad85be 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java @@ -1,3 +1,8 @@ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit BaseWindowedCharUpdateByOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ package io.deephaven.engine.table.impl.updateby.internal; import io.deephaven.api.updateby.OperationControl; @@ -9,9 +14,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.sources.FloatArraySource; -import io.deephaven.engine.table.impl.sources.FloatSparseArraySource; -import io.deephaven.engine.table.impl.sources.WritableRedirectedColumnSource; +import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.updateby.UpdateByWindowedOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -27,8 +30,6 @@ public abstract class BaseWindowedFloatUpdateByOperator extends UpdateByWindowed protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; - public float curVal = NULL_FLOAT; - // region extra-fields // endregion extra-fields @@ -36,7 +37,10 @@ protected abstract class Context extends UpdateByWindowedOperator.Context { public final ChunkSink.FillFromContext outputFillContext; public final WritableFloatChunk outputValues; - protected Context(final int chunkSize) { + public float curVal = NULL_FLOAT; + + protected Context(final int chunkSize, final int chunkCount) { + super(chunkCount); this.outputFillContext = outputSource.makeFillFromContext(chunkSize); this.outputValues = WritableFloatChunk.makeWritableChunk(chunkSize); } @@ -116,16 +120,15 @@ public void reset() { public BaseWindowedFloatUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final OperationControl control, @Nullable final String timestampColumnName, - final long reverseTimeScaleUnits, - final long forwardTimeScaleUnits, + final long reverseWindowScaleUnits, + final long forwardWindowScaleUnits, @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args - ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); - if(rowRedirection != null) { + ) { + super(pair, affectingColumns, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, rowRedirection); + if (rowRedirection != null) { // region create-dense this.maybeInnerSource = new FloatArraySource(); // endregion create-dense @@ -144,10 +147,6 @@ public BaseWindowedFloatUpdateByOperator(@NotNull final MatchPair pair, // region extra-methods // endregion extra-methods - @Override - public void initializeUpdate(@NotNull UpdateContext context) { - } - @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java index fb5e9c96270..0efb65c5ea9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java @@ -39,7 +39,8 @@ protected abstract class Context extends UpdateByWindowedOperator.Context { public int curVal = NULL_INT; - protected Context(final int chunkSize) { + protected Context(final int chunkSize, final int chunkCount) { + super(chunkCount); this.outputFillContext = outputSource.makeFillFromContext(chunkSize); this.outputValues = WritableIntChunk.makeWritableChunk(chunkSize); } @@ -119,15 +120,14 @@ public void reset() { public BaseWindowedIntUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final OperationControl control, @Nullable final String timestampColumnName, - final long reverseTimeScaleUnits, - final long forwardTimeScaleUnits, + final long reverseWindowScaleUnits, + final long forwardWindowScaleUnits, @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); + super(pair, affectingColumns, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, rowRedirection); if (rowRedirection != null) { // region create-dense this.maybeInnerSource = new IntegerArraySource(); @@ -147,10 +147,6 @@ public BaseWindowedIntUpdateByOperator(@NotNull final MatchPair pair, // region extra-methods // endregion extra-methods - @Override - public void initializeUpdate(@NotNull UpdateContext context) { - } - @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java index 2f1d34863f2..1818385b878 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java @@ -39,7 +39,8 @@ protected abstract class Context extends UpdateByWindowedOperator.Context { public long curVal = NULL_LONG; - protected Context(final int chunkSize) { + protected Context(final int chunkSize, final int chunkCount) { + super(chunkCount); this.outputFillContext = outputSource.makeFillFromContext(chunkSize); this.outputValues = WritableLongChunk.makeWritableChunk(chunkSize); } @@ -119,15 +120,14 @@ public void reset() { public BaseWindowedLongUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final OperationControl control, @Nullable final String timestampColumnName, - final long reverseTimeScaleUnits, - final long forwardTimeScaleUnits, + final long reverseWindowScaleUnits, + final long forwardWindowScaleUnits, @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); + super(pair, affectingColumns, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, rowRedirection); if (rowRedirection != null) { // region create-dense this.maybeInnerSource = new LongArraySource(); @@ -147,10 +147,6 @@ public BaseWindowedLongUpdateByOperator(@NotNull final MatchPair pair, // region extra-methods // endregion extra-methods - @Override - public void initializeUpdate(@NotNull UpdateContext context) { - } - @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java index 5883d1d8256..5c1e0de1041 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java @@ -42,7 +42,8 @@ protected abstract class Context extends UpdateByWindowedOperator.Context { public T curVal = null; - protected Context(final int chunkSize) { + protected Context(final int chunkSize, final int chunkCount) { + super(chunkCount); this.outputFillContext = outputSource.makeFillFromContext(chunkSize); this.outputValues = WritableObjectChunk.makeWritableChunk(chunkSize); } @@ -122,19 +123,18 @@ public void reset() { public BaseWindowedObjectUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final OperationControl control, @Nullable final String timestampColumnName, - final long reverseTimeScaleUnits, - final long forwardTimeScaleUnits, + final long reverseWindowScaleUnits, + final long forwardWindowScaleUnits, @Nullable final RowRedirection rowRedirection // region extra-constructor-args , final Class colType // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); + super(pair, affectingColumns, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, rowRedirection); if (rowRedirection != null) { // region create-dense - this.maybeInnerSource = new ObjectArraySource(colType); + this.maybeInnerSource = new ObjectArraySource<>(colType); // endregion create-dense this.outputSource = new WritableRedirectedColumnSource<>(rowRedirection, maybeInnerSource, 0); } else { @@ -152,10 +152,6 @@ public BaseWindowedObjectUpdateByOperator(@NotNull final MatchPair pair, // region extra-methods // endregion extra-methods - @Override - public void initializeUpdate(@NotNull UpdateContext context) { - } - @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java index 4156d242975..7f333aaa340 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java @@ -39,7 +39,8 @@ protected abstract class Context extends UpdateByWindowedOperator.Context { public short curVal = NULL_SHORT; - protected Context(final int chunkSize) { + protected Context(final int chunkSize, final int chunkCount) { + super(chunkCount); this.outputFillContext = outputSource.makeFillFromContext(chunkSize); this.outputValues = WritableShortChunk.makeWritableChunk(chunkSize); } @@ -119,15 +120,14 @@ public void reset() { public BaseWindowedShortUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final OperationControl control, @Nullable final String timestampColumnName, - final long reverseTimeScaleUnits, - final long forwardTimeScaleUnits, + final long reverseWindowScaleUnits, + final long forwardWindowScaleUnits, @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); + super(pair, affectingColumns, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, rowRedirection); if (rowRedirection != null) { // region create-dense this.maybeInnerSource = new ShortArraySource(); @@ -147,10 +147,6 @@ public BaseWindowedShortUpdateByOperator(@NotNull final MatchPair pair, // region extra-methods // endregion extra-methods - @Override - public void initializeUpdate(@NotNull UpdateContext context) { - } - @Override public void startTrackingPrev() { outputSource.startTrackingPrevValues(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java index e475066c626..c8d3cbdc2c4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java @@ -26,8 +26,8 @@ public class ByteCumMinMaxOperator extends BaseByteUpdateByOperator { protected class Context extends BaseByteUpdateByOperator.Context { public ByteChunk byteValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -67,7 +67,7 @@ public ByteCumMinMaxOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java index 97616a166de..3b74335c1a3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java @@ -26,8 +26,8 @@ public class DoubleCumMinMaxOperator extends BaseDoubleUpdateByOperator { protected class Context extends BaseDoubleUpdateByOperator.Context { public DoubleChunk doubleValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -67,7 +67,7 @@ public DoubleCumMinMaxOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java index 6a0077e6e28..f30ab671f86 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java @@ -21,8 +21,8 @@ public class FloatCumMinMaxOperator extends BaseFloatUpdateByOperator { protected class Context extends BaseFloatUpdateByOperator.Context { public FloatChunk floatValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -62,7 +62,7 @@ public FloatCumMinMaxOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java index c7f16cc62ed..0f49afd028d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java @@ -26,8 +26,8 @@ public class IntCumMinMaxOperator extends BaseIntUpdateByOperator { protected class Context extends BaseIntUpdateByOperator.Context { public IntChunk intValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -67,7 +67,7 @@ public IntCumMinMaxOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java index 503ab7cefc9..564679dbe93 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java @@ -34,8 +34,8 @@ public class LongCumMinMaxOperator extends BaseLongUpdateByOperator { protected class Context extends BaseLongUpdateByOperator.Context { public LongChunk longValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -77,7 +77,7 @@ public LongCumMinMaxOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java index 05c169c60f4..476805da716 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java @@ -21,8 +21,8 @@ public class ShortCumMinMaxOperator extends BaseShortUpdateByOperator { protected class Context extends BaseShortUpdateByOperator.Context { public ShortChunk shortValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -62,7 +62,7 @@ public ShortCumMinMaxOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java index 24ed54f4ac0..26e3cc61b39 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java @@ -20,8 +20,8 @@ public final class BigDecimalCumProdOperator extends BaseObjectUpdateByOperator< protected class Context extends BaseObjectUpdateByOperator.Context { public ObjectChunk objectValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -61,7 +61,7 @@ public BigDecimalCumProdOperator(@NotNull final MatchPair inputPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java index 1a773e0f776..cc56add1eb3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java @@ -16,8 +16,8 @@ public final class BigIntegerCumProdOperator extends BaseObjectUpdateByOperator< protected class Context extends BaseObjectUpdateByOperator.Context { public ObjectChunk objectValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -55,7 +55,7 @@ public BigIntegerCumProdOperator(@NotNull final MatchPair inputPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java index a0033df0b62..ffe9c770f71 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java @@ -25,8 +25,8 @@ public class ByteCumProdOperator extends BaseLongUpdateByOperator { protected class Context extends BaseLongUpdateByOperator.Context { public ByteChunk byteValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -61,7 +61,7 @@ public ByteCumProdOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java index ebc603c882a..197bb172b13 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java @@ -24,8 +24,8 @@ public class DoubleCumProdOperator extends BaseDoubleUpdateByOperator { protected class Context extends BaseDoubleUpdateByOperator.Context { public DoubleChunk doubleValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -60,8 +60,8 @@ public DoubleCumProdOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java index 5efd40cb4da..72110018cdf 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java @@ -19,8 +19,8 @@ public class FloatCumProdOperator extends BaseFloatUpdateByOperator { protected class Context extends BaseFloatUpdateByOperator.Context { public FloatChunk floatValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -55,8 +55,8 @@ public FloatCumProdOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java index 2a0ebab5a26..70613de699f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java @@ -25,8 +25,8 @@ public class IntCumProdOperator extends BaseLongUpdateByOperator { protected class Context extends BaseLongUpdateByOperator.Context { public IntChunk intValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -61,7 +61,7 @@ public IntCumProdOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java index 07caa35394f..873ae7e8ab6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java @@ -25,8 +25,8 @@ public class LongCumProdOperator extends BaseLongUpdateByOperator { protected class Context extends BaseLongUpdateByOperator.Context { public LongChunk longValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -61,7 +61,7 @@ public LongCumProdOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java index feceae27c15..cc561008802 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java @@ -20,8 +20,8 @@ public class ShortCumProdOperator extends BaseLongUpdateByOperator { protected class Context extends BaseLongUpdateByOperator.Context { public ShortChunk shortValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -56,7 +56,7 @@ public ShortCumProdOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java index 016ea10ac86..0611f922d63 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java @@ -24,8 +24,8 @@ protected class Context extends BaseWindowedObjectUpdateByOperator.C protected ObjectChunk objectInfluencerValuesChunk; protected RingBuffer objectWindowValues; - protected Context(final int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); objectWindowValues = new RingBuffer<>(RING_BUFFER_INITIAL_CAPACITY); } @@ -87,19 +87,18 @@ public void writeToOutputChunk(int outIdx) { @NotNull @Override - public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize) { - return new Context(chunkSize); + public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } public BigDecimalRollingSumOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final OperationControl control, @Nullable final String timestampColumnName, - final long reverseTimeScaleUnits, - final long forwardTimeScaleUnits, + final long reverseWindowScaleUnits, + final long forwardWindowScaleUnits, @Nullable final RowRedirection rowRedirection, @NotNull final MathContext mathContext) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, + super(pair, affectingColumns, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, rowRedirection, BigDecimal.class); this.mathContext = mathContext; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java index d0d32926a61..855092033c9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java @@ -22,8 +22,8 @@ protected class Context extends BaseWindowedObjectUpdateByOperator.C protected ObjectChunk objectInfluencerValuesChunk; protected RingBuffer objectWindowValues; - protected Context(final int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); objectWindowValues = new RingBuffer<>(RING_BUFFER_INITIAL_CAPACITY); } @@ -84,21 +84,20 @@ public void writeToOutputChunk(int outIdx) { @NotNull @Override - public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize) { - return new Context(chunkSize); + public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } public BigIntegerRollingSumOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final OperationControl control, @Nullable final String timestampColumnName, - final long reverseTimeScaleUnits, - final long forwardTimeScaleUnits, + final long reverseWindowScaleUnits, + final long forwardWindowScaleUnits, @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection, BigInteger.class); + super(pair, affectingColumns, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, rowRedirection, BigInteger.class); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java index 936c510993b..d0068e2c02e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java @@ -30,8 +30,8 @@ protected class Context extends BaseWindowedLongUpdateByOperator.Context { protected ByteRingBuffer byteWindowValues; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); byteWindowValues = new ByteRingBuffer(RING_BUFFER_INITIAL_CAPACITY, true); } @@ -96,22 +96,21 @@ public void writeToOutputChunk(int outIdx) { @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } public ByteRollingSumOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final OperationControl control, @Nullable final String timestampColumnName, - final long reverseTimeScaleUnits, - final long forwardTimeScaleUnits, + final long reverseWindowScaleUnits, + final long forwardWindowScaleUnits, @Nullable final RowRedirection rowRedirection // region extra-constructor-args ,final byte nullValue // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); + super(pair, affectingColumns, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, rowRedirection); // region constructor this.nullValue = nullValue; // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java index 5474d1e8261..8be0abded2d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java @@ -33,8 +33,8 @@ protected class Context extends BaseWindowedDoubleUpdateByOperator.Context { protected DoubleChunk doubleInfluencerValuesChunk; protected PairwiseDoubleRingBuffer doublePairwiseSum; - protected Context(final int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); doublePairwiseSum = new PairwiseDoubleRingBuffer(PAIRWISE_BUFFER_INITIAL_SIZE, 0.0f, (a, b) -> { if (a == NULL_DOUBLE) { return b; @@ -95,21 +95,20 @@ public void writeToOutputChunk(int outIdx) { @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } public DoubleRollingSumOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final OperationControl control, @Nullable final String timestampColumnName, - final long reverseTimeScaleUnits, - final long forwardTimeScaleUnits, + final long reverseWindowScaleUnits, + final long forwardWindowScaleUnits, @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); + super(pair, affectingColumns, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java index 0ed74b9af37..7e831d528fd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java @@ -28,8 +28,8 @@ protected class Context extends BaseWindowedFloatUpdateByOperator.Context { protected FloatChunk floatInfluencerValuesChunk; protected PairwiseFloatRingBuffer floatPairwiseSum; - protected Context(final int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); floatPairwiseSum = new PairwiseFloatRingBuffer(PAIRWISE_BUFFER_INITIAL_SIZE, 0.0f, (a, b) -> { if (a == NULL_FLOAT) { return b; @@ -90,21 +90,20 @@ public void writeToOutputChunk(int outIdx) { @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } public FloatRollingSumOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final OperationControl control, @Nullable final String timestampColumnName, - final long reverseTimeScaleUnits, - final long forwardTimeScaleUnits, + final long reverseWindowScaleUnits, + final long forwardWindowScaleUnits, @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); + super(pair, affectingColumns, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java index c0bb7d97648..e890db91ee8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java @@ -29,8 +29,8 @@ protected class Context extends BaseWindowedLongUpdateByOperator.Context { protected IntRingBuffer intWindowValues; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); intWindowValues = new IntRingBuffer(RING_BUFFER_INITIAL_CAPACITY, true); } @@ -95,21 +95,20 @@ public void writeToOutputChunk(int outIdx) { @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } public IntRollingSumOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final OperationControl control, @Nullable final String timestampColumnName, - final long reverseTimeScaleUnits, - final long forwardTimeScaleUnits, + final long reverseWindowScaleUnits, + final long forwardWindowScaleUnits, @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); + super(pair, affectingColumns, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java index a1f5e89cd44..e2a78c81f70 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java @@ -29,8 +29,8 @@ protected class Context extends BaseWindowedLongUpdateByOperator.Context { protected LongRingBuffer longWindowValues; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); longWindowValues = new LongRingBuffer(RING_BUFFER_INITIAL_CAPACITY, true); } @@ -95,21 +95,20 @@ public void writeToOutputChunk(int outIdx) { @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } public LongRollingSumOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final OperationControl control, @Nullable final String timestampColumnName, - final long reverseTimeScaleUnits, - final long forwardTimeScaleUnits, + final long reverseWindowScaleUnits, + final long forwardWindowScaleUnits, @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); + super(pair, affectingColumns, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java index c6407781d1e..28ad32a2cbb 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java @@ -24,8 +24,8 @@ protected class Context extends BaseWindowedLongUpdateByOperator.Context { protected ShortRingBuffer shortWindowValues; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); shortWindowValues = new ShortRingBuffer(RING_BUFFER_INITIAL_CAPACITY, true); } @@ -90,21 +90,20 @@ public void writeToOutputChunk(int outIdx) { @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } public ShortRollingSumOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @NotNull final OperationControl control, @Nullable final String timestampColumnName, - final long reverseTimeScaleUnits, - final long forwardTimeScaleUnits, + final long reverseWindowScaleUnits, + final long forwardWindowScaleUnits, @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, reverseTimeScaleUnits, forwardTimeScaleUnits, rowRedirection); + super(pair, affectingColumns, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java index 3d27c0728d2..6e6275c13bf 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java @@ -20,8 +20,8 @@ public final class BigDecimalCumSumOperator extends BaseObjectUpdateByOperator.Context { public ObjectChunk objectValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -57,7 +57,7 @@ public BigDecimalCumSumOperator(@NotNull final MatchPair inputPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java index bc463efd278..adfacc5a552 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java @@ -16,8 +16,8 @@ public final class BigIntegerCumSumOperator extends BaseObjectUpdateByOperator.Context { public ObjectChunk objectValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -50,7 +50,7 @@ public BigIntegerCumSumOperator(@NotNull final MatchPair inputPair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java index f7c94aba0d9..a972d555f51 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java @@ -26,8 +26,8 @@ public class ByteCumSumOperator extends BaseLongUpdateByOperator { protected class Context extends BaseLongUpdateByOperator.Context { public ByteChunk byteValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -64,7 +64,7 @@ public ByteCumSumOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java index 3fdd0dece62..a510eccf0ab 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java @@ -24,8 +24,8 @@ public class DoubleCumSumOperator extends BaseDoubleUpdateByOperator { protected class Context extends BaseDoubleUpdateByOperator.Context { public DoubleChunk doubleValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -60,7 +60,7 @@ public DoubleCumSumOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java index 8893d3eeb9d..1596472e5df 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java @@ -19,8 +19,8 @@ public class FloatCumSumOperator extends BaseFloatUpdateByOperator { protected class Context extends BaseFloatUpdateByOperator.Context { public FloatChunk floatValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -55,7 +55,7 @@ public FloatCumSumOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java index c2d55c041fa..83e3b04d42d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java @@ -25,8 +25,8 @@ public class IntCumSumOperator extends BaseLongUpdateByOperator { protected class Context extends BaseLongUpdateByOperator.Context { public IntChunk intValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -61,7 +61,7 @@ public IntCumSumOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java index 300b0f52eef..eda98e9c230 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java @@ -25,8 +25,8 @@ public class LongCumSumOperator extends BaseLongUpdateByOperator { protected class Context extends BaseLongUpdateByOperator.Context { public LongChunk longValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -61,7 +61,7 @@ public LongCumSumOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java index a0967d15637..8d92fd4e186 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java @@ -20,8 +20,8 @@ public class ShortCumSumOperator extends BaseLongUpdateByOperator { protected class Context extends BaseLongUpdateByOperator.Context { public ShortChunk shortValueChunk; - protected Context(int chunkSize) { - super(chunkSize); + protected Context(final int chunkSize, final int chunkCount) { + super(chunkSize, chunkCount); } @Override @@ -56,7 +56,7 @@ public ShortCumSumOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(int chunkSize) { - return new Context(chunkSize); + public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java index ae1dc15e601..250b4c9f667 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java @@ -329,7 +329,8 @@ private void doTestStaticBucketed(boolean grouped, int prevTicks, int postTicks) final PartitionedTable preOp = t.partitionBy("Sym"); final PartitionedTable postOp = summed.partitionBy("Sym"); - final String[] columns = t.getDefinition().getColumnStream().map(ColumnDefinition::getName).toArray(String[]::new); + final String[] columns = + t.getDefinition().getColumnStream().map(ColumnDefinition::getName).toArray(String[]::new); preOp.partitionedTransform(postOp, (source, actual) -> { Arrays.stream(columns).forEach(col -> { diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java index 4b199b82754..077ce922bd2 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java @@ -89,36 +89,33 @@ protected Table e() { final String[] columnNamesArray = base.getDefinition().getColumnNamesArray(); final Collection clauses = List.of( + UpdateByOperation.Fill(), + UpdateByOperation.RollingSum(100, 0, + makeOpColNames(columnNamesArray, "_rollsumticksrev", "Sym", "ts", "boolCol")), + UpdateByOperation.RollingSum("ts", Duration.ofMinutes(15), Duration.ofMinutes(0), + makeOpColNames(columnNamesArray, "_rollsumtimerev", "Sym", "ts", "boolCol")), + UpdateByOperation.RollingSum(0, 100, + makeOpColNames(columnNamesArray, "_rollsumticksfwd", "Sym", "ts", "boolCol")), UpdateByOperation.RollingSum(-50, 100, - makeOpColNames(columnNamesArray, "_rollsumticksfwdex", "Sym", "ts", "boolCol"))); - -// UpdateByOperation.Fill(), -// UpdateByOperation.RollingSum(100, 0, -// makeOpColNames(columnNamesArray, "_rollsumticksrev", "Sym", "ts", "boolCol")), -// UpdateByOperation.RollingSum("ts", Duration.ofMinutes(15), Duration.ofMinutes(0), -// makeOpColNames(columnNamesArray, "_rollsumtimerev", "Sym", "ts", "boolCol")), -// UpdateByOperation.RollingSum(0, 100, -// makeOpColNames(columnNamesArray, "_rollsumticksfwd", "Sym", "ts", "boolCol")), -// UpdateByOperation.RollingSum(-50, 100, -// makeOpColNames(columnNamesArray, "_rollsumticksfwdex", "Sym", "ts", "boolCol")), -// UpdateByOperation.RollingSum("ts", Duration.ofMinutes(0), Duration.ofMinutes(15), -// makeOpColNames(columnNamesArray, "_rollsumtimefwd", "Sym", "ts", "boolCol")), -// UpdateByOperation.RollingSum("ts", Duration.ofMinutes(-10), Duration.ofMinutes(15), -// makeOpColNames(columnNamesArray, "_rollsumtimefwdex", "Sym", "ts", "boolCol")), -// UpdateByOperation.RollingSum(50, 50, -// makeOpColNames(columnNamesArray, "_rollsumticksfwdrev", "Sym", "ts", -// "boolCol")), -// UpdateByOperation.RollingSum("ts", Duration.ofMinutes(5), Duration.ofMinutes(5), -// makeOpColNames(columnNamesArray, "_rollsumtimebothfwdrev", "Sym", "ts", -// "boolCol")), -// -// UpdateByOperation.Ema(skipControl, "ts", 10 * MINUTE, -// makeOpColNames(columnNamesArray, "_ema", "Sym", "ts", "boolCol")), -// UpdateByOperation.CumSum(makeOpColNames(columnNamesArray, "_sum", "Sym", "ts")), -// UpdateByOperation.CumMin(makeOpColNames(columnNamesArray, "_min", "boolCol")), -// UpdateByOperation.CumMax(makeOpColNames(columnNamesArray, "_max", "boolCol")), -// UpdateByOperation -// .CumProd(makeOpColNames(columnNamesArray, "_prod", "Sym", "ts", "boolCol"))); + makeOpColNames(columnNamesArray, "_rollsumticksfwdex", "Sym", "ts", "boolCol")), + UpdateByOperation.RollingSum("ts", Duration.ofMinutes(0), Duration.ofMinutes(15), + makeOpColNames(columnNamesArray, "_rollsumtimefwd", "Sym", "ts", "boolCol")), + UpdateByOperation.RollingSum("ts", Duration.ofMinutes(-10), Duration.ofMinutes(15), + makeOpColNames(columnNamesArray, "_rollsumtimefwdex", "Sym", "ts", "boolCol")), + UpdateByOperation.RollingSum(50, 50, + makeOpColNames(columnNamesArray, "_rollsumticksfwdrev", "Sym", "ts", + "boolCol")), + UpdateByOperation.RollingSum("ts", Duration.ofMinutes(5), Duration.ofMinutes(5), + makeOpColNames(columnNamesArray, "_rollsumtimebothfwdrev", "Sym", "ts", + "boolCol")), + + UpdateByOperation.Ema(skipControl, "ts", 10 * MINUTE, + makeOpColNames(columnNamesArray, "_ema", "Sym", "ts", "boolCol")), + UpdateByOperation.CumSum(makeOpColNames(columnNamesArray, "_sum", "Sym", "ts")), + UpdateByOperation.CumMin(makeOpColNames(columnNamesArray, "_min", "boolCol")), + UpdateByOperation.CumMax(makeOpColNames(columnNamesArray, "_max", "boolCol")), + UpdateByOperation + .CumProd(makeOpColNames(columnNamesArray, "_prod", "Sym", "ts", "boolCol"))); final UpdateByControl control = UpdateByControl.builder().useRedirection(redirected).build(); return bucketed ? base.updateBy(control, clauses, ColumnName.from("Sym")) diff --git a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java index e8c5d6b4fae..755dd5aa243 100644 --- a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java +++ b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java @@ -50,7 +50,7 @@ public static void main(String[] args) throws IOException { "long getFirstReprocessKey" }; - files = ReplicatePrimitiveCode.charToAllButBooleanAndFloats( + files = ReplicatePrimitiveCode.charToAllButBoolean( "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java", exemptions); for (final String f : files) { @@ -67,9 +67,6 @@ public static void main(String[] args) throws IOException { fixupStandardObject(objectResult, "BaseObjectUpdateByOperator", true, "this\\(pair, affectingColumns, rowRedirection, null, 0\\);", "this(pair, affectingColumns, rowRedirection, null, 0, colType);"); - ReplicatePrimitiveCode.floatToAllFloatingPoints( - "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java", - exemptions); replicateNumericOperator( "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java", @@ -85,7 +82,7 @@ public static void main(String[] args) throws IOException { "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java"); // Replicate the rolling versions of UpdateByOperations - files = ReplicatePrimitiveCode.charToAllButBooleanAndFloats( + files = ReplicatePrimitiveCode.charToAllButBoolean( "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java", exemptions); for (final String f : files) { @@ -100,9 +97,6 @@ public static void main(String[] args) throws IOException { objectResult = ReplicatePrimitiveCode.charToObject( "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java"); fixupStandardObject(objectResult, "BaseWindowedObjectUpdateByOperator", true); - ReplicatePrimitiveCode.floatToAllFloatingPoints( - "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java", - exemptions); replicateNumericOperator( "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java", @@ -289,7 +283,7 @@ private static void fixupStandardObject(String objectResult, final String classN " ObjectArraySource ", " ObjectArraySource ", "ObjectChunk", "ObjectChunk", "SizedObjectChunk", "SizedObjectChunk<>", - "new ObjectArraySource\\(\\);", "new ObjectArraySource(colType);", + "new ObjectArraySource\\(\\);", "new ObjectArraySource<>(colType);", "new ObjectSparseArraySource\\(\\);", "new ObjectSparseArraySource<>(colType);", "(?:QueryConstants\\.)?NULL_OBJECT", "null", "Object lastValidValue", "T lastValidValue", From d98dd599a0bb80adbae631d571f57802b8b021d8 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Mon, 23 Jan 2023 10:37:34 -0800 Subject: [PATCH 090/123] Changes to UpdateByWindowTime --- .../table/impl/updateby/UpdateByWindow.java | 5 +- .../impl/updateby/UpdateByWindowTicks.java | 2 +- .../impl/updateby/UpdateByWindowTime.java | 87 +++++++++++-------- .../impl/updateby/ema/ByteEMAOperator.java | 34 ++++---- .../impl/updateby/ema/IntEMAOperator.java | 34 ++++---- .../impl/updateby/ema/LongEMAOperator.java | 34 ++++---- .../impl/updateby/ema/ShortEMAOperator.java | 20 ++--- 7 files changed, 115 insertions(+), 101 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java index 51f663087fd..983e6a32fb8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java @@ -84,8 +84,11 @@ class UpdateByWindowBucketContext implements SafeCloseable { @Override public void close() { + // For efficiency, we occasionally use the source rowsets. Must be careful not to close in these cases try (final SafeCloseable ignoredRs1 = affectedRows == sourceRowSet ? null : affectedRows; - final SafeCloseable ignoredRs2 = influencerRows == affectedRows ? null : influencerRows) { + final SafeCloseable ignoredRs2 = + influencerRows == affectedRows || influencerRows == timestampValidRowSet ? null + : influencerRows) { } SafeCloseableArray.close(opContext); if (inputSources != null) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java index 50ee477d639..35e02ef3257 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java @@ -27,7 +27,7 @@ */ class UpdateByWindowTicks extends UpdateByWindow { /** growth rate after the contexts have exceeded the poolable chunk size */ - public static final double CONTEXT_GROWTH_PERCENTAGE = 0.25; + private static final double CONTEXT_GROWTH_PERCENTAGE = 0.25; private static final int WINDOW_CHUNK_SIZE = 4096; private final long prevUnits; private final long fwdUnits; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java index 6fcc6bbba78..a68f4af0f55 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java @@ -2,6 +2,7 @@ import gnu.trove.list.array.TIntArrayList; import gnu.trove.set.hash.TIntHashSet; +import io.deephaven.base.ringbuffer.LongRingBuffer; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableIntChunk; @@ -31,14 +32,16 @@ */ class UpdateByWindowTime extends UpdateByWindow { /** growth rate after the contexts have exceeded the poolable chunk size */ - public static final double CONTEXT_GROWTH_PERCENTAGE = 0.25; + private static final double CONTEXT_GROWTH_PERCENTAGE = 0.25; private static final int WINDOW_CHUNK_SIZE = 4096; + private static final int RING_BUFFER_INITIAL_SIZE = 512; protected final long prevUnits; protected final long fwdUnits; public class UpdateByWindowTimeBucketContext extends UpdateByWindowBucketContext { protected final ChunkSource.GetContext influencerTimestampContext; final ChunkSource.GetContext timestampColumnGetContext; + final LongRingBuffer timestampWindowBuffer; protected int currentGetContextSize; public UpdateByWindowTimeBucketContext(final TrackingRowSet sourceRowSet, @@ -51,6 +54,7 @@ public UpdateByWindowTimeBucketContext(final TrackingRowSet sourceRowSet, influencerTimestampContext = timestampColumnSource.makeGetContext(WINDOW_CHUNK_SIZE); timestampColumnGetContext = timestampColumnSource.makeGetContext(WINDOW_CHUNK_SIZE); + timestampWindowBuffer = new LongRingBuffer(RING_BUFFER_INITIAL_SIZE, true); } @Override @@ -211,7 +215,7 @@ public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, if (ctx.initialStep) { ctx.affectedRows = ctx.sourceRowSet; - ctx.influencerRows = computeInfluencerRowsTime(ctx, ctx.affectedRows, prevUnits, fwdUnits, false); + ctx.influencerRows = ctx.timestampValidRowSet; // mark all operators as affected by this update context.dirtyOperatorIndices = IntStream.range(0, operators.length).toArray(); @@ -269,7 +273,7 @@ public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, modifiedAffectedPrev.retain(ctx.timestampValidRowSet); tmpAffected.insert(modifiedAffectedPrev); } - // naturally need to compute all modified rows + // compute all modified rows tmpAffected.insert(upstream.modified()); } @@ -278,11 +282,11 @@ public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, computeAffectedRowsTime(ctx, upstream.added(), prevUnits, fwdUnits, false)) { tmpAffected.insert(addedAffected); } - // naturally need to compute all new rows + // compute all new rows tmpAffected.insert(upstream.added()); } - // other rows can be affected by removes or mods + // other rows can be affected by removes if (upstream.removed().isNonempty()) { try (final WritableRowSet removedAffected = computeAffectedRowsTime(ctx, upstream.removed(), prevUnits, fwdUnits, true)) { @@ -326,8 +330,6 @@ public void processRows(UpdateByWindowBucketContext context, boolean initialStep } try (final RowSequence.Iterator it = ctx.affectedRows.getRowSequenceIterator(); - final LongColumnIterator influencerTsHeadIt = - new LongColumnIterator(context.timestampColumnSource, context.influencerRows); final LongColumnIterator influencerTsTailIt = new LongColumnIterator(context.timestampColumnSource, context.influencerRows); final RowSequence.Iterator influencerKeyIt = ctx.influencerRows.getRowSequenceIterator(); @@ -338,9 +340,9 @@ public void processRows(UpdateByWindowBucketContext context, boolean initialStep final WritableIntChunk popChunk = WritableIntChunk.makeWritableChunk(ctx.workingChunkSize)) { - long currentHeadTs = nextLongOrMax(influencerTsHeadIt); long currentTailTs = nextLongOrMax(influencerTsTailIt); + while (it.hasMore()) { final RowSequence chunkRs = it.getNextRowSequenceWithLength(ctx.workingChunkSize); final int chunkRsSize = chunkRs.intSize(); @@ -350,8 +352,8 @@ public void processRows(UpdateByWindowBucketContext context, boolean initialStep final LongChunk timestampChunk = ctx.timestampColumnSource.getChunk(localTimestampContext, chunkRs).asLongChunk(); - // chunk processing - long totalCount = 0; + // we need to use a builder because we are potentially skipping rows from the influencer rowset + final RowSetBuilderSequential chunkBuilder = RowSetFactory.builderSequential(); for (int ii = 0; ii < chunkRsSize; ii++) { // read the current timestamp @@ -369,14 +371,27 @@ public void processRows(UpdateByWindowBucketContext context, boolean initialStep // pop out all values from the current window that are not in the new window long popCount = 0; - while (currentHeadTs < head) { - currentHeadTs = nextLongOrMax(influencerTsHeadIt); + while (!ctx.timestampWindowBuffer.isEmpty() && ctx.timestampWindowBuffer.front() < head) { + ctx.timestampWindowBuffer.remove(); popCount++; } + // advance the keyIt and timestamp iterators until we are within the window. This only happens + // when initialStep == true because we have not created the minimum set of rows but include all + // non-null timestamp rows in our influencer values + long skipCount = 0; + while (currentTailTs < head) { + Assert.eqTrue(initialStep, "initialStep when skipping rows"); + currentTailTs = nextLongOrMax(influencerTsTailIt); + skipCount++; + } + influencerKeyIt.getNextRowSequenceWithLength(skipCount); + // push in all values that are in the new window (inclusive of tail) long pushCount = 0; while (currentTailTs <= tail) { + // add this value to the buffer before advancing + ctx.timestampWindowBuffer.add(currentTailTs); currentTailTs = nextLongOrMax(influencerTsTailIt); pushCount++; } @@ -385,33 +400,35 @@ public void processRows(UpdateByWindowBucketContext context, boolean initialStep popChunk.set(ii, Math.toIntExact(popCount)); pushChunk.set(ii, Math.toIntExact(pushCount)); - totalCount += pushCount; + // store the influencer keys + chunkBuilder.appendRowSequence(influencerKeyIt.getNextRowSequenceWithLength(pushCount)); } // execute the operators - final RowSequence chunkInfluencerRs = influencerKeyIt.getNextRowSequenceWithLength(totalCount); - ensureGetContextSize(ctx, chunkInfluencerRs.size()); - - Arrays.fill(ctx.inputSourceChunks, null); - for (int opIdx : context.dirtyOperatorIndices) { - UpdateByWindowedOperator.Context opCtx = - (UpdateByWindowedOperator.Context) context.opContext[opIdx]; - // prep the chunk array needed by the accumulate call - final int[] srcIndices = operatorInputSourceSlots[opIdx]; - for (int ii = 0; ii < srcIndices.length; ii++) { - int srcIdx = srcIndices[ii]; - // chunk prep - prepareValuesChunkForSource(ctx, srcIdx, chunkInfluencerRs); - opCtx.chunkArr[ii] = ctx.inputSourceChunks[srcIdx]; - } + try (final RowSequence chunkInfluencerRs = chunkBuilder.build()) { + ensureGetContextSize(ctx, chunkInfluencerRs.size()); + + Arrays.fill(ctx.inputSourceChunks, null); + for (int opIdx : context.dirtyOperatorIndices) { + UpdateByWindowedOperator.Context opCtx = + (UpdateByWindowedOperator.Context) context.opContext[opIdx]; + // prep the chunk array needed by the accumulate call + final int[] srcIndices = operatorInputSourceSlots[opIdx]; + for (int ii = 0; ii < srcIndices.length; ii++) { + int srcIdx = srcIndices[ii]; + // chunk prep + prepareValuesChunkForSource(ctx, srcIdx, chunkInfluencerRs); + opCtx.chunkArr[ii] = ctx.inputSourceChunks[srcIdx]; + } - // make the specialized call for windowed operators - ((UpdateByWindowedOperator.Context) ctx.opContext[opIdx]).accumulate( - chunkRs, - opCtx.chunkArr, - pushChunk, - popChunk, - chunkRsSize); + // make the specialized call for windowed operators + ((UpdateByWindowedOperator.Context) ctx.opContext[opIdx]).accumulate( + chunkRs, + opCtx.chunkArr, + pushChunk, + popChunk, + chunkRsSize); + } } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java index 2698860cc27..1c3e237b6ce 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java @@ -45,10 +45,10 @@ public void accumulate(RowSequence inputKeys, // read the value from the values chunk final byte input = byteValueChunk.get(ii); - if(input == NULL_BYTE) { + if (input == NULL_BYTE) { handleBadData(this, true, false); } else { - if(curVal == NULL_DOUBLE) { + if (curVal == NULL_DOUBLE) { curVal = input; } else { curVal = alpha * curVal + (oneMinusAlpha * input); @@ -65,21 +65,19 @@ public void accumulate(RowSequence inputKeys, //noinspection ConstantConditions final boolean isNull = input == NULL_BYTE; final boolean isNullTime = timestamp == NULL_LONG; - if(isNull) { + if (isNull) { handleBadData(this, true, false); } else if (isNullTime) { // no change to curVal and lastStamp + } else if (curVal == NULL_DOUBLE) { + curVal = input; + lastStamp = timestamp; } else { - if(curVal == NULL_DOUBLE) { - curVal = input; - lastStamp = timestamp; - } else { - final long dt = timestamp - lastStamp; - // alpha is dynamic, based on time - final double alpha = Math.exp(-dt / (double) reverseWindowScaleUnits); - curVal = alpha * curVal + ((1 - alpha) * input); - lastStamp = timestamp; - } + final long dt = timestamp - lastStamp; + // alpha is dynamic, based on time + final double alpha = Math.exp(-dt / (double) reverseWindowScaleUnits); + curVal = alpha * curVal + (1 - alpha) * input; + lastStamp = timestamp; } outputValues.set(ii, curVal); } @@ -108,11 +106,11 @@ public void push(long key, int pos, int count) { /** * An operator that computes an EMA from a byte column using an exponential decay function. * - * @param pair the {@link MatchPair} that defines the input/output for this operation + * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this ema - * @param control defines how to handle {@code null} input values. - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is - * measured in ticks, otherwise it is measured in nanoseconds + * @param control defines how to handle {@code null} input values. + * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is + * measured in ticks, otherwise it is measured in nanoseconds */ public ByteEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @@ -123,7 +121,7 @@ public ByteEMAOperator(@NotNull final MatchPair pair, final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args - ) { + ) { super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, rowRedirection); this.valueSource = valueSource; // region constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java index 82e0e953521..112e6cf4df3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java @@ -45,10 +45,10 @@ public void accumulate(RowSequence inputKeys, // read the value from the values chunk final int input = intValueChunk.get(ii); - if(input == NULL_INT) { + if (input == NULL_INT) { handleBadData(this, true, false); } else { - if(curVal == NULL_DOUBLE) { + if (curVal == NULL_DOUBLE) { curVal = input; } else { curVal = alpha * curVal + (oneMinusAlpha * input); @@ -65,21 +65,19 @@ public void accumulate(RowSequence inputKeys, //noinspection ConstantConditions final boolean isNull = input == NULL_INT; final boolean isNullTime = timestamp == NULL_LONG; - if(isNull) { + if (isNull) { handleBadData(this, true, false); } else if (isNullTime) { // no change to curVal and lastStamp + } else if (curVal == NULL_DOUBLE) { + curVal = input; + lastStamp = timestamp; } else { - if(curVal == NULL_DOUBLE) { - curVal = input; - lastStamp = timestamp; - } else { - final long dt = timestamp - lastStamp; - // alpha is dynamic, based on time - final double alpha = Math.exp(-dt / (double) reverseWindowScaleUnits); - curVal = alpha * curVal + ((1 - alpha) * input); - lastStamp = timestamp; - } + final long dt = timestamp - lastStamp; + // alpha is dynamic, based on time + final double alpha = Math.exp(-dt / (double) reverseWindowScaleUnits); + curVal = alpha * curVal + (1 - alpha) * input; + lastStamp = timestamp; } outputValues.set(ii, curVal); } @@ -108,11 +106,11 @@ public void push(long key, int pos, int count) { /** * An operator that computes an EMA from a int column using an exponential decay function. * - * @param pair the {@link MatchPair} that defines the input/output for this operation + * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this ema - * @param control defines how to handle {@code null} input values. - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is - * measured in ticks, otherwise it is measured in nanoseconds + * @param control defines how to handle {@code null} input values. + * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is + * measured in ticks, otherwise it is measured in nanoseconds */ public IntEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @@ -123,7 +121,7 @@ public IntEMAOperator(@NotNull final MatchPair pair, final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args - ) { + ) { super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, rowRedirection); this.valueSource = valueSource; // region constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java index 6d936d3f7d4..1c4db317a28 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java @@ -45,10 +45,10 @@ public void accumulate(RowSequence inputKeys, // read the value from the values chunk final long input = longValueChunk.get(ii); - if(input == NULL_LONG) { + if (input == NULL_LONG) { handleBadData(this, true, false); } else { - if(curVal == NULL_DOUBLE) { + if (curVal == NULL_DOUBLE) { curVal = input; } else { curVal = alpha * curVal + (oneMinusAlpha * input); @@ -65,21 +65,19 @@ public void accumulate(RowSequence inputKeys, //noinspection ConstantConditions final boolean isNull = input == NULL_LONG; final boolean isNullTime = timestamp == NULL_LONG; - if(isNull) { + if (isNull) { handleBadData(this, true, false); } else if (isNullTime) { // no change to curVal and lastStamp + } else if (curVal == NULL_DOUBLE) { + curVal = input; + lastStamp = timestamp; } else { - if(curVal == NULL_DOUBLE) { - curVal = input; - lastStamp = timestamp; - } else { - final long dt = timestamp - lastStamp; - // alpha is dynamic, based on time - final double alpha = Math.exp(-dt / (double) reverseWindowScaleUnits); - curVal = alpha * curVal + ((1 - alpha) * input); - lastStamp = timestamp; - } + final long dt = timestamp - lastStamp; + // alpha is dynamic, based on time + final double alpha = Math.exp(-dt / (double) reverseWindowScaleUnits); + curVal = alpha * curVal + (1 - alpha) * input; + lastStamp = timestamp; } outputValues.set(ii, curVal); } @@ -108,11 +106,11 @@ public void push(long key, int pos, int count) { /** * An operator that computes an EMA from a long column using an exponential decay function. * - * @param pair the {@link MatchPair} that defines the input/output for this operation + * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this ema - * @param control defines how to handle {@code null} input values. - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is - * measured in ticks, otherwise it is measured in nanoseconds + * @param control defines how to handle {@code null} input values. + * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is + * measured in ticks, otherwise it is measured in nanoseconds */ public LongEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @@ -123,7 +121,7 @@ public LongEMAOperator(@NotNull final MatchPair pair, final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args - ) { + ) { super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, rowRedirection); this.valueSource = valueSource; // region constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java index 5b9484069f2..022835198dd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java @@ -40,10 +40,10 @@ public void accumulate(RowSequence inputKeys, // read the value from the values chunk final short input = shortValueChunk.get(ii); - if(input == NULL_SHORT) { + if (input == NULL_SHORT) { handleBadData(this, true, false); } else { - if(curVal == NULL_DOUBLE) { + if (curVal == NULL_DOUBLE) { curVal = input; } else { curVal = alpha * curVal + (oneMinusAlpha * input); @@ -60,13 +60,13 @@ public void accumulate(RowSequence inputKeys, //noinspection ConstantConditions final boolean isNull = input == NULL_SHORT; final boolean isNullTime = timestamp == NULL_LONG; - if(isNull) { + if (isNull) { handleBadData(this, true, false); } else if (isNullTime) { // no change to curVal and lastStamp } else if (curVal == NULL_DOUBLE) { - curVal = input; - lastStamp = timestamp; + curVal = input; + lastStamp = timestamp; } else { final long dt = timestamp - lastStamp; // alpha is dynamic, based on time @@ -101,11 +101,11 @@ public void push(long key, int pos, int count) { /** * An operator that computes an EMA from a short column using an exponential decay function. * - * @param pair the {@link MatchPair} that defines the input/output for this operation + * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this ema - * @param control defines how to handle {@code null} input values. - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is - * measured in ticks, otherwise it is measured in nanoseconds + * @param control defines how to handle {@code null} input values. + * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is + * measured in ticks, otherwise it is measured in nanoseconds */ public ShortEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @@ -116,7 +116,7 @@ public ShortEMAOperator(@NotNull final MatchPair pair, final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args - ) { + ) { super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, rowRedirection); this.valueSource = valueSource; // region constructor From 3e975852b4be86cf995880be306ab5ba0f3430bb Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Mon, 23 Jan 2023 11:48:31 -0800 Subject: [PATCH 091/123] Cleanup in ring buffers and tests --- .../base/ringbuffer/ByteRingBuffer.java | 69 +++++++++++++------ .../base/ringbuffer/CharRingBuffer.java | 69 +++++++++++++------ .../base/ringbuffer/DoubleRingBuffer.java | 69 +++++++++++++------ .../base/ringbuffer/FloatRingBuffer.java | 69 +++++++++++++------ .../base/ringbuffer/IntRingBuffer.java | 69 +++++++++++++------ .../base/ringbuffer/LongRingBuffer.java | 69 +++++++++++++------ .../base/ringbuffer/ShortRingBuffer.java | 69 +++++++++++++------ .../base/ringbuffer/ByteRingBufferTest.java | 40 +++++++---- .../base/ringbuffer/CharRingBufferTest.java | 40 +++++++---- .../base/ringbuffer/DoubleRingBufferTest.java | 40 +++++++---- .../base/ringbuffer/FloatRingBufferTest.java | 40 +++++++---- .../base/ringbuffer/IntRingBufferTest.java | 40 +++++++---- .../base/ringbuffer/LongRingBufferTest.java | 40 +++++++---- .../base/ringbuffer/ShortRingBufferTest.java | 40 +++++++---- 14 files changed, 511 insertions(+), 252 deletions(-) diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java index 3267e2864a3..9ab7a5bea0b 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java @@ -8,6 +8,9 @@ */ package io.deephaven.base.ringbuffer; +import io.deephaven.base.ArrayUtil; +import io.deephaven.base.verify.Assert; + import java.io.Serializable; import java.util.NoSuchElementException; @@ -20,25 +23,32 @@ public class ByteRingBuffer implements Serializable { protected final boolean growable; protected byte[] storage; - protected int head, tail; + protected int head, tail, size; private void grow(int increase) { if (growable) { - final int size = size(); + // assert that we are not asking for the impossible + Assert.eqTrue(ArrayUtil.MAX_ARRAY_SIZE - increase >= size, "ByteRingBuffer size <= MAX_ARRAY_SIZE"); - final int minLength = size + increase + 1; + final int minLength = size + increase; int newLength = storage.length * 2; while (newLength < minLength) { - newLength = newLength * 2; + newLength = Math.min(newLength * 2, ArrayUtil.MAX_ARRAY_SIZE); } byte[] newStorage = new byte[newLength]; - if (tail >= head) { - System.arraycopy(storage, head, newStorage, 0, size); - } else { - final int firstCopyLen = storage.length - head; - System.arraycopy(storage, head, newStorage, 0, firstCopyLen); - System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); - } + + // three scenarios: size is zero so nothing to copy, head is before tail so only one copy needed, head + // after tail so two copies needed. Assuming that copying zero bytes is a fast operation, we will always + // make two calls for simplicity and branch-prediction friendliness. + + // compute the size of the first copy + final int firstCopyLen = Math.min(storage.length - head, size); + + // do the copying + System.arraycopy(storage, head, newStorage, 0, firstCopyLen); + System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); + + // reset the pointers tail = size; head = 0; storage = newStorage; @@ -50,7 +60,7 @@ private void grow() { } public boolean isFull() { - return (tail + 1) % storage.length == head; + return size == storage.length; } /** @@ -70,28 +80,35 @@ public ByteRingBuffer(int capacity) { */ public ByteRingBuffer(int capacity, boolean growable) { this.growable = growable; - this.storage = new byte[capacity + 1]; + if (growable) { + // use next larger power of 2 + this.storage = new byte[Integer.highestOneBit(capacity - 1) << 1]; + } else { + // might as well use exact size and not over-allocate + this.storage = new byte[capacity]; + } + this.tail = this.head = 0; } public boolean isEmpty() { - return tail == head; + return size == 0; } public int size() { - return tail >= head ? (tail - head) : (tail + (storage.length - head)); + return size; } public int capacity() { - return storage.length - 1; + return storage.length; } public int remaining() { - return capacity() - size(); + return storage.length - size; } public void clear() { - tail = head = 0; + size = tail = head = 0; } /** @@ -112,6 +129,7 @@ public boolean add(byte e) { } storage[tail] = e; tail = (tail + 1) % storage.length; + size++; return true; } @@ -142,6 +160,7 @@ public void ensureRemaining(int count) { public void addUnsafe(byte e) { storage[tail] = e; tail = (tail + 1) % storage.length; + size++; } /** @@ -158,6 +177,7 @@ public byte addOverwrite(byte e, byte notFullResult) { } storage[tail] = e; tail = (tail + 1) % storage.length; + size++; return result; } @@ -174,11 +194,12 @@ public boolean offer(byte e) { } storage[tail] = e; tail = (tail + 1) % storage.length; + size++; return true; } public byte[] remove(int count) { - if (size() < count) { + if (size < count) { throw new NoSuchElementException(); } final byte[] result = new byte[count]; @@ -191,6 +212,7 @@ public byte[] remove(int count) { System.arraycopy(storage, 0, result, firstCopyLen, count - firstCopyLen); } head = (head + count) % storage.length; + size -= count; return result; } @@ -200,12 +222,14 @@ public byte remove() { } byte e = storage[head]; head = (head + 1) % storage.length; + size--; return e; } public byte removeUnsafe() { byte e = storage[head]; head = (head + 1) % storage.length; + size--; return e; } @@ -215,6 +239,7 @@ public byte poll(byte onEmpty) { } byte e = storage[head]; head = (head + 1) % storage.length; + size--; return e; } @@ -237,7 +262,7 @@ public byte front() { } public byte front(int offset) { - if (offset >= size()) { + if (offset >= size) { throw new NoSuchElementException(); } return storage[(head + offset) % storage.length]; @@ -265,7 +290,7 @@ public class Iterator { int count = -1; public boolean hasNext() { - return count + 1 < size(); + return count + 1 < size; } public byte next() { @@ -279,7 +304,7 @@ public void remove() { } public byte[] getAll() { - byte[] result = new byte[size()]; + byte[] result = new byte[size]; if (result.length > 0) { if (tail > head) { System.arraycopy(storage, head, result, 0, tail - head); diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java index fd559a01219..191ae75263e 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java @@ -3,6 +3,9 @@ */ package io.deephaven.base.ringbuffer; +import io.deephaven.base.ArrayUtil; +import io.deephaven.base.verify.Assert; + import java.io.Serializable; import java.util.NoSuchElementException; @@ -15,25 +18,32 @@ public class CharRingBuffer implements Serializable { protected final boolean growable; protected char[] storage; - protected int head, tail; + protected int head, tail, size; private void grow(int increase) { if (growable) { - final int size = size(); + // assert that we are not asking for the impossible + Assert.eqTrue(ArrayUtil.MAX_ARRAY_SIZE - increase >= size, "CharRingBuffer size <= MAX_ARRAY_SIZE"); - final int minLength = size + increase + 1; + final int minLength = size + increase; int newLength = storage.length * 2; while (newLength < minLength) { - newLength = newLength * 2; + newLength = Math.min(newLength * 2, ArrayUtil.MAX_ARRAY_SIZE); } char[] newStorage = new char[newLength]; - if (tail >= head) { - System.arraycopy(storage, head, newStorage, 0, size); - } else { - final int firstCopyLen = storage.length - head; - System.arraycopy(storage, head, newStorage, 0, firstCopyLen); - System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); - } + + // three scenarios: size is zero so nothing to copy, head is before tail so only one copy needed, head + // after tail so two copies needed. Assuming that copying zero bytes is a fast operation, we will always + // make two calls for simplicity and branch-prediction friendliness. + + // compute the size of the first copy + final int firstCopyLen = Math.min(storage.length - head, size); + + // do the copying + System.arraycopy(storage, head, newStorage, 0, firstCopyLen); + System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); + + // reset the pointers tail = size; head = 0; storage = newStorage; @@ -45,7 +55,7 @@ private void grow() { } public boolean isFull() { - return (tail + 1) % storage.length == head; + return size == storage.length; } /** @@ -65,28 +75,35 @@ public CharRingBuffer(int capacity) { */ public CharRingBuffer(int capacity, boolean growable) { this.growable = growable; - this.storage = new char[capacity + 1]; + if (growable) { + // use next larger power of 2 + this.storage = new char[Integer.highestOneBit(capacity - 1) << 1]; + } else { + // might as well use exact size and not over-allocate + this.storage = new char[capacity]; + } + this.tail = this.head = 0; } public boolean isEmpty() { - return tail == head; + return size == 0; } public int size() { - return tail >= head ? (tail - head) : (tail + (storage.length - head)); + return size; } public int capacity() { - return storage.length - 1; + return storage.length; } public int remaining() { - return capacity() - size(); + return storage.length - size; } public void clear() { - tail = head = 0; + size = tail = head = 0; } /** @@ -107,6 +124,7 @@ public boolean add(char e) { } storage[tail] = e; tail = (tail + 1) % storage.length; + size++; return true; } @@ -137,6 +155,7 @@ public void ensureRemaining(int count) { public void addUnsafe(char e) { storage[tail] = e; tail = (tail + 1) % storage.length; + size++; } /** @@ -153,6 +172,7 @@ public char addOverwrite(char e, char notFullResult) { } storage[tail] = e; tail = (tail + 1) % storage.length; + size++; return result; } @@ -169,11 +189,12 @@ public boolean offer(char e) { } storage[tail] = e; tail = (tail + 1) % storage.length; + size++; return true; } public char[] remove(int count) { - if (size() < count) { + if (size < count) { throw new NoSuchElementException(); } final char[] result = new char[count]; @@ -186,6 +207,7 @@ public char[] remove(int count) { System.arraycopy(storage, 0, result, firstCopyLen, count - firstCopyLen); } head = (head + count) % storage.length; + size -= count; return result; } @@ -195,12 +217,14 @@ public char remove() { } char e = storage[head]; head = (head + 1) % storage.length; + size--; return e; } public char removeUnsafe() { char e = storage[head]; head = (head + 1) % storage.length; + size--; return e; } @@ -210,6 +234,7 @@ public char poll(char onEmpty) { } char e = storage[head]; head = (head + 1) % storage.length; + size--; return e; } @@ -232,7 +257,7 @@ public char front() { } public char front(int offset) { - if (offset >= size()) { + if (offset >= size) { throw new NoSuchElementException(); } return storage[(head + offset) % storage.length]; @@ -260,7 +285,7 @@ public class Iterator { int count = -1; public boolean hasNext() { - return count + 1 < size(); + return count + 1 < size; } public char next() { @@ -274,7 +299,7 @@ public void remove() { } public char[] getAll() { - char[] result = new char[size()]; + char[] result = new char[size]; if (result.length > 0) { if (tail > head) { System.arraycopy(storage, head, result, 0, tail - head); diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java index 998b0c81e28..f3be8951ac2 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java @@ -8,6 +8,9 @@ */ package io.deephaven.base.ringbuffer; +import io.deephaven.base.ArrayUtil; +import io.deephaven.base.verify.Assert; + import java.io.Serializable; import java.util.NoSuchElementException; @@ -20,25 +23,32 @@ public class DoubleRingBuffer implements Serializable { protected final boolean growable; protected double[] storage; - protected int head, tail; + protected int head, tail, size; private void grow(int increase) { if (growable) { - final int size = size(); + // assert that we are not asking for the impossible + Assert.eqTrue(ArrayUtil.MAX_ARRAY_SIZE - increase >= size, "DoubleRingBuffer size <= MAX_ARRAY_SIZE"); - final int minLength = size + increase + 1; + final int minLength = size + increase; int newLength = storage.length * 2; while (newLength < minLength) { - newLength = newLength * 2; + newLength = Math.min(newLength * 2, ArrayUtil.MAX_ARRAY_SIZE); } double[] newStorage = new double[newLength]; - if (tail >= head) { - System.arraycopy(storage, head, newStorage, 0, size); - } else { - final int firstCopyLen = storage.length - head; - System.arraycopy(storage, head, newStorage, 0, firstCopyLen); - System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); - } + + // three scenarios: size is zero so nothing to copy, head is before tail so only one copy needed, head + // after tail so two copies needed. Assuming that copying zero bytes is a fast operation, we will always + // make two calls for simplicity and branch-prediction friendliness. + + // compute the size of the first copy + final int firstCopyLen = Math.min(storage.length - head, size); + + // do the copying + System.arraycopy(storage, head, newStorage, 0, firstCopyLen); + System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); + + // reset the pointers tail = size; head = 0; storage = newStorage; @@ -50,7 +60,7 @@ private void grow() { } public boolean isFull() { - return (tail + 1) % storage.length == head; + return size == storage.length; } /** @@ -70,28 +80,35 @@ public DoubleRingBuffer(int capacity) { */ public DoubleRingBuffer(int capacity, boolean growable) { this.growable = growable; - this.storage = new double[capacity + 1]; + if (growable) { + // use next larger power of 2 + this.storage = new double[Integer.highestOneBit(capacity - 1) << 1]; + } else { + // might as well use exact size and not over-allocate + this.storage = new double[capacity]; + } + this.tail = this.head = 0; } public boolean isEmpty() { - return tail == head; + return size == 0; } public int size() { - return tail >= head ? (tail - head) : (tail + (storage.length - head)); + return size; } public int capacity() { - return storage.length - 1; + return storage.length; } public int remaining() { - return capacity() - size(); + return storage.length - size; } public void clear() { - tail = head = 0; + size = tail = head = 0; } /** @@ -112,6 +129,7 @@ public boolean add(double e) { } storage[tail] = e; tail = (tail + 1) % storage.length; + size++; return true; } @@ -142,6 +160,7 @@ public void ensureRemaining(int count) { public void addUnsafe(double e) { storage[tail] = e; tail = (tail + 1) % storage.length; + size++; } /** @@ -158,6 +177,7 @@ public double addOverwrite(double e, double notFullResult) { } storage[tail] = e; tail = (tail + 1) % storage.length; + size++; return result; } @@ -174,11 +194,12 @@ public boolean offer(double e) { } storage[tail] = e; tail = (tail + 1) % storage.length; + size++; return true; } public double[] remove(int count) { - if (size() < count) { + if (size < count) { throw new NoSuchElementException(); } final double[] result = new double[count]; @@ -191,6 +212,7 @@ public double[] remove(int count) { System.arraycopy(storage, 0, result, firstCopyLen, count - firstCopyLen); } head = (head + count) % storage.length; + size -= count; return result; } @@ -200,12 +222,14 @@ public double remove() { } double e = storage[head]; head = (head + 1) % storage.length; + size--; return e; } public double removeUnsafe() { double e = storage[head]; head = (head + 1) % storage.length; + size--; return e; } @@ -215,6 +239,7 @@ public double poll(double onEmpty) { } double e = storage[head]; head = (head + 1) % storage.length; + size--; return e; } @@ -237,7 +262,7 @@ public double front() { } public double front(int offset) { - if (offset >= size()) { + if (offset >= size) { throw new NoSuchElementException(); } return storage[(head + offset) % storage.length]; @@ -265,7 +290,7 @@ public class Iterator { int count = -1; public boolean hasNext() { - return count + 1 < size(); + return count + 1 < size; } public double next() { @@ -279,7 +304,7 @@ public void remove() { } public double[] getAll() { - double[] result = new double[size()]; + double[] result = new double[size]; if (result.length > 0) { if (tail > head) { System.arraycopy(storage, head, result, 0, tail - head); diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java index 3fcc3ece33c..fe3f5907c45 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java @@ -8,6 +8,9 @@ */ package io.deephaven.base.ringbuffer; +import io.deephaven.base.ArrayUtil; +import io.deephaven.base.verify.Assert; + import java.io.Serializable; import java.util.NoSuchElementException; @@ -20,25 +23,32 @@ public class FloatRingBuffer implements Serializable { protected final boolean growable; protected float[] storage; - protected int head, tail; + protected int head, tail, size; private void grow(int increase) { if (growable) { - final int size = size(); + // assert that we are not asking for the impossible + Assert.eqTrue(ArrayUtil.MAX_ARRAY_SIZE - increase >= size, "FloatRingBuffer size <= MAX_ARRAY_SIZE"); - final int minLength = size + increase + 1; + final int minLength = size + increase; int newLength = storage.length * 2; while (newLength < minLength) { - newLength = newLength * 2; + newLength = Math.min(newLength * 2, ArrayUtil.MAX_ARRAY_SIZE); } float[] newStorage = new float[newLength]; - if (tail >= head) { - System.arraycopy(storage, head, newStorage, 0, size); - } else { - final int firstCopyLen = storage.length - head; - System.arraycopy(storage, head, newStorage, 0, firstCopyLen); - System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); - } + + // three scenarios: size is zero so nothing to copy, head is before tail so only one copy needed, head + // after tail so two copies needed. Assuming that copying zero bytes is a fast operation, we will always + // make two calls for simplicity and branch-prediction friendliness. + + // compute the size of the first copy + final int firstCopyLen = Math.min(storage.length - head, size); + + // do the copying + System.arraycopy(storage, head, newStorage, 0, firstCopyLen); + System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); + + // reset the pointers tail = size; head = 0; storage = newStorage; @@ -50,7 +60,7 @@ private void grow() { } public boolean isFull() { - return (tail + 1) % storage.length == head; + return size == storage.length; } /** @@ -70,28 +80,35 @@ public FloatRingBuffer(int capacity) { */ public FloatRingBuffer(int capacity, boolean growable) { this.growable = growable; - this.storage = new float[capacity + 1]; + if (growable) { + // use next larger power of 2 + this.storage = new float[Integer.highestOneBit(capacity - 1) << 1]; + } else { + // might as well use exact size and not over-allocate + this.storage = new float[capacity]; + } + this.tail = this.head = 0; } public boolean isEmpty() { - return tail == head; + return size == 0; } public int size() { - return tail >= head ? (tail - head) : (tail + (storage.length - head)); + return size; } public int capacity() { - return storage.length - 1; + return storage.length; } public int remaining() { - return capacity() - size(); + return storage.length - size; } public void clear() { - tail = head = 0; + size = tail = head = 0; } /** @@ -112,6 +129,7 @@ public boolean add(float e) { } storage[tail] = e; tail = (tail + 1) % storage.length; + size++; return true; } @@ -142,6 +160,7 @@ public void ensureRemaining(int count) { public void addUnsafe(float e) { storage[tail] = e; tail = (tail + 1) % storage.length; + size++; } /** @@ -158,6 +177,7 @@ public float addOverwrite(float e, float notFullResult) { } storage[tail] = e; tail = (tail + 1) % storage.length; + size++; return result; } @@ -174,11 +194,12 @@ public boolean offer(float e) { } storage[tail] = e; tail = (tail + 1) % storage.length; + size++; return true; } public float[] remove(int count) { - if (size() < count) { + if (size < count) { throw new NoSuchElementException(); } final float[] result = new float[count]; @@ -191,6 +212,7 @@ public float[] remove(int count) { System.arraycopy(storage, 0, result, firstCopyLen, count - firstCopyLen); } head = (head + count) % storage.length; + size -= count; return result; } @@ -200,12 +222,14 @@ public float remove() { } float e = storage[head]; head = (head + 1) % storage.length; + size--; return e; } public float removeUnsafe() { float e = storage[head]; head = (head + 1) % storage.length; + size--; return e; } @@ -215,6 +239,7 @@ public float poll(float onEmpty) { } float e = storage[head]; head = (head + 1) % storage.length; + size--; return e; } @@ -237,7 +262,7 @@ public float front() { } public float front(int offset) { - if (offset >= size()) { + if (offset >= size) { throw new NoSuchElementException(); } return storage[(head + offset) % storage.length]; @@ -265,7 +290,7 @@ public class Iterator { int count = -1; public boolean hasNext() { - return count + 1 < size(); + return count + 1 < size; } public float next() { @@ -279,7 +304,7 @@ public void remove() { } public float[] getAll() { - float[] result = new float[size()]; + float[] result = new float[size]; if (result.length > 0) { if (tail > head) { System.arraycopy(storage, head, result, 0, tail - head); diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java index 85431bca67e..40a19fa7ff7 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java @@ -8,6 +8,9 @@ */ package io.deephaven.base.ringbuffer; +import io.deephaven.base.ArrayUtil; +import io.deephaven.base.verify.Assert; + import java.io.Serializable; import java.util.NoSuchElementException; @@ -20,25 +23,32 @@ public class IntRingBuffer implements Serializable { protected final boolean growable; protected int[] storage; - protected int head, tail; + protected int head, tail, size; private void grow(int increase) { if (growable) { - final int size = size(); + // assert that we are not asking for the impossible + Assert.eqTrue(ArrayUtil.MAX_ARRAY_SIZE - increase >= size, "IntRingBuffer size <= MAX_ARRAY_SIZE"); - final int minLength = size + increase + 1; + final int minLength = size + increase; int newLength = storage.length * 2; while (newLength < minLength) { - newLength = newLength * 2; + newLength = Math.min(newLength * 2, ArrayUtil.MAX_ARRAY_SIZE); } int[] newStorage = new int[newLength]; - if (tail >= head) { - System.arraycopy(storage, head, newStorage, 0, size); - } else { - final int firstCopyLen = storage.length - head; - System.arraycopy(storage, head, newStorage, 0, firstCopyLen); - System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); - } + + // three scenarios: size is zero so nothing to copy, head is before tail so only one copy needed, head + // after tail so two copies needed. Assuming that copying zero bytes is a fast operation, we will always + // make two calls for simplicity and branch-prediction friendliness. + + // compute the size of the first copy + final int firstCopyLen = Math.min(storage.length - head, size); + + // do the copying + System.arraycopy(storage, head, newStorage, 0, firstCopyLen); + System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); + + // reset the pointers tail = size; head = 0; storage = newStorage; @@ -50,7 +60,7 @@ private void grow() { } public boolean isFull() { - return (tail + 1) % storage.length == head; + return size == storage.length; } /** @@ -70,28 +80,35 @@ public IntRingBuffer(int capacity) { */ public IntRingBuffer(int capacity, boolean growable) { this.growable = growable; - this.storage = new int[capacity + 1]; + if (growable) { + // use next larger power of 2 + this.storage = new int[Integer.highestOneBit(capacity - 1) << 1]; + } else { + // might as well use exact size and not over-allocate + this.storage = new int[capacity]; + } + this.tail = this.head = 0; } public boolean isEmpty() { - return tail == head; + return size == 0; } public int size() { - return tail >= head ? (tail - head) : (tail + (storage.length - head)); + return size; } public int capacity() { - return storage.length - 1; + return storage.length; } public int remaining() { - return capacity() - size(); + return storage.length - size; } public void clear() { - tail = head = 0; + size = tail = head = 0; } /** @@ -112,6 +129,7 @@ public boolean add(int e) { } storage[tail] = e; tail = (tail + 1) % storage.length; + size++; return true; } @@ -142,6 +160,7 @@ public void ensureRemaining(int count) { public void addUnsafe(int e) { storage[tail] = e; tail = (tail + 1) % storage.length; + size++; } /** @@ -158,6 +177,7 @@ public int addOverwrite(int e, int notFullResult) { } storage[tail] = e; tail = (tail + 1) % storage.length; + size++; return result; } @@ -174,11 +194,12 @@ public boolean offer(int e) { } storage[tail] = e; tail = (tail + 1) % storage.length; + size++; return true; } public int[] remove(int count) { - if (size() < count) { + if (size < count) { throw new NoSuchElementException(); } final int[] result = new int[count]; @@ -191,6 +212,7 @@ public int[] remove(int count) { System.arraycopy(storage, 0, result, firstCopyLen, count - firstCopyLen); } head = (head + count) % storage.length; + size -= count; return result; } @@ -200,12 +222,14 @@ public int remove() { } int e = storage[head]; head = (head + 1) % storage.length; + size--; return e; } public int removeUnsafe() { int e = storage[head]; head = (head + 1) % storage.length; + size--; return e; } @@ -215,6 +239,7 @@ public int poll(int onEmpty) { } int e = storage[head]; head = (head + 1) % storage.length; + size--; return e; } @@ -237,7 +262,7 @@ public int front() { } public int front(int offset) { - if (offset >= size()) { + if (offset >= size) { throw new NoSuchElementException(); } return storage[(head + offset) % storage.length]; @@ -265,7 +290,7 @@ public class Iterator { int count = -1; public boolean hasNext() { - return count + 1 < size(); + return count + 1 < size; } public int next() { @@ -279,7 +304,7 @@ public void remove() { } public int[] getAll() { - int[] result = new int[size()]; + int[] result = new int[size]; if (result.length > 0) { if (tail > head) { System.arraycopy(storage, head, result, 0, tail - head); diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java index f5f097a5901..cfce647e2ed 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java @@ -8,6 +8,9 @@ */ package io.deephaven.base.ringbuffer; +import io.deephaven.base.ArrayUtil; +import io.deephaven.base.verify.Assert; + import java.io.Serializable; import java.util.NoSuchElementException; @@ -20,25 +23,32 @@ public class LongRingBuffer implements Serializable { protected final boolean growable; protected long[] storage; - protected int head, tail; + protected int head, tail, size; private void grow(int increase) { if (growable) { - final int size = size(); + // assert that we are not asking for the impossible + Assert.eqTrue(ArrayUtil.MAX_ARRAY_SIZE - increase >= size, "LongRingBuffer size <= MAX_ARRAY_SIZE"); - final int minLength = size + increase + 1; + final int minLength = size + increase; int newLength = storage.length * 2; while (newLength < minLength) { - newLength = newLength * 2; + newLength = Math.min(newLength * 2, ArrayUtil.MAX_ARRAY_SIZE); } long[] newStorage = new long[newLength]; - if (tail >= head) { - System.arraycopy(storage, head, newStorage, 0, size); - } else { - final int firstCopyLen = storage.length - head; - System.arraycopy(storage, head, newStorage, 0, firstCopyLen); - System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); - } + + // three scenarios: size is zero so nothing to copy, head is before tail so only one copy needed, head + // after tail so two copies needed. Assuming that copying zero bytes is a fast operation, we will always + // make two calls for simplicity and branch-prediction friendliness. + + // compute the size of the first copy + final int firstCopyLen = Math.min(storage.length - head, size); + + // do the copying + System.arraycopy(storage, head, newStorage, 0, firstCopyLen); + System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); + + // reset the pointers tail = size; head = 0; storage = newStorage; @@ -50,7 +60,7 @@ private void grow() { } public boolean isFull() { - return (tail + 1) % storage.length == head; + return size == storage.length; } /** @@ -70,28 +80,35 @@ public LongRingBuffer(int capacity) { */ public LongRingBuffer(int capacity, boolean growable) { this.growable = growable; - this.storage = new long[capacity + 1]; + if (growable) { + // use next larger power of 2 + this.storage = new long[Integer.highestOneBit(capacity - 1) << 1]; + } else { + // might as well use exact size and not over-allocate + this.storage = new long[capacity]; + } + this.tail = this.head = 0; } public boolean isEmpty() { - return tail == head; + return size == 0; } public int size() { - return tail >= head ? (tail - head) : (tail + (storage.length - head)); + return size; } public int capacity() { - return storage.length - 1; + return storage.length; } public int remaining() { - return capacity() - size(); + return storage.length - size; } public void clear() { - tail = head = 0; + size = tail = head = 0; } /** @@ -112,6 +129,7 @@ public boolean add(long e) { } storage[tail] = e; tail = (tail + 1) % storage.length; + size++; return true; } @@ -142,6 +160,7 @@ public void ensureRemaining(int count) { public void addUnsafe(long e) { storage[tail] = e; tail = (tail + 1) % storage.length; + size++; } /** @@ -158,6 +177,7 @@ public long addOverwrite(long e, long notFullResult) { } storage[tail] = e; tail = (tail + 1) % storage.length; + size++; return result; } @@ -174,11 +194,12 @@ public boolean offer(long e) { } storage[tail] = e; tail = (tail + 1) % storage.length; + size++; return true; } public long[] remove(int count) { - if (size() < count) { + if (size < count) { throw new NoSuchElementException(); } final long[] result = new long[count]; @@ -191,6 +212,7 @@ public long[] remove(int count) { System.arraycopy(storage, 0, result, firstCopyLen, count - firstCopyLen); } head = (head + count) % storage.length; + size -= count; return result; } @@ -200,12 +222,14 @@ public long remove() { } long e = storage[head]; head = (head + 1) % storage.length; + size--; return e; } public long removeUnsafe() { long e = storage[head]; head = (head + 1) % storage.length; + size--; return e; } @@ -215,6 +239,7 @@ public long poll(long onEmpty) { } long e = storage[head]; head = (head + 1) % storage.length; + size--; return e; } @@ -237,7 +262,7 @@ public long front() { } public long front(int offset) { - if (offset >= size()) { + if (offset >= size) { throw new NoSuchElementException(); } return storage[(head + offset) % storage.length]; @@ -265,7 +290,7 @@ public class Iterator { int count = -1; public boolean hasNext() { - return count + 1 < size(); + return count + 1 < size; } public long next() { @@ -279,7 +304,7 @@ public void remove() { } public long[] getAll() { - long[] result = new long[size()]; + long[] result = new long[size]; if (result.length > 0) { if (tail > head) { System.arraycopy(storage, head, result, 0, tail - head); diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java index ec870a0350f..49ceaab58a7 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java @@ -8,6 +8,9 @@ */ package io.deephaven.base.ringbuffer; +import io.deephaven.base.ArrayUtil; +import io.deephaven.base.verify.Assert; + import java.io.Serializable; import java.util.NoSuchElementException; @@ -20,25 +23,32 @@ public class ShortRingBuffer implements Serializable { protected final boolean growable; protected short[] storage; - protected int head, tail; + protected int head, tail, size; private void grow(int increase) { if (growable) { - final int size = size(); + // assert that we are not asking for the impossible + Assert.eqTrue(ArrayUtil.MAX_ARRAY_SIZE - increase >= size, "ShortRingBuffer size <= MAX_ARRAY_SIZE"); - final int minLength = size + increase + 1; + final int minLength = size + increase; int newLength = storage.length * 2; while (newLength < minLength) { - newLength = newLength * 2; + newLength = Math.min(newLength * 2, ArrayUtil.MAX_ARRAY_SIZE); } short[] newStorage = new short[newLength]; - if (tail >= head) { - System.arraycopy(storage, head, newStorage, 0, size); - } else { - final int firstCopyLen = storage.length - head; - System.arraycopy(storage, head, newStorage, 0, firstCopyLen); - System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); - } + + // three scenarios: size is zero so nothing to copy, head is before tail so only one copy needed, head + // after tail so two copies needed. Assuming that copying zero bytes is a fast operation, we will always + // make two calls for simplicity and branch-prediction friendliness. + + // compute the size of the first copy + final int firstCopyLen = Math.min(storage.length - head, size); + + // do the copying + System.arraycopy(storage, head, newStorage, 0, firstCopyLen); + System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); + + // reset the pointers tail = size; head = 0; storage = newStorage; @@ -50,7 +60,7 @@ private void grow() { } public boolean isFull() { - return (tail + 1) % storage.length == head; + return size == storage.length; } /** @@ -70,28 +80,35 @@ public ShortRingBuffer(int capacity) { */ public ShortRingBuffer(int capacity, boolean growable) { this.growable = growable; - this.storage = new short[capacity + 1]; + if (growable) { + // use next larger power of 2 + this.storage = new short[Integer.highestOneBit(capacity - 1) << 1]; + } else { + // might as well use exact size and not over-allocate + this.storage = new short[capacity]; + } + this.tail = this.head = 0; } public boolean isEmpty() { - return tail == head; + return size == 0; } public int size() { - return tail >= head ? (tail - head) : (tail + (storage.length - head)); + return size; } public int capacity() { - return storage.length - 1; + return storage.length; } public int remaining() { - return capacity() - size(); + return storage.length - size; } public void clear() { - tail = head = 0; + size = tail = head = 0; } /** @@ -112,6 +129,7 @@ public boolean add(short e) { } storage[tail] = e; tail = (tail + 1) % storage.length; + size++; return true; } @@ -142,6 +160,7 @@ public void ensureRemaining(int count) { public void addUnsafe(short e) { storage[tail] = e; tail = (tail + 1) % storage.length; + size++; } /** @@ -158,6 +177,7 @@ public short addOverwrite(short e, short notFullResult) { } storage[tail] = e; tail = (tail + 1) % storage.length; + size++; return result; } @@ -174,11 +194,12 @@ public boolean offer(short e) { } storage[tail] = e; tail = (tail + 1) % storage.length; + size++; return true; } public short[] remove(int count) { - if (size() < count) { + if (size < count) { throw new NoSuchElementException(); } final short[] result = new short[count]; @@ -191,6 +212,7 @@ public short[] remove(int count) { System.arraycopy(storage, 0, result, firstCopyLen, count - firstCopyLen); } head = (head + count) % storage.length; + size -= count; return result; } @@ -200,12 +222,14 @@ public short remove() { } short e = storage[head]; head = (head + 1) % storage.length; + size--; return e; } public short removeUnsafe() { short e = storage[head]; head = (head + 1) % storage.length; + size--; return e; } @@ -215,6 +239,7 @@ public short poll(short onEmpty) { } short e = storage[head]; head = (head + 1) % storage.length; + size--; return e; } @@ -237,7 +262,7 @@ public short front() { } public short front(int offset) { - if (offset >= size()) { + if (offset >= size) { throw new NoSuchElementException(); } return storage[(head + offset) % storage.length]; @@ -265,7 +290,7 @@ public class Iterator { int count = -1; public boolean hasNext() { - return count + 1 < size(); + return count + 1 < size; } public short next() { @@ -279,7 +304,7 @@ public void remove() { } public short[] getAll() { - short[] result = new short[size()]; + short[] result = new short[size]; if (result.length > 0) { if (tail > head) { System.arraycopy(storage, head, result, 0, tail - head); diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/ByteRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/ByteRingBufferTest.java index f37bc31a0af..7fdaaf8990b 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/ByteRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/ByteRingBufferTest.java @@ -8,6 +8,8 @@ */ package io.deephaven.base.ringbuffer; +import io.deephaven.base.ArrayUtil; +import io.deephaven.base.verify.AssertionFailure; import junit.framework.TestCase; import org.junit.Assert; import org.junit.Test; @@ -115,7 +117,6 @@ public void testAddRemove() { assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); assertContents(rb, A, B, C); - assertFull(rb); assertRemove(rb, 3, A); assertContents(rb, B, C); @@ -170,7 +171,6 @@ public void testAddRemove() { assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); assertContents(rb, A, B, C); - assertFull(rb); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); @@ -196,8 +196,6 @@ public void testOfferPoll() { assertOffer(rb, B, 2, A); assertOffer(rb, C, 3, A); - assertFull(rb); - assertPoll(rb, 3, A); assertPoll(rb, 2, B); assertPoll(rb, 1, C); @@ -234,10 +232,9 @@ public void testOfferPoll() { assertOffer(rb, A, 1, A); assertOffer(rb, B, 2, A); assertOffer(rb, C, 3, A); - assertFull(rb); + assertOffer(rb, D, 4, A); - assertAdd(rb, D, 4, A); // need one add to grow it - assertOffer(rb, E, 5, A); // NOTE: assumes capacity grows by at least a factor of two + assertAdd(rb, E, 5, A); // need one add to grow it from 4 to 8 assertOffer(rb, F, 6, A); assertPoll(rb, 6, A); @@ -250,19 +247,21 @@ public void testOfferPoll() { } public void testGrowSimple() { - ByteRingBuffer rb = new ByteRingBuffer(5); + ByteRingBuffer rb = new ByteRingBuffer(4); assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); assertAdd(rb, D, 4, A); - assertAdd(rb, E, 5, A); assertFull(rb); + // remove one so head != 0 + assertRemove(rb, 4, A); + + assertAdd(rb, E, 4, B); // this will grow; the elements are in a single contiguous block - assertAdd(rb, F, 6, A); + assertAdd(rb, F, 5, B); - assertRemove(rb, 6, A); assertRemove(rb, 5, B); assertRemove(rb, 4, C); assertRemove(rb, 3, D); @@ -287,7 +286,6 @@ public void testGrowComplex() { assertAdd(rb, C, 3, A); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); - assertFull(rb); // this will grow; the elements are in two blocks assertAdd(rb, F, 6, A); @@ -310,7 +308,6 @@ public void testIterator() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertFull(rb); iter = rb.iterator(); assertTrue(iter.hasNext()); @@ -382,7 +379,7 @@ public void testBackWhenEmpty() { } public void testBackTailIsZero() { - ByteRingBuffer rb = new ByteRingBuffer(5); + ByteRingBuffer rb = new ByteRingBuffer(5, false); assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); @@ -544,4 +541,19 @@ public void testAddRemoveUnsafe() { } } } + + public void testOverflow() { + ByteRingBuffer rbA = new ByteRingBuffer(0); + // this should throw + assertThrows(AssertionFailure.class, + () -> rbA.ensureRemaining(ArrayUtil.MAX_ARRAY_SIZE + 1)); + + ByteRingBuffer rbB = new ByteRingBuffer(100); + for (int i = 0; i < 100; i++) { + rbB.addUnsafe(A); + } + // this should throw + assertThrows(AssertionFailure.class, + () -> rbB.ensureRemaining(ArrayUtil.MAX_ARRAY_SIZE - 100 + 1)); + } } diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java index b753d49dbe2..e3f2efcef53 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java @@ -3,6 +3,8 @@ */ package io.deephaven.base.ringbuffer; +import io.deephaven.base.ArrayUtil; +import io.deephaven.base.verify.AssertionFailure; import junit.framework.TestCase; import org.junit.Assert; import org.junit.Test; @@ -110,7 +112,6 @@ public void testAddRemove() { assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); assertContents(rb, A, B, C); - assertFull(rb); assertRemove(rb, 3, A); assertContents(rb, B, C); @@ -165,7 +166,6 @@ public void testAddRemove() { assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); assertContents(rb, A, B, C); - assertFull(rb); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); @@ -191,8 +191,6 @@ public void testOfferPoll() { assertOffer(rb, B, 2, A); assertOffer(rb, C, 3, A); - assertFull(rb); - assertPoll(rb, 3, A); assertPoll(rb, 2, B); assertPoll(rb, 1, C); @@ -229,10 +227,9 @@ public void testOfferPoll() { assertOffer(rb, A, 1, A); assertOffer(rb, B, 2, A); assertOffer(rb, C, 3, A); - assertFull(rb); + assertOffer(rb, D, 4, A); - assertAdd(rb, D, 4, A); // need one add to grow it - assertOffer(rb, E, 5, A); // NOTE: assumes capacity grows by at least a factor of two + assertAdd(rb, E, 5, A); // need one add to grow it from 4 to 8 assertOffer(rb, F, 6, A); assertPoll(rb, 6, A); @@ -245,19 +242,21 @@ public void testOfferPoll() { } public void testGrowSimple() { - CharRingBuffer rb = new CharRingBuffer(5); + CharRingBuffer rb = new CharRingBuffer(4); assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); assertAdd(rb, D, 4, A); - assertAdd(rb, E, 5, A); assertFull(rb); + // remove one so head != 0 + assertRemove(rb, 4, A); + + assertAdd(rb, E, 4, B); // this will grow; the elements are in a single contiguous block - assertAdd(rb, F, 6, A); + assertAdd(rb, F, 5, B); - assertRemove(rb, 6, A); assertRemove(rb, 5, B); assertRemove(rb, 4, C); assertRemove(rb, 3, D); @@ -282,7 +281,6 @@ public void testGrowComplex() { assertAdd(rb, C, 3, A); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); - assertFull(rb); // this will grow; the elements are in two blocks assertAdd(rb, F, 6, A); @@ -305,7 +303,6 @@ public void testIterator() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertFull(rb); iter = rb.iterator(); assertTrue(iter.hasNext()); @@ -377,7 +374,7 @@ public void testBackWhenEmpty() { } public void testBackTailIsZero() { - CharRingBuffer rb = new CharRingBuffer(5); + CharRingBuffer rb = new CharRingBuffer(5, false); assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); @@ -539,4 +536,19 @@ public void testAddRemoveUnsafe() { } } } + + public void testOverflow() { + CharRingBuffer rbA = new CharRingBuffer(0); + // this should throw + assertThrows(AssertionFailure.class, + () -> rbA.ensureRemaining(ArrayUtil.MAX_ARRAY_SIZE + 1)); + + CharRingBuffer rbB = new CharRingBuffer(100); + for (int i = 0; i < 100; i++) { + rbB.addUnsafe(A); + } + // this should throw + assertThrows(AssertionFailure.class, + () -> rbB.ensureRemaining(ArrayUtil.MAX_ARRAY_SIZE - 100 + 1)); + } } diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/DoubleRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/DoubleRingBufferTest.java index 145c897c43a..b7a082d310f 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/DoubleRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/DoubleRingBufferTest.java @@ -8,6 +8,8 @@ */ package io.deephaven.base.ringbuffer; +import io.deephaven.base.ArrayUtil; +import io.deephaven.base.verify.AssertionFailure; import junit.framework.TestCase; import org.junit.Assert; import org.junit.Test; @@ -115,7 +117,6 @@ public void testAddRemove() { assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); assertContents(rb, A, B, C); - assertFull(rb); assertRemove(rb, 3, A); assertContents(rb, B, C); @@ -170,7 +171,6 @@ public void testAddRemove() { assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); assertContents(rb, A, B, C); - assertFull(rb); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); @@ -196,8 +196,6 @@ public void testOfferPoll() { assertOffer(rb, B, 2, A); assertOffer(rb, C, 3, A); - assertFull(rb); - assertPoll(rb, 3, A); assertPoll(rb, 2, B); assertPoll(rb, 1, C); @@ -234,10 +232,9 @@ public void testOfferPoll() { assertOffer(rb, A, 1, A); assertOffer(rb, B, 2, A); assertOffer(rb, C, 3, A); - assertFull(rb); + assertOffer(rb, D, 4, A); - assertAdd(rb, D, 4, A); // need one add to grow it - assertOffer(rb, E, 5, A); // NOTE: assumes capacity grows by at least a factor of two + assertAdd(rb, E, 5, A); // need one add to grow it from 4 to 8 assertOffer(rb, F, 6, A); assertPoll(rb, 6, A); @@ -250,19 +247,21 @@ public void testOfferPoll() { } public void testGrowSimple() { - DoubleRingBuffer rb = new DoubleRingBuffer(5); + DoubleRingBuffer rb = new DoubleRingBuffer(4); assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); assertAdd(rb, D, 4, A); - assertAdd(rb, E, 5, A); assertFull(rb); + // remove one so head != 0 + assertRemove(rb, 4, A); + + assertAdd(rb, E, 4, B); // this will grow; the elements are in a single contiguous block - assertAdd(rb, F, 6, A); + assertAdd(rb, F, 5, B); - assertRemove(rb, 6, A); assertRemove(rb, 5, B); assertRemove(rb, 4, C); assertRemove(rb, 3, D); @@ -287,7 +286,6 @@ public void testGrowComplex() { assertAdd(rb, C, 3, A); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); - assertFull(rb); // this will grow; the elements are in two blocks assertAdd(rb, F, 6, A); @@ -310,7 +308,6 @@ public void testIterator() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertFull(rb); iter = rb.iterator(); assertTrue(iter.hasNext()); @@ -382,7 +379,7 @@ public void testBackWhenEmpty() { } public void testBackTailIsZero() { - DoubleRingBuffer rb = new DoubleRingBuffer(5); + DoubleRingBuffer rb = new DoubleRingBuffer(5, false); assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); @@ -544,4 +541,19 @@ public void testAddRemoveUnsafe() { } } } + + public void testOverflow() { + DoubleRingBuffer rbA = new DoubleRingBuffer(0); + // this should throw + assertThrows(AssertionFailure.class, + () -> rbA.ensureRemaining(ArrayUtil.MAX_ARRAY_SIZE + 1)); + + DoubleRingBuffer rbB = new DoubleRingBuffer(100); + for (int i = 0; i < 100; i++) { + rbB.addUnsafe(A); + } + // this should throw + assertThrows(AssertionFailure.class, + () -> rbB.ensureRemaining(ArrayUtil.MAX_ARRAY_SIZE - 100 + 1)); + } } diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/FloatRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/FloatRingBufferTest.java index 74e336732cf..92b5f3e54d2 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/FloatRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/FloatRingBufferTest.java @@ -8,6 +8,8 @@ */ package io.deephaven.base.ringbuffer; +import io.deephaven.base.ArrayUtil; +import io.deephaven.base.verify.AssertionFailure; import junit.framework.TestCase; import org.junit.Assert; import org.junit.Test; @@ -115,7 +117,6 @@ public void testAddRemove() { assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); assertContents(rb, A, B, C); - assertFull(rb); assertRemove(rb, 3, A); assertContents(rb, B, C); @@ -170,7 +171,6 @@ public void testAddRemove() { assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); assertContents(rb, A, B, C); - assertFull(rb); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); @@ -196,8 +196,6 @@ public void testOfferPoll() { assertOffer(rb, B, 2, A); assertOffer(rb, C, 3, A); - assertFull(rb); - assertPoll(rb, 3, A); assertPoll(rb, 2, B); assertPoll(rb, 1, C); @@ -234,10 +232,9 @@ public void testOfferPoll() { assertOffer(rb, A, 1, A); assertOffer(rb, B, 2, A); assertOffer(rb, C, 3, A); - assertFull(rb); + assertOffer(rb, D, 4, A); - assertAdd(rb, D, 4, A); // need one add to grow it - assertOffer(rb, E, 5, A); // NOTE: assumes capacity grows by at least a factor of two + assertAdd(rb, E, 5, A); // need one add to grow it from 4 to 8 assertOffer(rb, F, 6, A); assertPoll(rb, 6, A); @@ -250,19 +247,21 @@ public void testOfferPoll() { } public void testGrowSimple() { - FloatRingBuffer rb = new FloatRingBuffer(5); + FloatRingBuffer rb = new FloatRingBuffer(4); assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); assertAdd(rb, D, 4, A); - assertAdd(rb, E, 5, A); assertFull(rb); + // remove one so head != 0 + assertRemove(rb, 4, A); + + assertAdd(rb, E, 4, B); // this will grow; the elements are in a single contiguous block - assertAdd(rb, F, 6, A); + assertAdd(rb, F, 5, B); - assertRemove(rb, 6, A); assertRemove(rb, 5, B); assertRemove(rb, 4, C); assertRemove(rb, 3, D); @@ -287,7 +286,6 @@ public void testGrowComplex() { assertAdd(rb, C, 3, A); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); - assertFull(rb); // this will grow; the elements are in two blocks assertAdd(rb, F, 6, A); @@ -310,7 +308,6 @@ public void testIterator() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertFull(rb); iter = rb.iterator(); assertTrue(iter.hasNext()); @@ -382,7 +379,7 @@ public void testBackWhenEmpty() { } public void testBackTailIsZero() { - FloatRingBuffer rb = new FloatRingBuffer(5); + FloatRingBuffer rb = new FloatRingBuffer(5, false); assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); @@ -544,4 +541,19 @@ public void testAddRemoveUnsafe() { } } } + + public void testOverflow() { + FloatRingBuffer rbA = new FloatRingBuffer(0); + // this should throw + assertThrows(AssertionFailure.class, + () -> rbA.ensureRemaining(ArrayUtil.MAX_ARRAY_SIZE + 1)); + + FloatRingBuffer rbB = new FloatRingBuffer(100); + for (int i = 0; i < 100; i++) { + rbB.addUnsafe(A); + } + // this should throw + assertThrows(AssertionFailure.class, + () -> rbB.ensureRemaining(ArrayUtil.MAX_ARRAY_SIZE - 100 + 1)); + } } diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/IntRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/IntRingBufferTest.java index 7a6461009cd..e0388e537fd 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/IntRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/IntRingBufferTest.java @@ -8,6 +8,8 @@ */ package io.deephaven.base.ringbuffer; +import io.deephaven.base.ArrayUtil; +import io.deephaven.base.verify.AssertionFailure; import junit.framework.TestCase; import org.junit.Assert; import org.junit.Test; @@ -115,7 +117,6 @@ public void testAddRemove() { assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); assertContents(rb, A, B, C); - assertFull(rb); assertRemove(rb, 3, A); assertContents(rb, B, C); @@ -170,7 +171,6 @@ public void testAddRemove() { assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); assertContents(rb, A, B, C); - assertFull(rb); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); @@ -196,8 +196,6 @@ public void testOfferPoll() { assertOffer(rb, B, 2, A); assertOffer(rb, C, 3, A); - assertFull(rb); - assertPoll(rb, 3, A); assertPoll(rb, 2, B); assertPoll(rb, 1, C); @@ -234,10 +232,9 @@ public void testOfferPoll() { assertOffer(rb, A, 1, A); assertOffer(rb, B, 2, A); assertOffer(rb, C, 3, A); - assertFull(rb); + assertOffer(rb, D, 4, A); - assertAdd(rb, D, 4, A); // need one add to grow it - assertOffer(rb, E, 5, A); // NOTE: assumes capacity grows by at least a factor of two + assertAdd(rb, E, 5, A); // need one add to grow it from 4 to 8 assertOffer(rb, F, 6, A); assertPoll(rb, 6, A); @@ -250,19 +247,21 @@ public void testOfferPoll() { } public void testGrowSimple() { - IntRingBuffer rb = new IntRingBuffer(5); + IntRingBuffer rb = new IntRingBuffer(4); assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); assertAdd(rb, D, 4, A); - assertAdd(rb, E, 5, A); assertFull(rb); + // remove one so head != 0 + assertRemove(rb, 4, A); + + assertAdd(rb, E, 4, B); // this will grow; the elements are in a single contiguous block - assertAdd(rb, F, 6, A); + assertAdd(rb, F, 5, B); - assertRemove(rb, 6, A); assertRemove(rb, 5, B); assertRemove(rb, 4, C); assertRemove(rb, 3, D); @@ -287,7 +286,6 @@ public void testGrowComplex() { assertAdd(rb, C, 3, A); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); - assertFull(rb); // this will grow; the elements are in two blocks assertAdd(rb, F, 6, A); @@ -310,7 +308,6 @@ public void testIterator() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertFull(rb); iter = rb.iterator(); assertTrue(iter.hasNext()); @@ -382,7 +379,7 @@ public void testBackWhenEmpty() { } public void testBackTailIsZero() { - IntRingBuffer rb = new IntRingBuffer(5); + IntRingBuffer rb = new IntRingBuffer(5, false); assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); @@ -544,4 +541,19 @@ public void testAddRemoveUnsafe() { } } } + + public void testOverflow() { + IntRingBuffer rbA = new IntRingBuffer(0); + // this should throw + assertThrows(AssertionFailure.class, + () -> rbA.ensureRemaining(ArrayUtil.MAX_ARRAY_SIZE + 1)); + + IntRingBuffer rbB = new IntRingBuffer(100); + for (int i = 0; i < 100; i++) { + rbB.addUnsafe(A); + } + // this should throw + assertThrows(AssertionFailure.class, + () -> rbB.ensureRemaining(ArrayUtil.MAX_ARRAY_SIZE - 100 + 1)); + } } diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/LongRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/LongRingBufferTest.java index 48ab6a2dbca..b72c910f55d 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/LongRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/LongRingBufferTest.java @@ -8,6 +8,8 @@ */ package io.deephaven.base.ringbuffer; +import io.deephaven.base.ArrayUtil; +import io.deephaven.base.verify.AssertionFailure; import junit.framework.TestCase; import org.junit.Assert; import org.junit.Test; @@ -115,7 +117,6 @@ public void testAddRemove() { assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); assertContents(rb, A, B, C); - assertFull(rb); assertRemove(rb, 3, A); assertContents(rb, B, C); @@ -170,7 +171,6 @@ public void testAddRemove() { assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); assertContents(rb, A, B, C); - assertFull(rb); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); @@ -196,8 +196,6 @@ public void testOfferPoll() { assertOffer(rb, B, 2, A); assertOffer(rb, C, 3, A); - assertFull(rb); - assertPoll(rb, 3, A); assertPoll(rb, 2, B); assertPoll(rb, 1, C); @@ -234,10 +232,9 @@ public void testOfferPoll() { assertOffer(rb, A, 1, A); assertOffer(rb, B, 2, A); assertOffer(rb, C, 3, A); - assertFull(rb); + assertOffer(rb, D, 4, A); - assertAdd(rb, D, 4, A); // need one add to grow it - assertOffer(rb, E, 5, A); // NOTE: assumes capacity grows by at least a factor of two + assertAdd(rb, E, 5, A); // need one add to grow it from 4 to 8 assertOffer(rb, F, 6, A); assertPoll(rb, 6, A); @@ -250,19 +247,21 @@ public void testOfferPoll() { } public void testGrowSimple() { - LongRingBuffer rb = new LongRingBuffer(5); + LongRingBuffer rb = new LongRingBuffer(4); assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); assertAdd(rb, D, 4, A); - assertAdd(rb, E, 5, A); assertFull(rb); + // remove one so head != 0 + assertRemove(rb, 4, A); + + assertAdd(rb, E, 4, B); // this will grow; the elements are in a single contiguous block - assertAdd(rb, F, 6, A); + assertAdd(rb, F, 5, B); - assertRemove(rb, 6, A); assertRemove(rb, 5, B); assertRemove(rb, 4, C); assertRemove(rb, 3, D); @@ -287,7 +286,6 @@ public void testGrowComplex() { assertAdd(rb, C, 3, A); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); - assertFull(rb); // this will grow; the elements are in two blocks assertAdd(rb, F, 6, A); @@ -310,7 +308,6 @@ public void testIterator() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertFull(rb); iter = rb.iterator(); assertTrue(iter.hasNext()); @@ -382,7 +379,7 @@ public void testBackWhenEmpty() { } public void testBackTailIsZero() { - LongRingBuffer rb = new LongRingBuffer(5); + LongRingBuffer rb = new LongRingBuffer(5, false); assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); @@ -544,4 +541,19 @@ public void testAddRemoveUnsafe() { } } } + + public void testOverflow() { + LongRingBuffer rbA = new LongRingBuffer(0); + // this should throw + assertThrows(AssertionFailure.class, + () -> rbA.ensureRemaining(ArrayUtil.MAX_ARRAY_SIZE + 1)); + + LongRingBuffer rbB = new LongRingBuffer(100); + for (int i = 0; i < 100; i++) { + rbB.addUnsafe(A); + } + // this should throw + assertThrows(AssertionFailure.class, + () -> rbB.ensureRemaining(ArrayUtil.MAX_ARRAY_SIZE - 100 + 1)); + } } diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/ShortRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/ShortRingBufferTest.java index fc64b402e94..28cda474a29 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/ShortRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/ShortRingBufferTest.java @@ -8,6 +8,8 @@ */ package io.deephaven.base.ringbuffer; +import io.deephaven.base.ArrayUtil; +import io.deephaven.base.verify.AssertionFailure; import junit.framework.TestCase; import org.junit.Assert; import org.junit.Test; @@ -115,7 +117,6 @@ public void testAddRemove() { assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); assertContents(rb, A, B, C); - assertFull(rb); assertRemove(rb, 3, A); assertContents(rb, B, C); @@ -170,7 +171,6 @@ public void testAddRemove() { assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); assertContents(rb, A, B, C); - assertFull(rb); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); @@ -196,8 +196,6 @@ public void testOfferPoll() { assertOffer(rb, B, 2, A); assertOffer(rb, C, 3, A); - assertFull(rb); - assertPoll(rb, 3, A); assertPoll(rb, 2, B); assertPoll(rb, 1, C); @@ -234,10 +232,9 @@ public void testOfferPoll() { assertOffer(rb, A, 1, A); assertOffer(rb, B, 2, A); assertOffer(rb, C, 3, A); - assertFull(rb); + assertOffer(rb, D, 4, A); - assertAdd(rb, D, 4, A); // need one add to grow it - assertOffer(rb, E, 5, A); // NOTE: assumes capacity grows by at least a factor of two + assertAdd(rb, E, 5, A); // need one add to grow it from 4 to 8 assertOffer(rb, F, 6, A); assertPoll(rb, 6, A); @@ -250,19 +247,21 @@ public void testOfferPoll() { } public void testGrowSimple() { - ShortRingBuffer rb = new ShortRingBuffer(5); + ShortRingBuffer rb = new ShortRingBuffer(4); assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); assertAdd(rb, D, 4, A); - assertAdd(rb, E, 5, A); assertFull(rb); + // remove one so head != 0 + assertRemove(rb, 4, A); + + assertAdd(rb, E, 4, B); // this will grow; the elements are in a single contiguous block - assertAdd(rb, F, 6, A); + assertAdd(rb, F, 5, B); - assertRemove(rb, 6, A); assertRemove(rb, 5, B); assertRemove(rb, 4, C); assertRemove(rb, 3, D); @@ -287,7 +286,6 @@ public void testGrowComplex() { assertAdd(rb, C, 3, A); assertAdd(rb, D, 4, A); assertAdd(rb, E, 5, A); - assertFull(rb); // this will grow; the elements are in two blocks assertAdd(rb, F, 6, A); @@ -310,7 +308,6 @@ public void testIterator() { assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); assertAdd(rb, C, 3, A); - assertFull(rb); iter = rb.iterator(); assertTrue(iter.hasNext()); @@ -382,7 +379,7 @@ public void testBackWhenEmpty() { } public void testBackTailIsZero() { - ShortRingBuffer rb = new ShortRingBuffer(5); + ShortRingBuffer rb = new ShortRingBuffer(5, false); assertAdd(rb, A, 1, A); assertAdd(rb, B, 2, A); @@ -544,4 +541,19 @@ public void testAddRemoveUnsafe() { } } } + + public void testOverflow() { + ShortRingBuffer rbA = new ShortRingBuffer(0); + // this should throw + assertThrows(AssertionFailure.class, + () -> rbA.ensureRemaining(ArrayUtil.MAX_ARRAY_SIZE + 1)); + + ShortRingBuffer rbB = new ShortRingBuffer(100); + for (int i = 0; i < 100; i++) { + rbB.addUnsafe(A); + } + // this should throw + assertThrows(AssertionFailure.class, + () -> rbB.ensureRemaining(ArrayUtil.MAX_ARRAY_SIZE - 100 + 1)); + } } From 825db35e017a48429115bd18f4016f039a83e815 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 24 Jan 2023 10:39:47 -0800 Subject: [PATCH 092/123] More ring and pairwise buffer cleanup --- .../base/ringbuffer/ByteRingBuffer.java | 39 ++- .../base/ringbuffer/CharRingBuffer.java | 39 ++- .../base/ringbuffer/DoubleRingBuffer.java | 39 ++- .../base/ringbuffer/FloatRingBuffer.java | 39 ++- .../base/ringbuffer/IntRingBuffer.java | 39 ++- .../base/ringbuffer/LongRingBuffer.java | 39 ++- .../base/ringbuffer/ShortRingBuffer.java | 39 ++- .../impl/updateby/UpdateByWindowTime.java | 6 +- .../internal/PairwiseDoubleRingBuffer.java | 286 +++++++++++------- .../internal/PairwiseFloatRingBuffer.java | 284 ++++++++++------- .../rollingsum/DoubleRollingSumOperator.java | 2 - .../rollingsum/FloatRollingSumOperator.java | 2 - .../PairwiseDoubleRingBufferTest.java | 1 - .../internal/PairwiseFloatRingBufferTest.java | 1 - 14 files changed, 520 insertions(+), 335 deletions(-) diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java index 9ab7a5bea0b..4d86dc8ec89 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java @@ -21,10 +21,21 @@ * buffer instead. */ public class ByteRingBuffer implements Serializable { + private static int SYSTEM_ARRAYCOPY_THRESHOLD = 16; protected final boolean growable; protected byte[] storage; protected int head, tail, size; + private void fastCopy(byte[] src, int srcPos, byte[] dest, int destPos, int length) { + if (length < SYSTEM_ARRAYCOPY_THRESHOLD) { + for (int ii = 0; ii < length; ii++) { + dest[destPos + ii] = src[srcPos + ii]; + } + } else { + System.arraycopy(src, srcPos, dest, 0, length); + } + } + private void grow(int increase) { if (growable) { // assert that we are not asking for the impossible @@ -38,15 +49,15 @@ private void grow(int increase) { byte[] newStorage = new byte[newLength]; // three scenarios: size is zero so nothing to copy, head is before tail so only one copy needed, head - // after tail so two copies needed. Assuming that copying zero bytes is a fast operation, we will always + // after tail so two copies needed. Assuming that copying zero bytes is a fast operation, we will always // make two calls for simplicity and branch-prediction friendliness. // compute the size of the first copy final int firstCopyLen = Math.min(storage.length - head, size); // do the copying - System.arraycopy(storage, head, newStorage, 0, firstCopyLen); - System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); + fastCopy(storage, head, newStorage, 0, firstCopyLen); + fastCopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); // reset the pointers tail = size; @@ -82,13 +93,13 @@ public ByteRingBuffer(int capacity, boolean growable) { this.growable = growable; if (growable) { // use next larger power of 2 - this.storage = new byte[Integer.highestOneBit(capacity - 1) << 1]; + storage = new byte[Integer.highestOneBit(capacity - 1) << 1]; } else { // might as well use exact size and not over-allocate - this.storage = new byte[capacity]; + storage = new byte[capacity]; } - this.tail = this.head = 0; + tail = head = 0; } public boolean isEmpty() { @@ -138,7 +149,7 @@ public boolean add(byte e) { * {@code growable}, this may result in an internal growth operation. This call should be used in conjunction with * {@link #addUnsafe(byte)}. * - * @param count the amount of empty entries in the buffer after this call + * @param count the minimum number of empty entries in the buffer after this call * @throws UnsupportedOperationException when {@code growable} is {@code false} and buffer is full */ public void ensureRemaining(int count) { @@ -153,7 +164,7 @@ public void ensureRemaining(int count) { /** * Add values unsafely (will silently overwrite values if the buffer is full). This call should be used in - * conjunction with * {@link #ensureRemaining(int)}. + * conjunction with {@link #ensureRemaining(int)}. * * @param e the value to add to the buffer */ @@ -206,10 +217,10 @@ public byte[] remove(int count) { final int firstCopyLen = storage.length - head; if (tail >= head || firstCopyLen >= count) { - System.arraycopy(storage, head, result, 0, count); + fastCopy(storage, head, result, 0, count); } else { - System.arraycopy(storage, head, result, 0, firstCopyLen); - System.arraycopy(storage, 0, result, firstCopyLen, count - firstCopyLen); + fastCopy(storage, head, result, 0, firstCopyLen); + fastCopy(storage, 0, result, firstCopyLen, count - firstCopyLen); } head = (head + count) % storage.length; size -= count; @@ -307,10 +318,10 @@ public byte[] getAll() { byte[] result = new byte[size]; if (result.length > 0) { if (tail > head) { - System.arraycopy(storage, head, result, 0, tail - head); + fastCopy(storage, head, result, 0, tail - head); } else { - System.arraycopy(storage, head, result, 0, storage.length - head); - System.arraycopy(storage, 0, result, storage.length - head, tail); + fastCopy(storage, head, result, 0, storage.length - head); + fastCopy(storage, 0, result, storage.length - head, tail); } } return result; diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java index 191ae75263e..63e0a329761 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java @@ -16,10 +16,21 @@ * buffer instead. */ public class CharRingBuffer implements Serializable { + private static int SYSTEM_ARRAYCOPY_THRESHOLD = 16; protected final boolean growable; protected char[] storage; protected int head, tail, size; + private void fastCopy(char[] src, int srcPos, char[] dest, int destPos, int length) { + if (length < SYSTEM_ARRAYCOPY_THRESHOLD) { + for (int ii = 0; ii < length; ii++) { + dest[destPos + ii] = src[srcPos + ii]; + } + } else { + System.arraycopy(src, srcPos, dest, 0, length); + } + } + private void grow(int increase) { if (growable) { // assert that we are not asking for the impossible @@ -33,15 +44,15 @@ private void grow(int increase) { char[] newStorage = new char[newLength]; // three scenarios: size is zero so nothing to copy, head is before tail so only one copy needed, head - // after tail so two copies needed. Assuming that copying zero bytes is a fast operation, we will always + // after tail so two copies needed. Assuming that copying zero bytes is a fast operation, we will always // make two calls for simplicity and branch-prediction friendliness. // compute the size of the first copy final int firstCopyLen = Math.min(storage.length - head, size); // do the copying - System.arraycopy(storage, head, newStorage, 0, firstCopyLen); - System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); + fastCopy(storage, head, newStorage, 0, firstCopyLen); + fastCopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); // reset the pointers tail = size; @@ -77,13 +88,13 @@ public CharRingBuffer(int capacity, boolean growable) { this.growable = growable; if (growable) { // use next larger power of 2 - this.storage = new char[Integer.highestOneBit(capacity - 1) << 1]; + storage = new char[Integer.highestOneBit(capacity - 1) << 1]; } else { // might as well use exact size and not over-allocate - this.storage = new char[capacity]; + storage = new char[capacity]; } - this.tail = this.head = 0; + tail = head = 0; } public boolean isEmpty() { @@ -133,7 +144,7 @@ public boolean add(char e) { * {@code growable}, this may result in an internal growth operation. This call should be used in conjunction with * {@link #addUnsafe(char)}. * - * @param count the amount of empty entries in the buffer after this call + * @param count the minimum number of empty entries in the buffer after this call * @throws UnsupportedOperationException when {@code growable} is {@code false} and buffer is full */ public void ensureRemaining(int count) { @@ -148,7 +159,7 @@ public void ensureRemaining(int count) { /** * Add values unsafely (will silently overwrite values if the buffer is full). This call should be used in - * conjunction with * {@link #ensureRemaining(int)}. + * conjunction with {@link #ensureRemaining(int)}. * * @param e the value to add to the buffer */ @@ -201,10 +212,10 @@ public char[] remove(int count) { final int firstCopyLen = storage.length - head; if (tail >= head || firstCopyLen >= count) { - System.arraycopy(storage, head, result, 0, count); + fastCopy(storage, head, result, 0, count); } else { - System.arraycopy(storage, head, result, 0, firstCopyLen); - System.arraycopy(storage, 0, result, firstCopyLen, count - firstCopyLen); + fastCopy(storage, head, result, 0, firstCopyLen); + fastCopy(storage, 0, result, firstCopyLen, count - firstCopyLen); } head = (head + count) % storage.length; size -= count; @@ -302,10 +313,10 @@ public char[] getAll() { char[] result = new char[size]; if (result.length > 0) { if (tail > head) { - System.arraycopy(storage, head, result, 0, tail - head); + fastCopy(storage, head, result, 0, tail - head); } else { - System.arraycopy(storage, head, result, 0, storage.length - head); - System.arraycopy(storage, 0, result, storage.length - head, tail); + fastCopy(storage, head, result, 0, storage.length - head); + fastCopy(storage, 0, result, storage.length - head, tail); } } return result; diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java index f3be8951ac2..93956fad2b9 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java @@ -21,10 +21,21 @@ * buffer instead. */ public class DoubleRingBuffer implements Serializable { + private static int SYSTEM_ARRAYCOPY_THRESHOLD = 16; protected final boolean growable; protected double[] storage; protected int head, tail, size; + private void fastCopy(double[] src, int srcPos, double[] dest, int destPos, int length) { + if (length < SYSTEM_ARRAYCOPY_THRESHOLD) { + for (int ii = 0; ii < length; ii++) { + dest[destPos + ii] = src[srcPos + ii]; + } + } else { + System.arraycopy(src, srcPos, dest, 0, length); + } + } + private void grow(int increase) { if (growable) { // assert that we are not asking for the impossible @@ -38,15 +49,15 @@ private void grow(int increase) { double[] newStorage = new double[newLength]; // three scenarios: size is zero so nothing to copy, head is before tail so only one copy needed, head - // after tail so two copies needed. Assuming that copying zero bytes is a fast operation, we will always + // after tail so two copies needed. Assuming that copying zero bytes is a fast operation, we will always // make two calls for simplicity and branch-prediction friendliness. // compute the size of the first copy final int firstCopyLen = Math.min(storage.length - head, size); // do the copying - System.arraycopy(storage, head, newStorage, 0, firstCopyLen); - System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); + fastCopy(storage, head, newStorage, 0, firstCopyLen); + fastCopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); // reset the pointers tail = size; @@ -82,13 +93,13 @@ public DoubleRingBuffer(int capacity, boolean growable) { this.growable = growable; if (growable) { // use next larger power of 2 - this.storage = new double[Integer.highestOneBit(capacity - 1) << 1]; + storage = new double[Integer.highestOneBit(capacity - 1) << 1]; } else { // might as well use exact size and not over-allocate - this.storage = new double[capacity]; + storage = new double[capacity]; } - this.tail = this.head = 0; + tail = head = 0; } public boolean isEmpty() { @@ -138,7 +149,7 @@ public boolean add(double e) { * {@code growable}, this may result in an internal growth operation. This call should be used in conjunction with * {@link #addUnsafe(double)}. * - * @param count the amount of empty entries in the buffer after this call + * @param count the minimum number of empty entries in the buffer after this call * @throws UnsupportedOperationException when {@code growable} is {@code false} and buffer is full */ public void ensureRemaining(int count) { @@ -153,7 +164,7 @@ public void ensureRemaining(int count) { /** * Add values unsafely (will silently overwrite values if the buffer is full). This call should be used in - * conjunction with * {@link #ensureRemaining(int)}. + * conjunction with {@link #ensureRemaining(int)}. * * @param e the value to add to the buffer */ @@ -206,10 +217,10 @@ public double[] remove(int count) { final int firstCopyLen = storage.length - head; if (tail >= head || firstCopyLen >= count) { - System.arraycopy(storage, head, result, 0, count); + fastCopy(storage, head, result, 0, count); } else { - System.arraycopy(storage, head, result, 0, firstCopyLen); - System.arraycopy(storage, 0, result, firstCopyLen, count - firstCopyLen); + fastCopy(storage, head, result, 0, firstCopyLen); + fastCopy(storage, 0, result, firstCopyLen, count - firstCopyLen); } head = (head + count) % storage.length; size -= count; @@ -307,10 +318,10 @@ public double[] getAll() { double[] result = new double[size]; if (result.length > 0) { if (tail > head) { - System.arraycopy(storage, head, result, 0, tail - head); + fastCopy(storage, head, result, 0, tail - head); } else { - System.arraycopy(storage, head, result, 0, storage.length - head); - System.arraycopy(storage, 0, result, storage.length - head, tail); + fastCopy(storage, head, result, 0, storage.length - head); + fastCopy(storage, 0, result, storage.length - head, tail); } } return result; diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java index fe3f5907c45..7ce2b4e22b4 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java @@ -21,10 +21,21 @@ * buffer instead. */ public class FloatRingBuffer implements Serializable { + private static int SYSTEM_ARRAYCOPY_THRESHOLD = 16; protected final boolean growable; protected float[] storage; protected int head, tail, size; + private void fastCopy(float[] src, int srcPos, float[] dest, int destPos, int length) { + if (length < SYSTEM_ARRAYCOPY_THRESHOLD) { + for (int ii = 0; ii < length; ii++) { + dest[destPos + ii] = src[srcPos + ii]; + } + } else { + System.arraycopy(src, srcPos, dest, 0, length); + } + } + private void grow(int increase) { if (growable) { // assert that we are not asking for the impossible @@ -38,15 +49,15 @@ private void grow(int increase) { float[] newStorage = new float[newLength]; // three scenarios: size is zero so nothing to copy, head is before tail so only one copy needed, head - // after tail so two copies needed. Assuming that copying zero bytes is a fast operation, we will always + // after tail so two copies needed. Assuming that copying zero bytes is a fast operation, we will always // make two calls for simplicity and branch-prediction friendliness. // compute the size of the first copy final int firstCopyLen = Math.min(storage.length - head, size); // do the copying - System.arraycopy(storage, head, newStorage, 0, firstCopyLen); - System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); + fastCopy(storage, head, newStorage, 0, firstCopyLen); + fastCopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); // reset the pointers tail = size; @@ -82,13 +93,13 @@ public FloatRingBuffer(int capacity, boolean growable) { this.growable = growable; if (growable) { // use next larger power of 2 - this.storage = new float[Integer.highestOneBit(capacity - 1) << 1]; + storage = new float[Integer.highestOneBit(capacity - 1) << 1]; } else { // might as well use exact size and not over-allocate - this.storage = new float[capacity]; + storage = new float[capacity]; } - this.tail = this.head = 0; + tail = head = 0; } public boolean isEmpty() { @@ -138,7 +149,7 @@ public boolean add(float e) { * {@code growable}, this may result in an internal growth operation. This call should be used in conjunction with * {@link #addUnsafe(float)}. * - * @param count the amount of empty entries in the buffer after this call + * @param count the minimum number of empty entries in the buffer after this call * @throws UnsupportedOperationException when {@code growable} is {@code false} and buffer is full */ public void ensureRemaining(int count) { @@ -153,7 +164,7 @@ public void ensureRemaining(int count) { /** * Add values unsafely (will silently overwrite values if the buffer is full). This call should be used in - * conjunction with * {@link #ensureRemaining(int)}. + * conjunction with {@link #ensureRemaining(int)}. * * @param e the value to add to the buffer */ @@ -206,10 +217,10 @@ public float[] remove(int count) { final int firstCopyLen = storage.length - head; if (tail >= head || firstCopyLen >= count) { - System.arraycopy(storage, head, result, 0, count); + fastCopy(storage, head, result, 0, count); } else { - System.arraycopy(storage, head, result, 0, firstCopyLen); - System.arraycopy(storage, 0, result, firstCopyLen, count - firstCopyLen); + fastCopy(storage, head, result, 0, firstCopyLen); + fastCopy(storage, 0, result, firstCopyLen, count - firstCopyLen); } head = (head + count) % storage.length; size -= count; @@ -307,10 +318,10 @@ public float[] getAll() { float[] result = new float[size]; if (result.length > 0) { if (tail > head) { - System.arraycopy(storage, head, result, 0, tail - head); + fastCopy(storage, head, result, 0, tail - head); } else { - System.arraycopy(storage, head, result, 0, storage.length - head); - System.arraycopy(storage, 0, result, storage.length - head, tail); + fastCopy(storage, head, result, 0, storage.length - head); + fastCopy(storage, 0, result, storage.length - head, tail); } } return result; diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java index 40a19fa7ff7..f773609d701 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java @@ -21,10 +21,21 @@ * buffer instead. */ public class IntRingBuffer implements Serializable { + private static int SYSTEM_ARRAYCOPY_THRESHOLD = 16; protected final boolean growable; protected int[] storage; protected int head, tail, size; + private void fastCopy(int[] src, int srcPos, int[] dest, int destPos, int length) { + if (length < SYSTEM_ARRAYCOPY_THRESHOLD) { + for (int ii = 0; ii < length; ii++) { + dest[destPos + ii] = src[srcPos + ii]; + } + } else { + System.arraycopy(src, srcPos, dest, 0, length); + } + } + private void grow(int increase) { if (growable) { // assert that we are not asking for the impossible @@ -38,15 +49,15 @@ private void grow(int increase) { int[] newStorage = new int[newLength]; // three scenarios: size is zero so nothing to copy, head is before tail so only one copy needed, head - // after tail so two copies needed. Assuming that copying zero bytes is a fast operation, we will always + // after tail so two copies needed. Assuming that copying zero bytes is a fast operation, we will always // make two calls for simplicity and branch-prediction friendliness. // compute the size of the first copy final int firstCopyLen = Math.min(storage.length - head, size); // do the copying - System.arraycopy(storage, head, newStorage, 0, firstCopyLen); - System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); + fastCopy(storage, head, newStorage, 0, firstCopyLen); + fastCopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); // reset the pointers tail = size; @@ -82,13 +93,13 @@ public IntRingBuffer(int capacity, boolean growable) { this.growable = growable; if (growable) { // use next larger power of 2 - this.storage = new int[Integer.highestOneBit(capacity - 1) << 1]; + storage = new int[Integer.highestOneBit(capacity - 1) << 1]; } else { // might as well use exact size and not over-allocate - this.storage = new int[capacity]; + storage = new int[capacity]; } - this.tail = this.head = 0; + tail = head = 0; } public boolean isEmpty() { @@ -138,7 +149,7 @@ public boolean add(int e) { * {@code growable}, this may result in an internal growth operation. This call should be used in conjunction with * {@link #addUnsafe(int)}. * - * @param count the amount of empty entries in the buffer after this call + * @param count the minimum number of empty entries in the buffer after this call * @throws UnsupportedOperationException when {@code growable} is {@code false} and buffer is full */ public void ensureRemaining(int count) { @@ -153,7 +164,7 @@ public void ensureRemaining(int count) { /** * Add values unsafely (will silently overwrite values if the buffer is full). This call should be used in - * conjunction with * {@link #ensureRemaining(int)}. + * conjunction with {@link #ensureRemaining(int)}. * * @param e the value to add to the buffer */ @@ -206,10 +217,10 @@ public int[] remove(int count) { final int firstCopyLen = storage.length - head; if (tail >= head || firstCopyLen >= count) { - System.arraycopy(storage, head, result, 0, count); + fastCopy(storage, head, result, 0, count); } else { - System.arraycopy(storage, head, result, 0, firstCopyLen); - System.arraycopy(storage, 0, result, firstCopyLen, count - firstCopyLen); + fastCopy(storage, head, result, 0, firstCopyLen); + fastCopy(storage, 0, result, firstCopyLen, count - firstCopyLen); } head = (head + count) % storage.length; size -= count; @@ -307,10 +318,10 @@ public int[] getAll() { int[] result = new int[size]; if (result.length > 0) { if (tail > head) { - System.arraycopy(storage, head, result, 0, tail - head); + fastCopy(storage, head, result, 0, tail - head); } else { - System.arraycopy(storage, head, result, 0, storage.length - head); - System.arraycopy(storage, 0, result, storage.length - head, tail); + fastCopy(storage, head, result, 0, storage.length - head); + fastCopy(storage, 0, result, storage.length - head, tail); } } return result; diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java index cfce647e2ed..8973d9c11fc 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java @@ -21,10 +21,21 @@ * buffer instead. */ public class LongRingBuffer implements Serializable { + private static int SYSTEM_ARRAYCOPY_THRESHOLD = 16; protected final boolean growable; protected long[] storage; protected int head, tail, size; + private void fastCopy(long[] src, int srcPos, long[] dest, int destPos, int length) { + if (length < SYSTEM_ARRAYCOPY_THRESHOLD) { + for (int ii = 0; ii < length; ii++) { + dest[destPos + ii] = src[srcPos + ii]; + } + } else { + System.arraycopy(src, srcPos, dest, 0, length); + } + } + private void grow(int increase) { if (growable) { // assert that we are not asking for the impossible @@ -38,15 +49,15 @@ private void grow(int increase) { long[] newStorage = new long[newLength]; // three scenarios: size is zero so nothing to copy, head is before tail so only one copy needed, head - // after tail so two copies needed. Assuming that copying zero bytes is a fast operation, we will always + // after tail so two copies needed. Assuming that copying zero bytes is a fast operation, we will always // make two calls for simplicity and branch-prediction friendliness. // compute the size of the first copy final int firstCopyLen = Math.min(storage.length - head, size); // do the copying - System.arraycopy(storage, head, newStorage, 0, firstCopyLen); - System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); + fastCopy(storage, head, newStorage, 0, firstCopyLen); + fastCopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); // reset the pointers tail = size; @@ -82,13 +93,13 @@ public LongRingBuffer(int capacity, boolean growable) { this.growable = growable; if (growable) { // use next larger power of 2 - this.storage = new long[Integer.highestOneBit(capacity - 1) << 1]; + storage = new long[Integer.highestOneBit(capacity - 1) << 1]; } else { // might as well use exact size and not over-allocate - this.storage = new long[capacity]; + storage = new long[capacity]; } - this.tail = this.head = 0; + tail = head = 0; } public boolean isEmpty() { @@ -138,7 +149,7 @@ public boolean add(long e) { * {@code growable}, this may result in an internal growth operation. This call should be used in conjunction with * {@link #addUnsafe(long)}. * - * @param count the amount of empty entries in the buffer after this call + * @param count the minimum number of empty entries in the buffer after this call * @throws UnsupportedOperationException when {@code growable} is {@code false} and buffer is full */ public void ensureRemaining(int count) { @@ -153,7 +164,7 @@ public void ensureRemaining(int count) { /** * Add values unsafely (will silently overwrite values if the buffer is full). This call should be used in - * conjunction with * {@link #ensureRemaining(int)}. + * conjunction with {@link #ensureRemaining(int)}. * * @param e the value to add to the buffer */ @@ -206,10 +217,10 @@ public long[] remove(int count) { final int firstCopyLen = storage.length - head; if (tail >= head || firstCopyLen >= count) { - System.arraycopy(storage, head, result, 0, count); + fastCopy(storage, head, result, 0, count); } else { - System.arraycopy(storage, head, result, 0, firstCopyLen); - System.arraycopy(storage, 0, result, firstCopyLen, count - firstCopyLen); + fastCopy(storage, head, result, 0, firstCopyLen); + fastCopy(storage, 0, result, firstCopyLen, count - firstCopyLen); } head = (head + count) % storage.length; size -= count; @@ -307,10 +318,10 @@ public long[] getAll() { long[] result = new long[size]; if (result.length > 0) { if (tail > head) { - System.arraycopy(storage, head, result, 0, tail - head); + fastCopy(storage, head, result, 0, tail - head); } else { - System.arraycopy(storage, head, result, 0, storage.length - head); - System.arraycopy(storage, 0, result, storage.length - head, tail); + fastCopy(storage, head, result, 0, storage.length - head); + fastCopy(storage, 0, result, storage.length - head, tail); } } return result; diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java index 49ceaab58a7..c83cc1d6f7c 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java @@ -21,10 +21,21 @@ * buffer instead. */ public class ShortRingBuffer implements Serializable { + private static int SYSTEM_ARRAYCOPY_THRESHOLD = 16; protected final boolean growable; protected short[] storage; protected int head, tail, size; + private void fastCopy(short[] src, int srcPos, short[] dest, int destPos, int length) { + if (length < SYSTEM_ARRAYCOPY_THRESHOLD) { + for (int ii = 0; ii < length; ii++) { + dest[destPos + ii] = src[srcPos + ii]; + } + } else { + System.arraycopy(src, srcPos, dest, 0, length); + } + } + private void grow(int increase) { if (growable) { // assert that we are not asking for the impossible @@ -38,15 +49,15 @@ private void grow(int increase) { short[] newStorage = new short[newLength]; // three scenarios: size is zero so nothing to copy, head is before tail so only one copy needed, head - // after tail so two copies needed. Assuming that copying zero bytes is a fast operation, we will always + // after tail so two copies needed. Assuming that copying zero bytes is a fast operation, we will always // make two calls for simplicity and branch-prediction friendliness. // compute the size of the first copy final int firstCopyLen = Math.min(storage.length - head, size); // do the copying - System.arraycopy(storage, head, newStorage, 0, firstCopyLen); - System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); + fastCopy(storage, head, newStorage, 0, firstCopyLen); + fastCopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); // reset the pointers tail = size; @@ -82,13 +93,13 @@ public ShortRingBuffer(int capacity, boolean growable) { this.growable = growable; if (growable) { // use next larger power of 2 - this.storage = new short[Integer.highestOneBit(capacity - 1) << 1]; + storage = new short[Integer.highestOneBit(capacity - 1) << 1]; } else { // might as well use exact size and not over-allocate - this.storage = new short[capacity]; + storage = new short[capacity]; } - this.tail = this.head = 0; + tail = head = 0; } public boolean isEmpty() { @@ -138,7 +149,7 @@ public boolean add(short e) { * {@code growable}, this may result in an internal growth operation. This call should be used in conjunction with * {@link #addUnsafe(short)}. * - * @param count the amount of empty entries in the buffer after this call + * @param count the minimum number of empty entries in the buffer after this call * @throws UnsupportedOperationException when {@code growable} is {@code false} and buffer is full */ public void ensureRemaining(int count) { @@ -153,7 +164,7 @@ public void ensureRemaining(int count) { /** * Add values unsafely (will silently overwrite values if the buffer is full). This call should be used in - * conjunction with * {@link #ensureRemaining(int)}. + * conjunction with {@link #ensureRemaining(int)}. * * @param e the value to add to the buffer */ @@ -206,10 +217,10 @@ public short[] remove(int count) { final int firstCopyLen = storage.length - head; if (tail >= head || firstCopyLen >= count) { - System.arraycopy(storage, head, result, 0, count); + fastCopy(storage, head, result, 0, count); } else { - System.arraycopy(storage, head, result, 0, firstCopyLen); - System.arraycopy(storage, 0, result, firstCopyLen, count - firstCopyLen); + fastCopy(storage, head, result, 0, firstCopyLen); + fastCopy(storage, 0, result, firstCopyLen, count - firstCopyLen); } head = (head + count) % storage.length; size -= count; @@ -307,10 +318,10 @@ public short[] getAll() { short[] result = new short[size]; if (result.length > 0) { if (tail > head) { - System.arraycopy(storage, head, result, 0, tail - head); + fastCopy(storage, head, result, 0, tail - head); } else { - System.arraycopy(storage, head, result, 0, storage.length - head); - System.arraycopy(storage, 0, result, storage.length - head, tail); + fastCopy(storage, head, result, 0, storage.length - head); + fastCopy(storage, 0, result, storage.length - head, tail); } } return result; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java index a68f4af0f55..43ff32d9de0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java @@ -376,9 +376,9 @@ public void processRows(UpdateByWindowBucketContext context, boolean initialStep popCount++; } - // advance the keyIt and timestamp iterators until we are within the window. This only happens - // when initialStep == true because we have not created the minimum set of rows but include all - // non-null timestamp rows in our influencer values + // advance the influencerKeyIt and timestamp iterators until we are within the window. This only + // happens when initialStep == true because we have not created the minimum set of rows but include + // all non-null timestamp rows in our influencer values long skipCount = 0; while (currentTailTs < head) { Assert.eqTrue(initialStep, "initialStep when skipping rows"); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java index 486d736a840..20c7df8ba7f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java @@ -8,13 +8,18 @@ */ package io.deephaven.engine.table.impl.updateby.internal; -import gnu.trove.list.array.TIntArrayList; +import io.deephaven.base.ArrayUtil; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.WritableDoubleChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.*; import io.deephaven.util.SafeCloseable; +import org.apache.commons.lang3.mutable.MutableInt; import java.util.NoSuchElementException; +import static io.deephaven.util.QueryConstants.NULL_INT; + /*** * Store this data in the form of a binary tree where the latter half of the chunk is treated as a ring buffer and * pairwise results of the `DoubleFunction` are stored in the parent nodes. We do lazy evaluation by maintaining a @@ -26,10 +31,10 @@ */ public class PairwiseDoubleRingBuffer implements SafeCloseable { + private static final int PAIRWISE_MAX_CAPACITY = ArrayUtil.MAX_ARRAY_SIZE / 2; // use a sized double chunk for underlying storage private WritableDoubleChunk storageChunk; - private final TIntArrayList dirtyIndices; - private boolean allDirty; + private final DoubleFunction pairwiseFunction; private final double emptyVal; @@ -40,6 +45,12 @@ public class PairwiseDoubleRingBuffer implements SafeCloseable { private int head; private int tail; + private int size; + + private int dirtyPushHead; + private int dirtyPushTail; + private int dirtyPopHead; + private int dirtyPopTail; @FunctionalInterface public interface DoubleFunction { @@ -66,22 +77,22 @@ public interface DoubleFunction { * result is available */ public PairwiseDoubleRingBuffer(int initialSize, double emptyVal, DoubleFunction pairwiseFunction) { - // increase to next power of two - this.capacity = Integer.highestOneBit(initialSize) * 2; + Assert.eqTrue(PAIRWISE_MAX_CAPACITY >= initialSize, "PairwiseDoubleRingBuffer initialSize <= PAIRWISE_MAX_CAPACITY"); + + // use next larger power of 2 + this.capacity = Integer.highestOneBit(initialSize - 1) << 1; this.chunkSize = capacity * 2; this.storageChunk = WritableDoubleChunk.makeWritableChunk(chunkSize); - this.dirtyIndices = new TIntArrayList(chunkSize); this.pairwiseFunction = pairwiseFunction; this.emptyVal = emptyVal; - this.storageChunk.fillWithValue(0, chunkSize, emptyVal); - this.head = this.tail = this.capacity; - this.allDirty = false; + storageChunk.fillWithValue(0, chunkSize, emptyVal); + clear(); } private void evaluateRangeFast(int start, int end) { // everything in this range needs to be reevaluated - for (int left = start & 0xFFFFFFFE; left < end; left += 2) { + for (int left = start & 0xFFFFFFFE; left <= end; left += 2) { final int right = left + 1; final int parent = left / 2; @@ -92,57 +103,121 @@ private void evaluateRangeFast(int start, int end) { // compute & store final double computeVal = pairwiseFunction.apply(leftVal, rightVal); storageChunk.set(parent, computeVal); + } + } + + private void evaluateTree(int startA, int endA) { + while (endA > 1) { + // compute this level + evaluateRangeFast(startA, endA); - // mark the parent dirty - dirtyIndices.add(parent); + // compute the new parents + startA /= 2; + endA /= 2; } } - public double evaluate() { - // if all dirty, recompute all values - if (allDirty) { - if (head < tail) { - evaluateRangeFast(head, tail); + private void evaluateTree(int startA, int endA, int startB, int endB) { + while (endB > 1) { + if (endA >= startB - 1) { + // all collapse together into a single range + evaluateTree(startA, endB); + return; } else { - evaluateRangeFast(head, chunkSize); - evaluateRangeFast(capacity, tail); + // compute this level + evaluateRangeFast(startA, endA); + evaluateRangeFast(startB, endB); + + // compute the new parents + startA /= 2; + endA /= 2; + startB /= 2; + endB /= 2; } } + } - // work through all the dirty bits from high to low until none remain. - int dirtyIndex = 0; - while (dirtyIndex < dirtyIndices.size()) { - final int left = dirtyIndices.get(dirtyIndex) & 0xFFFFFFFE; // clear the final bit to force evenness - final int right = left + 1; + private void evaluateTree(int startA, int endA, int startB, int endB, int startC, int endC) { + while (endC > 1) { + if (endA >= startC - 1 || (endA >= startB - 1 && endB >= startC - 1)) { + // all collapse together into a single range + evaluateTree(startA, endC); + return; + } else if (endA >= startB - 1) { + // A and B collapse + evaluateTree(startA, endB, startC, endC); + return; + } else if (endB >= startC - 1) { + // B and C collapse + evaluateTree(startA, endA, startB, endC); + return; + } else { + // no collapse + evaluateRangeFast(startA, endA); + evaluateRangeFast(startB, endB); + evaluateRangeFast(startC, endC); + + // compute the new parents + startA /= 2; + endA /= 2; + startB /= 2; + endB /= 2; + startC /= 2; + endC /= 2; + } + } + } - // this isn't the typical parent = (n-1)/2 because the tree is right-shifted by one - final int parent = left / 2; + public double evaluate() { + final boolean pushDirty = dirtyPushHead != NULL_INT; + final boolean popDirty = dirtyPopHead != NULL_INT; - // load the data values - final double leftVal = storageChunk.get(left); - final double rightVal = storageChunk.get(right); - final double parentVal = storageChunk.get(parent); + if (size == 0) { + return emptyVal; + } - final double computeVal = pairwiseFunction.apply(leftVal, rightVal); - if (parentVal != computeVal) { - storageChunk.set(parent, computeVal); - // mark the parent dirty (if not the last) - if (parent > 1) { - dirtyIndices.add(parent); + if (!pushDirty && !popDirty) { + return storageChunk.get(1); + } + + // This is a nested complex set of `if` statements that are used to set the correct and minimal initial + // conditions for the evaluation. The calls to evaluateTree recurse no more than twice (as the ranges + // overlap and the calculation simplifies). + + if (pushDirty && popDirty) { + if (dirtyPushHead > dirtyPushTail && dirtyPopHead > dirtyPopTail) { + // both are wrapped + evaluateTree(capacity, Math.max(dirtyPushTail, dirtyPopTail), Math.min(dirtyPushHead, dirtyPopHead), chunkSize - 1); + } else if (dirtyPushHead > dirtyPushTail) { + // push wrapped, pop is not + evaluateTree(capacity, dirtyPushTail, dirtyPopHead, dirtyPopTail, dirtyPushHead, chunkSize - 1); + } else if (dirtyPushHead > dirtyPushTail) { + // pop wrapped, push is not + evaluateTree(capacity, dirtyPopTail, dirtyPushHead, dirtyPushTail, dirtyPopHead, chunkSize - 1); + } else { + // neither wrapped + if (dirtyPushHead > dirtyPopHead) { + evaluateTree(dirtyPopHead, dirtyPopTail, dirtyPushHead, dirtyPushTail); + } else { + evaluateTree(dirtyPushHead, dirtyPushTail, dirtyPopHead, dirtyPopTail); } } - // how far should we advance - final int nextIndex = dirtyIndex + 1; - if (nextIndex < dirtyIndices.size() && dirtyIndices.get(nextIndex) == right) { - dirtyIndex += 2; + } else if (pushDirty) { + if (dirtyPushHead > dirtyPushTail) { + evaluateTree(capacity, dirtyPushTail, dirtyPushHead, chunkSize - 1); } else { - dirtyIndex++; + evaluateTree(dirtyPushHead, dirtyPushTail); + } + } else if (popDirty) { + if (dirtyPopHead > dirtyPopTail) { + evaluateTree(capacity, dirtyPopTail, dirtyPopHead, chunkSize - 1); + } else { + evaluateTree(dirtyPopHead, dirtyPopTail); } } - allDirty = false; - dirtyIndices.clear(); - // final value is in index 1 + clearDirty(); + return storageChunk.get(1); } @@ -150,14 +225,15 @@ private void grow(int increase) { int oldCapacity = capacity; int oldChunkSize = chunkSize; - int size = size(); + // assert that we are not asking for the impossible + Assert.eqTrue(PAIRWISE_MAX_CAPACITY - increase >= size, "PairwiseDoubleRingBuffer size <= PAIRWISE_MAX_CAPACITY"); final int minLength = size + increase; // double the current capacity until there is sufficient space for the increase while (capacity <= minLength) { capacity *= 2; - chunkSize = capacity * 2; + chunkSize = Math.min(capacity * 2, PAIRWISE_MAX_CAPACITY); } // transfer to the new chunk @@ -168,14 +244,11 @@ private void grow(int increase) { storageChunk.fillWithValue(0, capacity, emptyVal); // move the data to the new chunk, note that we store the ring data in the second half of the array + final int firstCopyLen = Math.min(oldChunkSize - head, size); - if (tail >= head) { - storageChunk.copyFromTypedChunk(oldChunk, head, capacity, size); - } else { - final int firstCopyLen = oldChunkSize - head; - storageChunk.copyFromTypedChunk(oldChunk, head, capacity, firstCopyLen); - storageChunk.copyFromTypedChunk(oldChunk, oldCapacity, capacity + firstCopyLen , size - firstCopyLen); - } + // do the copying + storageChunk.copyFromTypedChunk(oldChunk, head, capacity, firstCopyLen); + storageChunk.copyFromTypedChunk(oldChunk, oldCapacity, capacity + firstCopyLen , size - firstCopyLen); tail = capacity + size; // fill the unused storage with the empty value @@ -188,8 +261,13 @@ private void grow(int increase) { // TODO: investigate moving precomputed results also. Since we are re-ordering the data values, would be // tricky to maintain order but a recursive function could probably do it efficiently. For now, make life easy // by setting all input dirty so the tree is recomputed on next `evaluate()` - this.dirtyIndices.clear(); - allDirty = true; + + // treat all these like pushed data + dirtyPushHead = head; + dirtyPushTail = size; + + dirtyPopHead = NULL_INT; + dirtyPopTail = 0; } private void grow() { @@ -200,25 +278,19 @@ public void push(double val) { if (isFull()) { grow(); } - // add the new data - storageChunk.set(tail, val); - if (!allDirty) { - dirtyIndices.add(tail); - } - - // move the tail - tail = ((tail + 1) % capacity) + capacity; + pushUnsafe(val); } public void pushUnsafe(double val) { - // add the new data storageChunk.set(tail, val); - if (!allDirty) { - dirtyIndices.add(tail); + if (dirtyPushHead == NULL_INT) { + dirtyPushHead = tail; } + dirtyPushTail = tail; // move the tail tail = ((tail + 1) % capacity) + capacity; + size++; } /** @@ -243,26 +315,21 @@ public double pop() { if (isEmpty()) { throw new NoSuchElementException(); } - double val = storageChunk.get(head); - storageChunk.set(head, emptyVal); - if (!allDirty) { - dirtyIndices.add(head); - } - - // move the head - head = ((head + 1) % capacity) + capacity; - return val; + return popUnsafe(); } public double popUnsafe() { double val = storageChunk.get(head); storageChunk.set(head, emptyVal); - if (!allDirty) { - dirtyIndices.add(head); + + if (dirtyPopHead == NULL_INT) { + dirtyPopHead = head; } + dirtyPopTail = head; // move the head head = ((head + 1) % capacity) + capacity; + size--; return val; } @@ -271,48 +338,34 @@ public double[] pop(int count) { throw new NoSuchElementException(); } final double[] result = new double[count]; - final int firstCopyLen = chunkSize - head; - - if (tail > head || firstCopyLen >= count) { - storageChunk.copyToArray(head, result, 0, count); - storageChunk.fillWithValue(head, count, emptyVal); - if (!allDirty) { - for (int ii = 0; ii < count; ii++) { - dirtyIndices.add(head + ii); - } - } - } else { - storageChunk.copyToArray(head, result, 0, firstCopyLen); - storageChunk.fillWithValue(head, firstCopyLen, emptyVal); - storageChunk.copyToArray(capacity, result, firstCopyLen, count - firstCopyLen); - storageChunk.fillWithValue(capacity, count - firstCopyLen, emptyVal); - if (!allDirty) { - for (int ii = 0; ii < firstCopyLen; ii++) { - dirtyIndices.add(head + ii); - } - for (int ii = 0; ii < count - firstCopyLen; ii++) { - dirtyIndices.add(capacity + ii); - } - } + + final int firstCopyLen = Math.min(chunkSize - head, count); + storageChunk.copyToArray(head, result, 0, firstCopyLen); + storageChunk.fillWithValue(head, firstCopyLen, emptyVal); + storageChunk.copyToArray(capacity, result, firstCopyLen, count - firstCopyLen); + storageChunk.fillWithValue(capacity, count - firstCopyLen, emptyVal); + + if (dirtyPopHead == NULL_INT) { + dirtyPopHead = head; } + dirtyPopTail = ((head + count - 1) % capacity) + capacity;; // move the head head = ((head + count) % capacity) + capacity; + size -= count; return result; } public boolean isFull() { - return ((tail + 1) % capacity) + capacity == head; + return size == capacity; } public int size() { - return tail >= head - ? (tail - head) : - (tail + (capacity - head)); + return size; } public boolean isEmpty() { - return tail == head; + return size == 0; } public double peek(double onEmpty) { @@ -326,9 +379,7 @@ public double poll(double onEmpty) { if (isEmpty()) { return onEmpty; } - double e = storageChunk.get(head); - head = (head + 1) % capacity + capacity; - return e; + return popUnsafe(); } public double front() { @@ -336,7 +387,7 @@ public double front() { } public double front(int offset) { - if (offset < 0 || offset >= size()) { + if (offset < 0 || offset >= size) { throw new NoSuchElementException(); } return storageChunk.get((head + offset) % capacity + capacity); @@ -364,23 +415,30 @@ public double element() { } public int capacity() { - return capacity - 1; + return capacity; } public int remaining() { - return capacity() - size(); + return capacity - size; + } + + private void clearDirty() { + dirtyPushHead = dirtyPopHead = NULL_INT; + dirtyPushTail = dirtyPopTail = NULL_INT; } public void clear() { head = tail = capacity; - dirtyIndices.clear(); - allDirty = false; + size = 0; + + clearDirty(); } @Override public void close() { try (final WritableDoubleChunk ignoredChunk = storageChunk) { - // close the closable items + // close the closable items and assign null + storageChunk = null; } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java index 2bd2fbdb391..c7d9ee9ed9d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java @@ -3,13 +3,16 @@ */ package io.deephaven.engine.table.impl.updateby.internal; -import gnu.trove.list.array.TIntArrayList; +import io.deephaven.base.ArrayUtil; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.WritableFloatChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.util.SafeCloseable; import java.util.NoSuchElementException; +import static io.deephaven.util.QueryConstants.NULL_INT; + /*** * Store this data in the form of a binary tree where the latter half of the chunk is treated as a ring buffer and * pairwise results of the `FloatFunction` are stored in the parent nodes. We do lazy evaluation by maintaining a @@ -21,10 +24,10 @@ */ public class PairwiseFloatRingBuffer implements SafeCloseable { + private static final int PAIRWISE_MAX_CAPACITY = ArrayUtil.MAX_ARRAY_SIZE / 2; // use a sized float chunk for underlying storage private WritableFloatChunk storageChunk; - private final TIntArrayList dirtyIndices; - private boolean allDirty; + private final FloatFunction pairwiseFunction; private final float emptyVal; @@ -35,6 +38,12 @@ public class PairwiseFloatRingBuffer implements SafeCloseable { private int head; private int tail; + private int size; + + private int dirtyPushHead; + private int dirtyPushTail; + private int dirtyPopHead; + private int dirtyPopTail; @FunctionalInterface public interface FloatFunction { @@ -61,22 +70,22 @@ public interface FloatFunction { * result is available */ public PairwiseFloatRingBuffer(int initialSize, float emptyVal, FloatFunction pairwiseFunction) { - // increase to next power of two - this.capacity = Integer.highestOneBit(initialSize) * 2; + Assert.eqTrue(PAIRWISE_MAX_CAPACITY >= initialSize, "PairwiseFloatRingBuffer initialSize <= PAIRWISE_MAX_CAPACITY"); + + // use next larger power of 2 + this.capacity = Integer.highestOneBit(initialSize - 1) << 1; this.chunkSize = capacity * 2; this.storageChunk = WritableFloatChunk.makeWritableChunk(chunkSize); - this.dirtyIndices = new TIntArrayList(chunkSize); this.pairwiseFunction = pairwiseFunction; this.emptyVal = emptyVal; - this.storageChunk.fillWithValue(0, chunkSize, emptyVal); - this.head = this.tail = this.capacity; - this.allDirty = false; + storageChunk.fillWithValue(0, chunkSize, emptyVal); + clear(); } private void evaluateRangeFast(int start, int end) { // everything in this range needs to be reevaluated - for (int left = start & 0xFFFFFFFE; left < end; left += 2) { + for (int left = start & 0xFFFFFFFE; left <= end; left += 2) { final int right = left + 1; final int parent = left / 2; @@ -87,57 +96,121 @@ private void evaluateRangeFast(int start, int end) { // compute & store final float computeVal = pairwiseFunction.apply(leftVal, rightVal); storageChunk.set(parent, computeVal); + } + } + + private void evaluateTree(int startA, int endA) { + while (endA > 1) { + // compute this level + evaluateRangeFast(startA, endA); - // mark the parent dirty - dirtyIndices.add(parent); + // compute the new parents + startA /= 2; + endA /= 2; } } - public float evaluate() { - // if all dirty, recompute all values - if (allDirty) { - if (head < tail) { - evaluateRangeFast(head, tail); + private void evaluateTree(int startA, int endA, int startB, int endB) { + while (endB > 1) { + if (endA >= startB - 1) { + // all collapse together into a single range + evaluateTree(startA, endB); + return; } else { - evaluateRangeFast(head, chunkSize); - evaluateRangeFast(capacity, tail); + // compute this level + evaluateRangeFast(startA, endA); + evaluateRangeFast(startB, endB); + + // compute the new parents + startA /= 2; + endA /= 2; + startB /= 2; + endB /= 2; } } + } - // work through all the dirty bits from high to low until none remain. - int dirtyIndex = 0; - while (dirtyIndex < dirtyIndices.size()) { - final int left = dirtyIndices.get(dirtyIndex) & 0xFFFFFFFE; // clear the final bit to force evenness - final int right = left + 1; + private void evaluateTree(int startA, int endA, int startB, int endB, int startC, int endC) { + while (endC > 1) { + if (endA >= startC - 1 || (endA >= startB - 1 && endB >= startC - 1)) { + // all collapse together into a single range + evaluateTree(startA, endC); + return; + } else if (endA >= startB - 1) { + // A and B collapse + evaluateTree(startA, endB, startC, endC); + return; + } else if (endB >= startC - 1) { + // B and C collapse + evaluateTree(startA, endA, startB, endC); + return; + } else { + // no collapse + evaluateRangeFast(startA, endA); + evaluateRangeFast(startB, endB); + evaluateRangeFast(startC, endC); + + // compute the new parents + startA /= 2; + endA /= 2; + startB /= 2; + endB /= 2; + startC /= 2; + endC /= 2; + } + } + } - // this isn't the typical parent = (n-1)/2 because the tree is right-shifted by one - final int parent = left / 2; + public float evaluate() { + final boolean pushDirty = dirtyPushHead != NULL_INT; + final boolean popDirty = dirtyPopHead != NULL_INT; - // load the data values - final float leftVal = storageChunk.get(left); - final float rightVal = storageChunk.get(right); - final float parentVal = storageChunk.get(parent); + if (size == 0) { + return emptyVal; + } - final float computeVal = pairwiseFunction.apply(leftVal, rightVal); - if (parentVal != computeVal) { - storageChunk.set(parent, computeVal); - // mark the parent dirty (if not the last) - if (parent > 1) { - dirtyIndices.add(parent); + if (!pushDirty && !popDirty) { + return storageChunk.get(1); + } + + // This is a nested complex set of `if` statements that are used to set the correct and minimal initial + // conditions for the evaluation. The calls to evaluateTree recurse no more than twice (as the ranges + // overlap and the calculation simplifies). + + if (pushDirty && popDirty) { + if (dirtyPushHead > dirtyPushTail && dirtyPopHead > dirtyPopTail) { + // both are wrapped + evaluateTree(capacity, Math.max(dirtyPushTail, dirtyPopTail), Math.min(dirtyPushHead, dirtyPopHead), chunkSize - 1); + } else if (dirtyPushHead > dirtyPushTail) { + // push wrapped, pop is not + evaluateTree(capacity, dirtyPushTail, dirtyPopHead, dirtyPopTail, dirtyPushHead, chunkSize - 1); + } else if (dirtyPopHead > dirtyPopTail) { + // pop wrapped, push is not + evaluateTree(capacity, dirtyPopTail, dirtyPushHead, dirtyPushTail, dirtyPopHead, chunkSize - 1); + } else { + // neither wrapped + if (dirtyPushHead > dirtyPopHead) { + evaluateTree(dirtyPopHead, dirtyPopTail, dirtyPushHead, dirtyPushTail); + } else { + evaluateTree(dirtyPushHead, dirtyPushTail, dirtyPopHead, dirtyPopTail); } } - // how far should we advance - final int nextIndex = dirtyIndex + 1; - if (nextIndex < dirtyIndices.size() && dirtyIndices.get(nextIndex) == right) { - dirtyIndex += 2; + } else if (pushDirty) { + if (dirtyPushHead > dirtyPushTail) { + evaluateTree(capacity, dirtyPushTail, dirtyPushHead, chunkSize - 1); } else { - dirtyIndex++; + evaluateTree(dirtyPushHead, dirtyPushTail); + } + } else if (popDirty) { + if (dirtyPopHead > dirtyPopTail) { + evaluateTree(capacity, dirtyPopTail, dirtyPopHead, chunkSize - 1); + } else { + evaluateTree(dirtyPopHead, dirtyPopTail); } } - allDirty = false; - dirtyIndices.clear(); - // final value is in index 1 + clearDirty(); + return storageChunk.get(1); } @@ -145,14 +218,15 @@ private void grow(int increase) { int oldCapacity = capacity; int oldChunkSize = chunkSize; - int size = size(); + // assert that we are not asking for the impossible + Assert.eqTrue(PAIRWISE_MAX_CAPACITY - increase >= size, "PairwiseFloatRingBuffer size <= PAIRWISE_MAX_CAPACITY"); final int minLength = size + increase; // double the current capacity until there is sufficient space for the increase while (capacity <= minLength) { capacity *= 2; - chunkSize = capacity * 2; + chunkSize = Math.min(capacity * 2, PAIRWISE_MAX_CAPACITY); } // transfer to the new chunk @@ -163,14 +237,11 @@ private void grow(int increase) { storageChunk.fillWithValue(0, capacity, emptyVal); // move the data to the new chunk, note that we store the ring data in the second half of the array + final int firstCopyLen = Math.min(oldChunkSize - head, size); - if (tail >= head) { - storageChunk.copyFromTypedChunk(oldChunk, head, capacity, size); - } else { - final int firstCopyLen = oldChunkSize - head; - storageChunk.copyFromTypedChunk(oldChunk, head, capacity, firstCopyLen); - storageChunk.copyFromTypedChunk(oldChunk, oldCapacity, capacity + firstCopyLen , size - firstCopyLen); - } + // do the copying + storageChunk.copyFromTypedChunk(oldChunk, head, capacity, firstCopyLen); + storageChunk.copyFromTypedChunk(oldChunk, oldCapacity, capacity + firstCopyLen , size - firstCopyLen); tail = capacity + size; // fill the unused storage with the empty value @@ -183,8 +254,13 @@ private void grow(int increase) { // TODO: investigate moving precomputed results also. Since we are re-ordering the data values, would be // tricky to maintain order but a recursive function could probably do it efficiently. For now, make life easy // by setting all input dirty so the tree is recomputed on next `evaluate()` - this.dirtyIndices.clear(); - allDirty = true; + + // treat all these like pushed data + dirtyPushHead = head; + dirtyPushTail = size; + + dirtyPopHead = NULL_INT; + dirtyPopTail = 0; } private void grow() { @@ -195,25 +271,19 @@ public void push(float val) { if (isFull()) { grow(); } - // add the new data - storageChunk.set(tail, val); - if (!allDirty) { - dirtyIndices.add(tail); - } - - // move the tail - tail = ((tail + 1) % capacity) + capacity; + pushUnsafe(val); } public void pushUnsafe(float val) { - // add the new data storageChunk.set(tail, val); - if (!allDirty) { - dirtyIndices.add(tail); + if (dirtyPushHead == NULL_INT) { + dirtyPushHead = tail; } + dirtyPushTail = tail; // move the tail tail = ((tail + 1) % capacity) + capacity; + size++; } /** @@ -238,26 +308,21 @@ public float pop() { if (isEmpty()) { throw new NoSuchElementException(); } - float val = storageChunk.get(head); - storageChunk.set(head, emptyVal); - if (!allDirty) { - dirtyIndices.add(head); - } - - // move the head - head = ((head + 1) % capacity) + capacity; - return val; + return popUnsafe(); } public float popUnsafe() { float val = storageChunk.get(head); storageChunk.set(head, emptyVal); - if (!allDirty) { - dirtyIndices.add(head); + + if (dirtyPopHead == NULL_INT) { + dirtyPopHead = head; } + dirtyPopTail = head; // move the head head = ((head + 1) % capacity) + capacity; + size--; return val; } @@ -266,48 +331,34 @@ public float[] pop(int count) { throw new NoSuchElementException(); } final float[] result = new float[count]; - final int firstCopyLen = chunkSize - head; - - if (tail > head || firstCopyLen >= count) { - storageChunk.copyToArray(head, result, 0, count); - storageChunk.fillWithValue(head, count, emptyVal); - if (!allDirty) { - for (int ii = 0; ii < count; ii++) { - dirtyIndices.add(head + ii); - } - } - } else { - storageChunk.copyToArray(head, result, 0, firstCopyLen); - storageChunk.fillWithValue(head, firstCopyLen, emptyVal); - storageChunk.copyToArray(capacity, result, firstCopyLen, count - firstCopyLen); - storageChunk.fillWithValue(capacity, count - firstCopyLen, emptyVal); - if (!allDirty) { - for (int ii = 0; ii < firstCopyLen; ii++) { - dirtyIndices.add(head + ii); - } - for (int ii = 0; ii < count - firstCopyLen; ii++) { - dirtyIndices.add(capacity + ii); - } - } + + final int firstCopyLen = Math.min(chunkSize - head, count); + storageChunk.copyToArray(head, result, 0, firstCopyLen); + storageChunk.fillWithValue(head, firstCopyLen, emptyVal); + storageChunk.copyToArray(capacity, result, firstCopyLen, count - firstCopyLen); + storageChunk.fillWithValue(capacity, count - firstCopyLen, emptyVal); + + if (dirtyPopHead == NULL_INT) { + dirtyPopHead = head; } + dirtyPopTail = ((head + count - 1) % capacity) + capacity;; // move the head head = ((head + count) % capacity) + capacity; + size -= count; return result; } public boolean isFull() { - return ((tail + 1) % capacity) + capacity == head; + return size == capacity; } public int size() { - return tail >= head - ? (tail - head) : - (tail + (capacity - head)); + return size; } public boolean isEmpty() { - return tail == head; + return size == 0; } public float peek(float onEmpty) { @@ -321,9 +372,7 @@ public float poll(float onEmpty) { if (isEmpty()) { return onEmpty; } - float e = storageChunk.get(head); - head = (head + 1) % capacity + capacity; - return e; + return popUnsafe(); } public float front() { @@ -331,7 +380,7 @@ public float front() { } public float front(int offset) { - if (offset < 0 || offset >= size()) { + if (offset < 0 || offset >= size) { throw new NoSuchElementException(); } return storageChunk.get((head + offset) % capacity + capacity); @@ -359,23 +408,30 @@ public float element() { } public int capacity() { - return capacity - 1; + return capacity; } public int remaining() { - return capacity() - size(); + return capacity - size; + } + + private void clearDirty() { + dirtyPushHead = dirtyPopHead = NULL_INT; + dirtyPushTail = dirtyPopTail = NULL_INT; } public void clear() { head = tail = capacity; - dirtyIndices.clear(); - allDirty = false; + size = 0; + + clearDirty(); } @Override public void close() { try (final WritableFloatChunk ignoredChunk = storageChunk) { - // close the closable items + // close the closable items and assign null + storageChunk = null; } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java index 8be0abded2d..308b10bd16f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java @@ -5,7 +5,6 @@ */ package io.deephaven.engine.table.impl.updateby.rollingsum; -import io.deephaven.api.updateby.OperationControl; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.DoubleChunk; @@ -22,7 +21,6 @@ import java.util.Map; import static io.deephaven.util.QueryConstants.NULL_DOUBLE; -import static io.deephaven.util.QueryConstants.NULL_SHORT; public class DoubleRollingSumOperator extends BaseWindowedDoubleUpdateByOperator { private static final int PAIRWISE_BUFFER_INITIAL_SIZE = 64; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java index 7e831d528fd..0c0a6549d91 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java @@ -1,6 +1,5 @@ package io.deephaven.engine.table.impl.updateby.rollingsum; -import io.deephaven.api.updateby.OperationControl; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.FloatChunk; @@ -17,7 +16,6 @@ import java.util.Map; import static io.deephaven.util.QueryConstants.NULL_FLOAT; -import static io.deephaven.util.QueryConstants.NULL_SHORT; public class FloatRollingSumOperator extends BaseWindowedFloatUpdateByOperator { private static final int PAIRWISE_BUFFER_INITIAL_SIZE = 64; diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBufferTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBufferTest.java index de5004221ac..ccea218d90e 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBufferTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBufferTest.java @@ -8,7 +8,6 @@ */ package io.deephaven.engine.table.impl.updateby.internal; -import io.deephaven.engine.table.impl.updateby.internal.PairwiseDoubleRingBuffer; import io.deephaven.test.types.OutOfBandTest; import junit.framework.TestCase; import org.junit.experimental.categories.Category; diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBufferTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBufferTest.java index a7098783182..97140fda810 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBufferTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBufferTest.java @@ -3,7 +3,6 @@ */ package io.deephaven.engine.table.impl.updateby.internal; -import io.deephaven.engine.table.impl.updateby.internal.PairwiseFloatRingBuffer; import io.deephaven.test.types.OutOfBandTest; import junit.framework.TestCase; import org.junit.experimental.categories.Category; From 3e958099e34ea29a2ba7b15d6a45bd448e231c92 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 24 Jan 2023 15:09:57 -0800 Subject: [PATCH 093/123] Properly replicated PairwiseDoubleRingBuffer, cleanup of repeated code --- .../base/ringbuffer/ByteRingBuffer.java | 58 +++------ .../base/ringbuffer/CharRingBuffer.java | 58 +++------ .../base/ringbuffer/DoubleRingBuffer.java | 58 +++------ .../base/ringbuffer/FloatRingBuffer.java | 58 +++------ .../base/ringbuffer/IntRingBuffer.java | 58 +++------ .../base/ringbuffer/LongRingBuffer.java | 58 +++------ .../base/ringbuffer/ShortRingBuffer.java | 58 +++------ .../base/ringbuffer/ByteRingBufferTest.java | 3 - .../base/ringbuffer/CharRingBufferTest.java | 3 - .../base/ringbuffer/DoubleRingBufferTest.java | 3 - .../base/ringbuffer/FloatRingBufferTest.java | 3 - .../base/ringbuffer/IntRingBufferTest.java | 3 - .../base/ringbuffer/LongRingBufferTest.java | 3 - .../base/ringbuffer/ShortRingBufferTest.java | 3 - .../internal/PairwiseDoubleRingBuffer.java | 59 ++++----- .../internal/PairwiseFloatRingBuffer.java | 55 ++++---- .../PairwiseDoubleRingBufferTest.java | 123 +++++++++--------- .../internal/PairwiseFloatRingBufferTest.java | 123 +++++++++--------- 18 files changed, 300 insertions(+), 487 deletions(-) diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java index 4d86dc8ec89..fffbd689417 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java @@ -26,27 +26,13 @@ public class ByteRingBuffer implements Serializable { protected byte[] storage; protected int head, tail, size; - private void fastCopy(byte[] src, int srcPos, byte[] dest, int destPos, int length) { - if (length < SYSTEM_ARRAYCOPY_THRESHOLD) { - for (int ii = 0; ii < length; ii++) { - dest[destPos + ii] = src[srcPos + ii]; - } - } else { - System.arraycopy(src, srcPos, dest, 0, length); - } - } - private void grow(int increase) { if (growable) { // assert that we are not asking for the impossible Assert.eqTrue(ArrayUtil.MAX_ARRAY_SIZE - increase >= size, "ByteRingBuffer size <= MAX_ARRAY_SIZE"); final int minLength = size + increase; - int newLength = storage.length * 2; - while (newLength < minLength) { - newLength = Math.min(newLength * 2, ArrayUtil.MAX_ARRAY_SIZE); - } - byte[] newStorage = new byte[newLength]; + byte[] newStorage = new byte[Integer.highestOneBit(minLength - 1) << 1]; // three scenarios: size is zero so nothing to copy, head is before tail so only one copy needed, head // after tail so two copies needed. Assuming that copying zero bytes is a fast operation, we will always @@ -55,9 +41,9 @@ private void grow(int increase) { // compute the size of the first copy final int firstCopyLen = Math.min(storage.length - head, size); - // do the copying - fastCopy(storage, head, newStorage, 0, firstCopyLen); - fastCopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); + // do the copying ( + System.arraycopy(storage, head, newStorage, 0, firstCopyLen); + System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); // reset the pointers tail = size; @@ -90,6 +76,8 @@ public ByteRingBuffer(int capacity) { * @param growable whether to allow growth when the buffer is full. */ public ByteRingBuffer(int capacity, boolean growable) { + Assert.eqTrue(capacity <= ArrayUtil.MAX_ARRAY_SIZE, "ByteRingBuffer size <= MAX_ARRAY_SIZE"); + this.growable = growable; if (growable) { // use next larger power of 2 @@ -138,9 +126,7 @@ public boolean add(byte e) { grow(); } } - storage[tail] = e; - tail = (tail + 1) % storage.length; - size++; + addUnsafe(e); return true; } @@ -186,9 +172,7 @@ public byte addOverwrite(byte e, byte notFullResult) { if (isFull()) { result = remove(); } - storage[tail] = e; - tail = (tail + 1) % storage.length; - size++; + addUnsafe(e); return result; } @@ -203,9 +187,7 @@ public boolean offer(byte e) { if (isFull()) { return false; } - storage[tail] = e; - tail = (tail + 1) % storage.length; - size++; + addUnsafe(e); return true; } @@ -217,10 +199,10 @@ public byte[] remove(int count) { final int firstCopyLen = storage.length - head; if (tail >= head || firstCopyLen >= count) { - fastCopy(storage, head, result, 0, count); + System.arraycopy(storage, head, result, 0, count); } else { - fastCopy(storage, head, result, 0, firstCopyLen); - fastCopy(storage, 0, result, firstCopyLen, count - firstCopyLen); + System.arraycopy(storage, head, result, 0, firstCopyLen); + System.arraycopy(storage, 0, result, firstCopyLen, count - firstCopyLen); } head = (head + count) % storage.length; size -= count; @@ -231,10 +213,7 @@ public byte remove() { if (isEmpty()) { throw new NoSuchElementException(); } - byte e = storage[head]; - head = (head + 1) % storage.length; - size--; - return e; + return removeUnsafe(); } public byte removeUnsafe() { @@ -248,10 +227,7 @@ public byte poll(byte onEmpty) { if (isEmpty()) { return onEmpty; } - byte e = storage[head]; - head = (head + 1) % storage.length; - size--; - return e; + return removeUnsafe(); } public byte element() { @@ -318,10 +294,10 @@ public byte[] getAll() { byte[] result = new byte[size]; if (result.length > 0) { if (tail > head) { - fastCopy(storage, head, result, 0, tail - head); + System.arraycopy(storage, head, result, 0, tail - head); } else { - fastCopy(storage, head, result, 0, storage.length - head); - fastCopy(storage, 0, result, storage.length - head, tail); + System.arraycopy(storage, head, result, 0, storage.length - head); + System.arraycopy(storage, 0, result, storage.length - head, tail); } } return result; diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java index 63e0a329761..740bd31c7d4 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java @@ -21,27 +21,13 @@ public class CharRingBuffer implements Serializable { protected char[] storage; protected int head, tail, size; - private void fastCopy(char[] src, int srcPos, char[] dest, int destPos, int length) { - if (length < SYSTEM_ARRAYCOPY_THRESHOLD) { - for (int ii = 0; ii < length; ii++) { - dest[destPos + ii] = src[srcPos + ii]; - } - } else { - System.arraycopy(src, srcPos, dest, 0, length); - } - } - private void grow(int increase) { if (growable) { // assert that we are not asking for the impossible Assert.eqTrue(ArrayUtil.MAX_ARRAY_SIZE - increase >= size, "CharRingBuffer size <= MAX_ARRAY_SIZE"); final int minLength = size + increase; - int newLength = storage.length * 2; - while (newLength < minLength) { - newLength = Math.min(newLength * 2, ArrayUtil.MAX_ARRAY_SIZE); - } - char[] newStorage = new char[newLength]; + char[] newStorage = new char[Integer.highestOneBit(minLength - 1) << 1]; // three scenarios: size is zero so nothing to copy, head is before tail so only one copy needed, head // after tail so two copies needed. Assuming that copying zero bytes is a fast operation, we will always @@ -50,9 +36,9 @@ private void grow(int increase) { // compute the size of the first copy final int firstCopyLen = Math.min(storage.length - head, size); - // do the copying - fastCopy(storage, head, newStorage, 0, firstCopyLen); - fastCopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); + // do the copying ( + System.arraycopy(storage, head, newStorage, 0, firstCopyLen); + System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); // reset the pointers tail = size; @@ -85,6 +71,8 @@ public CharRingBuffer(int capacity) { * @param growable whether to allow growth when the buffer is full. */ public CharRingBuffer(int capacity, boolean growable) { + Assert.eqTrue(capacity <= ArrayUtil.MAX_ARRAY_SIZE, "CharRingBuffer size <= MAX_ARRAY_SIZE"); + this.growable = growable; if (growable) { // use next larger power of 2 @@ -133,9 +121,7 @@ public boolean add(char e) { grow(); } } - storage[tail] = e; - tail = (tail + 1) % storage.length; - size++; + addUnsafe(e); return true; } @@ -181,9 +167,7 @@ public char addOverwrite(char e, char notFullResult) { if (isFull()) { result = remove(); } - storage[tail] = e; - tail = (tail + 1) % storage.length; - size++; + addUnsafe(e); return result; } @@ -198,9 +182,7 @@ public boolean offer(char e) { if (isFull()) { return false; } - storage[tail] = e; - tail = (tail + 1) % storage.length; - size++; + addUnsafe(e); return true; } @@ -212,10 +194,10 @@ public char[] remove(int count) { final int firstCopyLen = storage.length - head; if (tail >= head || firstCopyLen >= count) { - fastCopy(storage, head, result, 0, count); + System.arraycopy(storage, head, result, 0, count); } else { - fastCopy(storage, head, result, 0, firstCopyLen); - fastCopy(storage, 0, result, firstCopyLen, count - firstCopyLen); + System.arraycopy(storage, head, result, 0, firstCopyLen); + System.arraycopy(storage, 0, result, firstCopyLen, count - firstCopyLen); } head = (head + count) % storage.length; size -= count; @@ -226,10 +208,7 @@ public char remove() { if (isEmpty()) { throw new NoSuchElementException(); } - char e = storage[head]; - head = (head + 1) % storage.length; - size--; - return e; + return removeUnsafe(); } public char removeUnsafe() { @@ -243,10 +222,7 @@ public char poll(char onEmpty) { if (isEmpty()) { return onEmpty; } - char e = storage[head]; - head = (head + 1) % storage.length; - size--; - return e; + return removeUnsafe(); } public char element() { @@ -313,10 +289,10 @@ public char[] getAll() { char[] result = new char[size]; if (result.length > 0) { if (tail > head) { - fastCopy(storage, head, result, 0, tail - head); + System.arraycopy(storage, head, result, 0, tail - head); } else { - fastCopy(storage, head, result, 0, storage.length - head); - fastCopy(storage, 0, result, storage.length - head, tail); + System.arraycopy(storage, head, result, 0, storage.length - head); + System.arraycopy(storage, 0, result, storage.length - head, tail); } } return result; diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java index 93956fad2b9..d4372bcfecc 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java @@ -26,27 +26,13 @@ public class DoubleRingBuffer implements Serializable { protected double[] storage; protected int head, tail, size; - private void fastCopy(double[] src, int srcPos, double[] dest, int destPos, int length) { - if (length < SYSTEM_ARRAYCOPY_THRESHOLD) { - for (int ii = 0; ii < length; ii++) { - dest[destPos + ii] = src[srcPos + ii]; - } - } else { - System.arraycopy(src, srcPos, dest, 0, length); - } - } - private void grow(int increase) { if (growable) { // assert that we are not asking for the impossible Assert.eqTrue(ArrayUtil.MAX_ARRAY_SIZE - increase >= size, "DoubleRingBuffer size <= MAX_ARRAY_SIZE"); final int minLength = size + increase; - int newLength = storage.length * 2; - while (newLength < minLength) { - newLength = Math.min(newLength * 2, ArrayUtil.MAX_ARRAY_SIZE); - } - double[] newStorage = new double[newLength]; + double[] newStorage = new double[Integer.highestOneBit(minLength - 1) << 1]; // three scenarios: size is zero so nothing to copy, head is before tail so only one copy needed, head // after tail so two copies needed. Assuming that copying zero bytes is a fast operation, we will always @@ -55,9 +41,9 @@ private void grow(int increase) { // compute the size of the first copy final int firstCopyLen = Math.min(storage.length - head, size); - // do the copying - fastCopy(storage, head, newStorage, 0, firstCopyLen); - fastCopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); + // do the copying ( + System.arraycopy(storage, head, newStorage, 0, firstCopyLen); + System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); // reset the pointers tail = size; @@ -90,6 +76,8 @@ public DoubleRingBuffer(int capacity) { * @param growable whether to allow growth when the buffer is full. */ public DoubleRingBuffer(int capacity, boolean growable) { + Assert.eqTrue(capacity <= ArrayUtil.MAX_ARRAY_SIZE, "DoubleRingBuffer size <= MAX_ARRAY_SIZE"); + this.growable = growable; if (growable) { // use next larger power of 2 @@ -138,9 +126,7 @@ public boolean add(double e) { grow(); } } - storage[tail] = e; - tail = (tail + 1) % storage.length; - size++; + addUnsafe(e); return true; } @@ -186,9 +172,7 @@ public double addOverwrite(double e, double notFullResult) { if (isFull()) { result = remove(); } - storage[tail] = e; - tail = (tail + 1) % storage.length; - size++; + addUnsafe(e); return result; } @@ -203,9 +187,7 @@ public boolean offer(double e) { if (isFull()) { return false; } - storage[tail] = e; - tail = (tail + 1) % storage.length; - size++; + addUnsafe(e); return true; } @@ -217,10 +199,10 @@ public double[] remove(int count) { final int firstCopyLen = storage.length - head; if (tail >= head || firstCopyLen >= count) { - fastCopy(storage, head, result, 0, count); + System.arraycopy(storage, head, result, 0, count); } else { - fastCopy(storage, head, result, 0, firstCopyLen); - fastCopy(storage, 0, result, firstCopyLen, count - firstCopyLen); + System.arraycopy(storage, head, result, 0, firstCopyLen); + System.arraycopy(storage, 0, result, firstCopyLen, count - firstCopyLen); } head = (head + count) % storage.length; size -= count; @@ -231,10 +213,7 @@ public double remove() { if (isEmpty()) { throw new NoSuchElementException(); } - double e = storage[head]; - head = (head + 1) % storage.length; - size--; - return e; + return removeUnsafe(); } public double removeUnsafe() { @@ -248,10 +227,7 @@ public double poll(double onEmpty) { if (isEmpty()) { return onEmpty; } - double e = storage[head]; - head = (head + 1) % storage.length; - size--; - return e; + return removeUnsafe(); } public double element() { @@ -318,10 +294,10 @@ public double[] getAll() { double[] result = new double[size]; if (result.length > 0) { if (tail > head) { - fastCopy(storage, head, result, 0, tail - head); + System.arraycopy(storage, head, result, 0, tail - head); } else { - fastCopy(storage, head, result, 0, storage.length - head); - fastCopy(storage, 0, result, storage.length - head, tail); + System.arraycopy(storage, head, result, 0, storage.length - head); + System.arraycopy(storage, 0, result, storage.length - head, tail); } } return result; diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java index 7ce2b4e22b4..6c35f733ac3 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java @@ -26,27 +26,13 @@ public class FloatRingBuffer implements Serializable { protected float[] storage; protected int head, tail, size; - private void fastCopy(float[] src, int srcPos, float[] dest, int destPos, int length) { - if (length < SYSTEM_ARRAYCOPY_THRESHOLD) { - for (int ii = 0; ii < length; ii++) { - dest[destPos + ii] = src[srcPos + ii]; - } - } else { - System.arraycopy(src, srcPos, dest, 0, length); - } - } - private void grow(int increase) { if (growable) { // assert that we are not asking for the impossible Assert.eqTrue(ArrayUtil.MAX_ARRAY_SIZE - increase >= size, "FloatRingBuffer size <= MAX_ARRAY_SIZE"); final int minLength = size + increase; - int newLength = storage.length * 2; - while (newLength < minLength) { - newLength = Math.min(newLength * 2, ArrayUtil.MAX_ARRAY_SIZE); - } - float[] newStorage = new float[newLength]; + float[] newStorage = new float[Integer.highestOneBit(minLength - 1) << 1]; // three scenarios: size is zero so nothing to copy, head is before tail so only one copy needed, head // after tail so two copies needed. Assuming that copying zero bytes is a fast operation, we will always @@ -55,9 +41,9 @@ private void grow(int increase) { // compute the size of the first copy final int firstCopyLen = Math.min(storage.length - head, size); - // do the copying - fastCopy(storage, head, newStorage, 0, firstCopyLen); - fastCopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); + // do the copying ( + System.arraycopy(storage, head, newStorage, 0, firstCopyLen); + System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); // reset the pointers tail = size; @@ -90,6 +76,8 @@ public FloatRingBuffer(int capacity) { * @param growable whether to allow growth when the buffer is full. */ public FloatRingBuffer(int capacity, boolean growable) { + Assert.eqTrue(capacity <= ArrayUtil.MAX_ARRAY_SIZE, "FloatRingBuffer size <= MAX_ARRAY_SIZE"); + this.growable = growable; if (growable) { // use next larger power of 2 @@ -138,9 +126,7 @@ public boolean add(float e) { grow(); } } - storage[tail] = e; - tail = (tail + 1) % storage.length; - size++; + addUnsafe(e); return true; } @@ -186,9 +172,7 @@ public float addOverwrite(float e, float notFullResult) { if (isFull()) { result = remove(); } - storage[tail] = e; - tail = (tail + 1) % storage.length; - size++; + addUnsafe(e); return result; } @@ -203,9 +187,7 @@ public boolean offer(float e) { if (isFull()) { return false; } - storage[tail] = e; - tail = (tail + 1) % storage.length; - size++; + addUnsafe(e); return true; } @@ -217,10 +199,10 @@ public float[] remove(int count) { final int firstCopyLen = storage.length - head; if (tail >= head || firstCopyLen >= count) { - fastCopy(storage, head, result, 0, count); + System.arraycopy(storage, head, result, 0, count); } else { - fastCopy(storage, head, result, 0, firstCopyLen); - fastCopy(storage, 0, result, firstCopyLen, count - firstCopyLen); + System.arraycopy(storage, head, result, 0, firstCopyLen); + System.arraycopy(storage, 0, result, firstCopyLen, count - firstCopyLen); } head = (head + count) % storage.length; size -= count; @@ -231,10 +213,7 @@ public float remove() { if (isEmpty()) { throw new NoSuchElementException(); } - float e = storage[head]; - head = (head + 1) % storage.length; - size--; - return e; + return removeUnsafe(); } public float removeUnsafe() { @@ -248,10 +227,7 @@ public float poll(float onEmpty) { if (isEmpty()) { return onEmpty; } - float e = storage[head]; - head = (head + 1) % storage.length; - size--; - return e; + return removeUnsafe(); } public float element() { @@ -318,10 +294,10 @@ public float[] getAll() { float[] result = new float[size]; if (result.length > 0) { if (tail > head) { - fastCopy(storage, head, result, 0, tail - head); + System.arraycopy(storage, head, result, 0, tail - head); } else { - fastCopy(storage, head, result, 0, storage.length - head); - fastCopy(storage, 0, result, storage.length - head, tail); + System.arraycopy(storage, head, result, 0, storage.length - head); + System.arraycopy(storage, 0, result, storage.length - head, tail); } } return result; diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java index f773609d701..cdd08786569 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java @@ -26,27 +26,13 @@ public class IntRingBuffer implements Serializable { protected int[] storage; protected int head, tail, size; - private void fastCopy(int[] src, int srcPos, int[] dest, int destPos, int length) { - if (length < SYSTEM_ARRAYCOPY_THRESHOLD) { - for (int ii = 0; ii < length; ii++) { - dest[destPos + ii] = src[srcPos + ii]; - } - } else { - System.arraycopy(src, srcPos, dest, 0, length); - } - } - private void grow(int increase) { if (growable) { // assert that we are not asking for the impossible Assert.eqTrue(ArrayUtil.MAX_ARRAY_SIZE - increase >= size, "IntRingBuffer size <= MAX_ARRAY_SIZE"); final int minLength = size + increase; - int newLength = storage.length * 2; - while (newLength < minLength) { - newLength = Math.min(newLength * 2, ArrayUtil.MAX_ARRAY_SIZE); - } - int[] newStorage = new int[newLength]; + int[] newStorage = new int[Integer.highestOneBit(minLength - 1) << 1]; // three scenarios: size is zero so nothing to copy, head is before tail so only one copy needed, head // after tail so two copies needed. Assuming that copying zero bytes is a fast operation, we will always @@ -55,9 +41,9 @@ private void grow(int increase) { // compute the size of the first copy final int firstCopyLen = Math.min(storage.length - head, size); - // do the copying - fastCopy(storage, head, newStorage, 0, firstCopyLen); - fastCopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); + // do the copying ( + System.arraycopy(storage, head, newStorage, 0, firstCopyLen); + System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); // reset the pointers tail = size; @@ -90,6 +76,8 @@ public IntRingBuffer(int capacity) { * @param growable whether to allow growth when the buffer is full. */ public IntRingBuffer(int capacity, boolean growable) { + Assert.eqTrue(capacity <= ArrayUtil.MAX_ARRAY_SIZE, "IntRingBuffer size <= MAX_ARRAY_SIZE"); + this.growable = growable; if (growable) { // use next larger power of 2 @@ -138,9 +126,7 @@ public boolean add(int e) { grow(); } } - storage[tail] = e; - tail = (tail + 1) % storage.length; - size++; + addUnsafe(e); return true; } @@ -186,9 +172,7 @@ public int addOverwrite(int e, int notFullResult) { if (isFull()) { result = remove(); } - storage[tail] = e; - tail = (tail + 1) % storage.length; - size++; + addUnsafe(e); return result; } @@ -203,9 +187,7 @@ public boolean offer(int e) { if (isFull()) { return false; } - storage[tail] = e; - tail = (tail + 1) % storage.length; - size++; + addUnsafe(e); return true; } @@ -217,10 +199,10 @@ public int[] remove(int count) { final int firstCopyLen = storage.length - head; if (tail >= head || firstCopyLen >= count) { - fastCopy(storage, head, result, 0, count); + System.arraycopy(storage, head, result, 0, count); } else { - fastCopy(storage, head, result, 0, firstCopyLen); - fastCopy(storage, 0, result, firstCopyLen, count - firstCopyLen); + System.arraycopy(storage, head, result, 0, firstCopyLen); + System.arraycopy(storage, 0, result, firstCopyLen, count - firstCopyLen); } head = (head + count) % storage.length; size -= count; @@ -231,10 +213,7 @@ public int remove() { if (isEmpty()) { throw new NoSuchElementException(); } - int e = storage[head]; - head = (head + 1) % storage.length; - size--; - return e; + return removeUnsafe(); } public int removeUnsafe() { @@ -248,10 +227,7 @@ public int poll(int onEmpty) { if (isEmpty()) { return onEmpty; } - int e = storage[head]; - head = (head + 1) % storage.length; - size--; - return e; + return removeUnsafe(); } public int element() { @@ -318,10 +294,10 @@ public int[] getAll() { int[] result = new int[size]; if (result.length > 0) { if (tail > head) { - fastCopy(storage, head, result, 0, tail - head); + System.arraycopy(storage, head, result, 0, tail - head); } else { - fastCopy(storage, head, result, 0, storage.length - head); - fastCopy(storage, 0, result, storage.length - head, tail); + System.arraycopy(storage, head, result, 0, storage.length - head); + System.arraycopy(storage, 0, result, storage.length - head, tail); } } return result; diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java index 8973d9c11fc..9dd76c0f5da 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java @@ -26,27 +26,13 @@ public class LongRingBuffer implements Serializable { protected long[] storage; protected int head, tail, size; - private void fastCopy(long[] src, int srcPos, long[] dest, int destPos, int length) { - if (length < SYSTEM_ARRAYCOPY_THRESHOLD) { - for (int ii = 0; ii < length; ii++) { - dest[destPos + ii] = src[srcPos + ii]; - } - } else { - System.arraycopy(src, srcPos, dest, 0, length); - } - } - private void grow(int increase) { if (growable) { // assert that we are not asking for the impossible Assert.eqTrue(ArrayUtil.MAX_ARRAY_SIZE - increase >= size, "LongRingBuffer size <= MAX_ARRAY_SIZE"); final int minLength = size + increase; - int newLength = storage.length * 2; - while (newLength < minLength) { - newLength = Math.min(newLength * 2, ArrayUtil.MAX_ARRAY_SIZE); - } - long[] newStorage = new long[newLength]; + long[] newStorage = new long[Integer.highestOneBit(minLength - 1) << 1]; // three scenarios: size is zero so nothing to copy, head is before tail so only one copy needed, head // after tail so two copies needed. Assuming that copying zero bytes is a fast operation, we will always @@ -55,9 +41,9 @@ private void grow(int increase) { // compute the size of the first copy final int firstCopyLen = Math.min(storage.length - head, size); - // do the copying - fastCopy(storage, head, newStorage, 0, firstCopyLen); - fastCopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); + // do the copying ( + System.arraycopy(storage, head, newStorage, 0, firstCopyLen); + System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); // reset the pointers tail = size; @@ -90,6 +76,8 @@ public LongRingBuffer(int capacity) { * @param growable whether to allow growth when the buffer is full. */ public LongRingBuffer(int capacity, boolean growable) { + Assert.eqTrue(capacity <= ArrayUtil.MAX_ARRAY_SIZE, "LongRingBuffer size <= MAX_ARRAY_SIZE"); + this.growable = growable; if (growable) { // use next larger power of 2 @@ -138,9 +126,7 @@ public boolean add(long e) { grow(); } } - storage[tail] = e; - tail = (tail + 1) % storage.length; - size++; + addUnsafe(e); return true; } @@ -186,9 +172,7 @@ public long addOverwrite(long e, long notFullResult) { if (isFull()) { result = remove(); } - storage[tail] = e; - tail = (tail + 1) % storage.length; - size++; + addUnsafe(e); return result; } @@ -203,9 +187,7 @@ public boolean offer(long e) { if (isFull()) { return false; } - storage[tail] = e; - tail = (tail + 1) % storage.length; - size++; + addUnsafe(e); return true; } @@ -217,10 +199,10 @@ public long[] remove(int count) { final int firstCopyLen = storage.length - head; if (tail >= head || firstCopyLen >= count) { - fastCopy(storage, head, result, 0, count); + System.arraycopy(storage, head, result, 0, count); } else { - fastCopy(storage, head, result, 0, firstCopyLen); - fastCopy(storage, 0, result, firstCopyLen, count - firstCopyLen); + System.arraycopy(storage, head, result, 0, firstCopyLen); + System.arraycopy(storage, 0, result, firstCopyLen, count - firstCopyLen); } head = (head + count) % storage.length; size -= count; @@ -231,10 +213,7 @@ public long remove() { if (isEmpty()) { throw new NoSuchElementException(); } - long e = storage[head]; - head = (head + 1) % storage.length; - size--; - return e; + return removeUnsafe(); } public long removeUnsafe() { @@ -248,10 +227,7 @@ public long poll(long onEmpty) { if (isEmpty()) { return onEmpty; } - long e = storage[head]; - head = (head + 1) % storage.length; - size--; - return e; + return removeUnsafe(); } public long element() { @@ -318,10 +294,10 @@ public long[] getAll() { long[] result = new long[size]; if (result.length > 0) { if (tail > head) { - fastCopy(storage, head, result, 0, tail - head); + System.arraycopy(storage, head, result, 0, tail - head); } else { - fastCopy(storage, head, result, 0, storage.length - head); - fastCopy(storage, 0, result, storage.length - head, tail); + System.arraycopy(storage, head, result, 0, storage.length - head); + System.arraycopy(storage, 0, result, storage.length - head, tail); } } return result; diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java index c83cc1d6f7c..5f69a540618 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java @@ -26,27 +26,13 @@ public class ShortRingBuffer implements Serializable { protected short[] storage; protected int head, tail, size; - private void fastCopy(short[] src, int srcPos, short[] dest, int destPos, int length) { - if (length < SYSTEM_ARRAYCOPY_THRESHOLD) { - for (int ii = 0; ii < length; ii++) { - dest[destPos + ii] = src[srcPos + ii]; - } - } else { - System.arraycopy(src, srcPos, dest, 0, length); - } - } - private void grow(int increase) { if (growable) { // assert that we are not asking for the impossible Assert.eqTrue(ArrayUtil.MAX_ARRAY_SIZE - increase >= size, "ShortRingBuffer size <= MAX_ARRAY_SIZE"); final int minLength = size + increase; - int newLength = storage.length * 2; - while (newLength < minLength) { - newLength = Math.min(newLength * 2, ArrayUtil.MAX_ARRAY_SIZE); - } - short[] newStorage = new short[newLength]; + short[] newStorage = new short[Integer.highestOneBit(minLength - 1) << 1]; // three scenarios: size is zero so nothing to copy, head is before tail so only one copy needed, head // after tail so two copies needed. Assuming that copying zero bytes is a fast operation, we will always @@ -55,9 +41,9 @@ private void grow(int increase) { // compute the size of the first copy final int firstCopyLen = Math.min(storage.length - head, size); - // do the copying - fastCopy(storage, head, newStorage, 0, firstCopyLen); - fastCopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); + // do the copying ( + System.arraycopy(storage, head, newStorage, 0, firstCopyLen); + System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); // reset the pointers tail = size; @@ -90,6 +76,8 @@ public ShortRingBuffer(int capacity) { * @param growable whether to allow growth when the buffer is full. */ public ShortRingBuffer(int capacity, boolean growable) { + Assert.eqTrue(capacity <= ArrayUtil.MAX_ARRAY_SIZE, "ShortRingBuffer size <= MAX_ARRAY_SIZE"); + this.growable = growable; if (growable) { // use next larger power of 2 @@ -138,9 +126,7 @@ public boolean add(short e) { grow(); } } - storage[tail] = e; - tail = (tail + 1) % storage.length; - size++; + addUnsafe(e); return true; } @@ -186,9 +172,7 @@ public short addOverwrite(short e, short notFullResult) { if (isFull()) { result = remove(); } - storage[tail] = e; - tail = (tail + 1) % storage.length; - size++; + addUnsafe(e); return result; } @@ -203,9 +187,7 @@ public boolean offer(short e) { if (isFull()) { return false; } - storage[tail] = e; - tail = (tail + 1) % storage.length; - size++; + addUnsafe(e); return true; } @@ -217,10 +199,10 @@ public short[] remove(int count) { final int firstCopyLen = storage.length - head; if (tail >= head || firstCopyLen >= count) { - fastCopy(storage, head, result, 0, count); + System.arraycopy(storage, head, result, 0, count); } else { - fastCopy(storage, head, result, 0, firstCopyLen); - fastCopy(storage, 0, result, firstCopyLen, count - firstCopyLen); + System.arraycopy(storage, head, result, 0, firstCopyLen); + System.arraycopy(storage, 0, result, firstCopyLen, count - firstCopyLen); } head = (head + count) % storage.length; size -= count; @@ -231,10 +213,7 @@ public short remove() { if (isEmpty()) { throw new NoSuchElementException(); } - short e = storage[head]; - head = (head + 1) % storage.length; - size--; - return e; + return removeUnsafe(); } public short removeUnsafe() { @@ -248,10 +227,7 @@ public short poll(short onEmpty) { if (isEmpty()) { return onEmpty; } - short e = storage[head]; - head = (head + 1) % storage.length; - size--; - return e; + return removeUnsafe(); } public short element() { @@ -318,10 +294,10 @@ public short[] getAll() { short[] result = new short[size]; if (result.length > 0) { if (tail > head) { - fastCopy(storage, head, result, 0, tail - head); + System.arraycopy(storage, head, result, 0, tail - head); } else { - fastCopy(storage, head, result, 0, storage.length - head); - fastCopy(storage, 0, result, storage.length - head, tail); + System.arraycopy(storage, head, result, 0, storage.length - head); + System.arraycopy(storage, 0, result, storage.length - head, tail); } } return result; diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/ByteRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/ByteRingBufferTest.java index 7fdaaf8990b..9cdf79989ff 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/ByteRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/ByteRingBufferTest.java @@ -11,12 +11,9 @@ import io.deephaven.base.ArrayUtil; import io.deephaven.base.verify.AssertionFailure; import junit.framework.TestCase; -import org.junit.Assert; -import org.junit.Test; import java.util.NoSuchElementException; -import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertThrows; public class ByteRingBufferTest extends TestCase { diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java index e3f2efcef53..7684180806b 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/CharRingBufferTest.java @@ -6,12 +6,9 @@ import io.deephaven.base.ArrayUtil; import io.deephaven.base.verify.AssertionFailure; import junit.framework.TestCase; -import org.junit.Assert; -import org.junit.Test; import java.util.NoSuchElementException; -import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertThrows; public class CharRingBufferTest extends TestCase { diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/DoubleRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/DoubleRingBufferTest.java index b7a082d310f..d41cb132ee2 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/DoubleRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/DoubleRingBufferTest.java @@ -11,12 +11,9 @@ import io.deephaven.base.ArrayUtil; import io.deephaven.base.verify.AssertionFailure; import junit.framework.TestCase; -import org.junit.Assert; -import org.junit.Test; import java.util.NoSuchElementException; -import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertThrows; public class DoubleRingBufferTest extends TestCase { diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/FloatRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/FloatRingBufferTest.java index 92b5f3e54d2..1a52a3d65f5 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/FloatRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/FloatRingBufferTest.java @@ -11,12 +11,9 @@ import io.deephaven.base.ArrayUtil; import io.deephaven.base.verify.AssertionFailure; import junit.framework.TestCase; -import org.junit.Assert; -import org.junit.Test; import java.util.NoSuchElementException; -import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertThrows; public class FloatRingBufferTest extends TestCase { diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/IntRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/IntRingBufferTest.java index e0388e537fd..4d91d290379 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/IntRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/IntRingBufferTest.java @@ -11,12 +11,9 @@ import io.deephaven.base.ArrayUtil; import io.deephaven.base.verify.AssertionFailure; import junit.framework.TestCase; -import org.junit.Assert; -import org.junit.Test; import java.util.NoSuchElementException; -import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertThrows; public class IntRingBufferTest extends TestCase { diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/LongRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/LongRingBufferTest.java index b72c910f55d..fbd8bcedf7e 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/LongRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/LongRingBufferTest.java @@ -11,12 +11,9 @@ import io.deephaven.base.ArrayUtil; import io.deephaven.base.verify.AssertionFailure; import junit.framework.TestCase; -import org.junit.Assert; -import org.junit.Test; import java.util.NoSuchElementException; -import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertThrows; public class LongRingBufferTest extends TestCase { diff --git a/Base/src/test/java/io/deephaven/base/ringbuffer/ShortRingBufferTest.java b/Base/src/test/java/io/deephaven/base/ringbuffer/ShortRingBufferTest.java index 28cda474a29..1dfd9ebc99c 100644 --- a/Base/src/test/java/io/deephaven/base/ringbuffer/ShortRingBufferTest.java +++ b/Base/src/test/java/io/deephaven/base/ringbuffer/ShortRingBufferTest.java @@ -11,12 +11,9 @@ import io.deephaven.base.ArrayUtil; import io.deephaven.base.verify.AssertionFailure; import junit.framework.TestCase; -import org.junit.Assert; -import org.junit.Test; import java.util.NoSuchElementException; -import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertThrows; public class ShortRingBufferTest extends TestCase { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java index 20c7df8ba7f..671ed84c02f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java @@ -10,11 +10,11 @@ import io.deephaven.base.ArrayUtil; import io.deephaven.base.verify.Assert; +import io.deephaven.base.verify.AssertionFailure; import io.deephaven.chunk.WritableDoubleChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.*; import io.deephaven.util.SafeCloseable; -import org.apache.commons.lang3.mutable.MutableInt; +import io.deephaven.util.annotations.VisibleForTesting; import java.util.NoSuchElementException; @@ -35,7 +35,6 @@ public class PairwiseDoubleRingBuffer implements SafeCloseable { // use a sized double chunk for underlying storage private WritableDoubleChunk storageChunk; - private final DoubleFunction pairwiseFunction; private final double emptyVal; @@ -106,7 +105,8 @@ private void evaluateRangeFast(int start, int end) { } } - private void evaluateTree(int startA, int endA) { + @VisibleForTesting + public double evaluateTree(int startA, int endA) { while (endA > 1) { // compute this level evaluateRangeFast(startA, endA); @@ -115,14 +115,15 @@ private void evaluateTree(int startA, int endA) { startA /= 2; endA /= 2; } + return storageChunk.get(endA); } - private void evaluateTree(int startA, int endA, int startB, int endB) { + @VisibleForTesting + public double evaluateTree(int startA, int endA, int startB, int endB) { while (endB > 1) { if (endA >= startB - 1) { // all collapse together into a single range - evaluateTree(startA, endB); - return; + return evaluateTree(startA, endB); } else { // compute this level evaluateRangeFast(startA, endA); @@ -135,22 +136,21 @@ private void evaluateTree(int startA, int endA, int startB, int endB) { endB /= 2; } } + throw new AssertionFailure("should never reach this line"); } - private void evaluateTree(int startA, int endA, int startB, int endB, int startC, int endC) { + @VisibleForTesting + public double evaluateTree(int startA, int endA, int startB, int endB, int startC, int endC) { while (endC > 1) { if (endA >= startC - 1 || (endA >= startB - 1 && endB >= startC - 1)) { // all collapse together into a single range - evaluateTree(startA, endC); - return; + return evaluateTree(startA, endC); } else if (endA >= startB - 1) { // A and B collapse - evaluateTree(startA, endB, startC, endC); - return; + return evaluateTree(startA, endB, startC, endC); } else if (endB >= startC - 1) { // B and C collapse - evaluateTree(startA, endA, startB, endC); - return; + return evaluateTree(startA, endA, startB, endC); } else { // no collapse evaluateRangeFast(startA, endA); @@ -166,6 +166,7 @@ private void evaluateTree(int startA, int endA, int startB, int endB, int startC endC /= 2; } } + throw new AssertionFailure("should never reach this line"); } public double evaluate() { @@ -176,49 +177,49 @@ public double evaluate() { return emptyVal; } - if (!pushDirty && !popDirty) { - return storageChunk.get(1); - } - // This is a nested complex set of `if` statements that are used to set the correct and minimal initial // conditions for the evaluation. The calls to evaluateTree recurse no more than twice (as the ranges // overlap and the calculation simplifies). + final double value; + if (pushDirty && popDirty) { if (dirtyPushHead > dirtyPushTail && dirtyPopHead > dirtyPopTail) { // both are wrapped - evaluateTree(capacity, Math.max(dirtyPushTail, dirtyPopTail), Math.min(dirtyPushHead, dirtyPopHead), chunkSize - 1); + value = evaluateTree(capacity, Math.max(dirtyPushTail, dirtyPopTail), Math.min(dirtyPushHead, dirtyPopHead), chunkSize - 1); } else if (dirtyPushHead > dirtyPushTail) { // push wrapped, pop is not - evaluateTree(capacity, dirtyPushTail, dirtyPopHead, dirtyPopTail, dirtyPushHead, chunkSize - 1); - } else if (dirtyPushHead > dirtyPushTail) { + value = evaluateTree(capacity, dirtyPushTail, dirtyPopHead, dirtyPopTail, dirtyPushHead, chunkSize - 1); + } else if (dirtyPopHead > dirtyPopTail) { // pop wrapped, push is not - evaluateTree(capacity, dirtyPopTail, dirtyPushHead, dirtyPushTail, dirtyPopHead, chunkSize - 1); + value = evaluateTree(capacity, dirtyPopTail, dirtyPushHead, dirtyPushTail, dirtyPopHead, chunkSize - 1); } else { // neither wrapped if (dirtyPushHead > dirtyPopHead) { - evaluateTree(dirtyPopHead, dirtyPopTail, dirtyPushHead, dirtyPushTail); + value = evaluateTree(dirtyPopHead, dirtyPopTail, dirtyPushHead, dirtyPushTail); } else { - evaluateTree(dirtyPushHead, dirtyPushTail, dirtyPopHead, dirtyPopTail); + value = evaluateTree(dirtyPushHead, dirtyPushTail, dirtyPopHead, dirtyPopTail); } } } else if (pushDirty) { if (dirtyPushHead > dirtyPushTail) { - evaluateTree(capacity, dirtyPushTail, dirtyPushHead, chunkSize - 1); + value = evaluateTree(capacity, dirtyPushTail, dirtyPushHead, chunkSize - 1); } else { - evaluateTree(dirtyPushHead, dirtyPushTail); + value = evaluateTree(dirtyPushHead, dirtyPushTail); } } else if (popDirty) { if (dirtyPopHead > dirtyPopTail) { - evaluateTree(capacity, dirtyPopTail, dirtyPopHead, chunkSize - 1); + value = evaluateTree(capacity, dirtyPopTail, dirtyPopHead, chunkSize - 1); } else { - evaluateTree(dirtyPopHead, dirtyPopTail); + value = evaluateTree(dirtyPopHead, dirtyPopTail); } + } else { + value = storageChunk.get(1); } clearDirty(); - return storageChunk.get(1); + return value; } private void grow(int increase) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java index c7d9ee9ed9d..29ed42e29ea 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java @@ -5,9 +5,11 @@ import io.deephaven.base.ArrayUtil; import io.deephaven.base.verify.Assert; +import io.deephaven.base.verify.AssertionFailure; import io.deephaven.chunk.WritableFloatChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.util.SafeCloseable; +import io.deephaven.util.annotations.VisibleForTesting; import java.util.NoSuchElementException; @@ -28,7 +30,6 @@ public class PairwiseFloatRingBuffer implements SafeCloseable { // use a sized float chunk for underlying storage private WritableFloatChunk storageChunk; - private final FloatFunction pairwiseFunction; private final float emptyVal; @@ -99,7 +100,8 @@ private void evaluateRangeFast(int start, int end) { } } - private void evaluateTree(int startA, int endA) { + @VisibleForTesting + public float evaluateTree(int startA, int endA) { while (endA > 1) { // compute this level evaluateRangeFast(startA, endA); @@ -108,14 +110,15 @@ private void evaluateTree(int startA, int endA) { startA /= 2; endA /= 2; } + return storageChunk.get(endA); } - private void evaluateTree(int startA, int endA, int startB, int endB) { + @VisibleForTesting + public float evaluateTree(int startA, int endA, int startB, int endB) { while (endB > 1) { if (endA >= startB - 1) { // all collapse together into a single range - evaluateTree(startA, endB); - return; + return evaluateTree(startA, endB); } else { // compute this level evaluateRangeFast(startA, endA); @@ -128,22 +131,21 @@ private void evaluateTree(int startA, int endA, int startB, int endB) { endB /= 2; } } + throw new AssertionFailure("should never reach this line"); } - private void evaluateTree(int startA, int endA, int startB, int endB, int startC, int endC) { + @VisibleForTesting + public float evaluateTree(int startA, int endA, int startB, int endB, int startC, int endC) { while (endC > 1) { if (endA >= startC - 1 || (endA >= startB - 1 && endB >= startC - 1)) { // all collapse together into a single range - evaluateTree(startA, endC); - return; + return evaluateTree(startA, endC); } else if (endA >= startB - 1) { // A and B collapse - evaluateTree(startA, endB, startC, endC); - return; + return evaluateTree(startA, endB, startC, endC); } else if (endB >= startC - 1) { // B and C collapse - evaluateTree(startA, endA, startB, endC); - return; + return evaluateTree(startA, endA, startB, endC); } else { // no collapse evaluateRangeFast(startA, endA); @@ -159,6 +161,7 @@ private void evaluateTree(int startA, int endA, int startB, int endB, int startC endC /= 2; } } + throw new AssertionFailure("should never reach this line"); } public float evaluate() { @@ -169,49 +172,49 @@ public float evaluate() { return emptyVal; } - if (!pushDirty && !popDirty) { - return storageChunk.get(1); - } - // This is a nested complex set of `if` statements that are used to set the correct and minimal initial // conditions for the evaluation. The calls to evaluateTree recurse no more than twice (as the ranges // overlap and the calculation simplifies). + final float value; + if (pushDirty && popDirty) { if (dirtyPushHead > dirtyPushTail && dirtyPopHead > dirtyPopTail) { // both are wrapped - evaluateTree(capacity, Math.max(dirtyPushTail, dirtyPopTail), Math.min(dirtyPushHead, dirtyPopHead), chunkSize - 1); + value = evaluateTree(capacity, Math.max(dirtyPushTail, dirtyPopTail), Math.min(dirtyPushHead, dirtyPopHead), chunkSize - 1); } else if (dirtyPushHead > dirtyPushTail) { // push wrapped, pop is not - evaluateTree(capacity, dirtyPushTail, dirtyPopHead, dirtyPopTail, dirtyPushHead, chunkSize - 1); + value = evaluateTree(capacity, dirtyPushTail, dirtyPopHead, dirtyPopTail, dirtyPushHead, chunkSize - 1); } else if (dirtyPopHead > dirtyPopTail) { // pop wrapped, push is not - evaluateTree(capacity, dirtyPopTail, dirtyPushHead, dirtyPushTail, dirtyPopHead, chunkSize - 1); + value = evaluateTree(capacity, dirtyPopTail, dirtyPushHead, dirtyPushTail, dirtyPopHead, chunkSize - 1); } else { // neither wrapped if (dirtyPushHead > dirtyPopHead) { - evaluateTree(dirtyPopHead, dirtyPopTail, dirtyPushHead, dirtyPushTail); + value = evaluateTree(dirtyPopHead, dirtyPopTail, dirtyPushHead, dirtyPushTail); } else { - evaluateTree(dirtyPushHead, dirtyPushTail, dirtyPopHead, dirtyPopTail); + value = evaluateTree(dirtyPushHead, dirtyPushTail, dirtyPopHead, dirtyPopTail); } } } else if (pushDirty) { if (dirtyPushHead > dirtyPushTail) { - evaluateTree(capacity, dirtyPushTail, dirtyPushHead, chunkSize - 1); + value = evaluateTree(capacity, dirtyPushTail, dirtyPushHead, chunkSize - 1); } else { - evaluateTree(dirtyPushHead, dirtyPushTail); + value = evaluateTree(dirtyPushHead, dirtyPushTail); } } else if (popDirty) { if (dirtyPopHead > dirtyPopTail) { - evaluateTree(capacity, dirtyPopTail, dirtyPopHead, chunkSize - 1); + value = evaluateTree(capacity, dirtyPopTail, dirtyPopHead, chunkSize - 1); } else { - evaluateTree(dirtyPopHead, dirtyPopTail); + value = evaluateTree(dirtyPopHead, dirtyPopTail); } + } else { + value = storageChunk.get(1); } clearDirty(); - return storageChunk.get(1); + return value; } private void grow(int increase) { diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBufferTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBufferTest.java index ccea218d90e..2986d50142d 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBufferTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBufferTest.java @@ -96,8 +96,8 @@ public void testAddRemove() { assertEquals(rb.front(0), A); assertEquals(rb.front(1), B); assertEquals(rb.front(2), C); - assertEquals(rb.back(),C); - assertEquals(rb.peekBack(NULL_DOUBLE),C); + assertEquals(rb.back(), C); + assertEquals(rb.peekBack(NULL_DOUBLE), C); assertRemove(rb, 3, A); assertRemove(rb, 2, B); @@ -149,42 +149,42 @@ public void testAddRemove() { assertEmpty(rb); } - public void testGrowSimple() { - PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(5, NULL_DOUBLE, Double::min); - - assertAdd(rb, A, 1, A); - assertAdd(rb, B, 2, A); - assertAdd(rb, C, 3, A); - assertAdd(rb, D, 4, A); - assertAdd(rb, E, 5, A); - - // this will grow; the elements are in a single contiguous block - assertAdd(rb, F, 6, A); - - assertRemove(rb, 6, A); - assertRemove(rb, 5, B); - assertRemove(rb, 4, C); - assertRemove(rb, 3, D); - assertRemove(rb, 2, E); - assertRemove(rb, 1, F); - assertEmpty(rb); - - rb.pushEmptyValue(); - assertEquals(rb.front(), NULL_DOUBLE); - try { - rb.front(-1); - fail("expected a NoSuchElement exception"); - } catch (NoSuchElementException x) { - // expected - } - try { - rb.front(5); - fail("expected a NoSuchElement exception"); - } catch (NoSuchElementException x) { - // expected - } - assertEquals(rb.poll(0.0f), NULL_DOUBLE); - } + public void testGrowSimple() { + PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(5, NULL_DOUBLE, Double::min); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + + // this will grow; the elements are in a single contiguous block + assertAdd(rb, F, 6, A); + + assertRemove(rb, 6, A); + assertRemove(rb, 5, B); + assertRemove(rb, 4, C); + assertRemove(rb, 3, D); + assertRemove(rb, 2, E); + assertRemove(rb, 1, F); + assertEmpty(rb); + + rb.pushEmptyValue(); + assertEquals(rb.front(), NULL_DOUBLE); + try { + rb.front(-1); + fail("expected a NoSuchElement exception"); + } catch (NoSuchElementException x) { + // expected + } + try { + rb.front(5); + fail("expected a NoSuchElement exception"); + } catch (NoSuchElementException x) { + // expected + } + assertEquals(rb.poll(0.0f), NULL_DOUBLE); + } public void testGrowComplex() { PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(5, NULL_DOUBLE, Double::min); @@ -235,9 +235,9 @@ public void testWhenEmpty() { } catch (NoSuchElementException x) { // expected } - assertEquals(rb.poll((double)-1), (double)-1); - assertEquals(rb.peek((double)-1), (double)-1); - assertEquals(rb.peekBack((double)-1), (double)-1); + assertEquals(rb.poll((double) -1), (double) -1); + assertEquals(rb.peek((double) -1), (double) -1); + assertEquals(rb.peekBack((double) -1), (double) -1); } public void testBack() { @@ -342,20 +342,24 @@ public void testEvaluateSumLargeAmounts() { } } + /*** + * Return the sum of 0 to N-1 + */ + private double sum0toN(double n) { + return ((double) n * (double) (n - 1) / (double) 2); + } + public void testEvaluationEdgeCase() { - try (final PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(3, -Double.MAX_VALUE, Double::max)) { - // move the head and tail off zero - for (double i = 0; i < 500; i++) { - rb.push(i); - } - for (double i = 0; i < 500; i++) { - rb.pop(); + try (final PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(512, (double)0, Double::sum)) { + for (double i = 0; i < 512; i++) { + rb.pushUnsafe(i); } + // eval single range (internal collapse expected) + assertEquals(sum0toN(512), rb.evaluateTree(512, 1023)); + assertEquals(sum0toN(512), rb.evaluateTree(512, 600, 601, 1023)); + assertEquals(sum0toN(512), rb.evaluateTree(512, 600, 601, 700, 701, 1023)); - for (double i = 0; i < 100; i++) { - rb.push(i); - } - assertEquals((double)99, rb.evaluate()); // last value added is max + //TODO: more edge cases } } @@ -373,30 +377,26 @@ public void testPushPopUnsafe() { // do it again with an offset rb.ensureRemaining(500); for (double i = 0; i < 500; i++) { - rb.pushUnsafe(i + (double)1000); + rb.pushUnsafe(i + (double) 1000); } for (double i = 0; i < 500; i++) { - assertEquals(rb.popUnsafe(), i + (double)1000); + assertEquals(rb.popUnsafe(), i + (double) 1000); } for (double i = 0; i < 500; i++) { - rb.pushUnsafe(i + (double)1000); + rb.pushUnsafe(i + (double) 1000); } rb.clear(); for (double i = 0; i < 100; i++) { rb.push(i); } - assertEquals((double)99, rb.evaluate()); // last value added is max + assertEquals((double) 99, rb.evaluate()); // last value added is max } } - private double sum1toN(double n) { - return ((double)n * (double)(n+1) / (double)2); - } - public void testPopMultiple() { - try (final PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(3, (double)0, Double::sum)) { + try (final PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(3, (double) 0, Double::sum)) { for (int step = 0; step < 10; step++) { rb.ensureRemaining(500); @@ -416,5 +416,4 @@ public void testPopMultiple() { } } } - } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBufferTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBufferTest.java index 97140fda810..cc26f6df15c 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBufferTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBufferTest.java @@ -91,8 +91,8 @@ public void testAddRemove() { assertEquals(rb.front(0), A); assertEquals(rb.front(1), B); assertEquals(rb.front(2), C); - assertEquals(rb.back(),C); - assertEquals(rb.peekBack(NULL_FLOAT),C); + assertEquals(rb.back(), C); + assertEquals(rb.peekBack(NULL_FLOAT), C); assertRemove(rb, 3, A); assertRemove(rb, 2, B); @@ -144,42 +144,42 @@ public void testAddRemove() { assertEmpty(rb); } - public void testGrowSimple() { - PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(5, NULL_FLOAT, Float::min); - - assertAdd(rb, A, 1, A); - assertAdd(rb, B, 2, A); - assertAdd(rb, C, 3, A); - assertAdd(rb, D, 4, A); - assertAdd(rb, E, 5, A); - - // this will grow; the elements are in a single contiguous block - assertAdd(rb, F, 6, A); - - assertRemove(rb, 6, A); - assertRemove(rb, 5, B); - assertRemove(rb, 4, C); - assertRemove(rb, 3, D); - assertRemove(rb, 2, E); - assertRemove(rb, 1, F); - assertEmpty(rb); - - rb.pushEmptyValue(); - assertEquals(rb.front(), NULL_FLOAT); - try { - rb.front(-1); - fail("expected a NoSuchElement exception"); - } catch (NoSuchElementException x) { - // expected - } - try { - rb.front(5); - fail("expected a NoSuchElement exception"); - } catch (NoSuchElementException x) { - // expected - } - assertEquals(rb.poll(0.0f), NULL_FLOAT); - } + public void testGrowSimple() { + PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(5, NULL_FLOAT, Float::min); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + + // this will grow; the elements are in a single contiguous block + assertAdd(rb, F, 6, A); + + assertRemove(rb, 6, A); + assertRemove(rb, 5, B); + assertRemove(rb, 4, C); + assertRemove(rb, 3, D); + assertRemove(rb, 2, E); + assertRemove(rb, 1, F); + assertEmpty(rb); + + rb.pushEmptyValue(); + assertEquals(rb.front(), NULL_FLOAT); + try { + rb.front(-1); + fail("expected a NoSuchElement exception"); + } catch (NoSuchElementException x) { + // expected + } + try { + rb.front(5); + fail("expected a NoSuchElement exception"); + } catch (NoSuchElementException x) { + // expected + } + assertEquals(rb.poll(0.0f), NULL_FLOAT); + } public void testGrowComplex() { PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(5, NULL_FLOAT, Float::min); @@ -230,9 +230,9 @@ public void testWhenEmpty() { } catch (NoSuchElementException x) { // expected } - assertEquals(rb.poll((float)-1), (float)-1); - assertEquals(rb.peek((float)-1), (float)-1); - assertEquals(rb.peekBack((float)-1), (float)-1); + assertEquals(rb.poll((float) -1), (float) -1); + assertEquals(rb.peek((float) -1), (float) -1); + assertEquals(rb.peekBack((float) -1), (float) -1); } public void testBack() { @@ -337,20 +337,24 @@ public void testEvaluateSumLargeAmounts() { } } + /*** + * Return the sum of 0 to N-1 + */ + private float sum0toN(float n) { + return ((float) n * (float) (n - 1) / (float) 2); + } + public void testEvaluationEdgeCase() { - try (final PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(3, -Float.MAX_VALUE, Float::max)) { - // move the head and tail off zero - for (float i = 0; i < 500; i++) { - rb.push(i); - } - for (float i = 0; i < 500; i++) { - rb.pop(); + try (final PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(512, (float)0, Float::sum)) { + for (float i = 0; i < 512; i++) { + rb.pushUnsafe(i); } + // eval single range (internal collapse expected) + assertEquals(sum0toN(512), rb.evaluateTree(512, 1023)); + assertEquals(sum0toN(512), rb.evaluateTree(512, 600, 601, 1023)); + assertEquals(sum0toN(512), rb.evaluateTree(512, 600, 601, 700, 701, 1023)); - for (float i = 0; i < 100; i++) { - rb.push(i); - } - assertEquals((float)99, rb.evaluate()); // last value added is max + //TODO: more edge cases } } @@ -368,30 +372,26 @@ public void testPushPopUnsafe() { // do it again with an offset rb.ensureRemaining(500); for (float i = 0; i < 500; i++) { - rb.pushUnsafe(i + (float)1000); + rb.pushUnsafe(i + (float) 1000); } for (float i = 0; i < 500; i++) { - assertEquals(rb.popUnsafe(), i + (float)1000); + assertEquals(rb.popUnsafe(), i + (float) 1000); } for (float i = 0; i < 500; i++) { - rb.pushUnsafe(i + (float)1000); + rb.pushUnsafe(i + (float) 1000); } rb.clear(); for (float i = 0; i < 100; i++) { rb.push(i); } - assertEquals((float)99, rb.evaluate()); // last value added is max + assertEquals((float) 99, rb.evaluate()); // last value added is max } } - private float sum1toN(float n) { - return ((float)n * (float)(n+1) / (float)2); - } - public void testPopMultiple() { - try (final PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(3, (float)0, Float::sum)) { + try (final PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(3, (float) 0, Float::sum)) { for (int step = 0; step < 10; step++) { rb.ensureRemaining(500); @@ -411,5 +411,4 @@ public void testPopMultiple() { } } } - } From 20d15c524850957bc9f3051eb2168fdcd05552db Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 24 Jan 2023 19:02:24 -0800 Subject: [PATCH 094/123] More cleanup on RingBuffer/PairwiseBuffer classes --- .../base/ringbuffer/ByteRingBuffer.java | 12 +- .../base/ringbuffer/CharRingBuffer.java | 12 +- .../base/ringbuffer/DoubleRingBuffer.java | 12 +- .../base/ringbuffer/FloatRingBuffer.java | 12 +- .../base/ringbuffer/IntRingBuffer.java | 12 +- .../base/ringbuffer/LongRingBuffer.java | 12 +- .../base/ringbuffer/ShortRingBuffer.java | 12 +- .../internal/PairwiseDoubleRingBuffer.java | 37 +- .../internal/PairwiseFloatRingBuffer.java | 37 +- .../PairwiseDoubleRingBufferTest.java | 480 +++++++++++------- .../internal/PairwiseFloatRingBufferTest.java | 480 +++++++++++------- 11 files changed, 680 insertions(+), 438 deletions(-) diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java index fffbd689417..2dee7736bb6 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/ByteRingBuffer.java @@ -21,7 +21,6 @@ * buffer instead. */ public class ByteRingBuffer implements Serializable { - private static int SYSTEM_ARRAYCOPY_THRESHOLD = 16; protected final boolean growable; protected byte[] storage; protected int head, tail, size; @@ -31,17 +30,18 @@ private void grow(int increase) { // assert that we are not asking for the impossible Assert.eqTrue(ArrayUtil.MAX_ARRAY_SIZE - increase >= size, "ByteRingBuffer size <= MAX_ARRAY_SIZE"); - final int minLength = size + increase; - byte[] newStorage = new byte[Integer.highestOneBit(minLength - 1) << 1]; + // make sure we cap out at ArrayUtil.MAX_ARRAY_SIZE + final int newLength = + Math.toIntExact(Math.min(ArrayUtil.MAX_ARRAY_SIZE, Long.highestOneBit(size + increase - 1) << 1)); + byte[] newStorage = new byte[newLength]; // three scenarios: size is zero so nothing to copy, head is before tail so only one copy needed, head - // after tail so two copies needed. Assuming that copying zero bytes is a fast operation, we will always - // make two calls for simplicity and branch-prediction friendliness. + // after tail so two copies needed. Make two calls for simplicity and branch-prediction friendliness. // compute the size of the first copy final int firstCopyLen = Math.min(storage.length - head, size); - // do the copying ( + // do the copying System.arraycopy(storage, head, newStorage, 0, firstCopyLen); System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java index 740bd31c7d4..94a1c8af1ff 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/CharRingBuffer.java @@ -16,7 +16,6 @@ * buffer instead. */ public class CharRingBuffer implements Serializable { - private static int SYSTEM_ARRAYCOPY_THRESHOLD = 16; protected final boolean growable; protected char[] storage; protected int head, tail, size; @@ -26,17 +25,18 @@ private void grow(int increase) { // assert that we are not asking for the impossible Assert.eqTrue(ArrayUtil.MAX_ARRAY_SIZE - increase >= size, "CharRingBuffer size <= MAX_ARRAY_SIZE"); - final int minLength = size + increase; - char[] newStorage = new char[Integer.highestOneBit(minLength - 1) << 1]; + // make sure we cap out at ArrayUtil.MAX_ARRAY_SIZE + final int newLength = + Math.toIntExact(Math.min(ArrayUtil.MAX_ARRAY_SIZE, Long.highestOneBit(size + increase - 1) << 1)); + char[] newStorage = new char[newLength]; // three scenarios: size is zero so nothing to copy, head is before tail so only one copy needed, head - // after tail so two copies needed. Assuming that copying zero bytes is a fast operation, we will always - // make two calls for simplicity and branch-prediction friendliness. + // after tail so two copies needed. Make two calls for simplicity and branch-prediction friendliness. // compute the size of the first copy final int firstCopyLen = Math.min(storage.length - head, size); - // do the copying ( + // do the copying System.arraycopy(storage, head, newStorage, 0, firstCopyLen); System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java index d4372bcfecc..54bdbb1d89d 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/DoubleRingBuffer.java @@ -21,7 +21,6 @@ * buffer instead. */ public class DoubleRingBuffer implements Serializable { - private static int SYSTEM_ARRAYCOPY_THRESHOLD = 16; protected final boolean growable; protected double[] storage; protected int head, tail, size; @@ -31,17 +30,18 @@ private void grow(int increase) { // assert that we are not asking for the impossible Assert.eqTrue(ArrayUtil.MAX_ARRAY_SIZE - increase >= size, "DoubleRingBuffer size <= MAX_ARRAY_SIZE"); - final int minLength = size + increase; - double[] newStorage = new double[Integer.highestOneBit(minLength - 1) << 1]; + // make sure we cap out at ArrayUtil.MAX_ARRAY_SIZE + final int newLength = + Math.toIntExact(Math.min(ArrayUtil.MAX_ARRAY_SIZE, Long.highestOneBit(size + increase - 1) << 1)); + double[] newStorage = new double[newLength]; // three scenarios: size is zero so nothing to copy, head is before tail so only one copy needed, head - // after tail so two copies needed. Assuming that copying zero bytes is a fast operation, we will always - // make two calls for simplicity and branch-prediction friendliness. + // after tail so two copies needed. Make two calls for simplicity and branch-prediction friendliness. // compute the size of the first copy final int firstCopyLen = Math.min(storage.length - head, size); - // do the copying ( + // do the copying System.arraycopy(storage, head, newStorage, 0, firstCopyLen); System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java index 6c35f733ac3..ea35c8879b7 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/FloatRingBuffer.java @@ -21,7 +21,6 @@ * buffer instead. */ public class FloatRingBuffer implements Serializable { - private static int SYSTEM_ARRAYCOPY_THRESHOLD = 16; protected final boolean growable; protected float[] storage; protected int head, tail, size; @@ -31,17 +30,18 @@ private void grow(int increase) { // assert that we are not asking for the impossible Assert.eqTrue(ArrayUtil.MAX_ARRAY_SIZE - increase >= size, "FloatRingBuffer size <= MAX_ARRAY_SIZE"); - final int minLength = size + increase; - float[] newStorage = new float[Integer.highestOneBit(minLength - 1) << 1]; + // make sure we cap out at ArrayUtil.MAX_ARRAY_SIZE + final int newLength = + Math.toIntExact(Math.min(ArrayUtil.MAX_ARRAY_SIZE, Long.highestOneBit(size + increase - 1) << 1)); + float[] newStorage = new float[newLength]; // three scenarios: size is zero so nothing to copy, head is before tail so only one copy needed, head - // after tail so two copies needed. Assuming that copying zero bytes is a fast operation, we will always - // make two calls for simplicity and branch-prediction friendliness. + // after tail so two copies needed. Make two calls for simplicity and branch-prediction friendliness. // compute the size of the first copy final int firstCopyLen = Math.min(storage.length - head, size); - // do the copying ( + // do the copying System.arraycopy(storage, head, newStorage, 0, firstCopyLen); System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java index cdd08786569..f049531e9c1 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/IntRingBuffer.java @@ -21,7 +21,6 @@ * buffer instead. */ public class IntRingBuffer implements Serializable { - private static int SYSTEM_ARRAYCOPY_THRESHOLD = 16; protected final boolean growable; protected int[] storage; protected int head, tail, size; @@ -31,17 +30,18 @@ private void grow(int increase) { // assert that we are not asking for the impossible Assert.eqTrue(ArrayUtil.MAX_ARRAY_SIZE - increase >= size, "IntRingBuffer size <= MAX_ARRAY_SIZE"); - final int minLength = size + increase; - int[] newStorage = new int[Integer.highestOneBit(minLength - 1) << 1]; + // make sure we cap out at ArrayUtil.MAX_ARRAY_SIZE + final int newLength = + Math.toIntExact(Math.min(ArrayUtil.MAX_ARRAY_SIZE, Long.highestOneBit(size + increase - 1) << 1)); + int[] newStorage = new int[newLength]; // three scenarios: size is zero so nothing to copy, head is before tail so only one copy needed, head - // after tail so two copies needed. Assuming that copying zero bytes is a fast operation, we will always - // make two calls for simplicity and branch-prediction friendliness. + // after tail so two copies needed. Make two calls for simplicity and branch-prediction friendliness. // compute the size of the first copy final int firstCopyLen = Math.min(storage.length - head, size); - // do the copying ( + // do the copying System.arraycopy(storage, head, newStorage, 0, firstCopyLen); System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java index 9dd76c0f5da..e4580514ed1 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/LongRingBuffer.java @@ -21,7 +21,6 @@ * buffer instead. */ public class LongRingBuffer implements Serializable { - private static int SYSTEM_ARRAYCOPY_THRESHOLD = 16; protected final boolean growable; protected long[] storage; protected int head, tail, size; @@ -31,17 +30,18 @@ private void grow(int increase) { // assert that we are not asking for the impossible Assert.eqTrue(ArrayUtil.MAX_ARRAY_SIZE - increase >= size, "LongRingBuffer size <= MAX_ARRAY_SIZE"); - final int minLength = size + increase; - long[] newStorage = new long[Integer.highestOneBit(minLength - 1) << 1]; + // make sure we cap out at ArrayUtil.MAX_ARRAY_SIZE + final int newLength = + Math.toIntExact(Math.min(ArrayUtil.MAX_ARRAY_SIZE, Long.highestOneBit(size + increase - 1) << 1)); + long[] newStorage = new long[newLength]; // three scenarios: size is zero so nothing to copy, head is before tail so only one copy needed, head - // after tail so two copies needed. Assuming that copying zero bytes is a fast operation, we will always - // make two calls for simplicity and branch-prediction friendliness. + // after tail so two copies needed. Make two calls for simplicity and branch-prediction friendliness. // compute the size of the first copy final int firstCopyLen = Math.min(storage.length - head, size); - // do the copying ( + // do the copying System.arraycopy(storage, head, newStorage, 0, firstCopyLen); System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); diff --git a/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java b/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java index 5f69a540618..ad88fc20a98 100644 --- a/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java +++ b/Base/src/main/java/io/deephaven/base/ringbuffer/ShortRingBuffer.java @@ -21,7 +21,6 @@ * buffer instead. */ public class ShortRingBuffer implements Serializable { - private static int SYSTEM_ARRAYCOPY_THRESHOLD = 16; protected final boolean growable; protected short[] storage; protected int head, tail, size; @@ -31,17 +30,18 @@ private void grow(int increase) { // assert that we are not asking for the impossible Assert.eqTrue(ArrayUtil.MAX_ARRAY_SIZE - increase >= size, "ShortRingBuffer size <= MAX_ARRAY_SIZE"); - final int minLength = size + increase; - short[] newStorage = new short[Integer.highestOneBit(minLength - 1) << 1]; + // make sure we cap out at ArrayUtil.MAX_ARRAY_SIZE + final int newLength = + Math.toIntExact(Math.min(ArrayUtil.MAX_ARRAY_SIZE, Long.highestOneBit(size + increase - 1) << 1)); + short[] newStorage = new short[newLength]; // three scenarios: size is zero so nothing to copy, head is before tail so only one copy needed, head - // after tail so two copies needed. Assuming that copying zero bytes is a fast operation, we will always - // make two calls for simplicity and branch-prediction friendliness. + // after tail so two copies needed. Make two calls for simplicity and branch-prediction friendliness. // compute the size of the first copy final int firstCopyLen = Math.min(storage.length - head, size); - // do the copying ( + // do the copying System.arraycopy(storage, head, newStorage, 0, firstCopyLen); System.arraycopy(storage, 0, newStorage, firstCopyLen, size - firstCopyLen); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java index 671ed84c02f..6f494c256ac 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java @@ -8,9 +8,7 @@ */ package io.deephaven.engine.table.impl.updateby.internal; -import io.deephaven.base.ArrayUtil; import io.deephaven.base.verify.Assert; -import io.deephaven.base.verify.AssertionFailure; import io.deephaven.chunk.WritableDoubleChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.util.SafeCloseable; @@ -31,7 +29,10 @@ */ public class PairwiseDoubleRingBuffer implements SafeCloseable { - private static final int PAIRWISE_MAX_CAPACITY = ArrayUtil.MAX_ARRAY_SIZE / 2; + /** We are limited to a buffer size exactly a power of two. Since we store the values and the pairwise tree in the + * same buffer, we are limited to a max capacity of 2^29 (500M) entries + */ + private static final int PAIRWISE_MAX_CAPACITY = 1 << 29; // use a sized double chunk for underlying storage private WritableDoubleChunk storageChunk; @@ -76,7 +77,7 @@ public interface DoubleFunction { * result is available */ public PairwiseDoubleRingBuffer(int initialSize, double emptyVal, DoubleFunction pairwiseFunction) { - Assert.eqTrue(PAIRWISE_MAX_CAPACITY >= initialSize, "PairwiseDoubleRingBuffer initialSize <= PAIRWISE_MAX_CAPACITY"); + Assert.eqTrue(initialSize <= PAIRWISE_MAX_CAPACITY, "PairwiseDoubleRingBuffer initialSize <= PAIRWISE_MAX_CAPACITY"); // use next larger power of 2 this.capacity = Integer.highestOneBit(initialSize - 1) << 1; @@ -85,7 +86,6 @@ public PairwiseDoubleRingBuffer(int initialSize, double emptyVal, DoubleFunction this.pairwiseFunction = pairwiseFunction; this.emptyVal = emptyVal; - storageChunk.fillWithValue(0, chunkSize, emptyVal); clear(); } @@ -136,7 +136,7 @@ public double evaluateTree(int startA, int endA, int startB, int endB) { endB /= 2; } } - throw new AssertionFailure("should never reach this line"); + throw Assert.statementNeverExecuted(); } @VisibleForTesting @@ -166,17 +166,13 @@ public double evaluateTree(int startA, int endA, int startB, int endB, int start endC /= 2; } } - throw new AssertionFailure("should never reach this line"); + throw Assert.statementNeverExecuted(); } public double evaluate() { final boolean pushDirty = dirtyPushHead != NULL_INT; final boolean popDirty = dirtyPopHead != NULL_INT; - if (size == 0) { - return emptyVal; - } - // This is a nested complex set of `if` statements that are used to set the correct and minimal initial // conditions for the evaluation. The calls to evaluateTree recurse no more than twice (as the ranges // overlap and the calculation simplifies). @@ -229,16 +225,10 @@ private void grow(int increase) { // assert that we are not asking for the impossible Assert.eqTrue(PAIRWISE_MAX_CAPACITY - increase >= size, "PairwiseDoubleRingBuffer size <= PAIRWISE_MAX_CAPACITY"); - final int minLength = size + increase; - - // double the current capacity until there is sufficient space for the increase - while (capacity <= minLength) { - capacity *= 2; - chunkSize = Math.min(capacity * 2, PAIRWISE_MAX_CAPACITY); - } - - // transfer to the new chunk WritableDoubleChunk oldChunk = storageChunk; + + capacity = Integer.highestOneBit(size + increase - 1) << 1; + chunkSize = capacity * 2; storageChunk = WritableDoubleChunk.makeWritableChunk(chunkSize); // fill the pairwise tree (0 to capacity) with empty value @@ -268,7 +258,7 @@ private void grow(int increase) { dirtyPushTail = size; dirtyPopHead = NULL_INT; - dirtyPopTail = 0; + dirtyPopTail = NULL_INT; } private void grow() { @@ -349,7 +339,7 @@ public double[] pop(int count) { if (dirtyPopHead == NULL_INT) { dirtyPopHead = head; } - dirtyPopTail = ((head + count - 1) % capacity) + capacity;; + dirtyPopTail = ((head + count - 1) % capacity) + capacity; // move the head head = ((head + count) % capacity) + capacity; @@ -429,6 +419,9 @@ private void clearDirty() { } public void clear() { + // fill with the empty value + storageChunk.fillWithValue(0, chunkSize, emptyVal); + head = tail = capacity; size = 0; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java index 29ed42e29ea..4567bc1abb3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java @@ -3,9 +3,7 @@ */ package io.deephaven.engine.table.impl.updateby.internal; -import io.deephaven.base.ArrayUtil; import io.deephaven.base.verify.Assert; -import io.deephaven.base.verify.AssertionFailure; import io.deephaven.chunk.WritableFloatChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.util.SafeCloseable; @@ -26,7 +24,10 @@ */ public class PairwiseFloatRingBuffer implements SafeCloseable { - private static final int PAIRWISE_MAX_CAPACITY = ArrayUtil.MAX_ARRAY_SIZE / 2; + /** We are limited to a buffer size exactly a power of two. Since we store the values and the pairwise tree in the + * same buffer, we are limited to a max capacity of 2^29 (500M) entries + */ + private static final int PAIRWISE_MAX_CAPACITY = 1 << 29; // use a sized float chunk for underlying storage private WritableFloatChunk storageChunk; @@ -71,7 +72,7 @@ public interface FloatFunction { * result is available */ public PairwiseFloatRingBuffer(int initialSize, float emptyVal, FloatFunction pairwiseFunction) { - Assert.eqTrue(PAIRWISE_MAX_CAPACITY >= initialSize, "PairwiseFloatRingBuffer initialSize <= PAIRWISE_MAX_CAPACITY"); + Assert.eqTrue(initialSize <= PAIRWISE_MAX_CAPACITY, "PairwiseFloatRingBuffer initialSize <= PAIRWISE_MAX_CAPACITY"); // use next larger power of 2 this.capacity = Integer.highestOneBit(initialSize - 1) << 1; @@ -80,7 +81,6 @@ public PairwiseFloatRingBuffer(int initialSize, float emptyVal, FloatFunction pa this.pairwiseFunction = pairwiseFunction; this.emptyVal = emptyVal; - storageChunk.fillWithValue(0, chunkSize, emptyVal); clear(); } @@ -131,7 +131,7 @@ public float evaluateTree(int startA, int endA, int startB, int endB) { endB /= 2; } } - throw new AssertionFailure("should never reach this line"); + throw Assert.statementNeverExecuted(); } @VisibleForTesting @@ -161,17 +161,13 @@ public float evaluateTree(int startA, int endA, int startB, int endB, int startC endC /= 2; } } - throw new AssertionFailure("should never reach this line"); + throw Assert.statementNeverExecuted(); } public float evaluate() { final boolean pushDirty = dirtyPushHead != NULL_INT; final boolean popDirty = dirtyPopHead != NULL_INT; - if (size == 0) { - return emptyVal; - } - // This is a nested complex set of `if` statements that are used to set the correct and minimal initial // conditions for the evaluation. The calls to evaluateTree recurse no more than twice (as the ranges // overlap and the calculation simplifies). @@ -224,16 +220,10 @@ private void grow(int increase) { // assert that we are not asking for the impossible Assert.eqTrue(PAIRWISE_MAX_CAPACITY - increase >= size, "PairwiseFloatRingBuffer size <= PAIRWISE_MAX_CAPACITY"); - final int minLength = size + increase; - - // double the current capacity until there is sufficient space for the increase - while (capacity <= minLength) { - capacity *= 2; - chunkSize = Math.min(capacity * 2, PAIRWISE_MAX_CAPACITY); - } - - // transfer to the new chunk WritableFloatChunk oldChunk = storageChunk; + + capacity = Integer.highestOneBit(size + increase - 1) << 1; + chunkSize = capacity * 2; storageChunk = WritableFloatChunk.makeWritableChunk(chunkSize); // fill the pairwise tree (0 to capacity) with empty value @@ -263,7 +253,7 @@ private void grow(int increase) { dirtyPushTail = size; dirtyPopHead = NULL_INT; - dirtyPopTail = 0; + dirtyPopTail = NULL_INT; } private void grow() { @@ -344,7 +334,7 @@ public float[] pop(int count) { if (dirtyPopHead == NULL_INT) { dirtyPopHead = head; } - dirtyPopTail = ((head + count - 1) % capacity) + capacity;; + dirtyPopTail = ((head + count - 1) % capacity) + capacity; // move the head head = ((head + count) % capacity) + capacity; @@ -424,6 +414,9 @@ private void clearDirty() { } public void clear() { + // fill with the empty value + storageChunk.fillWithValue(0, chunkSize, emptyVal); + head = tail = capacity; size = 0; diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBufferTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBufferTest.java index 2986d50142d..dac81fac5b7 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBufferTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBufferTest.java @@ -13,6 +13,7 @@ import org.junit.experimental.categories.Category; import java.util.NoSuchElementException; +import java.util.Random; import static io.deephaven.util.QueryConstants.NULL_DOUBLE; @@ -42,9 +43,9 @@ private void assertNotEmpty(PairwiseDoubleRingBuffer rb, int expectedSize, doubl assertFalse(rb.isEmpty()); assertEquals(expectedSize, rb.size()); - assertTrue(expectedHead == rb.peek(Long.MIN_VALUE)); + assertEquals(expectedHead, rb.peek(Long.MIN_VALUE)); try { - assertTrue(expectedHead == rb.element()); + assertEquals(expectedHead, rb.element()); } catch (NoSuchElementException x) { fail("queue should not be empty"); } @@ -58,7 +59,7 @@ private void assertAdd(PairwiseDoubleRingBuffer rb, double newHead, int expected private void assertRemove(PairwiseDoubleRingBuffer rb, int expectedSize, double expectedHead) { assertNotEmpty(rb, expectedSize, expectedHead); try { - assertTrue(expectedHead == rb.pop()); + assertEquals(expectedHead, rb.pop()); } catch (NoSuchElementException x) { fail("queue should not be empty"); } @@ -74,195 +75,201 @@ private void assertRemove(PairwiseDoubleRingBuffer rb, int expectedSize, double public void testAddRemove() { - PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(3, NULL_DOUBLE, Double::min); + try (PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(3, NULL_DOUBLE, Double::min)) { - assertEmpty(rb); + assertEmpty(rb); - // move the head and tail off zero - for (double i = 0; i < 1000; i++) { - rb.push(i); - } - for (double i = 0; i < 1000; i++) { - rb.pop(); - } + // move the head and tail off zero + for (double i = 0; i < 1000; i++) { + rb.push(i); + } + for (double i = 0; i < 1000; i++) { + rb.pop(); + } - assertEmpty(rb); - - assertAdd(rb, A, 1, A); - assertAdd(rb, B, 2, A); - assertAdd(rb, C, 3, A); - // take this opportunity to test some front()/back() functions - assertEquals(rb.front(), A); - assertEquals(rb.front(0), A); - assertEquals(rb.front(1), B); - assertEquals(rb.front(2), C); - assertEquals(rb.back(), C); - assertEquals(rb.peekBack(NULL_DOUBLE), C); - - assertRemove(rb, 3, A); - assertRemove(rb, 2, B); - assertRemove(rb, 1, C); - assertEmpty(rb); - - assertAdd(rb, A, 1, A); - assertAdd(rb, B, 2, A); - assertRemove(rb, 2, A); - assertAdd(rb, C, 2, B); - assertRemove(rb, 2, B); - assertRemove(rb, 1, C); - assertEmpty(rb); - - assertAdd(rb, A, 1, A); - assertRemove(rb, 1, A); - assertEmpty(rb); - assertAdd(rb, B, 1, B); - assertRemove(rb, 1, B); - assertEmpty(rb); - assertAdd(rb, C, 1, C); - assertRemove(rb, 1, C); - assertEmpty(rb); - - assertAdd(rb, D, 1, D); - assertRemove(rb, 1, D); - assertEmpty(rb); - assertAdd(rb, E, 1, E); - assertRemove(rb, 1, E); - assertEmpty(rb); - assertAdd(rb, F, 1, F); - assertRemove(rb, 1, F); - assertEmpty(rb); - - assertAdd(rb, A, 1, A); - assertAdd(rb, B, 2, A); - assertAdd(rb, C, 3, A); - - assertAdd(rb, D, 4, A); - assertAdd(rb, E, 5, A); - assertAdd(rb, F, 6, A); - - assertRemove(rb, 6, A); - assertRemove(rb, 5, B); - assertRemove(rb, 4, C); - assertRemove(rb, 3, D); - assertRemove(rb, 2, E); - assertRemove(rb, 1, F); - assertEmpty(rb); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + // take this opportunity to test some front()/back() functions + assertEquals(rb.front(), A); + assertEquals(rb.front(0), A); + assertEquals(rb.front(1), B); + assertEquals(rb.front(2), C); + assertEquals(rb.back(), C); + assertEquals(rb.peekBack(NULL_DOUBLE), C); + + assertRemove(rb, 3, A); + assertRemove(rb, 2, B); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertRemove(rb, 2, A); + assertAdd(rb, C, 2, B); + assertRemove(rb, 2, B); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertRemove(rb, 1, A); + assertEmpty(rb); + assertAdd(rb, B, 1, B); + assertRemove(rb, 1, B); + assertEmpty(rb); + assertAdd(rb, C, 1, C); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, D, 1, D); + assertRemove(rb, 1, D); + assertEmpty(rb); + assertAdd(rb, E, 1, E); + assertRemove(rb, 1, E); + assertEmpty(rb); + assertAdd(rb, F, 1, F); + assertRemove(rb, 1, F); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + assertAdd(rb, F, 6, A); + + assertRemove(rb, 6, A); + assertRemove(rb, 5, B); + assertRemove(rb, 4, C); + assertRemove(rb, 3, D); + assertRemove(rb, 2, E); + assertRemove(rb, 1, F); + assertEmpty(rb); + } } public void testGrowSimple() { - PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(5, NULL_DOUBLE, Double::min); - - assertAdd(rb, A, 1, A); - assertAdd(rb, B, 2, A); - assertAdd(rb, C, 3, A); - assertAdd(rb, D, 4, A); - assertAdd(rb, E, 5, A); - - // this will grow; the elements are in a single contiguous block - assertAdd(rb, F, 6, A); - - assertRemove(rb, 6, A); - assertRemove(rb, 5, B); - assertRemove(rb, 4, C); - assertRemove(rb, 3, D); - assertRemove(rb, 2, E); - assertRemove(rb, 1, F); - assertEmpty(rb); - - rb.pushEmptyValue(); - assertEquals(rb.front(), NULL_DOUBLE); - try { - rb.front(-1); - fail("expected a NoSuchElement exception"); - } catch (NoSuchElementException x) { - // expected - } - try { - rb.front(5); - fail("expected a NoSuchElement exception"); - } catch (NoSuchElementException x) { - // expected + try (PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(5, NULL_DOUBLE, Double::min)) { + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + + // this will grow; the elements are in a single contiguous block + assertAdd(rb, F, 6, A); + + assertRemove(rb, 6, A); + assertRemove(rb, 5, B); + assertRemove(rb, 4, C); + assertRemove(rb, 3, D); + assertRemove(rb, 2, E); + assertRemove(rb, 1, F); + assertEmpty(rb); + + rb.pushEmptyValue(); + assertEquals(rb.front(), NULL_DOUBLE); + try { + rb.front(-1); + fail("expected a NoSuchElement exception"); + } catch (NoSuchElementException x) { + // expected + } + try { + rb.front(5); + fail("expected a NoSuchElement exception"); + } catch (NoSuchElementException x) { + // expected + } + assertEquals(rb.poll(0.0f), NULL_DOUBLE); } - assertEquals(rb.poll(0.0f), NULL_DOUBLE); } public void testGrowComplex() { - PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(5, NULL_DOUBLE, Double::min); - - assertAdd(rb, A, 1, A); - assertAdd(rb, B, 2, A); - assertAdd(rb, C, 3, A); - assertRemove(rb, 3, A); - assertRemove(rb, 2, B); - assertRemove(rb, 1, C); - assertEmpty(rb); - - assertAdd(rb, A, 1, A); - assertAdd(rb, B, 2, A); - assertAdd(rb, C, 3, A); - assertAdd(rb, D, 4, A); - assertAdd(rb, E, 5, A); - - // this will grow; the elements are in two blocks - assertAdd(rb, F, 6, A); - - assertRemove(rb, 6, A); - assertRemove(rb, 5, B); - assertRemove(rb, 4, C); - assertRemove(rb, 3, D); - assertRemove(rb, 2, E); - assertRemove(rb, 1, F); - assertEmpty(rb); + try (PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(5, NULL_DOUBLE, Double::min)) { + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertRemove(rb, 3, A); + assertRemove(rb, 2, B); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + + // this will grow; the elements are in two blocks + assertAdd(rb, F, 6, A); + + assertRemove(rb, 6, A); + assertRemove(rb, 5, B); + assertRemove(rb, 4, C); + assertRemove(rb, 3, D); + assertRemove(rb, 2, E); + assertRemove(rb, 1, F); + assertEmpty(rb); + } } public void testWhenEmpty() { - PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(5, NULL_DOUBLE, Double::min); - try { - rb.back(); - fail("expected a NoSuchElement exception"); - } catch (NoSuchElementException x) { - // expected - } - try { - rb.pop(); - fail("expected a NoSuchElement exception"); - } catch (NoSuchElementException x) { - // expected - } - try { - rb.front(); - fail("expected a NoSuchElement exception"); - } catch (NoSuchElementException x) { - // expected + try (PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(5, NULL_DOUBLE, Double::min)) { + try { + rb.back(); + fail("expected a NoSuchElement exception"); + } catch (NoSuchElementException x) { + // expected + } + try { + rb.pop(); + fail("expected a NoSuchElement exception"); + } catch (NoSuchElementException x) { + // expected + } + try { + rb.front(); + fail("expected a NoSuchElement exception"); + } catch (NoSuchElementException x) { + // expected + } + assertEquals(rb.poll((double) -1), (double) -1); + assertEquals(rb.peek((double) -1), (double) -1); + assertEquals(rb.peekBack((double) -1), (double) -1); } - assertEquals(rb.poll((double) -1), (double) -1); - assertEquals(rb.peek((double) -1), (double) -1); - assertEquals(rb.peekBack((double) -1), (double) -1); } public void testBack() { - PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(5, NULL_DOUBLE, Double::min); + try (PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(5, NULL_DOUBLE, Double::min)) { - assertAdd(rb, A, 1, A); - assertEquals(rb.back(), A); + assertAdd(rb, A, 1, A); + assertEquals(rb.back(), A); - assertAdd(rb, B, 2, A); - assertAdd(rb, C, 3, A); - assertEquals(rb.back(), C); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertEquals(rb.back(), C); + } } public void testBackTailIsZero() { - PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(5, NULL_DOUBLE, Double::min); + try (PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(5, NULL_DOUBLE, Double::min)) { - assertAdd(rb, A, 1, A); - assertAdd(rb, B, 2, A); - assertAdd(rb, C, 3, A); - assertAdd(rb, D, 4, A); - assertAdd(rb, E, 5, A); + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); - assertRemove(rb, 5, A); - assertAdd(rb, F, 5, B); - assertEquals(rb.back(), F); + assertRemove(rb, 5, A); + assertAdd(rb, F, 5, B); + assertEquals(rb.back(), F); + } } public void testLargeAmounts() { @@ -346,20 +353,134 @@ public void testEvaluateSumLargeAmounts() { * Return the sum of 0 to N-1 */ private double sum0toN(double n) { - return ((double) n * (double) (n - 1) / (double) 2); + if (n == (double) 0) { + return (double) 0; // not negative zero, sigh + } + return (n * (n - 1) / (double) 2); } public void testEvaluationEdgeCase() { - try (final PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(512, (double)0, Double::sum)) { - for (double i = 0; i < 512; i++) { + try (final PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(512, (double) 0, Double::sum)) { + + final int PRIME = 97; + final double PRIME_SUM = sum0toN(PRIME); + final Random rnd = new Random(0xDEADBEEF); + + // specific test for push or wrap araound + for (int step = 0; step < 100; step++) { + for (double i = 0; i < PRIME; i++) { + rb.pushUnsafe(i); + } + assertEquals(PRIME_SUM, rb.evaluate()); + + for (double i = 0; i < PRIME; i++) { + rb.popUnsafe(); + } + assertEquals((double) 0, rb.evaluate()); + // not a copy/paste error, call this twice + assertEquals((double) 0, rb.evaluate()); + } + + // specific test for push & wrap araound + for (int step = 0; step < 100; step++) { + for (double i = 0; i < PRIME; i++) { + rb.pushUnsafe(i); + } + for (double i = 0; i < PRIME; i++) { + rb.popUnsafe(); + } + assertEquals((double) 0, rb.evaluate()); + // not a copy/paste error, call this twice + assertEquals((double) 0, rb.evaluate()); + } + + // push random amounts and test + for (int step = 0; step < 100; step++) { + final int OFFSET = rnd.nextInt(PRIME); + + for (double i = 0; i < OFFSET; i++) { + rb.pushUnsafe(i); + } + assertEquals(sum0toN(OFFSET), rb.evaluate()); + + for (double i = OFFSET; i < PRIME; i++) { + rb.pushUnsafe(i); + } + assertEquals(PRIME_SUM, rb.evaluate()); + + for (double i = 0; i < PRIME; i++) { + rb.popUnsafe(); + } + assertEquals((double) 0, rb.evaluate()); + } + + // pop random amounts and test + for (int step = 0; step < 100; step++) { + final int OFFSET = rnd.nextInt(PRIME); + for (double i = 0; i < PRIME; i++) { + rb.pushUnsafe(i); + } + assertEquals(PRIME_SUM, rb.evaluate()); + + for (double i = 0; i < OFFSET; i++) { + rb.popUnsafe(); + } + assertEquals(PRIME_SUM - sum0toN(OFFSET), rb.evaluate()); + + for (double i = 0; i < (PRIME - OFFSET); i++) { + rb.popUnsafe(); + } + assertEquals((double) 0, rb.evaluate()); + } + } + + try (final PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(512, (double) 0, Double::sum)) { + // need to get the buffer to state where we have clean pushes and a wrapped pop + + // move the pointers close to the end + for (double i = 0; i < 500; i++) { + rb.pushUnsafe(i); + rb.popUnsafe(); + } + assertEquals((double) 0, rb.evaluate()); + assertEmpty(rb); + + // push past the end + for (double i = 0; i < 200; i++) { + rb.pushUnsafe(i); + } + assertEquals(sum0toN(200), rb.evaluate()); + + // one more push to dirty the pushes + rb.pushUnsafe(201); + + // pop past the end + for (double i = 0; i < 200; i++) { + rb.popUnsafe(); + } + + // only thing in the buffer is the final push + assertEquals((double) 201, rb.evaluate()); + } + + try (final PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(512, (double) 0, Double::sum)) { + // cue some hacky stuff to improve code coverage + for (double i = 0; i < 100; i++) { rb.pushUnsafe(i); } - // eval single range (internal collapse expected) - assertEquals(sum0toN(512), rb.evaluateTree(512, 1023)); - assertEquals(sum0toN(512), rb.evaluateTree(512, 600, 601, 1023)); - assertEquals(sum0toN(512), rb.evaluateTree(512, 600, 601, 700, 701, 1023)); - //TODO: more edge cases + final double sum = sum0toN(100); + + // evaluateTree() doesn't reset the dirty bits so these calls can follow each other + + // no consecutive ranges + assertEquals(sum, rb.evaluateTree(512, 611, 712, 811, 912, 1023)); + // first two ranges are consecutive + assertEquals(sum, rb.evaluateTree(512, 611, 612, 811, 912, 1023)); + // second two ranges are consecutive + assertEquals(sum, rb.evaluateTree(512, 611, 712, 811, 812, 1023)); + // all ranges are consecutive + assertEquals(sum, rb.evaluateTree(512, 611, 612, 811, 812, 1023)); } } @@ -408,6 +529,13 @@ public void testPopMultiple() { rb.evaluate(); } + try { + double[] values = rb.pop(501); + fail("popping more values than size() should fail"); + } catch (NoSuchElementException x) { + // expected + } + double[] values = rb.pop(500); for (double i = 0; i < 500; i++) { assertEquals(values[(int) i], i); diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBufferTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBufferTest.java index cc26f6df15c..5f02ac1d97e 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBufferTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBufferTest.java @@ -8,6 +8,7 @@ import org.junit.experimental.categories.Category; import java.util.NoSuchElementException; +import java.util.Random; import static io.deephaven.util.QueryConstants.NULL_FLOAT; @@ -37,9 +38,9 @@ private void assertNotEmpty(PairwiseFloatRingBuffer rb, int expectedSize, float assertFalse(rb.isEmpty()); assertEquals(expectedSize, rb.size()); - assertTrue(expectedHead == rb.peek(Long.MIN_VALUE)); + assertEquals(expectedHead, rb.peek(Long.MIN_VALUE)); try { - assertTrue(expectedHead == rb.element()); + assertEquals(expectedHead, rb.element()); } catch (NoSuchElementException x) { fail("queue should not be empty"); } @@ -53,7 +54,7 @@ private void assertAdd(PairwiseFloatRingBuffer rb, float newHead, int expectedSi private void assertRemove(PairwiseFloatRingBuffer rb, int expectedSize, float expectedHead) { assertNotEmpty(rb, expectedSize, expectedHead); try { - assertTrue(expectedHead == rb.pop()); + assertEquals(expectedHead, rb.pop()); } catch (NoSuchElementException x) { fail("queue should not be empty"); } @@ -69,195 +70,201 @@ private void assertRemove(PairwiseFloatRingBuffer rb, int expectedSize, float ex public void testAddRemove() { - PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(3, NULL_FLOAT, Float::min); + try (PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(3, NULL_FLOAT, Float::min)) { - assertEmpty(rb); + assertEmpty(rb); - // move the head and tail off zero - for (float i = 0; i < 1000; i++) { - rb.push(i); - } - for (float i = 0; i < 1000; i++) { - rb.pop(); - } + // move the head and tail off zero + for (float i = 0; i < 1000; i++) { + rb.push(i); + } + for (float i = 0; i < 1000; i++) { + rb.pop(); + } - assertEmpty(rb); - - assertAdd(rb, A, 1, A); - assertAdd(rb, B, 2, A); - assertAdd(rb, C, 3, A); - // take this opportunity to test some front()/back() functions - assertEquals(rb.front(), A); - assertEquals(rb.front(0), A); - assertEquals(rb.front(1), B); - assertEquals(rb.front(2), C); - assertEquals(rb.back(), C); - assertEquals(rb.peekBack(NULL_FLOAT), C); - - assertRemove(rb, 3, A); - assertRemove(rb, 2, B); - assertRemove(rb, 1, C); - assertEmpty(rb); - - assertAdd(rb, A, 1, A); - assertAdd(rb, B, 2, A); - assertRemove(rb, 2, A); - assertAdd(rb, C, 2, B); - assertRemove(rb, 2, B); - assertRemove(rb, 1, C); - assertEmpty(rb); - - assertAdd(rb, A, 1, A); - assertRemove(rb, 1, A); - assertEmpty(rb); - assertAdd(rb, B, 1, B); - assertRemove(rb, 1, B); - assertEmpty(rb); - assertAdd(rb, C, 1, C); - assertRemove(rb, 1, C); - assertEmpty(rb); - - assertAdd(rb, D, 1, D); - assertRemove(rb, 1, D); - assertEmpty(rb); - assertAdd(rb, E, 1, E); - assertRemove(rb, 1, E); - assertEmpty(rb); - assertAdd(rb, F, 1, F); - assertRemove(rb, 1, F); - assertEmpty(rb); - - assertAdd(rb, A, 1, A); - assertAdd(rb, B, 2, A); - assertAdd(rb, C, 3, A); - - assertAdd(rb, D, 4, A); - assertAdd(rb, E, 5, A); - assertAdd(rb, F, 6, A); - - assertRemove(rb, 6, A); - assertRemove(rb, 5, B); - assertRemove(rb, 4, C); - assertRemove(rb, 3, D); - assertRemove(rb, 2, E); - assertRemove(rb, 1, F); - assertEmpty(rb); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + // take this opportunity to test some front()/back() functions + assertEquals(rb.front(), A); + assertEquals(rb.front(0), A); + assertEquals(rb.front(1), B); + assertEquals(rb.front(2), C); + assertEquals(rb.back(), C); + assertEquals(rb.peekBack(NULL_FLOAT), C); + + assertRemove(rb, 3, A); + assertRemove(rb, 2, B); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertRemove(rb, 2, A); + assertAdd(rb, C, 2, B); + assertRemove(rb, 2, B); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertRemove(rb, 1, A); + assertEmpty(rb); + assertAdd(rb, B, 1, B); + assertRemove(rb, 1, B); + assertEmpty(rb); + assertAdd(rb, C, 1, C); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, D, 1, D); + assertRemove(rb, 1, D); + assertEmpty(rb); + assertAdd(rb, E, 1, E); + assertRemove(rb, 1, E); + assertEmpty(rb); + assertAdd(rb, F, 1, F); + assertRemove(rb, 1, F); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + assertAdd(rb, F, 6, A); + + assertRemove(rb, 6, A); + assertRemove(rb, 5, B); + assertRemove(rb, 4, C); + assertRemove(rb, 3, D); + assertRemove(rb, 2, E); + assertRemove(rb, 1, F); + assertEmpty(rb); + } } public void testGrowSimple() { - PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(5, NULL_FLOAT, Float::min); - - assertAdd(rb, A, 1, A); - assertAdd(rb, B, 2, A); - assertAdd(rb, C, 3, A); - assertAdd(rb, D, 4, A); - assertAdd(rb, E, 5, A); - - // this will grow; the elements are in a single contiguous block - assertAdd(rb, F, 6, A); - - assertRemove(rb, 6, A); - assertRemove(rb, 5, B); - assertRemove(rb, 4, C); - assertRemove(rb, 3, D); - assertRemove(rb, 2, E); - assertRemove(rb, 1, F); - assertEmpty(rb); - - rb.pushEmptyValue(); - assertEquals(rb.front(), NULL_FLOAT); - try { - rb.front(-1); - fail("expected a NoSuchElement exception"); - } catch (NoSuchElementException x) { - // expected - } - try { - rb.front(5); - fail("expected a NoSuchElement exception"); - } catch (NoSuchElementException x) { - // expected + try (PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(5, NULL_FLOAT, Float::min)) { + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + + // this will grow; the elements are in a single contiguous block + assertAdd(rb, F, 6, A); + + assertRemove(rb, 6, A); + assertRemove(rb, 5, B); + assertRemove(rb, 4, C); + assertRemove(rb, 3, D); + assertRemove(rb, 2, E); + assertRemove(rb, 1, F); + assertEmpty(rb); + + rb.pushEmptyValue(); + assertEquals(rb.front(), NULL_FLOAT); + try { + rb.front(-1); + fail("expected a NoSuchElement exception"); + } catch (NoSuchElementException x) { + // expected + } + try { + rb.front(5); + fail("expected a NoSuchElement exception"); + } catch (NoSuchElementException x) { + // expected + } + assertEquals(rb.poll(0.0f), NULL_FLOAT); } - assertEquals(rb.poll(0.0f), NULL_FLOAT); } public void testGrowComplex() { - PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(5, NULL_FLOAT, Float::min); - - assertAdd(rb, A, 1, A); - assertAdd(rb, B, 2, A); - assertAdd(rb, C, 3, A); - assertRemove(rb, 3, A); - assertRemove(rb, 2, B); - assertRemove(rb, 1, C); - assertEmpty(rb); - - assertAdd(rb, A, 1, A); - assertAdd(rb, B, 2, A); - assertAdd(rb, C, 3, A); - assertAdd(rb, D, 4, A); - assertAdd(rb, E, 5, A); - - // this will grow; the elements are in two blocks - assertAdd(rb, F, 6, A); - - assertRemove(rb, 6, A); - assertRemove(rb, 5, B); - assertRemove(rb, 4, C); - assertRemove(rb, 3, D); - assertRemove(rb, 2, E); - assertRemove(rb, 1, F); - assertEmpty(rb); + try (PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(5, NULL_FLOAT, Float::min)) { + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertRemove(rb, 3, A); + assertRemove(rb, 2, B); + assertRemove(rb, 1, C); + assertEmpty(rb); + + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); + + // this will grow; the elements are in two blocks + assertAdd(rb, F, 6, A); + + assertRemove(rb, 6, A); + assertRemove(rb, 5, B); + assertRemove(rb, 4, C); + assertRemove(rb, 3, D); + assertRemove(rb, 2, E); + assertRemove(rb, 1, F); + assertEmpty(rb); + } } public void testWhenEmpty() { - PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(5, NULL_FLOAT, Float::min); - try { - rb.back(); - fail("expected a NoSuchElement exception"); - } catch (NoSuchElementException x) { - // expected - } - try { - rb.pop(); - fail("expected a NoSuchElement exception"); - } catch (NoSuchElementException x) { - // expected - } - try { - rb.front(); - fail("expected a NoSuchElement exception"); - } catch (NoSuchElementException x) { - // expected + try (PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(5, NULL_FLOAT, Float::min)) { + try { + rb.back(); + fail("expected a NoSuchElement exception"); + } catch (NoSuchElementException x) { + // expected + } + try { + rb.pop(); + fail("expected a NoSuchElement exception"); + } catch (NoSuchElementException x) { + // expected + } + try { + rb.front(); + fail("expected a NoSuchElement exception"); + } catch (NoSuchElementException x) { + // expected + } + assertEquals(rb.poll((float) -1), (float) -1); + assertEquals(rb.peek((float) -1), (float) -1); + assertEquals(rb.peekBack((float) -1), (float) -1); } - assertEquals(rb.poll((float) -1), (float) -1); - assertEquals(rb.peek((float) -1), (float) -1); - assertEquals(rb.peekBack((float) -1), (float) -1); } public void testBack() { - PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(5, NULL_FLOAT, Float::min); + try (PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(5, NULL_FLOAT, Float::min)) { - assertAdd(rb, A, 1, A); - assertEquals(rb.back(), A); + assertAdd(rb, A, 1, A); + assertEquals(rb.back(), A); - assertAdd(rb, B, 2, A); - assertAdd(rb, C, 3, A); - assertEquals(rb.back(), C); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertEquals(rb.back(), C); + } } public void testBackTailIsZero() { - PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(5, NULL_FLOAT, Float::min); + try (PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(5, NULL_FLOAT, Float::min)) { - assertAdd(rb, A, 1, A); - assertAdd(rb, B, 2, A); - assertAdd(rb, C, 3, A); - assertAdd(rb, D, 4, A); - assertAdd(rb, E, 5, A); + assertAdd(rb, A, 1, A); + assertAdd(rb, B, 2, A); + assertAdd(rb, C, 3, A); + assertAdd(rb, D, 4, A); + assertAdd(rb, E, 5, A); - assertRemove(rb, 5, A); - assertAdd(rb, F, 5, B); - assertEquals(rb.back(), F); + assertRemove(rb, 5, A); + assertAdd(rb, F, 5, B); + assertEquals(rb.back(), F); + } } public void testLargeAmounts() { @@ -341,20 +348,134 @@ public void testEvaluateSumLargeAmounts() { * Return the sum of 0 to N-1 */ private float sum0toN(float n) { - return ((float) n * (float) (n - 1) / (float) 2); + if (n == (float) 0) { + return (float) 0; // not negative zero, sigh + } + return (n * (n - 1) / (float) 2); } public void testEvaluationEdgeCase() { - try (final PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(512, (float)0, Float::sum)) { - for (float i = 0; i < 512; i++) { + try (final PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(512, (float) 0, Float::sum)) { + + final int PRIME = 97; + final float PRIME_SUM = sum0toN(PRIME); + final Random rnd = new Random(0xDEADBEEF); + + // specific test for push or wrap araound + for (int step = 0; step < 100; step++) { + for (float i = 0; i < PRIME; i++) { + rb.pushUnsafe(i); + } + assertEquals(PRIME_SUM, rb.evaluate()); + + for (float i = 0; i < PRIME; i++) { + rb.popUnsafe(); + } + assertEquals((float) 0, rb.evaluate()); + // not a copy/paste error, call this twice + assertEquals((float) 0, rb.evaluate()); + } + + // specific test for push & wrap araound + for (int step = 0; step < 100; step++) { + for (float i = 0; i < PRIME; i++) { + rb.pushUnsafe(i); + } + for (float i = 0; i < PRIME; i++) { + rb.popUnsafe(); + } + assertEquals((float) 0, rb.evaluate()); + // not a copy/paste error, call this twice + assertEquals((float) 0, rb.evaluate()); + } + + // push random amounts and test + for (int step = 0; step < 100; step++) { + final int OFFSET = rnd.nextInt(PRIME); + + for (float i = 0; i < OFFSET; i++) { + rb.pushUnsafe(i); + } + assertEquals(sum0toN(OFFSET), rb.evaluate()); + + for (float i = OFFSET; i < PRIME; i++) { + rb.pushUnsafe(i); + } + assertEquals(PRIME_SUM, rb.evaluate()); + + for (float i = 0; i < PRIME; i++) { + rb.popUnsafe(); + } + assertEquals((float) 0, rb.evaluate()); + } + + // pop random amounts and test + for (int step = 0; step < 100; step++) { + final int OFFSET = rnd.nextInt(PRIME); + for (float i = 0; i < PRIME; i++) { + rb.pushUnsafe(i); + } + assertEquals(PRIME_SUM, rb.evaluate()); + + for (float i = 0; i < OFFSET; i++) { + rb.popUnsafe(); + } + assertEquals(PRIME_SUM - sum0toN(OFFSET), rb.evaluate()); + + for (float i = 0; i < (PRIME - OFFSET); i++) { + rb.popUnsafe(); + } + assertEquals((float) 0, rb.evaluate()); + } + } + + try (final PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(512, (float) 0, Float::sum)) { + // need to get the buffer to state where we have clean pushes and a wrapped pop + + // move the pointers close to the end + for (float i = 0; i < 500; i++) { + rb.pushUnsafe(i); + rb.popUnsafe(); + } + assertEquals((float) 0, rb.evaluate()); + assertEmpty(rb); + + // push past the end + for (float i = 0; i < 200; i++) { + rb.pushUnsafe(i); + } + assertEquals(sum0toN(200), rb.evaluate()); + + // one more push to dirty the pushes + rb.pushUnsafe(201); + + // pop past the end + for (float i = 0; i < 200; i++) { + rb.popUnsafe(); + } + + // only thing in the buffer is the final push + assertEquals((float) 201, rb.evaluate()); + } + + try (final PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(512, (float) 0, Float::sum)) { + // cue some hacky stuff to improve code coverage + for (float i = 0; i < 100; i++) { rb.pushUnsafe(i); } - // eval single range (internal collapse expected) - assertEquals(sum0toN(512), rb.evaluateTree(512, 1023)); - assertEquals(sum0toN(512), rb.evaluateTree(512, 600, 601, 1023)); - assertEquals(sum0toN(512), rb.evaluateTree(512, 600, 601, 700, 701, 1023)); - //TODO: more edge cases + final float sum = sum0toN(100); + + // evaluateTree() doesn't reset the dirty bits so these calls can follow each other + + // no consecutive ranges + assertEquals(sum, rb.evaluateTree(512, 611, 712, 811, 912, 1023)); + // first two ranges are consecutive + assertEquals(sum, rb.evaluateTree(512, 611, 612, 811, 912, 1023)); + // second two ranges are consecutive + assertEquals(sum, rb.evaluateTree(512, 611, 712, 811, 812, 1023)); + // all ranges are consecutive + assertEquals(sum, rb.evaluateTree(512, 611, 612, 811, 812, 1023)); } } @@ -403,6 +524,13 @@ public void testPopMultiple() { rb.evaluate(); } + try { + float[] values = rb.pop(501); + fail("popping more values than size() should fail"); + } catch (NoSuchElementException x) { + // expected + } + float[] values = rb.pop(500); for (float i = 0; i < 500; i++) { assertEquals(values[(int) i], i); From 4e3854ac162390735ff1b63b118312561e85cbcb Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 25 Jan 2023 09:13:28 -0800 Subject: [PATCH 095/123] Implemented timestamp column modification check --- .../impl/updateby/UpdateByBucketHelper.java | 9 ++ .../table/impl/updateby/UpdateByWindow.java | 6 ++ .../updateby/UpdateByWindowCumulative.java | 5 +- .../impl/updateby/UpdateByWindowTicks.java | 3 +- .../impl/updateby/UpdateByWindowTime.java | 88 ++++++++++--------- 5 files changed, 65 insertions(+), 46 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java index b321d0fcfcf..a23eafd083a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java @@ -294,9 +294,17 @@ private int fillChunkWithNonNull(LongChunk keysChunk, LongChunk< public void prepareForUpdate(final TableUpdate upstream, final boolean initialStep) { Assert.eqFalse(isDirty, "UpdateBy bucket was marked dirty before processing an update"); + final boolean timestampsModified; + // add all the SSA data if (timestampColumnName != null) { processUpdateForSsa(upstream); + + // test whether any timestamps were modified + timestampsModified = + upstream.modified().isNonempty() && upstream.modifiedColumnSet().containsAny(timestampColumnSet); + } else { + timestampsModified = false; } final TrackingRowSet sourceRowSet = source.getRowSet(); @@ -308,6 +316,7 @@ public void prepareForUpdate(final TableUpdate upstream, final boolean initialSt timestampColumnSource, timestampSsa, timestampValidRowSet, + timestampsModified, control.chunkCapacityOrDefault(), initialStep); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java index 983e6a32fb8..2194af13891 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java @@ -41,6 +41,9 @@ class UpdateByWindowBucketContext implements SafeCloseable { protected final LongSegmentedSortedArray timestampSsa; /** This rowset will store row keys where the timestamp is not null (will mirror the SSA contents) */ protected final TrackingRowSet timestampValidRowSet; + + /** Were any timestamps modified in the current update? */ + protected final boolean timestampsModified; /** An array of context objects for each underlying operator */ protected final UpdateByOperator.UpdateContext[] opContext; /** Whether this is the creation phase of this window */ @@ -69,12 +72,14 @@ class UpdateByWindowBucketContext implements SafeCloseable { @Nullable final ColumnSource timestampColumnSource, @Nullable final LongSegmentedSortedArray timestampSsa, final TrackingRowSet timestampValidRowSet, + final boolean timestampsModified, final int chunkSize, final boolean initialStep) { this.sourceRowSet = sourceRowSet; this.timestampColumnSource = timestampColumnSource; this.timestampSsa = timestampSsa; this.timestampValidRowSet = timestampValidRowSet; + this.timestampsModified = timestampsModified; this.opContext = new UpdateByOperator.UpdateContext[operators.length]; @@ -102,6 +107,7 @@ abstract UpdateByWindowBucketContext makeWindowContext(final TrackingRowSet sour final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa, final TrackingRowSet timestampValidRowSet, + final boolean timestampsModified, final int chunkSize, final boolean isInitializeStep); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java index 4902b39b3d1..0f68e97054e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java @@ -45,10 +45,11 @@ UpdateByWindowBucketContext makeWindowContext(final TrackingRowSet sourceRowSet, final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa, final TrackingRowSet timestampValidRowSet, + final boolean timestampsModified, final int chunkSize, final boolean isInitializeStep) { return new UpdateByWindowBucketContext(sourceRowSet, timestampColumnSource, timestampSsa, timestampValidRowSet, - chunkSize, isInitializeStep); + timestampsModified, chunkSize, isInitializeStep); } @Override @@ -91,7 +92,7 @@ void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, @NotNu if (upstream.modifiedColumnSet().nonempty() && (op.getInputModifiedColumnSet() == null || upstream.modifiedColumnSet().containsAny(op.getInputModifiedColumnSet()))) { dirtyOperators.set(opIdx); - Arrays.stream(operatorInputSourceSlots[opIdx]).forEach(srcIdx -> dirtySourceIndices.set(srcIdx)); + Arrays.stream(operatorInputSourceSlots[opIdx]).forEach(dirtySourceIndices::set); } } context.isDirty = !dirtyOperators.isEmpty(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java index 35e02ef3257..3b5aea19c72 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java @@ -39,7 +39,7 @@ class UpdateByWindowTicksBucketContext extends UpdateByWindowBucketContext { UpdateByWindowTicksBucketContext(final TrackingRowSet sourceRowSet, final int chunkSize, final boolean initialStep) { - super(sourceRowSet, null, null, null, chunkSize, initialStep); + super(sourceRowSet, null, null, null, false, chunkSize, initialStep); } @Override @@ -84,6 +84,7 @@ UpdateByWindowBucketContext makeWindowContext(final TrackingRowSet sourceRowSet, final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa, final TrackingRowSet timestampValidRowSet, + final boolean timestampsModified, final int chunkSize, final boolean isInitializeStep) { return new UpdateByWindowTicksBucketContext(sourceRowSet, chunkSize, isInitializeStep); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java index 43ff32d9de0..fe70350ced3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java @@ -48,9 +48,11 @@ public UpdateByWindowTimeBucketContext(final TrackingRowSet sourceRowSet, @NotNull final ColumnSource timestampColumnSource, @Nullable final LongSegmentedSortedArray timestampSsa, final TrackingRowSet timestampValidRowSet, + final boolean timestampsModified, final int chunkSize, final boolean initialStep) { - super(sourceRowSet, timestampColumnSource, timestampSsa, timestampValidRowSet, chunkSize, initialStep); + super(sourceRowSet, timestampColumnSource, timestampSsa, timestampValidRowSet, timestampsModified, + chunkSize, initialStep); influencerTimestampContext = timestampColumnSource.makeGetContext(WINDOW_CHUNK_SIZE); timestampColumnGetContext = timestampColumnSource.makeGetContext(WINDOW_CHUNK_SIZE); @@ -93,10 +95,11 @@ public UpdateByWindowBucketContext makeWindowContext(final TrackingRowSet source final ColumnSource timestampColumnSource, final LongSegmentedSortedArray timestampSsa, final TrackingRowSet timestampValidRowSet, + final boolean timestampsModified, final int chunkSize, final boolean isInitializeStep) { return new UpdateByWindowTimeBucketContext(sourceRowSet, timestampColumnSource, timestampSsa, - timestampValidRowSet, chunkSize, isInitializeStep); + timestampValidRowSet, timestampsModified, chunkSize, isInitializeStep); } /** @@ -261,20 +264,22 @@ public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, final WritableRowSet tmpAffected = RowSetFactory.empty(); if (upstream.modified().isNonempty()) { - // modified timestamps will affect the current and previous values try (final RowSet modifiedAffected = computeAffectedRowsTime(ctx, upstream.modified(), prevUnits, fwdUnits, false)) { tmpAffected.insert(modifiedAffected); } - try (final WritableRowSet modifiedAffectedPrev = - computeAffectedRowsTime(ctx, upstream.getModifiedPreShift(), prevUnits, fwdUnits, true)) { - // we used the SSA (post-shift) to get these keys, no need to shift - // retain only the rows that still exist in the sourceRowSet - modifiedAffectedPrev.retain(ctx.timestampValidRowSet); - tmpAffected.insert(modifiedAffectedPrev); + if (ctx.timestampsModified) { + // modified timestamps will affect previous values + try (final WritableRowSet modifiedAffectedPrev = + computeAffectedRowsTime(ctx, upstream.getModifiedPreShift(), prevUnits, fwdUnits, true)) { + // we used the SSA (post-shift) to get these keys, no need to shift + // retain only the rows that still exist in the sourceRowSet + modifiedAffectedPrev.retain(ctx.timestampValidRowSet); + tmpAffected.insert(modifiedAffectedPrev); + } + // compute modified rows only if timestamps have changed + tmpAffected.insert(upstream.modified()); } - // compute all modified rows - tmpAffected.insert(upstream.modified()); } if (upstream.added().isNonempty()) { @@ -329,12 +334,12 @@ public void processRows(UpdateByWindowBucketContext context, boolean initialStep winOp.initializeUpdate(ctx.opContext[opIdx]); } - try (final RowSequence.Iterator it = ctx.affectedRows.getRowSequenceIterator(); + try (final RowSequence.Iterator affectedRowsIt = ctx.affectedRows.getRowSequenceIterator(); + final LongColumnIterator affectedTsIt = + new LongColumnIterator(context.timestampColumnSource, ctx.affectedRows); final LongColumnIterator influencerTsTailIt = new LongColumnIterator(context.timestampColumnSource, context.influencerRows); - final RowSequence.Iterator influencerKeyIt = ctx.influencerRows.getRowSequenceIterator(); - final ChunkSource.GetContext localTimestampContext = - ctx.timestampColumnSource.makeGetContext(ctx.workingChunkSize); + final RowSequence.Iterator influencerRowsIt = ctx.influencerRows.getRowSequenceIterator(); final WritableIntChunk pushChunk = WritableIntChunk.makeWritableChunk(ctx.workingChunkSize); final WritableIntChunk popChunk = @@ -342,26 +347,22 @@ public void processRows(UpdateByWindowBucketContext context, boolean initialStep long currentTailTs = nextLongOrMax(influencerTsTailIt); - - while (it.hasMore()) { - final RowSequence chunkRs = it.getNextRowSequenceWithLength(ctx.workingChunkSize); - final int chunkRsSize = chunkRs.intSize(); - + while (affectedRowsIt.hasMore()) { // NOTE: we did not put null values into our SSA and our influencer rowset is built using the // SSA. there should be no null timestamps considered in the rolling windows - final LongChunk timestampChunk = - ctx.timestampColumnSource.getChunk(localTimestampContext, chunkRs).asLongChunk(); - // we need to use a builder because we are potentially skipping rows from the influencer rowset - final RowSetBuilderSequential chunkBuilder = RowSetFactory.builderSequential(); + RowSetBuilderSequential builder = RowSetFactory.builderSequential(); + + int affectedRowIndex; + for (affectedRowIndex = 0; affectedRowIndex < ctx.workingChunkSize + && affectedTsIt.hasNext(); affectedRowIndex++) { - for (int ii = 0; ii < chunkRsSize; ii++) { // read the current timestamp - final long currentTimestamp = timestampChunk.get(ii); + final long currentTimestamp = affectedTsIt.nextLong(); if (currentTimestamp == NULL_LONG) { // this signifies that does not belong to a time window - popChunk.set(ii, NULL_INT); - pushChunk.set(ii, NULL_INT); + popChunk.set(affectedRowIndex, NULL_INT); + pushChunk.set(affectedRowIndex, NULL_INT); continue; } @@ -376,16 +377,16 @@ public void processRows(UpdateByWindowBucketContext context, boolean initialStep popCount++; } - // advance the influencerKeyIt and timestamp iterators until we are within the window. This only - // happens when initialStep == true because we have not created the minimum set of rows but include - // all non-null timestamp rows in our influencer values + // advance the keyIt and timestamp iterators until we are within the window. This only happens + // when initialStep == true because we have not created the minimum set of rows but include all + // non-null timestamp rows in our influencer values long skipCount = 0; while (currentTailTs < head) { Assert.eqTrue(initialStep, "initialStep when skipping rows"); currentTailTs = nextLongOrMax(influencerTsTailIt); skipCount++; } - influencerKeyIt.getNextRowSequenceWithLength(skipCount); + influencerRowsIt.getNextRowSequenceWithLength(skipCount); // push in all values that are in the new window (inclusive of tail) long pushCount = 0; @@ -395,18 +396,19 @@ public void processRows(UpdateByWindowBucketContext context, boolean initialStep currentTailTs = nextLongOrMax(influencerTsTailIt); pushCount++; } + builder.appendRowSequence(influencerRowsIt.getNextRowSequenceWithLength(pushCount)); // write the push and pop counts to the chunks - popChunk.set(ii, Math.toIntExact(popCount)); - pushChunk.set(ii, Math.toIntExact(pushCount)); - - // store the influencer keys - chunkBuilder.appendRowSequence(influencerKeyIt.getNextRowSequenceWithLength(pushCount)); + popChunk.set(affectedRowIndex, Math.toIntExact(popCount)); + pushChunk.set(affectedRowIndex, Math.toIntExact(pushCount)); } - // execute the operators - try (final RowSequence chunkInfluencerRs = chunkBuilder.build()) { - ensureGetContextSize(ctx, chunkInfluencerRs.size()); + final RowSequence chunkAffectedRows = affectedRowsIt.getNextRowSequenceWithLength(affectedRowIndex); + try (final RowSet chunkInfluencerRows = builder.build()) { + // final RowSequence chunkInfluencerRows = + // influencerRowsIt.getNextRowSequenceWithLength(totalPushCount); + // execute the operators + ensureGetContextSize(ctx, chunkInfluencerRows.size()); Arrays.fill(ctx.inputSourceChunks, null); for (int opIdx : context.dirtyOperatorIndices) { @@ -417,17 +419,17 @@ public void processRows(UpdateByWindowBucketContext context, boolean initialStep for (int ii = 0; ii < srcIndices.length; ii++) { int srcIdx = srcIndices[ii]; // chunk prep - prepareValuesChunkForSource(ctx, srcIdx, chunkInfluencerRs); + prepareValuesChunkForSource(ctx, srcIdx, chunkInfluencerRows); opCtx.chunkArr[ii] = ctx.inputSourceChunks[srcIdx]; } // make the specialized call for windowed operators ((UpdateByWindowedOperator.Context) ctx.opContext[opIdx]).accumulate( - chunkRs, + chunkAffectedRows, opCtx.chunkArr, pushChunk, popChunk, - chunkRsSize); + chunkAffectedRows.intSize()); } } } From ea478b70a8f41f805bbbc645973c7fa4fa0814c2 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 25 Jan 2023 17:32:39 -0800 Subject: [PATCH 096/123] UpdateBy operation spec changes --- .../api/updateby/spec/CumMinMaxSpec.java | 6 +- .../api/updateby/spec/CumProdSpec.java | 6 +- .../api/updateby/spec/CumSumSpec.java | 6 +- .../deephaven/api/updateby/spec/EmaSpec.java | 6 +- .../api/updateby/spec/RollingSumSpec.java | 85 +++++-------------- 5 files changed, 33 insertions(+), 76 deletions(-) diff --git a/table-api/src/main/java/io/deephaven/api/updateby/spec/CumMinMaxSpec.java b/table-api/src/main/java/io/deephaven/api/updateby/spec/CumMinMaxSpec.java index d63b7cdf8d1..daa648858d7 100644 --- a/table-api/src/main/java/io/deephaven/api/updateby/spec/CumMinMaxSpec.java +++ b/table-api/src/main/java/io/deephaven/api/updateby/spec/CumMinMaxSpec.java @@ -21,9 +21,9 @@ public static CumMinMaxSpec of(boolean isMax) { public final boolean applicableTo(Class inputType) { return // is primitive numeric? - inputType.equals(double.class) || inputType.equals(float.class) - || inputType.equals(int.class) || inputType.equals(long.class) || inputType.equals(short.class) - || inputType.equals(byte.class) + inputType == double.class || inputType == float.class + || inputType == int.class || inputType == long.class || inputType == short.class + || inputType == byte.class // is boxed numeric? || Number.class.isAssignableFrom(inputType) diff --git a/table-api/src/main/java/io/deephaven/api/updateby/spec/CumProdSpec.java b/table-api/src/main/java/io/deephaven/api/updateby/spec/CumProdSpec.java index 10070022a80..bce90dc9d17 100644 --- a/table-api/src/main/java/io/deephaven/api/updateby/spec/CumProdSpec.java +++ b/table-api/src/main/java/io/deephaven/api/updateby/spec/CumProdSpec.java @@ -17,9 +17,9 @@ public static CumProdSpec of() { public final boolean applicableTo(Class inputType) { return // is primitive numeric? - inputType.equals(double.class) || inputType.equals(float.class) - || inputType.equals(int.class) || inputType.equals(long.class) || inputType.equals(short.class) - || inputType.equals(byte.class) + inputType == double.class || inputType == float.class + || inputType == int.class || inputType == long.class || inputType == short.class + || inputType == byte.class // is boxed numeric? || Number.class.isAssignableFrom(inputType); diff --git a/table-api/src/main/java/io/deephaven/api/updateby/spec/CumSumSpec.java b/table-api/src/main/java/io/deephaven/api/updateby/spec/CumSumSpec.java index 6d78ba81047..cecffe68fc2 100644 --- a/table-api/src/main/java/io/deephaven/api/updateby/spec/CumSumSpec.java +++ b/table-api/src/main/java/io/deephaven/api/updateby/spec/CumSumSpec.java @@ -17,9 +17,9 @@ public static CumSumSpec of() { public final boolean applicableTo(Class inputType) { return // is primitive numeric? - inputType.equals(double.class) || inputType.equals(float.class) - || inputType.equals(int.class) || inputType.equals(long.class) || inputType.equals(short.class) - || inputType.equals(byte.class) + inputType == double.class || inputType == float.class + || inputType == int.class || inputType == long.class || inputType == short.class + || inputType == byte.class // is boxed numeric? || Number.class.isAssignableFrom(inputType) diff --git a/table-api/src/main/java/io/deephaven/api/updateby/spec/EmaSpec.java b/table-api/src/main/java/io/deephaven/api/updateby/spec/EmaSpec.java index c58ac2a764b..4a6f4c1d197 100644 --- a/table-api/src/main/java/io/deephaven/api/updateby/spec/EmaSpec.java +++ b/table-api/src/main/java/io/deephaven/api/updateby/spec/EmaSpec.java @@ -62,9 +62,9 @@ public final OperationControl controlOrDefault() { public final boolean applicableTo(Class inputType) { return // is primitive numeric? - inputType.equals(double.class) || inputType.equals(float.class) - || inputType.equals(int.class) || inputType.equals(long.class) || inputType.equals(short.class) - || inputType.equals(byte.class) + inputType == double.class || inputType == float.class + || inputType == int.class || inputType == long.class || inputType == short.class + || inputType == byte.class // is boxed numeric? || Number.class.isAssignableFrom(inputType); diff --git a/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java b/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java index 217d47bea43..7c012fac83d 100644 --- a/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java +++ b/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java @@ -17,22 +17,30 @@ public abstract class RollingSumSpec extends UpdateBySpecBase { // most common usages first, will complete the list later - public static RollingSumSpec ofTicks(long tickWindow) { - return of(WindowScale.ofTicks(tickWindow)); + public static RollingSumSpec ofTicks(long prevTicks) { + return of(WindowScale.ofTicks(prevTicks)); } - public static RollingSumSpec ofTicks(long prevTickWindow, long fwdTickWindow) { - return of(WindowScale.ofTicks(prevTickWindow), WindowScale.ofTicks(fwdTickWindow)); + public static RollingSumSpec ofTicks(long prevTicks, long fwdTicks) { + return of(WindowScale.ofTicks(prevTicks), WindowScale.ofTicks(fwdTicks)); } - public static RollingSumSpec ofTime(final String timestampCol, Duration prevWindowDuration) { - return of(WindowScale.ofTime(timestampCol, prevWindowDuration)); + public static RollingSumSpec ofTime(final String timestampCol, Duration prevDuration) { + return of(WindowScale.ofTime(timestampCol, prevDuration)); } - public static RollingSumSpec ofTime(final String timestampCol, Duration prevWindowDuration, - Duration fwdWindowDuration) { - return of(WindowScale.ofTime(timestampCol, prevWindowDuration), - WindowScale.ofTime(timestampCol, fwdWindowDuration)); + public static RollingSumSpec ofTime(final String timestampCol, Duration prevDuration, Duration fwdDuration) { + return of(WindowScale.ofTime(timestampCol, prevDuration), + WindowScale.ofTime(timestampCol, fwdDuration)); + } + + public static RollingSumSpec ofTime(final String timestampCol, long prevDuration) { + return of(WindowScale.ofTime(timestampCol, prevDuration)); + } + + public static RollingSumSpec ofTime(final String timestampCol, long prevDuration, long fwdDuration) { + return of(WindowScale.ofTime(timestampCol, prevDuration), + WindowScale.ofTime(timestampCol, fwdDuration)); } // general use constructors @@ -45,53 +53,6 @@ public static RollingSumSpec of(WindowScale prevWindowScale, WindowScale fwdWind return ImmutableRollingSumSpec.builder().prevTimeScale(prevWindowScale).fwdTimeScale(fwdWindowScale).build(); } - // public static RollingSumSpec of(WindowScale prevTimeScale) { - // return ImmutableWindowedOpSpec.builder().prevTimeScale(prevTimeScale).build(); - // } - // - // public static RollingSumSpec of(OperationControl control, WindowScale prevTimeScale, WindowScale fwdTimeScale) { - // return - // ImmutableWindowedOpSpec.builder().control(control).prevTimeScale(prevTimeScale).fwdTimeScale(fwdTimeScale).build(); - // } - // - // public static RollingSumSpec ofTime(final OperationControl control, - // final String timestampCol, - // long prevWindowTimeScaleNanos) { - // return of(control, WindowScale.ofTime(timestampCol, prevWindowTimeScaleNanos)); - // } - // - // public static RollingSumSpec ofTime(final OperationControl control, - // final String timestampCol, - // long prevWindowTimeScaleNanos, - // long fwdWindowTimeScaleNanos) { - // return of(control, WindowScale.ofTime(timestampCol, prevWindowTimeScaleNanos), WindowScale.ofTime(timestampCol, - // fwdWindowTimeScaleNanos)); - // } - // - // public static RollingSumSpec ofTime(final OperationControl control, - // final String timestampCol, - // Duration prevWindowDuration) { - // return of(control, WindowScale.ofTime(timestampCol, prevWindowDuration)); - // } - // - // - // public static RollingSumSpec ofTime(final OperationControl control, - // final String timestampCol, - // Duration prevWindowDuration, - // Duration fwdWindowDuration) { - // return of(control, WindowScale.ofTime(timestampCol, prevWindowDuration), WindowScale.ofTime(timestampCol, - // fwdWindowDuration)); - // } - // - // public static RollingSumSpec ofTicks(OperationControl control, long prevTickWindow) { - // return of(control, WindowScale.ofTicks(prevTickWindow)); - // } - // - // public static RollingSumSpec ofTicks(OperationControl control, long prevTickWindow, long fwdTickWindow) { - // return of(control, WindowScale.ofTicks(prevTickWindow), WindowScale.ofTicks(fwdTickWindow)); - // } - - public abstract Optional control(); public abstract WindowScale prevTimeScale(); @@ -104,17 +65,13 @@ public WindowScale fwdTimeScale() { return WindowScale.ofTicks(0); } - public final OperationControl controlOrDefault() { - return control().orElseGet(OperationControl::defaultInstance); - } - @Override public final boolean applicableTo(Class inputType) { return // is primitive numeric? - inputType.equals(double.class) || inputType.equals(float.class) - || inputType.equals(int.class) || inputType.equals(long.class) || inputType.equals(short.class) - || inputType.equals(byte.class) + inputType == double.class || inputType == float.class + || inputType == int.class || inputType == long.class || inputType == short.class + || inputType == byte.class // is boxed numeric? || Number.class.isAssignableFrom(inputType) From cecfd4807f9cbc32c81c723b6ad202a4954a2346 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 25 Jan 2023 17:33:33 -0800 Subject: [PATCH 097/123] Bug fixes to Pairwise ring buffers --- .../internal/PairwiseDoubleRingBuffer.java | 27 +++++++++--- .../internal/PairwiseFloatRingBuffer.java | 27 +++++++++--- .../PairwiseDoubleRingBufferTest.java | 42 +++++++++++++++++++ .../internal/PairwiseFloatRingBufferTest.java | 42 +++++++++++++++++++ 4 files changed, 128 insertions(+), 10 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java index 6f494c256ac..b1ec21fcaf7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java @@ -49,6 +49,7 @@ public class PairwiseDoubleRingBuffer implements SafeCloseable { private int dirtyPushHead; private int dirtyPushTail; + private boolean allDirty; private int dirtyPopHead; private int dirtyPopTail; @@ -179,7 +180,9 @@ public double evaluate() { final double value; - if (pushDirty && popDirty) { + if (allDirty) { + value = evaluateTree(capacity, chunkSize - 1); + } else if (pushDirty && popDirty) { if (dirtyPushHead > dirtyPushTail && dirtyPopHead > dirtyPopTail) { // both are wrapped value = evaluateTree(capacity, Math.max(dirtyPushTail, dirtyPopTail), Math.min(dirtyPushHead, dirtyPopHead), chunkSize - 1); @@ -190,11 +193,16 @@ public double evaluate() { // pop wrapped, push is not value = evaluateTree(capacity, dirtyPopTail, dirtyPushHead, dirtyPushTail, dirtyPopHead, chunkSize - 1); } else { - // neither wrapped - if (dirtyPushHead > dirtyPopHead) { - value = evaluateTree(dirtyPopHead, dirtyPopTail, dirtyPushHead, dirtyPushTail); + // neither wrapped, do they overlap? + if (dirtyPushTail >= dirtyPopHead && dirtyPopTail >= dirtyPushHead) { + value = evaluateTree(Math.min(dirtyPushHead, dirtyPopHead), Math.max(dirtyPushTail, dirtyPopTail)); } else { - value = evaluateTree(dirtyPushHead, dirtyPushTail, dirtyPopHead, dirtyPopTail); + // no overlap + if (dirtyPopHead < dirtyPushHead) { + value = evaluateTree(dirtyPopHead, dirtyPopTail, dirtyPushHead, dirtyPushTail); + } else { + value = evaluateTree(dirtyPushHead, dirtyPushTail, dirtyPopHead, dirtyPopTail); + } } } } else if (pushDirty) { @@ -276,6 +284,9 @@ public void pushUnsafe(double val) { storageChunk.set(tail, val); if (dirtyPushHead == NULL_INT) { dirtyPushHead = tail; + } else if (dirtyPushHead == tail) { + // wrapped around, everything will be dirty until evaluated + allDirty = true; } dirtyPushTail = tail; @@ -315,9 +326,13 @@ public double popUnsafe() { if (dirtyPopHead == NULL_INT) { dirtyPopHead = head; + } else if (dirtyPopHead == head) { + // wrapped around, everything will be dirty until evaluated + allDirty = true; } dirtyPopTail = head; + // move the head head = ((head + 1) % capacity) + capacity; size--; @@ -416,6 +431,8 @@ public int remaining() { private void clearDirty() { dirtyPushHead = dirtyPopHead = NULL_INT; dirtyPushTail = dirtyPopTail = NULL_INT; + + allDirty = false; } public void clear() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java index 4567bc1abb3..b49a5c4fc61 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java @@ -44,6 +44,7 @@ public class PairwiseFloatRingBuffer implements SafeCloseable { private int dirtyPushHead; private int dirtyPushTail; + private boolean allDirty; private int dirtyPopHead; private int dirtyPopTail; @@ -174,7 +175,9 @@ public float evaluate() { final float value; - if (pushDirty && popDirty) { + if (allDirty) { + value = evaluateTree(capacity, chunkSize - 1); + } else if (pushDirty && popDirty) { if (dirtyPushHead > dirtyPushTail && dirtyPopHead > dirtyPopTail) { // both are wrapped value = evaluateTree(capacity, Math.max(dirtyPushTail, dirtyPopTail), Math.min(dirtyPushHead, dirtyPopHead), chunkSize - 1); @@ -185,11 +188,16 @@ public float evaluate() { // pop wrapped, push is not value = evaluateTree(capacity, dirtyPopTail, dirtyPushHead, dirtyPushTail, dirtyPopHead, chunkSize - 1); } else { - // neither wrapped - if (dirtyPushHead > dirtyPopHead) { - value = evaluateTree(dirtyPopHead, dirtyPopTail, dirtyPushHead, dirtyPushTail); + // neither wrapped, do they overlap? + if (dirtyPushTail >= dirtyPopHead && dirtyPopTail >= dirtyPushHead) { + value = evaluateTree(Math.min(dirtyPushHead, dirtyPopHead), Math.max(dirtyPushTail, dirtyPopTail)); } else { - value = evaluateTree(dirtyPushHead, dirtyPushTail, dirtyPopHead, dirtyPopTail); + // no overlap + if (dirtyPopHead < dirtyPushHead) { + value = evaluateTree(dirtyPopHead, dirtyPopTail, dirtyPushHead, dirtyPushTail); + } else { + value = evaluateTree(dirtyPushHead, dirtyPushTail, dirtyPopHead, dirtyPopTail); + } } } } else if (pushDirty) { @@ -271,6 +279,9 @@ public void pushUnsafe(float val) { storageChunk.set(tail, val); if (dirtyPushHead == NULL_INT) { dirtyPushHead = tail; + } else if (dirtyPushHead == tail) { + // wrapped around, everything will be dirty until evaluated + allDirty = true; } dirtyPushTail = tail; @@ -310,9 +321,13 @@ public float popUnsafe() { if (dirtyPopHead == NULL_INT) { dirtyPopHead = head; + } else if (dirtyPopHead == head) { + // wrapped around, everything will be dirty until evaluated + allDirty = true; } dirtyPopTail = head; + // move the head head = ((head + 1) % capacity) + capacity; size--; @@ -411,6 +426,8 @@ public int remaining() { private void clearDirty() { dirtyPushHead = dirtyPopHead = NULL_INT; dirtyPushTail = dirtyPopTail = NULL_INT; + + allDirty = false; } public void clear() { diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBufferTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBufferTest.java index dac81fac5b7..9f633de4e89 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBufferTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBufferTest.java @@ -544,4 +544,46 @@ public void testPopMultiple() { } } } + + public void testSpecialCaseA() { + // overlapping push and pop ranges with popTail < pushTail + try (final PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(4, (double) 0, Double::sum)) { + rb.push((double)1); + assertEquals((double)1, rb.pop()); + rb.push((double)2); + rb.push((double)3); + assertEquals((double)5, rb.evaluate()); + } + } + + public void testSpecialCaseB() { + // push the full capacity while wrapped + try (final PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(64, (double) 0, Double::sum)) { + rb.push((double)1); + assertEquals((double)1, rb.pop()); + + for (int i = 0; i < 64; i++) { + rb.push((double)1); + } + assertEquals((double)64, rb.evaluate()); + } + } + + public void testSpecialCaseC() { + // overlapping push and pop ranges with popTail < pushTail + try (final PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(16, (double) 0, Double::sum)) { + // move pointers to middle of storage + for (int i = 0; i < 8; i++) { + rb.push((double)1); + rb.pop(); + } + assertEquals((double)0, rb.evaluate()); + + for (int i = 0; i < 11; i++) { + rb.push((double)1); + } + rb.pop(); + assertEquals((double)10, rb.evaluate()); + } + } } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBufferTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBufferTest.java index 5f02ac1d97e..e86322a375a 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBufferTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBufferTest.java @@ -539,4 +539,46 @@ public void testPopMultiple() { } } } + + public void testSpecialCaseA() { + // overlapping push and pop ranges with popTail < pushTail + try (final PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(4, (float) 0, Float::sum)) { + rb.push((float)1); + assertEquals((float)1, rb.pop()); + rb.push((float)2); + rb.push((float)3); + assertEquals((float)5, rb.evaluate()); + } + } + + public void testSpecialCaseB() { + // push the full capacity while wrapped + try (final PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(64, (float) 0, Float::sum)) { + rb.push((float)1); + assertEquals((float)1, rb.pop()); + + for (int i = 0; i < 64; i++) { + rb.push((float)1); + } + assertEquals((float)64, rb.evaluate()); + } + } + + public void testSpecialCaseC() { + // overlapping push and pop ranges with popTail < pushTail + try (final PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(16, (float) 0, Float::sum)) { + // move pointers to middle of storage + for (int i = 0; i < 8; i++) { + rb.push((float)1); + rb.pop(); + } + assertEquals((float)0, rb.evaluate()); + + for (int i = 0; i < 11; i++) { + rb.push((float)1); + } + rb.pop(); + assertEquals((float)10, rb.evaluate()); + } + } } From aaf7c58fcf446338a9fb6a998b6f0a25cd5cd8c1 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 25 Jan 2023 21:04:47 -0800 Subject: [PATCH 098/123] UpdateBy corrections and optimizations. Tests passing. --- .../engine/table/impl/updateby/UpdateBy.java | 46 +++--- .../updateby/UpdateByWindowCumulative.java | 7 + .../impl/updateby/UpdateByWindowTicks.java | 6 + .../impl/updateby/UpdateByWindowTime.java | 141 ++++++++++-------- .../impl/updateby/ZeroKeyUpdateByManager.java | 4 +- .../api/updateby/UpdateByOperation.java | 120 +++++++++++++-- 6 files changed, 227 insertions(+), 97 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java index 2bf1972748c..870cdc29768 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java @@ -206,6 +206,7 @@ protected UpdateBy( /** Release the input sources that will not be needed for the rest of this update */ private void releaseInputSources(int winIdx, ColumnSource[] maybeCachedInputSources, WritableRowSet[] inputSourceRowSets, int[] inputSourceReferenceCounts) { + final UpdateByWindow win = windows[winIdx]; final int[] uniqueWindowSources = win.getUniqueSourceIndices(); @@ -264,6 +265,7 @@ class PhasedUpdateProcessor implements LogOutputAppendable { final TableUpdate upstream; final boolean initialStep; final UpdateByBucketHelper[] dirtyBuckets; + final boolean[] dirtyWindows; /** The active set of sources to use for processing, each source may be cached or original */ final ColumnSource[] maybeCachedInputSources; @@ -291,6 +293,8 @@ class PhasedUpdateProcessor implements LogOutputAppendable { // determine which buckets we'll examine during this update dirtyBuckets = buckets.stream().filter(UpdateByBucketHelper::isDirty).toArray(UpdateByBucketHelper[]::new); + // which windows are dirty and need to be computed this cycle + dirtyWindows = new boolean[windows.length]; if (inputCacheNeeded) { maybeCachedInputSources = new ColumnSource[inputSources.length]; @@ -364,6 +368,7 @@ private void computeCachedColumnRowsets(final Runnable resumeAction) { (int) Arrays.stream(windows).filter(win -> win.isSourceInUse(srcIdx)).count(); } } + Arrays.fill(dirtyWindows, true); resumeAction.run(); return; } @@ -389,6 +394,8 @@ private void computeCachedColumnRowsets(final Runnable resumeAction) { } // at least one dirty bucket will need this source srcNeeded = true; + // this window must be computed + dirtyWindows[winIdx] = true; } } if (srcNeeded) { @@ -466,7 +473,7 @@ public void close() { // be exhausted and hasMore() will return false remaining -= PARALLEL_CACHE_CHUNK_SIZE; } - }, resumeAction::run, + }, resumeAction, this::onError); } @@ -475,15 +482,15 @@ public void close() { * when the work is complete */ private void cacheInputSources(final int winIdx, final Runnable resumeAction) { - if (inputCacheNeeded) { + if (inputCacheNeeded && dirtyWindows[winIdx]) { final UpdateByWindow win = windows[winIdx]; final int[] uniqueWindowSources = win.getUniqueSourceIndices(); jobScheduler.iterateParallel(ExecutionContext.getContextToRecord(), this, JobScheduler.JobContext::new, 0, uniqueWindowSources.length, - (context, idx, sourceComplete) -> { - createCachedColumnSource(uniqueWindowSources[idx], sourceComplete); - }, resumeAction, this::onError); + (context, idx, sourceComplete) -> createCachedColumnSource(uniqueWindowSources[idx], + sourceComplete), + resumeAction, this::onError); } else { // no work to do, continue resumeAction.run(); @@ -567,17 +574,21 @@ private void processWindows(final Runnable resumeAction) { } } - processWindowBuckets(winIdx, () -> { - if (inputCacheNeeded) { - // release the cached sources that are no longer needed - releaseInputSources(winIdx, maybeCachedInputSources, inputSourceRowSets, - inputSourceReferenceCounts); - } + if (dirtyWindows[winIdx]) { + processWindowBuckets(winIdx, () -> { + if (inputCacheNeeded) { + // release the cached sources that are no longer needed + releaseInputSources(winIdx, maybeCachedInputSources, inputSourceRowSets, + inputSourceReferenceCounts); + } - // signal that the work for this window is complete (will iterate to the next window - // sequentially) + // signal that the work for this window is complete (will iterate to the next window + // sequentially) + windowComplete.run(); + }); + } else { windowComplete.run(); - }); + } }); }, resumeAction, this::onError); } @@ -782,12 +793,7 @@ public void onUpdate(final TableUpdate upstream) { @Override public boolean canExecute(final long step) { - if (!upstreamSatisfied(step)) { - return false; - } - synchronized (buckets) { - return buckets.stream().allMatch(b -> b.result.satisfied(step)); - } + return upstreamSatisfied(step); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java index 0f68e97054e..1db34839378 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java @@ -109,6 +109,13 @@ void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, @NotNu context.affectedRows = smallestModifiedKey == Long.MAX_VALUE ? RowSetFactory.empty() : context.sourceRowSet.subSetByKeyRange(smallestModifiedKey, context.sourceRowSet.lastRowKey()); + + if (context.affectedRows.isEmpty()) { + // we really aren't dirty if no rows are affected by the update + context.isDirty = false; + return; + } + context.influencerRows = context.affectedRows; makeOperatorContexts(context); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java index 3b5aea19c72..13828d80d4b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java @@ -282,6 +282,12 @@ void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, @NotNu ctx.affectedRows = tmpAffected; + if (ctx.affectedRows.isEmpty()) { + // we really aren't dirty if no rows are affected by the update + ctx.isDirty = false; + return; + } + // now get influencer rows for the affected rows // generate position data rowsets for efficiently computed position offsets ctx.affectedRowPositions = ctx.sourceRowSet.invert(ctx.affectedRows); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java index fe70350ced3..f7eb7d15d19 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java @@ -264,20 +264,21 @@ public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, final WritableRowSet tmpAffected = RowSetFactory.empty(); if (upstream.modified().isNonempty()) { + // TODO: we can omit these checks if we can prove no input columns or timestamp columns were modified try (final RowSet modifiedAffected = computeAffectedRowsTime(ctx, upstream.modified(), prevUnits, fwdUnits, false)) { tmpAffected.insert(modifiedAffected); } + // modified timestamps and values will affect previous values + try (final WritableRowSet modifiedAffectedPrev = + computeAffectedRowsTime(ctx, upstream.getModifiedPreShift(), prevUnits, fwdUnits, true)) { + // we used the SSA (post-shift) to get these keys, no need to shift + // retain only the rows that still exist in the sourceRowSet + modifiedAffectedPrev.retain(ctx.timestampValidRowSet); + tmpAffected.insert(modifiedAffectedPrev); + } if (ctx.timestampsModified) { - // modified timestamps will affect previous values - try (final WritableRowSet modifiedAffectedPrev = - computeAffectedRowsTime(ctx, upstream.getModifiedPreShift(), prevUnits, fwdUnits, true)) { - // we used the SSA (post-shift) to get these keys, no need to shift - // retain only the rows that still exist in the sourceRowSet - modifiedAffectedPrev.retain(ctx.timestampValidRowSet); - tmpAffected.insert(modifiedAffectedPrev); - } - // compute modified rows only if timestamps have changed + // compute all modified rows only if timestamps have changed tmpAffected.insert(upstream.modified()); } } @@ -305,6 +306,12 @@ public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, ctx.affectedRows = tmpAffected; + if (ctx.affectedRows.isEmpty()) { + // we really aren't dirty if no rows are affected by the update + ctx.isDirty = false; + return; + } + // now get influencer rows for the affected rows ctx.influencerRows = computeInfluencerRowsTime(ctx, ctx.affectedRows, prevUnits, fwdUnits, false); @@ -345,30 +352,47 @@ public void processRows(UpdateByWindowBucketContext context, boolean initialStep final WritableIntChunk popChunk = WritableIntChunk.makeWritableChunk(ctx.workingChunkSize)) { - long currentTailTs = nextLongOrMax(influencerTsTailIt); + long influencerTs = nextLongOrMax(influencerTsTailIt); + + Assert.eqTrue(affectedTsIt.hasNext(), "affectedTsIt.hasNext()"); + + final long EXHAUSTED = -1L; + // long currentTimestamp = affectedTsIt.nextLong(); + long affectedTs = affectedTsIt.hasNext() ? affectedTsIt.nextLong() : EXHAUSTED; while (affectedRowsIt.hasMore()) { // NOTE: we did not put null values into our SSA and our influencer rowset is built using the // SSA. there should be no null timestamps considered in the rolling windows - RowSetBuilderSequential builder = RowSetFactory.builderSequential(); - int affectedRowIndex; - for (affectedRowIndex = 0; affectedRowIndex < ctx.workingChunkSize - && affectedTsIt.hasNext(); affectedRowIndex++) { + long totalPushCount = 0; + long skipCount = 0; - // read the current timestamp - final long currentTimestamp = affectedTsIt.nextLong(); - if (currentTimestamp == NULL_LONG) { + for (affectedRowIndex = 0; affectedRowIndex < ctx.workingChunkSize + && affectedTs != EXHAUSTED; affectedRowIndex++) { + if (affectedTs == NULL_LONG) { // this signifies that does not belong to a time window popChunk.set(affectedRowIndex, NULL_INT); pushChunk.set(affectedRowIndex, NULL_INT); + affectedTs = affectedTsIt.hasNext() ? affectedTsIt.nextLong() : EXHAUSTED; continue; } // compute the head and tail timestamps (inclusive) - final long head = currentTimestamp - prevUnits; - final long tail = currentTimestamp + fwdUnits; + final long head = affectedTs - prevUnits; + final long tail = affectedTs + fwdUnits; + + // advance the keyIt and timestamp iterators until we are within the window. This only happens + // when initialStep == true because we have not created the minimum set of rows but include all + // non-null timestamp rows in our influencer values + while (influencerTs < head) { + Assert.eqTrue(ctx.initialStep, "initialStep when skipping rows"); + influencerTs = nextLongOrMax(influencerTsTailIt); + skipCount++; + } + if (skipCount > 0) { + break; + } // pop out all values from the current window that are not in the new window long popCount = 0; @@ -377,60 +401,55 @@ public void processRows(UpdateByWindowBucketContext context, boolean initialStep popCount++; } - // advance the keyIt and timestamp iterators until we are within the window. This only happens - // when initialStep == true because we have not created the minimum set of rows but include all - // non-null timestamp rows in our influencer values - long skipCount = 0; - while (currentTailTs < head) { - Assert.eqTrue(initialStep, "initialStep when skipping rows"); - currentTailTs = nextLongOrMax(influencerTsTailIt); - skipCount++; - } - influencerRowsIt.getNextRowSequenceWithLength(skipCount); - // push in all values that are in the new window (inclusive of tail) long pushCount = 0; - while (currentTailTs <= tail) { + while (influencerTs <= tail) { // add this value to the buffer before advancing - ctx.timestampWindowBuffer.add(currentTailTs); - currentTailTs = nextLongOrMax(influencerTsTailIt); + ctx.timestampWindowBuffer.add(influencerTs); + influencerTs = nextLongOrMax(influencerTsTailIt); pushCount++; } - builder.appendRowSequence(influencerRowsIt.getNextRowSequenceWithLength(pushCount)); // write the push and pop counts to the chunks popChunk.set(affectedRowIndex, Math.toIntExact(popCount)); pushChunk.set(affectedRowIndex, Math.toIntExact(pushCount)); - } + totalPushCount += pushCount; + affectedTs = affectedTsIt.hasNext() ? affectedTsIt.nextLong() : EXHAUSTED; + } final RowSequence chunkAffectedRows = affectedRowsIt.getNextRowSequenceWithLength(affectedRowIndex); - try (final RowSet chunkInfluencerRows = builder.build()) { - // final RowSequence chunkInfluencerRows = - // influencerRowsIt.getNextRowSequenceWithLength(totalPushCount); - // execute the operators - ensureGetContextSize(ctx, chunkInfluencerRows.size()); - - Arrays.fill(ctx.inputSourceChunks, null); - for (int opIdx : context.dirtyOperatorIndices) { - UpdateByWindowedOperator.Context opCtx = - (UpdateByWindowedOperator.Context) context.opContext[opIdx]; - // prep the chunk array needed by the accumulate call - final int[] srcIndices = operatorInputSourceSlots[opIdx]; - for (int ii = 0; ii < srcIndices.length; ii++) { - int srcIdx = srcIndices[ii]; - // chunk prep - prepareValuesChunkForSource(ctx, srcIdx, chunkInfluencerRows); - opCtx.chunkArr[ii] = ctx.inputSourceChunks[srcIdx]; - } - - // make the specialized call for windowed operators - ((UpdateByWindowedOperator.Context) ctx.opContext[opIdx]).accumulate( - chunkAffectedRows, - opCtx.chunkArr, - pushChunk, - popChunk, - chunkAffectedRows.intSize()); + final RowSequence chunkInfluencerRows = influencerRowsIt.getNextRowSequenceWithLength(totalPushCount); + + // execute the operators + ensureGetContextSize(ctx, chunkInfluencerRows.size()); + + Arrays.fill(ctx.inputSourceChunks, null); + for (int opIdx : context.dirtyOperatorIndices) { + UpdateByWindowedOperator.Context opCtx = + (UpdateByWindowedOperator.Context) context.opContext[opIdx]; + // prep the chunk array needed by the accumulate call + final int[] srcIndices = operatorInputSourceSlots[opIdx]; + for (int ii = 0; ii < srcIndices.length; ii++) { + int srcIdx = srcIndices[ii]; + // chunk prep + prepareValuesChunkForSource(ctx, srcIdx, chunkInfluencerRows); + opCtx.chunkArr[ii] = ctx.inputSourceChunks[srcIdx]; } + + // make the specialized call for windowed operators + ((UpdateByWindowedOperator.Context) ctx.opContext[opIdx]).accumulate( + chunkAffectedRows, + opCtx.chunkArr, + pushChunk, + popChunk, + chunkAffectedRows.intSize()); + } + + // dump these rows + if (skipCount > 0) { + final long pos = context.influencerRows.find(influencerRowsIt.peekNextKey()) + skipCount; + final long key = context.influencerRows.get(pos); + influencerRowsIt.advance(key); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java index 66384170cfa..095fd20115a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java @@ -96,7 +96,7 @@ protected QueryTable result() { @Override protected boolean upstreamSatisfied(final long step) { - // for Zero-Key, only need to verify the source is satisfied - return source.satisfied(step); + // for Zero-Key, verify the source and the single bucket are satisfied + return source.satisfied(step) && zeroKeyUpdateBy.result.satisfied(step); } } diff --git a/table-api/src/main/java/io/deephaven/api/updateby/UpdateByOperation.java b/table-api/src/main/java/io/deephaven/api/updateby/UpdateByOperation.java index 2fa345a031e..822c28d5349 100644 --- a/table-api/src/main/java/io/deephaven/api/updateby/UpdateByOperation.java +++ b/table-api/src/main/java/io/deephaven/api/updateby/UpdateByOperation.java @@ -217,41 +217,133 @@ static UpdateByOperation Ema(OperationControl control, String timestampColumn, D /** * Create a {@link RollingSumSpec rolling sum} for the supplied column name pairs, using ticks as the windowing - * unit. Uses the default OperationControl settings. + * unit. Ticks are row counts and you may specify the previous window in number of rows to include. The current row + * is considered to belong to the reverse window, so calling this with {@code prevTicks = 1} will simply return the + * current row. Specifying {@code prevTicks = 10} will include the previous 9 rows to this one and this row for a + * total of 10 rows. * - * @param prevTimeTicks the look-behind window size (in rows/ticks) + * @param prevTicks the look-behind window size (in rows/ticks) * @param pairs The input/output column name pairs * @return The aggregation */ - static UpdateByOperation RollingSum(long prevTimeTicks, String... pairs) { - return RollingSumSpec.ofTicks(prevTimeTicks).clause(pairs); + static UpdateByOperation RollingSum(long prevTicks, String... pairs) { + return RollingSumSpec.ofTicks(prevTicks).clause(pairs); } /** * Create a {@link RollingSumSpec rolling sum} for the supplied column name pairs, using ticks as the windowing - * unit. Uses the default OperationControl settings. + * unit. Ticks are row counts and you may specify the previous and forward window in number of rows to include. The + * current row is considered to belong to the reverse window but not the forward window. Also, negative values are + * allowed and can be used to generate completely forward or completely reverse windows. Here are some examples of + * window values: + *
                + *
              • {@code prevTicks = 1, fwdTicks = 0} - contains only the current row
              • + *
              • {@code prevTicks = 10, fwdTicks = 0} - contains 9 previous rows and the current row
              • + *
              • {@code prevTicks = 0, fwdTicks = 10} - contains the following 10 rows, excludes the current row
              • + *
              • {@code prevTicks = 10, fwdTicks = 10} - contains the previous 9 rows, the current row and the 10 rows + * following
              • + *
              • {@code prevTicks = 10, fwdTicks = -5} - contains 5 rows, beginning at 9 rows before, ending at 5 rows before + * the current row (inclusive)
              • + *
              • {@code prevTicks = 11, fwdTicks = -1} - contains 10 rows, beginning at 10 rows before, ending at 1 row before + * the current row (inclusive)
              • + *
              • {@code prevTicks = -5, fwdTicks = 10} - contains 5 rows, beginning 5 rows following, ending at 10 rows + * following the current row (inclusive)
              • + *
              + * + * @param prevTicks the look-behind window size (in rows/ticks) + * @param fwdTicks the look-ahead window size (in rows/ticks) + * @param pairs The input/output column name pairs + * @return The aggregation + */ + static UpdateByOperation RollingSum(long prevTicks, long fwdTicks, String... pairs) { + return RollingSumSpec.ofTicks(prevTicks, fwdTicks).clause(pairs); + } + + /** + * Create a {@link RollingSumSpec rolling sum} for the supplied column name pairs, using time as the windowing unit. + * This function accepts {@link Duration duration} as the prev window parameter. A row that contains a {@code null} + * in the timestamp column belongs to no window and will not have a value computed or be considered in the windows + * of other rows. + * + * Here are some examples of window values: + *
                + *
              • {@code prevDuration = 0m} - contains rows that exactly match the current row timestamp
              • + *
              • {@code prevDuration = 10m} - contains rows from 10m earlier through the current row timestamp + * (inclusive)
              • + *
              * - * @param prevTimeTicks the look-behind window size (in rows/ticks) - * @param fwdTimeTicks the look-ahead window size (in rows/ticks) + * @param timestampCol the name of the timestamp column + * @param prevDuration the look-behind window size (in Duration) * @param pairs The input/output column name pairs * @return The aggregation */ - static UpdateByOperation RollingSum(long prevTimeTicks, long fwdTimeTicks, String... pairs) { - return RollingSumSpec.ofTicks(prevTimeTicks, fwdTimeTicks).clause(pairs); + static UpdateByOperation RollingSum(String timestampCol, Duration prevDuration, String... pairs) { + return RollingSumSpec.ofTime(timestampCol, prevDuration).clause(pairs); } /** * Create a {@link RollingSumSpec rolling sum} for the supplied column name pairs, using time as the windowing unit. - * Uses the default OperationControl settings. + * This function accepts {@link Duration durations} as the prev and forward window parameters. Negative values are + * allowed and can be used to generate completely forward or completely reverse windows. A row that contains a + * {@code null} in the timestamp column belongs to no window and will not have a value computed or be considered in + * the windows of other rows. * - * @param prevWindowDuration the look-behind window size (in Duration) - * @param fwdWindowDuration the look-ahead window size (in Duration) + * Here are some examples of window values: + *
                + *
              • {@code prevDuration = 0m, fwdDuration = 0m} - contains rows that exactly match the current row timestamp
              • + *
              • {@code prevDuration = 10m, fwdDuration = 0m} - contains rows from 10m earlier through the current row + * timestamp (inclusive)
              • + *
              • {@code prevDuration = 0m, fwdDuration = 10m} - contains rows from the current row through 10m following the + * current row timestamp (inclusive)
              • + *
              • {@code prevDuration = 10m, fwdDuration = 10m} - contains rows from 10m earlier through 10m following the + * current row timestamp (inclusive)
              • + *
              • {@code prevDuration = 10m, fwdDuration = -5m} - contains rows from 10m earlier through 5m before the current + * row timestamp (inclusive), this is a purely backwards looking window
              • + *
              • {@code prevDuration = -5m, fwdDuration = 10m} - contains rows from 5m following through 10m following the + * current row timestamp (inclusive), this is a purely forwards looking window
              • + *
              + * + * @param timestampCol the name of the timestamp column + * @param prevDuration the look-behind window size (in Duration) + * @param fwdDuration the look-ahead window size (in Duration) * @param pairs The input/output column name pairs * @return The aggregation */ - static UpdateByOperation RollingSum(String timestampCol, Duration prevWindowDuration, Duration fwdWindowDuration, + static UpdateByOperation RollingSum(String timestampCol, Duration prevDuration, Duration fwdDuration, String... pairs) { - return RollingSumSpec.ofTime(timestampCol, prevWindowDuration, fwdWindowDuration).clause(pairs); + return RollingSumSpec.ofTime(timestampCol, prevDuration, fwdDuration).clause(pairs); + } + + /** + * Create a {@link RollingSumSpec rolling sum} for the supplied column name pairs, using time as the windowing unit. + * This function accepts {@code nanoseconds} as the prev window parameters. A row that contains a {@code null} in + * the timestamp column belongs to no window and will not have a value computed or be considered in the windows of + * other rows. + * + * @param timestampCol the name of the timestamp column + * @param prevNanos the look-behind window size (in nanoseconds) + * @param pairs The input/output column name pairs + * @return The aggregation + */ + static UpdateByOperation RollingSum(String timestampCol, long prevNanos, String... pairs) { + return RollingSumSpec.ofTime(timestampCol, prevNanos).clause(pairs); + } + + /** + * Create a {@link RollingSumSpec rolling sum} for the supplied column name pairs, using time as the windowing unit. + * This function accepts {@code nanoseconds} as the prev and forward window parameters. Negative values are allowed + * and can be used to generate completely forward or completely reverse windows. A row that contains a {@code null} + * in the timestamp column belongs to no window and will not have a value computed or be considered in the windows + * of other rows. + * + * @param timestampCol the name of the timestamp column + * @param prevNanos the look-behind window size (in nanoseconds) + * @param fwdNanos the look-ahead window size (in nanoseconds) + * @param pairs The input/output column name pairs + * @return The aggregation + */ + static UpdateByOperation RollingSum(String timestampCol, long prevNanos, long fwdNanos, String... pairs) { + return RollingSumSpec.ofTime(timestampCol, prevNanos, fwdNanos).clause(pairs); } T walk(Visitor visitor); From 1a259e4af4a1f8c66322cd788a42ac43cadab88c Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Thu, 26 Jan 2023 12:57:45 -0800 Subject: [PATCH 099/123] Optimizing when to consider modified rows. --- .../impl/updateby/UpdateByBucketHelper.java | 24 +++---- .../table/impl/updateby/UpdateByWindow.java | 72 ++++++++++++++++--- .../updateby/UpdateByWindowCumulative.java | 59 ++++++--------- .../impl/updateby/UpdateByWindowTicks.java | 39 +++------- .../impl/updateby/UpdateByWindowTime.java | 64 ++++++----------- 5 files changed, 124 insertions(+), 134 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java index a23eafd083a..7ec7adc1336 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java @@ -18,11 +18,13 @@ import io.deephaven.engine.table.impl.sources.ReinterpretUtils; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.util.RowRedirection; +import io.deephaven.util.SafeCloseable; import io.deephaven.util.datastructures.linked.IntrusiveDoublyLinkedNode; import org.apache.commons.lang3.mutable.MutableLong; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import java.util.Arrays; import java.util.Map; import static io.deephaven.util.QueryConstants.NULL_LONG; @@ -127,25 +129,23 @@ UpdateByBucketHelperListener newListener(@NotNull final String description) { return new UpdateByBucketHelperListener(description, source); } - private void processUpdateForSsa(TableUpdate upstream) { + private void processUpdateForSsa(final TableUpdate upstream, final boolean timestampsModified) { if (upstream.empty()) { return; } - final boolean stampModified = upstream.modifiedColumnSet().containsAny(timestampColumnSet); - final int chunkSize = 1 << 12; // 4k - try (final RowSet addedAndModified = stampModified ? upstream.added().union(upstream.modified()) : null; + try (final RowSet addedAndModified = timestampsModified ? upstream.added().union(upstream.modified()) : null; final RowSet removedAndModifiedPreShift = - stampModified ? upstream.removed().union(upstream.getModifiedPreShift()) : null; + timestampsModified ? upstream.removed().union(upstream.getModifiedPreShift()) : null; final ChunkSource.GetContext context = timestampColumnSource.makeGetContext(chunkSize); final WritableLongChunk ssaValues = WritableLongChunk.makeWritableChunk(chunkSize); final WritableLongChunk ssaKeys = WritableLongChunk.makeWritableChunk(chunkSize); final WritableLongChunk nullTsKeys = WritableLongChunk.makeWritableChunk(chunkSize)) { - final RowSet restampAdditions = stampModified ? addedAndModified : upstream.added(); - final RowSet restampRemovals = stampModified ? removedAndModifiedPreShift : upstream.removed(); + final RowSet restampAdditions = timestampsModified ? addedAndModified : upstream.added(); + final RowSet restampRemovals = timestampsModified ? removedAndModifiedPreShift : upstream.removed(); // removes if (restampRemovals.isNonempty()) { @@ -298,11 +298,11 @@ public void prepareForUpdate(final TableUpdate upstream, final boolean initialSt // add all the SSA data if (timestampColumnName != null) { - processUpdateForSsa(upstream); - // test whether any timestamps were modified timestampsModified = upstream.modified().isNonempty() && upstream.modifiedColumnSet().containsAny(timestampColumnSet); + + processUpdateForSsa(upstream, timestampsModified); } else { timestampsModified = false; } @@ -368,10 +368,8 @@ public void processWindow(final int winIdx, final boolean initialStep) { * Close the window contexts and release resources for this bucket */ public void finalizeUpdate() { - for (int winIdx = 0; winIdx < windows.length; winIdx++) { - windowContexts[winIdx].close(); - windowContexts[winIdx] = null; - } + SafeCloseable.closeArray(windowContexts); + Arrays.fill(windowContexts, null); isDirty = false; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java index 2194af13891..5fa444b8d15 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java @@ -4,18 +4,21 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.*; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.TrackingRowSet; import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.util.SafeCloseable; -import io.deephaven.util.SafeCloseableArray; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import java.util.Arrays; +import java.util.BitSet; import java.util.Objects; +import java.util.stream.IntStream; abstract class UpdateByWindow { @Nullable @@ -45,7 +48,7 @@ class UpdateByWindowBucketContext implements SafeCloseable { /** Were any timestamps modified in the current update? */ protected final boolean timestampsModified; /** An array of context objects for each underlying operator */ - protected final UpdateByOperator.UpdateContext[] opContext; + protected final UpdateByOperator.UpdateContext[] opContexts; /** Whether this is the creation phase of this window */ protected final boolean initialStep; @@ -67,6 +70,8 @@ class UpdateByWindowBucketContext implements SafeCloseable { protected int[] dirtyOperatorIndices; /** Indicates which sources are needed to process this window context */ protected int[] dirtySourceIndices; + /** Were any input columns modified in the current update? */ + protected boolean inputModified; UpdateByWindowBucketContext(final TrackingRowSet sourceRowSet, @Nullable final ColumnSource timestampColumnSource, @@ -81,7 +86,7 @@ class UpdateByWindowBucketContext implements SafeCloseable { this.timestampValidRowSet = timestampValidRowSet; this.timestampsModified = timestampsModified; - this.opContext = new UpdateByOperator.UpdateContext[operators.length]; + this.opContexts = new UpdateByOperator.UpdateContext[operators.length]; this.workingChunkSize = chunkSize; this.initialStep = initialStep; @@ -95,10 +100,11 @@ public void close() { influencerRows == affectedRows || influencerRows == timestampValidRowSet ? null : influencerRows) { } - SafeCloseableArray.close(opContext); - if (inputSources != null) { - SafeCloseableArray.close(inputSourceGetContexts); - } + SafeCloseable.closeArray(opContexts); + Arrays.fill(opContexts, null); + + SafeCloseable.closeArray(inputSourceGetContexts); + Arrays.fill(inputSourceGetContexts, null); } } @@ -292,6 +298,56 @@ RowSet getInfluencerRows(final UpdateByWindowBucketContext context) { return context.influencerRows; } + /** + * Examines the {@link TableUpdate update} and set the context dirty bits appropriately + * + * @param context the window context that will manage the results. + * @param upstream the update to process. + */ + void processUpdateForContext(UpdateByWindowBucketContext context, @NotNull TableUpdate upstream) { + boolean addsOrRemoves = upstream.added().isNonempty() || upstream.removed().isNonempty(); + + if (addsOrRemoves) { + // mark all operators as affected by this update + context.dirtyOperatorIndices = IntStream.range(0, operators.length).toArray(); + context.dirtySourceIndices = getUniqueSourceIndices(); + context.isDirty = true; + + // still need to compute whether any input columns were modified + for (UpdateByOperator op : operators) { + final boolean opInputModified = upstream.modifiedColumnSet().nonempty() && + (op.getInputModifiedColumnSet() == null || + upstream.modifiedColumnSet().containsAny(op.getInputModifiedColumnSet())); + + if (opInputModified) { + context.inputModified = true; + break; + } + } + } else { + // determine which operators are affected by this update and whether any input columns were modified + BitSet dirtyOperators = new BitSet(); + BitSet dirtySourceIndices = new BitSet(); + + for (int opIdx = 0; opIdx < operators.length; opIdx++) { + UpdateByOperator op = operators[opIdx]; + final boolean opInputModified = upstream.modifiedColumnSet().nonempty() && + (op.getInputModifiedColumnSet() == null || + upstream.modifiedColumnSet().containsAny(op.getInputModifiedColumnSet())); + + if (opInputModified) { + dirtyOperators.set(opIdx); + Arrays.stream(operatorInputSourceSlots[opIdx]).forEach(dirtySourceIndices::set); + context.inputModified = true; + } + } + + context.isDirty = !dirtyOperators.isEmpty(); + context.dirtyOperatorIndices = dirtyOperators.stream().toArray(); + context.dirtySourceIndices = dirtySourceIndices.stream().toArray(); + } + } + // endregion /** diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java index 1db34839378..8a29ebe9d33 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java @@ -12,7 +12,6 @@ import org.jetbrains.annotations.Nullable; import java.util.Arrays; -import java.util.BitSet; import java.util.stream.IntStream; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; @@ -35,7 +34,7 @@ private void makeOperatorContexts(UpdateByWindowBucketContext context) { // create contexts for the affected operators for (int opIdx : context.dirtyOperatorIndices) { - context.opContext[opIdx] = operators[opIdx].makeUpdateContext(context.workingChunkSize, + context.opContexts[opIdx] = operators[opIdx].makeUpdateContext(context.workingChunkSize, operatorInputSourceSlots[opIdx].length); } } @@ -68,43 +67,21 @@ void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, @NotNu context.dirtySourceIndices = getUniqueSourceIndices(); makeOperatorContexts(context); - context.isDirty = !upstream.empty(); + context.isDirty = true; return; } // determine which operators are affected by this update - context.isDirty = false; - boolean allAffected = upstream.added().isNonempty() || - upstream.removed().isNonempty(); - - if (allAffected) { - // mark all operators as affected by this update - context.dirtyOperatorIndices = IntStream.range(0, operators.length).toArray(); - context.dirtySourceIndices = getUniqueSourceIndices(); - context.isDirty = true; - } else { - // determine which operators are affected by this update - BitSet dirtyOperators = new BitSet(); - BitSet dirtySourceIndices = new BitSet(); - - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - UpdateByOperator op = operators[opIdx]; - if (upstream.modifiedColumnSet().nonempty() && (op.getInputModifiedColumnSet() == null - || upstream.modifiedColumnSet().containsAny(op.getInputModifiedColumnSet()))) { - dirtyOperators.set(opIdx); - Arrays.stream(operatorInputSourceSlots[opIdx]).forEach(dirtySourceIndices::set); - } - } - context.isDirty = !dirtyOperators.isEmpty(); - context.dirtyOperatorIndices = dirtyOperators.stream().toArray(); - context.dirtySourceIndices = dirtySourceIndices.stream().toArray(); - } + processUpdateForContext(context, upstream); if (!context.isDirty) { return; } - long smallestModifiedKey = smallestAffectedKey(upstream, context.sourceRowSet); + // we can ignore modifications if they do not affect our input columns + final boolean inputModified = context.inputModified + || (timestampColumnName != null && context.timestampsModified); + long smallestModifiedKey = smallestAffectedKey(upstream, context.sourceRowSet, inputModified); context.affectedRows = smallestModifiedKey == Long.MAX_VALUE ? RowSetFactory.empty() @@ -129,7 +106,7 @@ void processRows(UpdateByWindowBucketContext context, final boolean initialStep) // always at the beginning of the RowSet at creation phase for (int opIdx : context.dirtyOperatorIndices) { UpdateByCumulativeOperator cumOp = (UpdateByCumulativeOperator) operators[opIdx]; - cumOp.initializeUpdate(context.opContext[opIdx], NULL_ROW_KEY, NULL_LONG); + cumOp.initializeUpdate(context.opContexts[opIdx], NULL_ROW_KEY, NULL_LONG); } } else { // find the key before the first affected row @@ -141,12 +118,12 @@ void processRows(UpdateByWindowBucketContext context, final boolean initialStep) UpdateByCumulativeOperator cumOp = (UpdateByCumulativeOperator) operators[opIdx]; if (cumOp.getTimestampColumnName() == null || keyBefore == NULL_ROW_KEY) { // this operator doesn't care about timestamps or we know we are at the beginning of the rowset - cumOp.initializeUpdate(context.opContext[opIdx], keyBefore, NULL_LONG); + cumOp.initializeUpdate(context.opContexts[opIdx], keyBefore, NULL_LONG); } else { // this operator cares about timestamps, so make sure it is starting from a valid value and // valid timestamp by looking backward until the conditions are met UpdateByCumulativeOperator.Context cumOpContext = - (UpdateByCumulativeOperator.Context) context.opContext[opIdx]; + (UpdateByCumulativeOperator.Context) context.opContexts[opIdx]; long potentialResetTimestamp = context.timestampColumnSource.getLong(keyBefore); if (potentialResetTimestamp == NULL_LONG || !cumOpContext.isValueValid(keyBefore)) { @@ -164,7 +141,7 @@ void processRows(UpdateByWindowBucketContext context, final boolean initialStep) } } // call the specialized version of `intializeUpdate()` for these operators - cumOp.initializeUpdate(context.opContext[opIdx], keyBefore, potentialResetTimestamp); + cumOp.initializeUpdate(context.opContexts[opIdx], keyBefore, potentialResetTimestamp); } } } @@ -184,7 +161,7 @@ void processRows(UpdateByWindowBucketContext context, final boolean initialStep) for (int opIdx : context.dirtyOperatorIndices) { UpdateByCumulativeOperator.Context opCtx = - (UpdateByCumulativeOperator.Context) context.opContext[opIdx]; + (UpdateByCumulativeOperator.Context) context.opContexts[opIdx]; // prep the chunk array needed by the accumulate call final int[] srcIndices = operatorInputSourceSlots[opIdx]; for (int ii = 0; ii < srcIndices.length; ii++) { @@ -195,7 +172,7 @@ void processRows(UpdateByWindowBucketContext context, final boolean initialStep) } // make the specialized call for cumulative operators - ((UpdateByCumulativeOperator.Context) context.opContext[opIdx]).accumulate( + ((UpdateByCumulativeOperator.Context) context.opContexts[opIdx]).accumulate( rs, opCtx.chunkArr, tsChunk, @@ -206,7 +183,7 @@ void processRows(UpdateByWindowBucketContext context, final boolean initialStep) // call `finishUpdate()` function for each operator for (int opIdx : context.dirtyOperatorIndices) { - operators[opIdx].finishUpdate(context.opContext[opIdx]); + operators[opIdx].finishUpdate(context.opContexts[opIdx]); } } @@ -216,11 +193,14 @@ void processRows(UpdateByWindowBucketContext context, final boolean initialStep) * * @param upstream the {@link TableUpdate update} from upstream * @param affectedRowSet the {@link TrackingRowSet rowset} for the current bucket + * @param inputModified whether the input columns for this window were modified * * @return the smallest key that participated in any part of the update. This will be the minimum of the first key * of each of added, modified and removed (post-shift) rows. */ - private static long smallestAffectedKey(@NotNull TableUpdate upstream, @NotNull TrackingRowSet affectedRowSet) { + private static long smallestAffectedKey(final @NotNull TableUpdate upstream, + final @NotNull TrackingRowSet affectedRowSet, + final boolean inputModified) { long smallestModifiedKey = Long.MAX_VALUE; if (upstream.removed().isNonempty()) { @@ -249,7 +229,8 @@ private static long smallestAffectedKey(@NotNull TableUpdate upstream, @NotNull smallestModifiedKey = Math.min(smallestModifiedKey, upstream.added().firstRowKey()); } - if (upstream.modified().isNonempty()) { + // consider the modifications only when input columns were modified + if (upstream.modified().isNonempty() && inputModified) { smallestModifiedKey = Math.min(smallestModifiedKey, upstream.modified().firstRowKey()); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java index 13828d80d4b..f62271f07ef 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java @@ -1,7 +1,5 @@ package io.deephaven.engine.table.impl.updateby; -import gnu.trove.list.array.TIntArrayList; -import gnu.trove.set.hash.TIntHashSet; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableIntChunk; @@ -74,7 +72,7 @@ private void makeOperatorContexts(UpdateByWindowBucketContext context) { // create contexts for the affected operators for (int opIdx : context.dirtyOperatorIndices) { - context.opContext[opIdx] = operators[opIdx].makeUpdateContext(context.workingChunkSize, + context.opContexts[opIdx] = operators[opIdx].makeUpdateContext(context.workingChunkSize, operatorInputSourceSlots[opIdx].length); } } @@ -212,29 +210,7 @@ void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, @NotNu } // determine which operators are affected by this update - boolean allAffected = upstream.added().isNonempty() || upstream.removed().isNonempty(); - - if (allAffected) { - // mark all operators as affected by this update - context.dirtyOperatorIndices = IntStream.range(0, operators.length).toArray(); - context.dirtySourceIndices = getUniqueSourceIndices(); - context.isDirty = true; - } else { - // determine which operators were affected by this update - TIntArrayList dirtyOperatorList = new TIntArrayList(operators.length); - TIntHashSet inputSourcesSet = new TIntHashSet(getUniqueSourceIndices().length); - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - UpdateByOperator op = operators[opIdx]; - if (upstream.modifiedColumnSet().nonempty() && (op.getInputModifiedColumnSet() == null - || upstream.modifiedColumnSet().containsAny(op.getInputModifiedColumnSet()))) { - dirtyOperatorList.add(opIdx); - inputSourcesSet.addAll(operatorInputSourceSlots[opIdx]); - context.isDirty = true; - } - } - context.dirtyOperatorIndices = dirtyOperatorList.toArray(); - context.dirtySourceIndices = inputSourcesSet.toArray(); - } + processUpdateForContext(context, upstream); if (!ctx.isDirty) { return; @@ -243,7 +219,8 @@ void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, @NotNu // need a writable rowset final WritableRowSet tmpAffected = RowSetFactory.empty(); - if (upstream.modified().isNonempty()) { + // consider the modifications only when input columns were modified + if (upstream.modified().isNonempty() && ctx.inputModified) { // compute the rows affected from these changes try (final WritableRowSet modifiedInverted = ctx.sourceRowSet.invert(upstream.modified()); final RowSet modifiedAffected = @@ -309,7 +286,7 @@ void processRows(UpdateByWindowBucketContext context, boolean initialStep) { for (int opIdx : context.dirtyOperatorIndices) { UpdateByWindowedOperator winOp = (UpdateByWindowedOperator) operators[opIdx]; // call the specialized version of `intializeUpdate()` for these operators - winOp.initializeUpdate(ctx.opContext[opIdx]); + winOp.initializeUpdate(ctx.opContexts[opIdx]); } try (final RowSequence.Iterator it = ctx.affectedRows.getRowSequenceIterator(); @@ -363,7 +340,7 @@ void processRows(UpdateByWindowBucketContext context, boolean initialStep) { Arrays.fill(ctx.inputSourceChunks, null); for (int opIdx : context.dirtyOperatorIndices) { UpdateByWindowedOperator.Context opCtx = - (UpdateByWindowedOperator.Context) context.opContext[opIdx]; + (UpdateByWindowedOperator.Context) context.opContexts[opIdx]; // prep the chunk array needed by the accumulate call final int[] srcIndices = operatorInputSourceSlots[opIdx]; for (int ii = 0; ii < srcIndices.length; ii++) { @@ -374,7 +351,7 @@ void processRows(UpdateByWindowBucketContext context, boolean initialStep) { } // make the specialized call for windowed operators - ((UpdateByWindowedOperator.Context) ctx.opContext[opIdx]).accumulate( + ((UpdateByWindowedOperator.Context) ctx.opContexts[opIdx]).accumulate( chunkRs, opCtx.chunkArr, pushChunk, @@ -386,7 +363,7 @@ void processRows(UpdateByWindowBucketContext context, boolean initialStep) { // call `finishUpdate()` function for each operator for (int opIdx : context.dirtyOperatorIndices) { - operators[opIdx].finishUpdate(context.opContext[opIdx]); + operators[opIdx].finishUpdate(context.opContexts[opIdx]); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java index f7eb7d15d19..85a0129f426 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java @@ -1,7 +1,5 @@ package io.deephaven.engine.table.impl.updateby; -import gnu.trove.list.array.TIntArrayList; -import gnu.trove.set.hash.TIntHashSet; import io.deephaven.base.ringbuffer.LongRingBuffer; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.LongChunk; @@ -85,7 +83,7 @@ protected void makeOperatorContexts(UpdateByWindowBucketContext context) { // create contexts for the affected operators for (int opIdx : context.dirtyOperatorIndices) { - context.opContext[opIdx] = operators[opIdx].makeUpdateContext(context.workingChunkSize, + context.opContexts[opIdx] = operators[opIdx].makeUpdateContext(context.workingChunkSize, operatorInputSourceSlots[opIdx].length); } } @@ -230,31 +228,7 @@ public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, } // determine which operators are affected by this update - ctx.isDirty = false; - boolean allAffected = upstream.added().isNonempty() || - upstream.removed().isNonempty(); - - if (allAffected) { - // mark all operators as affected by this update - context.dirtyOperatorIndices = IntStream.range(0, operators.length).toArray(); - context.dirtySourceIndices = getUniqueSourceIndices(); - context.isDirty = true; - } else { - // determine which operators are affected by this update - TIntArrayList dirtyOperatorList = new TIntArrayList(operators.length); - TIntHashSet inputSourcesSet = new TIntHashSet(getUniqueSourceIndices().length); - for (int opIdx = 0; opIdx < operators.length; opIdx++) { - UpdateByOperator op = operators[opIdx]; - if (upstream.modifiedColumnSet().nonempty() && (op.getInputModifiedColumnSet() == null - || upstream.modifiedColumnSet().containsAny(op.getInputModifiedColumnSet()))) { - dirtyOperatorList.add(opIdx); - inputSourcesSet.addAll(operatorInputSourceSlots[opIdx]); - context.isDirty = true; - } - } - context.dirtyOperatorIndices = dirtyOperatorList.toArray(); - context.dirtySourceIndices = inputSourcesSet.toArray(); - } + processUpdateForContext(context, upstream); if (!ctx.isDirty) { return; @@ -263,22 +237,26 @@ public void computeAffectedRowsAndOperators(UpdateByWindowBucketContext context, final WritableRowSet tmpAffected = RowSetFactory.empty(); - if (upstream.modified().isNonempty()) { - // TODO: we can omit these checks if we can prove no input columns or timestamp columns were modified + // consider the modifications only when input or timestamp columns were modified + if (upstream.modified().isNonempty() && (ctx.timestampsModified || ctx.inputModified)) { + // recompute all windows that have the modified rows in their window try (final RowSet modifiedAffected = computeAffectedRowsTime(ctx, upstream.modified(), prevUnits, fwdUnits, false)) { tmpAffected.insert(modifiedAffected); } - // modified timestamps and values will affect previous values - try (final WritableRowSet modifiedAffectedPrev = - computeAffectedRowsTime(ctx, upstream.getModifiedPreShift(), prevUnits, fwdUnits, true)) { - // we used the SSA (post-shift) to get these keys, no need to shift - // retain only the rows that still exist in the sourceRowSet - modifiedAffectedPrev.retain(ctx.timestampValidRowSet); - tmpAffected.insert(modifiedAffectedPrev); - } + if (ctx.timestampsModified) { - // compute all modified rows only if timestamps have changed + // recompute all windows that previously contained the modified rows, they may not contain this value + // after the timestamp modifications + try (final WritableRowSet modifiedAffectedPrev = + computeAffectedRowsTime(ctx, upstream.getModifiedPreShift(), prevUnits, fwdUnits, true)) { + // we used the SSA (post-shift) to get these keys, no need to shift + // retain only the rows that still exist in the sourceRowSet + modifiedAffectedPrev.retain(ctx.timestampValidRowSet); + tmpAffected.insert(modifiedAffectedPrev); + } + + // re-compute all modified rows, they have new windows after the timestamp modifications tmpAffected.insert(upstream.modified()); } } @@ -338,7 +316,7 @@ public void processRows(UpdateByWindowBucketContext context, boolean initialStep for (int opIdx : context.dirtyOperatorIndices) { UpdateByWindowedOperator winOp = (UpdateByWindowedOperator) operators[opIdx]; // call the specialized version of `intializeUpdate()` for these operators - winOp.initializeUpdate(ctx.opContext[opIdx]); + winOp.initializeUpdate(ctx.opContexts[opIdx]); } try (final RowSequence.Iterator affectedRowsIt = ctx.affectedRows.getRowSequenceIterator(); @@ -426,7 +404,7 @@ public void processRows(UpdateByWindowBucketContext context, boolean initialStep Arrays.fill(ctx.inputSourceChunks, null); for (int opIdx : context.dirtyOperatorIndices) { UpdateByWindowedOperator.Context opCtx = - (UpdateByWindowedOperator.Context) context.opContext[opIdx]; + (UpdateByWindowedOperator.Context) context.opContexts[opIdx]; // prep the chunk array needed by the accumulate call final int[] srcIndices = operatorInputSourceSlots[opIdx]; for (int ii = 0; ii < srcIndices.length; ii++) { @@ -437,7 +415,7 @@ public void processRows(UpdateByWindowBucketContext context, boolean initialStep } // make the specialized call for windowed operators - ((UpdateByWindowedOperator.Context) ctx.opContext[opIdx]).accumulate( + ((UpdateByWindowedOperator.Context) ctx.opContexts[opIdx]).accumulate( chunkAffectedRows, opCtx.chunkArr, pushChunk, @@ -456,7 +434,7 @@ public void processRows(UpdateByWindowBucketContext context, boolean initialStep // call `finishUpdate()` function for each operator for (int opIdx : context.dirtyOperatorIndices) { - operators[opIdx].finishUpdate(context.opContext[opIdx]); + operators[opIdx].finishUpdate(context.opContexts[opIdx]); } } } From 60e4624bff9d900464fd55ee4cb642b15c44d6ab Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Thu, 26 Jan 2023 14:52:05 -0800 Subject: [PATCH 100/123] PairwiseBuffer simplification, bug fix --- .../table/impl/updateby/UpdateByWindow.java | 6 ++- .../internal/PairwiseDoubleRingBuffer.java | 41 +++++++++---------- .../internal/PairwiseFloatRingBuffer.java | 41 +++++++++---------- .../PairwiseDoubleRingBufferTest.java | 30 ++++++++++++++ .../internal/PairwiseFloatRingBufferTest.java | 30 ++++++++++++++ 5 files changed, 102 insertions(+), 46 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java index 5fa444b8d15..040e24a6dfc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java @@ -103,8 +103,10 @@ public void close() { SafeCloseable.closeArray(opContexts); Arrays.fill(opContexts, null); - SafeCloseable.closeArray(inputSourceGetContexts); - Arrays.fill(inputSourceGetContexts, null); + if (inputSourceGetContexts != null) { + SafeCloseable.closeArray(inputSourceGetContexts); + Arrays.fill(inputSourceGetContexts, null); + } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java index b1ec21fcaf7..bcd009f1eaf 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBuffer.java @@ -90,6 +90,13 @@ public PairwiseDoubleRingBuffer(int initialSize, double emptyVal, DoubleFunction clear(); } + @VisibleForTesting + public static boolean rangesCollapse(final int x1, final int y1, final int x2, final int y2) { + // Ranges overlap when the start of each range is leq the end of the other. In this case, we want to extend + // detection to when the ranges are consecutive as well. Do this by adding one to the range ends then compare. + return x1 <= (y2 + 1) && x2 <= (y1 + 1); + } + private void evaluateRangeFast(int start, int end) { // everything in this range needs to be reevaluated for (int left = start & 0xFFFFFFFE; left <= end; left += 2) { @@ -122,9 +129,9 @@ public double evaluateTree(int startA, int endA) { @VisibleForTesting public double evaluateTree(int startA, int endA, int startB, int endB) { while (endB > 1) { - if (endA >= startB - 1) { + if (rangesCollapse(startA, endA, startB, endB)) { // all collapse together into a single range - return evaluateTree(startA, endB); + return evaluateTree(Math.min(startA, startB), Math.max(endA, endB)); } else { // compute this level evaluateRangeFast(startA, endA); @@ -143,15 +150,12 @@ public double evaluateTree(int startA, int endA, int startB, int endB) { @VisibleForTesting public double evaluateTree(int startA, int endA, int startB, int endB, int startC, int endC) { while (endC > 1) { - if (endA >= startC - 1 || (endA >= startB - 1 && endB >= startC - 1)) { - // all collapse together into a single range - return evaluateTree(startA, endC); - } else if (endA >= startB - 1) { - // A and B collapse - return evaluateTree(startA, endB, startC, endC); - } else if (endB >= startC - 1) { - // B and C collapse - return evaluateTree(startA, endA, startB, endC); + if (rangesCollapse(startA, endA, startB, endB)) { + // A and B overlap + return evaluateTree(Math.min(startA, startB), Math.max(endA, endB), startC, endC); + } else if (rangesCollapse(startB, endB, startC, endC)) { + // B and C overlap + return evaluateTree(startA, endA, Math.min(startB, startC), Math.max(endB, endC)); } else { // no collapse evaluateRangeFast(startA, endA); @@ -193,26 +197,19 @@ public double evaluate() { // pop wrapped, push is not value = evaluateTree(capacity, dirtyPopTail, dirtyPushHead, dirtyPushTail, dirtyPopHead, chunkSize - 1); } else { - // neither wrapped, do they overlap? - if (dirtyPushTail >= dirtyPopHead && dirtyPopTail >= dirtyPushHead) { - value = evaluateTree(Math.min(dirtyPushHead, dirtyPopHead), Math.max(dirtyPushTail, dirtyPopTail)); - } else { - // no overlap - if (dirtyPopHead < dirtyPushHead) { - value = evaluateTree(dirtyPopHead, dirtyPopTail, dirtyPushHead, dirtyPushTail); - } else { - value = evaluateTree(dirtyPushHead, dirtyPushTail, dirtyPopHead, dirtyPopTail); - } - } + // neither are wrapped, can evaluate directly + value = evaluateTree(dirtyPushHead, dirtyPushTail, dirtyPopHead, dirtyPopTail); } } else if (pushDirty) { if (dirtyPushHead > dirtyPushTail) { + // wrapped value = evaluateTree(capacity, dirtyPushTail, dirtyPushHead, chunkSize - 1); } else { value = evaluateTree(dirtyPushHead, dirtyPushTail); } } else if (popDirty) { if (dirtyPopHead > dirtyPopTail) { + // wrapped value = evaluateTree(capacity, dirtyPopTail, dirtyPopHead, chunkSize - 1); } else { value = evaluateTree(dirtyPopHead, dirtyPopTail); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java index b49a5c4fc61..8f3cc8931a4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBuffer.java @@ -85,6 +85,13 @@ public PairwiseFloatRingBuffer(int initialSize, float emptyVal, FloatFunction pa clear(); } + @VisibleForTesting + public static boolean rangesCollapse(final int x1, final int y1, final int x2, final int y2) { + // Ranges overlap when the start of each range is leq the end of the other. In this case, we want to extend + // detection to when the ranges are consecutive as well. Do this by adding one to the range ends then compare. + return x1 <= (y2 + 1) && x2 <= (y1 + 1); + } + private void evaluateRangeFast(int start, int end) { // everything in this range needs to be reevaluated for (int left = start & 0xFFFFFFFE; left <= end; left += 2) { @@ -117,9 +124,9 @@ public float evaluateTree(int startA, int endA) { @VisibleForTesting public float evaluateTree(int startA, int endA, int startB, int endB) { while (endB > 1) { - if (endA >= startB - 1) { + if (rangesCollapse(startA, endA, startB, endB)) { // all collapse together into a single range - return evaluateTree(startA, endB); + return evaluateTree(Math.min(startA, startB), Math.max(endA, endB)); } else { // compute this level evaluateRangeFast(startA, endA); @@ -138,15 +145,12 @@ public float evaluateTree(int startA, int endA, int startB, int endB) { @VisibleForTesting public float evaluateTree(int startA, int endA, int startB, int endB, int startC, int endC) { while (endC > 1) { - if (endA >= startC - 1 || (endA >= startB - 1 && endB >= startC - 1)) { - // all collapse together into a single range - return evaluateTree(startA, endC); - } else if (endA >= startB - 1) { - // A and B collapse - return evaluateTree(startA, endB, startC, endC); - } else if (endB >= startC - 1) { - // B and C collapse - return evaluateTree(startA, endA, startB, endC); + if (rangesCollapse(startA, endA, startB, endB)) { + // A and B overlap + return evaluateTree(Math.min(startA, startB), Math.max(endA, endB), startC, endC); + } else if (rangesCollapse(startB, endB, startC, endC)) { + // B and C overlap + return evaluateTree(startA, endA, Math.min(startB, startC), Math.max(endB, endC)); } else { // no collapse evaluateRangeFast(startA, endA); @@ -188,26 +192,19 @@ public float evaluate() { // pop wrapped, push is not value = evaluateTree(capacity, dirtyPopTail, dirtyPushHead, dirtyPushTail, dirtyPopHead, chunkSize - 1); } else { - // neither wrapped, do they overlap? - if (dirtyPushTail >= dirtyPopHead && dirtyPopTail >= dirtyPushHead) { - value = evaluateTree(Math.min(dirtyPushHead, dirtyPopHead), Math.max(dirtyPushTail, dirtyPopTail)); - } else { - // no overlap - if (dirtyPopHead < dirtyPushHead) { - value = evaluateTree(dirtyPopHead, dirtyPopTail, dirtyPushHead, dirtyPushTail); - } else { - value = evaluateTree(dirtyPushHead, dirtyPushTail, dirtyPopHead, dirtyPopTail); - } - } + // neither are wrapped, can evaluate directly + value = evaluateTree(dirtyPushHead, dirtyPushTail, dirtyPopHead, dirtyPopTail); } } else if (pushDirty) { if (dirtyPushHead > dirtyPushTail) { + // wrapped value = evaluateTree(capacity, dirtyPushTail, dirtyPushHead, chunkSize - 1); } else { value = evaluateTree(dirtyPushHead, dirtyPushTail); } } else if (popDirty) { if (dirtyPopHead > dirtyPopTail) { + // wrapped value = evaluateTree(capacity, dirtyPopTail, dirtyPopHead, chunkSize - 1); } else { value = evaluateTree(dirtyPopHead, dirtyPopTail); diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBufferTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBufferTest.java index 9f633de4e89..e28e4b9a294 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBufferTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseDoubleRingBufferTest.java @@ -545,6 +545,36 @@ public void testPopMultiple() { } } + public void testRangesCollapse() { + // overlapping + assertTrue(PairwiseDoubleRingBuffer.rangesCollapse(0, 2, 1, 3)); + assertTrue(PairwiseDoubleRingBuffer.rangesCollapse(1, 3, 0, 2)); + + assertTrue(PairwiseDoubleRingBuffer.rangesCollapse(0, 10, 1, 3)); + assertTrue(PairwiseDoubleRingBuffer.rangesCollapse(1, 3, 0, 10)); + + // fully contained + assertTrue(PairwiseDoubleRingBuffer.rangesCollapse(0, 3, 1, 2)); + assertTrue(PairwiseDoubleRingBuffer.rangesCollapse(1, 2, 0, 3)); + + assertTrue(PairwiseDoubleRingBuffer.rangesCollapse(1, 2, 1, 1)); + assertTrue(PairwiseDoubleRingBuffer.rangesCollapse(1, 1, 1, 2)); + + assertTrue(PairwiseDoubleRingBuffer.rangesCollapse(1, 2, 2, 2)); + assertTrue(PairwiseDoubleRingBuffer.rangesCollapse(2, 2, 1, 2)); + + // consecutive + assertTrue(PairwiseDoubleRingBuffer.rangesCollapse(0, 1, 2, 3)); + assertTrue(PairwiseDoubleRingBuffer.rangesCollapse(2, 3, 0, 1)); + + // non-overlapping, non consecutive + assertFalse(PairwiseDoubleRingBuffer.rangesCollapse(0, 1, 3, 4)); + assertFalse(PairwiseDoubleRingBuffer.rangesCollapse(3, 4, 0, 1)); + + assertFalse(PairwiseDoubleRingBuffer.rangesCollapse(0, 1, 9, 10)); + assertFalse(PairwiseDoubleRingBuffer.rangesCollapse(9, 10, 0, 1)); + } + public void testSpecialCaseA() { // overlapping push and pop ranges with popTail < pushTail try (final PairwiseDoubleRingBuffer rb = new PairwiseDoubleRingBuffer(4, (double) 0, Double::sum)) { diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBufferTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBufferTest.java index e86322a375a..16221df13bb 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBufferTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/internal/PairwiseFloatRingBufferTest.java @@ -540,6 +540,36 @@ public void testPopMultiple() { } } + public void testRangesCollapse() { + // overlapping + assertTrue(PairwiseFloatRingBuffer.rangesCollapse(0, 2, 1, 3)); + assertTrue(PairwiseFloatRingBuffer.rangesCollapse(1, 3, 0, 2)); + + assertTrue(PairwiseFloatRingBuffer.rangesCollapse(0, 10, 1, 3)); + assertTrue(PairwiseFloatRingBuffer.rangesCollapse(1, 3, 0, 10)); + + // fully contained + assertTrue(PairwiseFloatRingBuffer.rangesCollapse(0, 3, 1, 2)); + assertTrue(PairwiseFloatRingBuffer.rangesCollapse(1, 2, 0, 3)); + + assertTrue(PairwiseFloatRingBuffer.rangesCollapse(1, 2, 1, 1)); + assertTrue(PairwiseFloatRingBuffer.rangesCollapse(1, 1, 1, 2)); + + assertTrue(PairwiseFloatRingBuffer.rangesCollapse(1, 2, 2, 2)); + assertTrue(PairwiseFloatRingBuffer.rangesCollapse(2, 2, 1, 2)); + + // consecutive + assertTrue(PairwiseFloatRingBuffer.rangesCollapse(0, 1, 2, 3)); + assertTrue(PairwiseFloatRingBuffer.rangesCollapse(2, 3, 0, 1)); + + // non-overlapping, non consecutive + assertFalse(PairwiseFloatRingBuffer.rangesCollapse(0, 1, 3, 4)); + assertFalse(PairwiseFloatRingBuffer.rangesCollapse(3, 4, 0, 1)); + + assertFalse(PairwiseFloatRingBuffer.rangesCollapse(0, 1, 9, 10)); + assertFalse(PairwiseFloatRingBuffer.rangesCollapse(9, 10, 0, 1)); + } + public void testSpecialCaseA() { // overlapping push and pop ranges with popTail < pushTail try (final PairwiseFloatRingBuffer rb = new PairwiseFloatRingBuffer(4, (float) 0, Float::sum)) { From f77d57deea8c738c404f6a8f604701814850d1f7 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 27 Jan 2023 09:34:28 -0800 Subject: [PATCH 101/123] UpdateBy operator cleanup --- .../BucketedPartitionedUpdateByManager.java | 4 + .../table/impl/updateby/UpdateByOperator.java | 5 +- .../table/impl/updateby/UpdateByWindow.java | 2 +- .../updateby/UpdateByWindowCumulative.java | 3 + .../ema/BasePrimitiveEMAOperator.java | 19 +---- .../updateby/ema/BigDecimalEMAOperator.java | 35 ++++----- .../updateby/ema/BigIntegerEMAOperator.java | 20 ++--- .../updateby/ema/BigNumberEMAOperator.java | 35 ++++----- .../impl/updateby/ema/ByteEMAOperator.java | 25 ++++--- .../impl/updateby/ema/DoubleEMAOperator.java | 27 +++---- .../impl/updateby/ema/FloatEMAOperator.java | 27 +++---- .../impl/updateby/ema/IntEMAOperator.java | 25 ++++--- .../impl/updateby/ema/LongEMAOperator.java | 25 ++++--- .../impl/updateby/ema/ShortEMAOperator.java | 25 ++++--- .../updateby/fill/BooleanFillByOperator.java | 6 +- .../updateby/fill/ByteFillByOperator.java | 6 +- .../updateby/fill/CharFillByOperator.java | 6 +- .../updateby/fill/DoubleFillByOperator.java | 6 +- .../updateby/fill/FloatFillByOperator.java | 6 +- .../impl/updateby/fill/IntFillByOperator.java | 6 +- .../updateby/fill/LongFillByOperator.java | 6 +- .../updateby/fill/ObjectFillByOperator.java | 6 +- .../updateby/fill/ShortFillByOperator.java | 6 +- .../internal/BaseByteUpdateByOperator.java | 3 - .../internal/BaseCharUpdateByOperator.java | 3 - .../internal/BaseDoubleUpdateByOperator.java | 3 - .../internal/BaseFloatUpdateByOperator.java | 3 - .../internal/BaseIntUpdateByOperator.java | 3 - .../internal/BaseLongUpdateByOperator.java | 3 - .../internal/BaseObjectUpdateByOperator.java | 3 - .../internal/BaseShortUpdateByOperator.java | 3 - .../BaseWindowedByteUpdateByOperator.java | 3 - .../BaseWindowedCharUpdateByOperator.java | 3 - .../BaseWindowedDoubleUpdateByOperator.java | 3 - .../BaseWindowedFloatUpdateByOperator.java | 3 - .../BaseWindowedIntUpdateByOperator.java | 3 - .../BaseWindowedLongUpdateByOperator.java | 3 - .../BaseWindowedObjectUpdateByOperator.java | 3 - .../BaseWindowedShortUpdateByOperator.java | 3 - .../minmax/ByteCumMinMaxOperator.java | 74 +++++++++++++++++-- .../minmax/DoubleCumMinMaxOperator.java | 13 ++-- .../minmax/FloatCumMinMaxOperator.java | 13 ++-- .../updateby/minmax/IntCumMinMaxOperator.java | 74 +++++++++++++++++-- .../minmax/LongCumMinMaxOperator.java | 74 +++++++++++++++++-- .../minmax/ShortCumMinMaxOperator.java | 74 +++++++++++++++++-- .../prod/BigDecimalCumProdOperator.java | 12 +-- .../prod/BigIntegerCumProdOperator.java | 15 ++-- .../updateby/prod/ByteCumProdOperator.java | 9 +-- .../updateby/prod/DoubleCumProdOperator.java | 11 +-- .../updateby/prod/FloatCumProdOperator.java | 11 +-- .../updateby/prod/IntCumProdOperator.java | 9 +-- .../updateby/prod/LongCumProdOperator.java | 9 +-- .../updateby/prod/ShortCumProdOperator.java | 9 +-- .../BigDecimalRollingSumOperator.java | 1 + .../BigIntegerRollingSumOperator.java | 1 + .../rollingsum/ByteRollingSumOperator.java | 4 +- .../rollingsum/DoubleRollingSumOperator.java | 10 --- .../rollingsum/FloatRollingSumOperator.java | 10 --- .../rollingsum/IntRollingSumOperator.java | 4 +- .../rollingsum/LongRollingSumOperator.java | 4 +- .../rollingsum/ShortRollingSumOperator.java | 4 +- .../replicators/ReplicateUpdateBy.java | 4 +- 62 files changed, 486 insertions(+), 342 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java index 0b36163ba7e..5b36ac87a6b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java @@ -11,6 +11,7 @@ import io.deephaven.engine.table.impl.QueryTable; import io.deephaven.engine.table.impl.TableUpdateImpl; import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.updategraph.LogicalClock; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -93,6 +94,9 @@ protected BucketedPartitionedUpdateByManager( rowRedirection, control); + bucket.parentUpdateBy = this; + bucket.createdStep = LogicalClock.DEFAULT.currentStep(); + // add this to the bucket list synchronized (buckets) { buckets.offer(bucket); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperator.java index 809bb2b1e63..45f145cdfac 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperator.java @@ -16,6 +16,7 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import javax.annotation.OverridingMethodsMustInvokeSuper; import java.util.Map; /** @@ -58,8 +59,6 @@ public interface UpdateContext extends SafeCloseable { void setValuesChunk(@NotNull Chunk valuesChunk); - void setTimestampChunk(@NotNull LongChunk valuesChunk); - /** * Add values to the operators current data set * @@ -88,6 +87,8 @@ public interface UpdateContext extends SafeCloseable { * Reset the operator data values to a known state. This may occur during initialization or when a windowed * operator has an empty window */ + + @OverridingMethodsMustInvokeSuper void reset(); /** diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java index 040e24a6dfc..6e37b6302b9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java @@ -172,7 +172,7 @@ UpdateByOperator[] getOperators() { return operators; } - public int[] getUniqueSourceIndices() { + int[] getUniqueSourceIndices() { if (uniqueInputSourceIndices == null) { final TIntHashSet set = new TIntHashSet(); for (int opIdx = 0; opIdx < operators.length; opIdx++) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java index 8a29ebe9d33..7d0bcb56951 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java @@ -222,6 +222,9 @@ private static long smallestAffectedKey(final @NotNull TableUpdate upstream, // following the first remove if (smallestModifiedKey < affectedRowSet.lastRowKey()) { smallestModifiedKey = affectedRowSet.get(affectedRowSet.find(smallestModifiedKey) + 1); + } else { + // all removes are after the end of the current rowset + smallestModifiedKey = Long.MAX_VALUE; } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java index ec3c34ef87b..bd0bc50c609 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java @@ -25,24 +25,19 @@ public abstract class Context extends BaseDoubleUpdateByOperator.Context { long lastStamp = NULL_LONG; - @Override - public void setTimestampChunk(@NotNull final LongChunk valuesChunk) { - timestampValueChunk = valuesChunk; - } - Context(final int chunkSize, final int chunkCount) { super(chunkSize, chunkCount); } @Override public void reset() { - curVal = NULL_DOUBLE; + super.reset(); lastStamp = NULL_LONG; } } /** - * An operator that computes an EMA from a short column using an exponential decay function. + * An operator that computes an EMA from an input column using an exponential decay function. * * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this ema @@ -74,14 +69,8 @@ public void initializeUpdate(@NotNull final UpdateContext updateContext, super.initializeUpdate(updateContext, firstUnmodifiedKey, firstUnmodifiedTimestamp); final Context ctx = (Context) updateContext; - // If we set the last state to null, then we know it was a reset state and the timestamp must also - // have been reset. - if (ctx.curVal == NULL_DOUBLE || firstUnmodifiedKey == NULL_ROW_KEY) { - ctx.lastStamp = NULL_LONG; - } else { - // rely on the caller to validate this is a valid timestamp (or NULL_LONG when appropriate) - ctx.lastStamp = firstUnmodifiedTimestamp; - } + // rely on the caller to validate this is a valid timestamp (or NULL_LONG when appropriate) + ctx.lastStamp = firstUnmodifiedTimestamp; } void handleBadData(@NotNull final Context ctx, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java index 132e89ebcb0..c562cf06dcb 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java @@ -27,7 +27,6 @@ public void accumulate(RowSequence inputKeys, LongChunk tsChunk, int len) { setValuesChunk(valueChunkArr[0]); - setTimestampChunk(tsChunk); // chunk processing if (timestampColumnName == null) { @@ -67,15 +66,18 @@ public void accumulate(RowSequence inputKeys, } else { final long dt = timestamp - lastStamp; - // alpha is dynamic, based on time - BigDecimal alpha = BigDecimal.valueOf(Math.exp(-dt / (double) reverseWindowScaleUnits)); - BigDecimal oneMinusAlpha = - BigDecimal.ONE.subtract(alpha, control.bigValueContextOrDefault()); + if (dt != 0) { - curVal = curVal.multiply(alpha, control.bigValueContextOrDefault()) - .add(input.multiply(oneMinusAlpha, control.bigValueContextOrDefault()), - control.bigValueContextOrDefault()); - lastStamp = timestamp; + // alpha is dynamic, based on time + BigDecimal alpha = BigDecimal.valueOf(Math.exp(-dt / (double) reverseWindowScaleUnits)); + BigDecimal oneMinusAlpha = + BigDecimal.ONE.subtract(alpha, control.bigValueContextOrDefault()); + + curVal = curVal.multiply(alpha, control.bigValueContextOrDefault()) + .add(input.multiply(oneMinusAlpha, control.bigValueContextOrDefault()), + control.bigValueContextOrDefault()); + lastStamp = timestamp; + } } } outputValues.set(ii, curVal); @@ -95,14 +97,13 @@ public void push(long key, int pos, int count) { /** * An operator that computes an EMA from a BigDecimal column using an exponential decay function. * - * @param pair the {@link MatchPair} that defines the input/output for this operation - * @param affectingColumns the names of the columns that affect this ema - * @param control defines how to handle {@code null} input values. - * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in - * integer ticks. - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timeRecorder} is provided, this is measured - * in ticks, otherwise it is measured in nanoseconds - * @param rowRedirection the row redirection for the EMA output column + * @param pair the {@link MatchPair} that defines the input/output for this operation + * @param affectingColumns the names of the columns that affect this ema + * @param control defines how to handle {@code null} input values. + * @param timestampColumnName the name of the column containing timestamps for time-based calcuations + * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is measured in ticks, otherwise it is measured in nanoseconds + * @param rowRedirection the {@link RowRedirection} to use for dense output sources + * @param valueSource a reference to the input column source for this operation */ public BigDecimalEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java index 8c17fdd89d4..115451acc43 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java @@ -28,7 +28,6 @@ public void accumulate(RowSequence inputKeys, LongChunk tsChunk, int len) { setValuesChunk(valueChunkArr[0]); - setTimestampChunk(tsChunk); // chunk processing if (timestampColumnName == null) { @@ -69,9 +68,7 @@ public void accumulate(RowSequence inputKeys, lastStamp = timestamp; } else { final long dt = timestamp - lastStamp; - if (dt == 0) { - // preserve curVal and timestamp - } else { + if (dt != 0) { // alpha is dynamic, based on time BigDecimal alpha = BigDecimal.valueOf(Math.exp(-dt / (double) reverseWindowScaleUnits)); BigDecimal oneMinusAlpha = BigDecimal.ONE.subtract(alpha, control.bigValueContextOrDefault()); @@ -100,14 +97,13 @@ public void push(long key, int pos, int count) { /** * An operator that computes an EMA from a BigInteger column using an exponential decay function. * - * @param pair the {@link MatchPair} that defines the input/output for this operation - * @param affectingColumns the names of the columns that affect this ema - * @param control defines how to handle {@code null} input values. - * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in - * integer ticks. - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timeRecorder} is provided, this is measured - * in ticks, otherwise it is measured in nanoseconds - * @param rowRedirection the row redirection for the EMA output column + * @param pair the {@link MatchPair} that defines the input/output for this operation + * @param affectingColumns the names of the columns that affect this ema + * @param control defines how to handle {@code null} input values. + * @param timestampColumnName the name of the column containing timestamps for time-based calcuations + * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is measured in ticks, otherwise it is measured in nanoseconds + * @param rowRedirection the {@link RowRedirection} to use for dense output sources + * @param valueSource a reference to the input column source for this operation */ public BigIntegerEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java index e2669b46fbf..f11b9c56016 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java @@ -41,11 +41,6 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { objectValueChunk = valuesChunk.asObjectChunk(); } - @Override - public void setTimestampChunk(@NotNull final LongChunk valuesChunk) { - timestampValueChunk = valuesChunk; - } - @Override public boolean isValueValid(long atKey) { return valueSource.get(atKey) != null; @@ -59,21 +54,23 @@ public void reset() { } /** - * An operator that computes an EMA from a int column using an exponential decay function. + * An operator that computes an EMA from a big number column using an exponential decay function. * - * @param pair the {@link MatchPair} that defines the input/output for this operation - * @param affectingColumns the names of the columns that affect this ema - * @param control defines how to handle {@code null} input values. - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timeRecorder} is provided, this is measured - * in ticks, otherwise it is measured in nanoseconds + * @param pair the {@link MatchPair} that defines the input/output for this operation + * @param affectingColumns the names of the columns that affect this ema + * @param control defines how to handle {@code null} input values. + * @param timestampColumnName the name of the column containing timestamps for time-based calcuations + * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is measured in ticks, otherwise it is measured in nanoseconds + * @param rowRedirection the {@link RowRedirection} to use for dense output sources + * @param valueSource a reference to the input column source for this operation */ public BigNumberEMAOperator(@NotNull final MatchPair pair, - @NotNull final String[] affectingColumns, - @NotNull final OperationControl control, - @Nullable final String timestampColumnName, - final long timeScaleUnits, - @Nullable final RowRedirection rowRedirection, - final ColumnSource valueSource) { + @NotNull final String[] affectingColumns, + @NotNull final OperationControl control, + @Nullable final String timestampColumnName, + final long timeScaleUnits, + @Nullable final RowRedirection rowRedirection, + final ColumnSource valueSource) { super(pair, affectingColumns, rowRedirection, timestampColumnName, timeScaleUnits, BigDecimal.class); this.control = control; @@ -86,8 +83,8 @@ public BigNumberEMAOperator(@NotNull final MatchPair pair, @Override public void initializeUpdate(@NotNull final UpdateContext updateContext, - final long firstUnmodifiedKey, - final long firstUnmodifiedTimestamp) { + final long firstUnmodifiedKey, + final long firstUnmodifiedTimestamp) { super.initializeUpdate(updateContext, firstUnmodifiedKey, firstUnmodifiedTimestamp); final Context ctx = (Context) updateContext; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java index 1c3e237b6ce..c99a15935dd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java @@ -17,6 +17,8 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import java.math.BigDecimal; + import static io.deephaven.util.QueryConstants.*; public class ByteEMAOperator extends BasePrimitiveEMAOperator { @@ -36,7 +38,6 @@ public void accumulate(RowSequence inputKeys, LongChunk tsChunk, int len) { setValuesChunk(valueChunkArr[0]); - setTimestampChunk(tsChunk); // chunk processing if (timestampColumnName == null) { @@ -74,10 +75,12 @@ public void accumulate(RowSequence inputKeys, lastStamp = timestamp; } else { final long dt = timestamp - lastStamp; - // alpha is dynamic, based on time - final double alpha = Math.exp(-dt / (double) reverseWindowScaleUnits); - curVal = alpha * curVal + (1 - alpha) * input; - lastStamp = timestamp; + if (dt != 0) { + // alpha is dynamic, based on time + final double alpha = Math.exp(-dt / (double) reverseWindowScaleUnits); + curVal = alpha * curVal + (1 - alpha) * input; + lastStamp = timestamp; + } } outputValues.set(ii, curVal); } @@ -106,11 +109,13 @@ public void push(long key, int pos, int count) { /** * An operator that computes an EMA from a byte column using an exponential decay function. * - * @param pair the {@link MatchPair} that defines the input/output for this operation - * @param affectingColumns the names of the columns that affect this ema - * @param control defines how to handle {@code null} input values. - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is - * measured in ticks, otherwise it is measured in nanoseconds + * @param pair the {@link MatchPair} that defines the input/output for this operation + * @param affectingColumns the names of the columns that affect this ema + * @param control defines how to handle {@code null} input values. + * @param timestampColumnName the name of the column containing timestamps for time-based calcuations + * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is measured in ticks, otherwise it is measured in nanoseconds + * @param rowRedirection the {@link RowRedirection} to use for dense output sources + * @param valueSource a reference to the input column source for this operation */ public ByteEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java index edf7de855fe..e96d8068050 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java @@ -36,7 +36,6 @@ public void accumulate(RowSequence inputKeys, LongChunk tsChunk, int len) { setValuesChunk(valueChunkArr[0]); - setTimestampChunk(tsChunk); // chunk processing if (timestampColumnName == null) { @@ -77,13 +76,8 @@ public void accumulate(RowSequence inputKeys, curVal = input; lastStamp = timestamp; } else { - final boolean currentPoisoned = Double.isNaN(curVal); - if (currentPoisoned && lastStamp == NULL_LONG) { - // If the current EMA was a NaN, we should accept the first good timestamp so that - // we can handle reset behavior properly in the following else - lastStamp = timestamp; - } else { - final long dt = timestamp - lastStamp; + final long dt = timestamp - lastStamp; + if (dt != 0) { final double alpha = Math.exp(-dt / (double) reverseWindowScaleUnits); curVal = alpha * curVal + ((1 - alpha) * input); lastStamp = timestamp; @@ -108,9 +102,6 @@ public boolean isValueValid(long atKey) { if (value == NULL_DOUBLE) { return false; } - - // Note that we don't care about Reset because in that case the current EMA at this key would be null - // and the superclass will do the right thing. return !Double.isNaN(value) || control.onNanValueOrDefault() != BadDataBehavior.SKIP; } @@ -124,11 +115,13 @@ public void push(long key, int pos, int count) { /** * An operator that computes an EMA from a double column using an exponential decay function. * - * @param pair the {@link MatchPair} that defines the input/output for this operation - * @param affectingColumns the names of the columns that affect this ema - * @param control defines how to handle {@code null} input values. - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timeRecorder} is provided, this is measured - * in ticks, otherwise it is measured in nanoseconds + * @param pair the {@link MatchPair} that defines the input/output for this operation + * @param affectingColumns the names of the columns that affect this ema + * @param control defines how to handle {@code null} input values. + * @param timestampColumnName the name of the column containing timestamps for time-based calcuations + * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is measured in ticks, otherwise it is measured in nanoseconds + * @param rowRedirection the {@link RowRedirection} to use for dense output sources + * @param valueSource a reference to the input column source for this operation */ public DoubleEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @@ -139,7 +132,7 @@ public DoubleEMAOperator(@NotNull final MatchPair pair, final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args - ) { + ) { super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, rowRedirection); this.valueSource = valueSource; // region constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java index 350e09db701..56728214d19 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java @@ -31,7 +31,6 @@ public void accumulate(RowSequence inputKeys, LongChunk tsChunk, int len) { setValuesChunk(valueChunkArr[0]); - setTimestampChunk(tsChunk); // chunk processing if (timestampColumnName == null) { @@ -72,13 +71,8 @@ public void accumulate(RowSequence inputKeys, curVal = input; lastStamp = timestamp; } else { - final boolean currentPoisoned = Double.isNaN(curVal); - if (currentPoisoned && lastStamp == NULL_LONG) { - // If the current EMA was a NaN, we should accept the first good timestamp so that - // we can handle reset behavior properly in the following else - lastStamp = timestamp; - } else { - final long dt = timestamp - lastStamp; + final long dt = timestamp - lastStamp; + if (dt != 0) { final double alpha = Math.exp(-dt / (double) reverseWindowScaleUnits); curVal = alpha * curVal + ((1 - alpha) * input); lastStamp = timestamp; @@ -103,9 +97,6 @@ public boolean isValueValid(long atKey) { if (value == NULL_FLOAT) { return false; } - - // Note that we don't care about Reset because in that case the current EMA at this key would be null - // and the superclass will do the right thing. return !Float.isNaN(value) || control.onNanValueOrDefault() != BadDataBehavior.SKIP; } @@ -119,11 +110,13 @@ public void push(long key, int pos, int count) { /** * An operator that computes an EMA from a float column using an exponential decay function. * - * @param pair the {@link MatchPair} that defines the input/output for this operation - * @param affectingColumns the names of the columns that affect this ema - * @param control defines how to handle {@code null} input values. - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timeRecorder} is provided, this is measured - * in ticks, otherwise it is measured in nanoseconds + * @param pair the {@link MatchPair} that defines the input/output for this operation + * @param affectingColumns the names of the columns that affect this ema + * @param control defines how to handle {@code null} input values. + * @param timestampColumnName the name of the column containing timestamps for time-based calcuations + * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is measured in ticks, otherwise it is measured in nanoseconds + * @param rowRedirection the {@link RowRedirection} to use for dense output sources + * @param valueSource a reference to the input column source for this operation */ public FloatEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @@ -134,7 +127,7 @@ public FloatEMAOperator(@NotNull final MatchPair pair, final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args - ) { + ) { super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, rowRedirection); this.valueSource = valueSource; // region constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java index 112e6cf4df3..52c2483594b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java @@ -17,6 +17,8 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import java.math.BigDecimal; + import static io.deephaven.util.QueryConstants.*; public class IntEMAOperator extends BasePrimitiveEMAOperator { @@ -36,7 +38,6 @@ public void accumulate(RowSequence inputKeys, LongChunk tsChunk, int len) { setValuesChunk(valueChunkArr[0]); - setTimestampChunk(tsChunk); // chunk processing if (timestampColumnName == null) { @@ -74,10 +75,12 @@ public void accumulate(RowSequence inputKeys, lastStamp = timestamp; } else { final long dt = timestamp - lastStamp; - // alpha is dynamic, based on time - final double alpha = Math.exp(-dt / (double) reverseWindowScaleUnits); - curVal = alpha * curVal + (1 - alpha) * input; - lastStamp = timestamp; + if (dt != 0) { + // alpha is dynamic, based on time + final double alpha = Math.exp(-dt / (double) reverseWindowScaleUnits); + curVal = alpha * curVal + (1 - alpha) * input; + lastStamp = timestamp; + } } outputValues.set(ii, curVal); } @@ -106,11 +109,13 @@ public void push(long key, int pos, int count) { /** * An operator that computes an EMA from a int column using an exponential decay function. * - * @param pair the {@link MatchPair} that defines the input/output for this operation - * @param affectingColumns the names of the columns that affect this ema - * @param control defines how to handle {@code null} input values. - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is - * measured in ticks, otherwise it is measured in nanoseconds + * @param pair the {@link MatchPair} that defines the input/output for this operation + * @param affectingColumns the names of the columns that affect this ema + * @param control defines how to handle {@code null} input values. + * @param timestampColumnName the name of the column containing timestamps for time-based calcuations + * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is measured in ticks, otherwise it is measured in nanoseconds + * @param rowRedirection the {@link RowRedirection} to use for dense output sources + * @param valueSource a reference to the input column source for this operation */ public IntEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java index 1c4db317a28..9446e79dd73 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java @@ -17,6 +17,8 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import java.math.BigDecimal; + import static io.deephaven.util.QueryConstants.*; public class LongEMAOperator extends BasePrimitiveEMAOperator { @@ -36,7 +38,6 @@ public void accumulate(RowSequence inputKeys, LongChunk tsChunk, int len) { setValuesChunk(valueChunkArr[0]); - setTimestampChunk(tsChunk); // chunk processing if (timestampColumnName == null) { @@ -74,10 +75,12 @@ public void accumulate(RowSequence inputKeys, lastStamp = timestamp; } else { final long dt = timestamp - lastStamp; - // alpha is dynamic, based on time - final double alpha = Math.exp(-dt / (double) reverseWindowScaleUnits); - curVal = alpha * curVal + (1 - alpha) * input; - lastStamp = timestamp; + if (dt != 0) { + // alpha is dynamic, based on time + final double alpha = Math.exp(-dt / (double) reverseWindowScaleUnits); + curVal = alpha * curVal + (1 - alpha) * input; + lastStamp = timestamp; + } } outputValues.set(ii, curVal); } @@ -106,11 +109,13 @@ public void push(long key, int pos, int count) { /** * An operator that computes an EMA from a long column using an exponential decay function. * - * @param pair the {@link MatchPair} that defines the input/output for this operation - * @param affectingColumns the names of the columns that affect this ema - * @param control defines how to handle {@code null} input values. - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is - * measured in ticks, otherwise it is measured in nanoseconds + * @param pair the {@link MatchPair} that defines the input/output for this operation + * @param affectingColumns the names of the columns that affect this ema + * @param control defines how to handle {@code null} input values. + * @param timestampColumnName the name of the column containing timestamps for time-based calcuations + * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is measured in ticks, otherwise it is measured in nanoseconds + * @param rowRedirection the {@link RowRedirection} to use for dense output sources + * @param valueSource a reference to the input column source for this operation */ public LongEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java index 022835198dd..87e7ed52a3d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java @@ -12,6 +12,8 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import java.math.BigDecimal; + import static io.deephaven.util.QueryConstants.*; public class ShortEMAOperator extends BasePrimitiveEMAOperator { @@ -31,7 +33,6 @@ public void accumulate(RowSequence inputKeys, LongChunk tsChunk, int len) { setValuesChunk(valueChunkArr[0]); - setTimestampChunk(tsChunk); // chunk processing if (timestampColumnName == null) { @@ -69,10 +70,12 @@ public void accumulate(RowSequence inputKeys, lastStamp = timestamp; } else { final long dt = timestamp - lastStamp; - // alpha is dynamic, based on time - final double alpha = Math.exp(-dt / (double) reverseWindowScaleUnits); - curVal = alpha * curVal + (1 - alpha) * input; - lastStamp = timestamp; + if (dt != 0) { + // alpha is dynamic, based on time + final double alpha = Math.exp(-dt / (double) reverseWindowScaleUnits); + curVal = alpha * curVal + (1 - alpha) * input; + lastStamp = timestamp; + } } outputValues.set(ii, curVal); } @@ -101,11 +104,13 @@ public void push(long key, int pos, int count) { /** * An operator that computes an EMA from a short column using an exponential decay function. * - * @param pair the {@link MatchPair} that defines the input/output for this operation - * @param affectingColumns the names of the columns that affect this ema - * @param control defines how to handle {@code null} input values. - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is - * measured in ticks, otherwise it is measured in nanoseconds + * @param pair the {@link MatchPair} that defines the input/output for this operation + * @param affectingColumns the names of the columns that affect this ema + * @param control defines how to handle {@code null} input values. + * @param timestampColumnName the name of the column containing timestamps for time-based calcuations + * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is measured in ticks, otherwise it is measured in nanoseconds + * @param rowRedirection the {@link RowRedirection} to use for dense output sources + * @param valueSource a reference to the input column source for this operation */ public ShortEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java index 9f760ee5a0e..2a3c1bc929f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java @@ -44,9 +44,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { public void push(long key, int pos, int count) { Assert.eq(count, "push count", 1); - byte currentVal = booleanValueChunk.get(pos); - if(currentVal != NULL_BOOLEAN_AS_BYTE) { - curVal = currentVal; + byte val = booleanValueChunk.get(pos); + if(val != NULL_BOOLEAN_AS_BYTE) { + curVal = val; } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java index 813986ae2f4..713ce87dcdc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java @@ -37,9 +37,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { public void push(long key, int pos, int count) { Assert.eq(count, "push count", 1); - byte currentVal = byteValueChunk.get(pos); - if(currentVal != NULL_BYTE) { - curVal = currentVal; + byte val = byteValueChunk.get(pos); + if(val != NULL_BYTE) { + curVal = val; } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java index d3c15a0eef1..73a1ed15b51 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java @@ -32,9 +32,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { public void push(long key, int pos, int count) { Assert.eq(count, "push count", 1); - char currentVal = charValueChunk.get(pos); - if(currentVal != NULL_CHAR) { - curVal = currentVal; + char val = charValueChunk.get(pos); + if(val != NULL_CHAR) { + curVal = val; } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java index 5362f37ef41..d6ceb593c2b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java @@ -37,9 +37,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { public void push(long key, int pos, int count) { Assert.eq(count, "push count", 1); - double currentVal = doubleValueChunk.get(pos); - if(currentVal != NULL_DOUBLE) { - curVal = currentVal; + double val = doubleValueChunk.get(pos); + if(val != NULL_DOUBLE) { + curVal = val; } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java index f29e82a5a00..369caab1d2f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java @@ -37,9 +37,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { public void push(long key, int pos, int count) { Assert.eq(count, "push count", 1); - float currentVal = floatValueChunk.get(pos); - if(currentVal != NULL_FLOAT) { - curVal = currentVal; + float val = floatValueChunk.get(pos); + if(val != NULL_FLOAT) { + curVal = val; } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java index 82ecd5a1117..f37d2b6ba58 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java @@ -37,9 +37,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { public void push(long key, int pos, int count) { Assert.eq(count, "push count", 1); - int currentVal = intValueChunk.get(pos); - if(currentVal != NULL_INT) { - curVal = currentVal; + int val = intValueChunk.get(pos); + if(val != NULL_INT) { + curVal = val; } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java index f273d01660b..d5a0ee5b78d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java @@ -45,9 +45,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { public void push(long key, int pos, int count) { Assert.eq(count, "push count", 1); - long currentVal = longValueChunk.get(pos); - if(currentVal != NULL_LONG) { - curVal = currentVal; + long val = longValueChunk.get(pos); + if(val != NULL_LONG) { + curVal = val; } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java index 39484c130cd..ca9046d23a0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java @@ -38,9 +38,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { public void push(long key, int pos, int count) { Assert.eq(count, "push count", 1); - T currentVal = ObjectValueChunk.get(pos); - if(currentVal != null) { - curVal = currentVal; + T val = ObjectValueChunk.get(pos); + if(val != null) { + curVal = val; } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java index 9aaada7e18b..4f20e07da4a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java @@ -37,9 +37,9 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { public void push(long key, int pos, int count) { Assert.eq(count, "push count", 1); - short currentVal = shortValueChunk.get(pos); - if(currentVal != NULL_SHORT) { - curVal = currentVal; + short val = shortValueChunk.get(pos); + if(val != NULL_SHORT) { + curVal = val; } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java index 29c2642942c..f73da17ebe0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java @@ -57,7 +57,6 @@ public void accumulate(RowSequence inputKeys, int len) { setValuesChunk(valueChunkArr[0]); - setTimestampChunk(tsChunk); // chunk processing for (int ii = 0; ii < len; ii++) { @@ -79,8 +78,6 @@ public void close() { @Override public void setValuesChunk(@NotNull final Chunk valuesChunk) {} - @Override - public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} @Override public void writeToOutputChunk(int outIdx) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java index 4563a5df585..fd18745dd7e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java @@ -46,7 +46,6 @@ public void accumulate(RowSequence inputKeys, int len) { setValuesChunk(valueChunkArr[0]); - setTimestampChunk(tsChunk); // chunk processing for (int ii = 0; ii < len; ii++) { @@ -68,8 +67,6 @@ public void close() { @Override public void setValuesChunk(@NotNull final Chunk valuesChunk) {} - @Override - public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} @Override public void writeToOutputChunk(int outIdx) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java index 23dde0f795e..1b5a93f872a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java @@ -51,7 +51,6 @@ public void accumulate(RowSequence inputKeys, int len) { setValuesChunk(valueChunkArr[0]); - setTimestampChunk(tsChunk); // chunk processing for (int ii = 0; ii < len; ii++) { @@ -73,8 +72,6 @@ public void close() { @Override public void setValuesChunk(@NotNull final Chunk valuesChunk) {} - @Override - public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} @Override public void writeToOutputChunk(int outIdx) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java index aff25a6acba..be608bae468 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java @@ -51,7 +51,6 @@ public void accumulate(RowSequence inputKeys, int len) { setValuesChunk(valueChunkArr[0]); - setTimestampChunk(tsChunk); // chunk processing for (int ii = 0; ii < len; ii++) { @@ -73,8 +72,6 @@ public void close() { @Override public void setValuesChunk(@NotNull final Chunk valuesChunk) {} - @Override - public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} @Override public void writeToOutputChunk(int outIdx) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java index 81fd7cc666b..09d61b49281 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java @@ -51,7 +51,6 @@ public void accumulate(RowSequence inputKeys, int len) { setValuesChunk(valueChunkArr[0]); - setTimestampChunk(tsChunk); // chunk processing for (int ii = 0; ii < len; ii++) { @@ -73,8 +72,6 @@ public void close() { @Override public void setValuesChunk(@NotNull final Chunk valuesChunk) {} - @Override - public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} @Override public void writeToOutputChunk(int outIdx) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java index f1bad77e99f..51f588cb336 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java @@ -51,7 +51,6 @@ public void accumulate(RowSequence inputKeys, int len) { setValuesChunk(valueChunkArr[0]); - setTimestampChunk(tsChunk); // chunk processing for (int ii = 0; ii < len; ii++) { @@ -73,8 +72,6 @@ public void close() { @Override public void setValuesChunk(@NotNull final Chunk valuesChunk) {} - @Override - public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} @Override public void writeToOutputChunk(int outIdx) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java index 075a991d639..38658a457ad 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java @@ -54,7 +54,6 @@ public void accumulate(RowSequence inputKeys, int len) { setValuesChunk(valueChunkArr[0]); - setTimestampChunk(tsChunk); // chunk processing for (int ii = 0; ii < len; ii++) { @@ -76,8 +75,6 @@ public void close() { @Override public void setValuesChunk(@NotNull final Chunk valuesChunk) {} - @Override - public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} @Override public void writeToOutputChunk(int outIdx) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java index 23c061981ef..439d7e5c7d6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java @@ -51,7 +51,6 @@ public void accumulate(RowSequence inputKeys, int len) { setValuesChunk(valueChunkArr[0]); - setTimestampChunk(tsChunk); // chunk processing for (int ii = 0; ii < len; ii++) { @@ -73,8 +72,6 @@ public void close() { @Override public void setValuesChunk(@NotNull final Chunk valuesChunk) {} - @Override - public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} @Override public void writeToOutputChunk(int outIdx) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java index d7507a49814..1e11f9ed583 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java @@ -93,9 +93,6 @@ public void accumulate(RowSequence inputKeys, @Override public void setValuesChunk(@NotNull final Chunk valuesChunk) {} - @Override - public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} - @Override public void writeToOutputChunk(int outIdx) { outputValues.set(outIdx, curVal); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java index 76e66907d6b..efc26a4eff3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java @@ -82,9 +82,6 @@ public void accumulate(RowSequence inputKeys, @Override public void setValuesChunk(@NotNull final Chunk valuesChunk) {} - @Override - public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} - @Override public void writeToOutputChunk(int outIdx) { outputValues.set(outIdx, curVal); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java index 75485c7ad0c..d6c65c816dd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java @@ -87,9 +87,6 @@ public void accumulate(RowSequence inputKeys, @Override public void setValuesChunk(@NotNull final Chunk valuesChunk) {} - @Override - public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} - @Override public void writeToOutputChunk(int outIdx) { outputValues.set(outIdx, curVal); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java index 81094bd3509..ce7e159669f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java @@ -87,9 +87,6 @@ public void accumulate(RowSequence inputKeys, @Override public void setValuesChunk(@NotNull final Chunk valuesChunk) {} - @Override - public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} - @Override public void writeToOutputChunk(int outIdx) { outputValues.set(outIdx, curVal); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java index 88a80244393..07e211de8bf 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java @@ -87,9 +87,6 @@ public void accumulate(RowSequence inputKeys, @Override public void setValuesChunk(@NotNull final Chunk valuesChunk) {} - @Override - public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} - @Override public void writeToOutputChunk(int outIdx) { outputValues.set(outIdx, curVal); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java index fc7f7e0a393..34622e8aac1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java @@ -87,9 +87,6 @@ public void accumulate(RowSequence inputKeys, @Override public void setValuesChunk(@NotNull final Chunk valuesChunk) {} - @Override - public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} - @Override public void writeToOutputChunk(int outIdx) { outputValues.set(outIdx, curVal); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java index 6db7f05f079..bbaee240cbe 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java @@ -90,9 +90,6 @@ public void accumulate(RowSequence inputKeys, @Override public void setValuesChunk(@NotNull final Chunk valuesChunk) {} - @Override - public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} - @Override public void writeToOutputChunk(int outIdx) { outputValues.set(outIdx, curVal); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java index 4d554e18791..08d6f1be9d9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java @@ -87,9 +87,6 @@ public void accumulate(RowSequence inputKeys, @Override public void setValuesChunk(@NotNull final Chunk valuesChunk) {} - @Override - public void setTimestampChunk(@NotNull final LongChunk valuesChunk) {} - @Override public void writeToOutputChunk(int outIdx) { outputValues.set(outIdx, curVal); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java index c8d3cbdc2c4..d730f958de2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java @@ -1,8 +1,68 @@ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ByteCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ /* * --------------------------------------------------------------------------------------------------------------------- * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ShortCumMinMaxOperator and regenerate * --------------------------------------------------------------------------------------------------------------------- */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ByteCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ByteCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ByteCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ByteCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ByteCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ByteCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ByteCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ByteCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ByteCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ByteCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ByteCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ package io.deephaven.engine.table.impl.updateby.minmax; import io.deephaven.base.verify.Assert; @@ -15,6 +75,7 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import static io.deephaven.util.QueryConstants.NULL_LONG; import static io.deephaven.util.QueryConstants.NULL_BYTE; public class ByteCumMinMaxOperator extends BaseByteUpdateByOperator { @@ -39,15 +100,14 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { public void push(long key, int pos, int count) { Assert.eq(count, "push count", 1); - // read the value from the values chunk - final byte currentVal = byteValueChunk.get(pos); + final byte val = byteValueChunk.get(pos); if (curVal == NULL_BYTE) { - curVal = currentVal; - } else if (currentVal != NULL_BYTE) { - if ((isMax && currentVal > curVal) || - (!isMax && currentVal < curVal)) { - curVal = currentVal; + curVal = val; + } else if (val != NULL_BYTE) { + if ((isMax && val > curVal) || + (!isMax && val < curVal)) { + curVal = val; } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java index 3b74335c1a3..1b5d79c0cf8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java @@ -39,15 +39,14 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { public void push(long key, int pos, int count) { Assert.eq(count, "push count", 1); - // read the value from the values chunk - final double currentVal = doubleValueChunk.get(pos); + final double val = doubleValueChunk.get(pos); if(curVal == NULL_DOUBLE) { - curVal = currentVal; - } else if(currentVal != NULL_DOUBLE) { - if ((isMax && currentVal > curVal) || - (!isMax && currentVal < curVal)) { - curVal = currentVal; + curVal = val; + } else if(val != NULL_DOUBLE) { + if ((isMax && val > curVal) || + (!isMax && val < curVal)) { + curVal = val; } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java index f30ab671f86..dc46e11a1d4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java @@ -34,15 +34,14 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { public void push(long key, int pos, int count) { Assert.eq(count, "push count", 1); - // read the value from the values chunk - final float currentVal = floatValueChunk.get(pos); + final float val = floatValueChunk.get(pos); if(curVal == NULL_FLOAT) { - curVal = currentVal; - } else if(currentVal != NULL_FLOAT) { - if ((isMax && currentVal > curVal) || - (!isMax && currentVal < curVal)) { - curVal = currentVal; + curVal = val; + } else if(val != NULL_FLOAT) { + if ((isMax && val > curVal) || + (!isMax && val < curVal)) { + curVal = val; } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java index 0f49afd028d..8ddd854fc20 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java @@ -1,8 +1,68 @@ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit IntegerCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ /* * --------------------------------------------------------------------------------------------------------------------- * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ShortCumMinMaxOperator and regenerate * --------------------------------------------------------------------------------------------------------------------- */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit IntegerCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit IntegerCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit IntegerCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit IntegerCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit IntegerCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit IntegerCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit IntegerCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit IntegerCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit IntegerCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit IntegerCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit IntegerCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ package io.deephaven.engine.table.impl.updateby.minmax; import io.deephaven.base.verify.Assert; @@ -15,6 +75,7 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import static io.deephaven.util.QueryConstants.NULL_LONG; import static io.deephaven.util.QueryConstants.NULL_INT; public class IntCumMinMaxOperator extends BaseIntUpdateByOperator { @@ -39,15 +100,14 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { public void push(long key, int pos, int count) { Assert.eq(count, "push count", 1); - // read the value from the values chunk - final int currentVal = intValueChunk.get(pos); + final int val = intValueChunk.get(pos); if (curVal == NULL_INT) { - curVal = currentVal; - } else if (currentVal != NULL_INT) { - if ((isMax && currentVal > curVal) || - (!isMax && currentVal < curVal)) { - curVal = currentVal; + curVal = val; + } else if (val != NULL_INT) { + if ((isMax && val > curVal) || + (!isMax && val < curVal)) { + curVal = val; } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java index 564679dbe93..42a4ae56b57 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java @@ -1,8 +1,68 @@ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit LongCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ /* * --------------------------------------------------------------------------------------------------------------------- * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ShortCumMinMaxOperator and regenerate * --------------------------------------------------------------------------------------------------------------------- */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit LongCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit LongCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit LongCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit LongCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit LongCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit LongCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit LongCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit LongCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit LongCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit LongCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit LongCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ package io.deephaven.engine.table.impl.updateby.minmax; import io.deephaven.engine.table.ColumnSource; @@ -22,6 +82,7 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import static io.deephaven.util.QueryConstants.NULL_LONG; import static io.deephaven.util.QueryConstants.NULL_LONG; public class LongCumMinMaxOperator extends BaseLongUpdateByOperator { @@ -47,15 +108,14 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { public void push(long key, int pos, int count) { Assert.eq(count, "push count", 1); - // read the value from the values chunk - final long currentVal = longValueChunk.get(pos); + final long val = longValueChunk.get(pos); if (curVal == NULL_LONG) { - curVal = currentVal; - } else if (currentVal != NULL_LONG) { - if ((isMax && currentVal > curVal) || - (!isMax && currentVal < curVal)) { - curVal = currentVal; + curVal = val; + } else if (val != NULL_LONG) { + if ((isMax && val > curVal) || + (!isMax && val < curVal)) { + curVal = val; } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java index 476805da716..1f250b8bf0c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java @@ -1,3 +1,63 @@ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ShortCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ShortCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ShortCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ShortCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ShortCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ShortCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ShortCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ShortCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ShortCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ShortCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ShortCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ShortCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ package io.deephaven.engine.table.impl.updateby.minmax; import io.deephaven.base.verify.Assert; @@ -10,6 +70,7 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import static io.deephaven.util.QueryConstants.NULL_LONG; import static io.deephaven.util.QueryConstants.NULL_SHORT; public class ShortCumMinMaxOperator extends BaseShortUpdateByOperator { @@ -34,15 +95,14 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { public void push(long key, int pos, int count) { Assert.eq(count, "push count", 1); - // read the value from the values chunk - final short currentVal = shortValueChunk.get(pos); + final short val = shortValueChunk.get(pos); if (curVal == NULL_SHORT) { - curVal = currentVal; - } else if (currentVal != NULL_SHORT) { - if ((isMax && currentVal > curVal) || - (!isMax && currentVal < curVal)) { - curVal = currentVal; + curVal = val; + } else if (val != NULL_SHORT) { + if ((isMax && val > curVal) || + (!isMax && val < curVal)) { + curVal = val; } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java index 26e3cc61b39..4fe9a2d3228 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java @@ -33,16 +33,10 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { public void push(long key, int pos, int count) { Assert.eq(count, "push count", 1); - // read the value from the values chunk - final BigDecimal currentVal = objectValueChunk.get(pos); + final BigDecimal val = objectValueChunk.get(pos); - final boolean isCurrentNull = currentVal == null; - if (curVal == null) { - curVal = isCurrentNull ? null : currentVal; - } else { - if (!isCurrentNull) { - curVal = curVal.multiply(objectValueChunk.get(pos), mathContext); - } + if (val != null) { + curVal = curVal == null ? val : curVal.multiply(val, mathContext); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java index cc56add1eb3..2bdd5545e0b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java @@ -12,6 +12,9 @@ import java.math.BigInteger; +import static io.deephaven.util.QueryConstants.NULL_LONG; +import static io.deephaven.util.QueryConstants.NULL_SHORT; + public final class BigIntegerCumProdOperator extends BaseObjectUpdateByOperator { protected class Context extends BaseObjectUpdateByOperator.Context { public ObjectChunk objectValueChunk; @@ -29,16 +32,10 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { public void push(long key, int pos, int count) { Assert.eq(count, "push count", 1); - // read the value from the values chunk - final BigInteger currentVal = objectValueChunk.get(pos); + final BigInteger val = objectValueChunk.get(pos); - final boolean isCurrentNull = currentVal == null; - if(curVal == null) { - curVal = isCurrentNull ? null : currentVal; - } else { - if(!isCurrentNull) { - curVal = curVal.multiply(objectValueChunk.get(pos)); - } + if (val != null) { + curVal = curVal == null ? val : curVal.multiply(val); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java index ffe9c770f71..db92c73412c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java @@ -38,13 +38,10 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { public void push(long key, int pos, int count) { Assert.eq(count, "push count", 1); - // read the value from the values chunk - final byte currentVal = byteValueChunk.get(pos); + final byte val = byteValueChunk.get(pos); - if (curVal == NULL_LONG) { - curVal = currentVal == NULL_BYTE ? NULL_LONG : currentVal; - } else if (currentVal != NULL_BYTE) { - curVal *= currentVal; + if (val != NULL_BYTE) { + curVal = curVal == NULL_LONG ? val : curVal * val; } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java index 197bb172b13..3955755b76a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java @@ -15,7 +15,7 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.NULL_DOUBLE; +import static io.deephaven.util.QueryConstants.*; public class DoubleCumProdOperator extends BaseDoubleUpdateByOperator { // region extra-fields @@ -37,13 +37,10 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { public void push(long key, int pos, int count) { Assert.eq(count, "push count", 1); - // read the value from the values chunk - final double currentVal = doubleValueChunk.get(pos); + final double val = doubleValueChunk.get(pos); - if (curVal == NULL_DOUBLE) { - curVal = currentVal; - } else if (currentVal != NULL_DOUBLE) { - curVal *= currentVal; + if (val != NULL_DOUBLE) { + curVal = curVal == NULL_DOUBLE ? val : curVal * val; } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java index 72110018cdf..e14c83c938c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java @@ -10,7 +10,7 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.NULL_FLOAT; +import static io.deephaven.util.QueryConstants.*; public class FloatCumProdOperator extends BaseFloatUpdateByOperator { // region extra-fields @@ -32,13 +32,10 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { public void push(long key, int pos, int count) { Assert.eq(count, "push count", 1); - // read the value from the values chunk - final float currentVal = floatValueChunk.get(pos); + final float val = floatValueChunk.get(pos); - if (curVal == NULL_FLOAT) { - curVal = currentVal; - } else if (currentVal != NULL_FLOAT) { - curVal *= currentVal; + if (val != NULL_FLOAT) { + curVal = curVal == NULL_FLOAT ? val : curVal * val; } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java index 70613de699f..aad1b3898fc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java @@ -38,13 +38,10 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { public void push(long key, int pos, int count) { Assert.eq(count, "push count", 1); - // read the value from the values chunk - final int currentVal = intValueChunk.get(pos); + final int val = intValueChunk.get(pos); - if (curVal == NULL_LONG) { - curVal = currentVal == NULL_INT ? NULL_LONG : currentVal; - } else if (currentVal != NULL_INT) { - curVal *= currentVal; + if (val != NULL_INT) { + curVal = curVal == NULL_LONG ? val : curVal * val; } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java index 873ae7e8ab6..d31cef26e5d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java @@ -38,13 +38,10 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { public void push(long key, int pos, int count) { Assert.eq(count, "push count", 1); - // read the value from the values chunk - final long currentVal = longValueChunk.get(pos); + final long val = longValueChunk.get(pos); - if (curVal == NULL_LONG) { - curVal = currentVal == NULL_LONG ? NULL_LONG : currentVal; - } else if (currentVal != NULL_LONG) { - curVal *= currentVal; + if (val != NULL_LONG) { + curVal = curVal == NULL_LONG ? val : curVal * val; } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java index cc561008802..bc0a351e45c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java @@ -33,13 +33,10 @@ public void setValuesChunk(@NotNull final Chunk valuesChunk) { public void push(long key, int pos, int count) { Assert.eq(count, "push count", 1); - // read the value from the values chunk - final short currentVal = shortValueChunk.get(pos); + final short val = shortValueChunk.get(pos); - if (curVal == NULL_LONG) { - curVal = currentVal == NULL_SHORT ? NULL_LONG : currentVal; - } else if (currentVal != NULL_SHORT) { - curVal *= currentVal; + if (val != NULL_SHORT) { + curVal = curVal == NULL_LONG ? val : curVal * val; } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java index 0611f922d63..3f666356e67 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java @@ -79,6 +79,7 @@ public void pop(int count) { public void writeToOutputChunk(int outIdx) { if (objectWindowValues.size() == nullCount) { outputValues.set(outIdx, null); + curVal = null; } else { outputValues.set(outIdx, curVal); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java index 855092033c9..c1fe24d7816 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java @@ -76,6 +76,7 @@ public void pop(int count) { public void writeToOutputChunk(int outIdx) { if (objectWindowValues.size() == nullCount) { outputValues.set(outIdx, null); + curVal = null; } else { outputValues.set(outIdx, curVal); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java index d0068e2c02e..97887c64d01 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java @@ -5,7 +5,6 @@ */ package io.deephaven.engine.table.impl.updateby.rollingsum; -import io.deephaven.api.updateby.OperationControl; import io.deephaven.base.ringbuffer.ByteRingBuffer; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; @@ -17,7 +16,8 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.*; +import static io.deephaven.util.QueryConstants.NULL_LONG; +import static io.deephaven.util.QueryConstants.NULL_BYTE; public class ByteRollingSumOperator extends BaseWindowedLongUpdateByOperator { private static final int RING_BUFFER_INITIAL_CAPACITY = 512; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java index 308b10bd16f..6a546d847b3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java @@ -9,7 +9,6 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.DoubleChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedDoubleUpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.PairwiseDoubleRingBuffer; @@ -17,9 +16,6 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.util.Collections; -import java.util.Map; - import static io.deephaven.util.QueryConstants.NULL_DOUBLE; public class DoubleRollingSumOperator extends BaseWindowedDoubleUpdateByOperator { @@ -110,10 +106,4 @@ public DoubleRollingSumOperator(@NotNull final MatchPair pair, // region constructor // endregion constructor } - - @NotNull - @Override - public Map> getOutputColumns() { - return Collections.singletonMap(pair.leftColumn, outputSource); - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java index 0c0a6549d91..7d3d4976732 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java @@ -4,7 +4,6 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.FloatChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedFloatUpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.PairwiseFloatRingBuffer; @@ -12,9 +11,6 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.util.Collections; -import java.util.Map; - import static io.deephaven.util.QueryConstants.NULL_FLOAT; public class FloatRollingSumOperator extends BaseWindowedFloatUpdateByOperator { @@ -105,10 +101,4 @@ public FloatRollingSumOperator(@NotNull final MatchPair pair, // region constructor // endregion constructor } - - @NotNull - @Override - public Map> getOutputColumns() { - return Collections.singletonMap(pair.leftColumn, outputSource); - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java index e890db91ee8..a8a6ea1fe16 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java @@ -5,7 +5,6 @@ */ package io.deephaven.engine.table.impl.updateby.rollingsum; -import io.deephaven.api.updateby.OperationControl; import io.deephaven.base.ringbuffer.IntRingBuffer; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; @@ -17,7 +16,8 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.*; +import static io.deephaven.util.QueryConstants.NULL_LONG; +import static io.deephaven.util.QueryConstants.NULL_INT; public class IntRollingSumOperator extends BaseWindowedLongUpdateByOperator { private static final int RING_BUFFER_INITIAL_CAPACITY = 512; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java index e2a78c81f70..0f4075ff23b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java @@ -5,7 +5,6 @@ */ package io.deephaven.engine.table.impl.updateby.rollingsum; -import io.deephaven.api.updateby.OperationControl; import io.deephaven.base.ringbuffer.LongRingBuffer; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; @@ -17,7 +16,8 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.*; +import static io.deephaven.util.QueryConstants.NULL_LONG; +import static io.deephaven.util.QueryConstants.NULL_LONG; public class LongRollingSumOperator extends BaseWindowedLongUpdateByOperator { private static final int RING_BUFFER_INITIAL_CAPACITY = 512; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java index 28ad32a2cbb..7d474f1ec29 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java @@ -1,6 +1,5 @@ package io.deephaven.engine.table.impl.updateby.rollingsum; -import io.deephaven.api.updateby.OperationControl; import io.deephaven.base.ringbuffer.ShortRingBuffer; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; @@ -12,7 +11,8 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.*; +import static io.deephaven.util.QueryConstants.NULL_LONG; +import static io.deephaven.util.QueryConstants.NULL_SHORT; public class ShortRollingSumOperator extends BaseWindowedLongUpdateByOperator { private static final int RING_BUFFER_INITIAL_CAPACITY = 512; diff --git a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java index 755dd5aa243..342d95c3a19 100644 --- a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java +++ b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java @@ -74,6 +74,7 @@ public static void main(String[] args) throws IOException { replicateNumericOperator( "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java", "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java"); + replicateNumericOperator( "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java", "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java"); @@ -196,6 +197,7 @@ private static void fixupBoolean(String boolResult) throws IOException { "boolean singletonVal", "byte singletonVal", "QueryConstants", "BooleanUtils", "boolean curVal", "byte curVal", + "boolean val", "byte val", "getBoolean", "getByte", "boolean previousVal", "byte previousVal", "boolean currentVal", "byte currentVal", @@ -287,7 +289,7 @@ private static void fixupStandardObject(String objectResult, final String classN "new ObjectSparseArraySource\\(\\);", "new ObjectSparseArraySource<>(colType);", "(?:QueryConstants\\.)?NULL_OBJECT", "null", "Object lastValidValue", "T lastValidValue", - "Object currentVal", "T currentVal", + "Object val", "T val", "Object curVal", "T curVal", "Object previousVal", "T previousVal", "Object singletonVal", "T singletonVal", From 65f02a4aa626c2143f4572f2d479b968dbc75013 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 27 Jan 2023 12:30:52 -0800 Subject: [PATCH 102/123] Low hanging fruit exhausted --- .../engine/rowset/RowSetShiftData.java | 5 ++- .../engine/table/impl/updateby/UpdateBy.java | 5 +++ .../impl/updateby/UpdateByBucketHelper.java | 45 +++++++++++++------ .../updateby/ema/BigDecimalEMAOperator.java | 14 +++--- .../updateby/ema/BigNumberEMAOperator.java | 30 ++++++------- .../engine/table/impl/util/JobScheduler.java | 3 +- 6 files changed, 65 insertions(+), 37 deletions(-) diff --git a/engine/rowset/src/main/java/io/deephaven/engine/rowset/RowSetShiftData.java b/engine/rowset/src/main/java/io/deephaven/engine/rowset/RowSetShiftData.java index 8e64ced5020..0c62e5c86fd 100644 --- a/engine/rowset/src/main/java/io/deephaven/engine/rowset/RowSetShiftData.java +++ b/engine/rowset/src/main/java/io/deephaven/engine/rowset/RowSetShiftData.java @@ -329,7 +329,7 @@ public boolean apply(final WritableRowSet rowSet) { } /** - * Apply all shifts to {@code keyToShift}. Moves the single from pre-shift keyspace to post-shift keyspace. + * Apply all shifts to {@code keyToShift}. Moves the single row key from pre-shift keyspace to post-shift keyspace. * * @param keyToShift The single row key to shift * @return the key in post-shift space @@ -339,7 +339,8 @@ public long apply(final long keyToShift) { if (getBeginRange(shiftIdx) > keyToShift) { // no shift applies so we are already in post-shift space return keyToShift; - } else if (getEndRange(shiftIdx) >= keyToShift) { + } + if (getEndRange(shiftIdx) >= keyToShift) { // this shift applies, add the delta to get post-shift return keyToShift + getShiftDelta(shiftIdx); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java index 870cdc29768..dc5dc954b71 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java @@ -890,6 +890,11 @@ public static Table updateBy(@NotNull final QueryTable source, } } + if (!problems.isEmpty()) { + throw new UncheckedTableException(descriptionBuilder + ": resulting column names must be unique {" + + String.join(", ", problems) + "}"); + } + // We will divide the operators into similar windows for efficient processing. final KeyedObjectHashMap> windowMap = new KeyedObjectHashMap<>(new KeyedObjectKey<>() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java index 7ec7adc1336..dd74e62890f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java @@ -18,13 +18,13 @@ import io.deephaven.engine.table.impl.sources.ReinterpretUtils; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.engine.table.impl.util.RowRedirection; -import io.deephaven.util.SafeCloseable; +import io.deephaven.engine.updategraph.UpdateCommitter; +import io.deephaven.util.SafeCloseableArray; import io.deephaven.util.datastructures.linked.IntrusiveDoublyLinkedNode; import org.apache.commons.lang3.mutable.MutableLong; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.util.Arrays; import java.util.Map; import static io.deephaven.util.QueryConstants.NULL_LONG; @@ -56,10 +56,15 @@ class UpdateByBucketHelper extends IntrusiveDoublyLinkedNode.Impl committer; + public UpdateBy parentUpdateBy; + public long createdStep; + /** * Perform updateBy operations on a single bucket of data (either zero-key or already limited through partitioning) * @@ -172,7 +177,7 @@ private void processUpdateForSsa(final TableUpdate upstream, final boolean times timestampValidRowSet.close(); timestampValidRowSet = source.getRowSet().writableCast(); } else { - timestampValidRowSet.remove(ssaKeys, 0, ssaKeys.size()); + timestampValidRowSet.writableCast().remove(ssaKeys, 0, ssaKeys.size()); } } } @@ -205,8 +210,12 @@ private void processUpdateForSsa(final TableUpdate upstream, final boolean times final LongChunk shiftValues = timestampColumnSource.getPrevChunk(getContext, subRowSet).asLongChunk(); - - timestampSsa.applyShiftReverse(shiftValues, subRowSet.asRowKeyChunk(), sit.shiftDelta()); + if (sit.polarityReversed()) { + timestampSsa.applyShiftReverse(shiftValues, subRowSet.asRowKeyChunk(), + sit.shiftDelta()); + } else { + timestampSsa.applyShift(shiftValues, subRowSet.asRowKeyChunk(), sit.shiftDelta()); + } } } } @@ -232,20 +241,21 @@ private void processUpdateForSsa(final TableUpdate upstream, final boolean times if (timestampValidRowSet == source.getRowSet()) { if (nullTimestampCount > 0) { // make a copy and remove the nulls - timestampValidRowSet = source.getRowSet().copy().toTracking().writableCast(); - timestampValidRowSet.remove(nullTsKeys, 0, nullTsKeys.size()); + timestampValidRowSet = source.getRowSet().copy().toTracking(); + timestampValidRowSet.writableCast().remove(nullTsKeys, 0, nullTsKeys.size()); } } else { - timestampValidRowSet.insert(ssaKeys, 0, ssaKeys.size()); + timestampValidRowSet.writableCast().insert(ssaKeys, 0, ssaKeys.size()); } } } } } + Assert.eq(nullTimestampCount, "nullTimestampCount", source.size() - timestampValidRowSet.size()); } /** - * helper function to fill a LongChunk while skipping values that are NULL_LONG. Used to populate an SSA from a + * Helper function to fill a LongChunk while skipping values that are NULL_LONG. Used to populate an SSA from a * source containing null values * * @return the number of NULL values found in the set @@ -264,7 +274,7 @@ private int fillChunkWithNonNull(LongChunk keysChunk, LongChunk< for (int i = 0; i < valuesChunk.size(); i++) { long ts = valuesChunk.get(i); if (ts == NULL_LONG) { - // null timestamps will not cause problems + // track the nulls added during this operation nullTimestampKeys.add(keysChunk.get(i)); nullCount++; continue; @@ -368,8 +378,7 @@ public void processWindow(final int winIdx, final boolean initialStep) { * Close the window contexts and release resources for this bucket */ public void finalizeUpdate() { - SafeCloseable.closeArray(windowContexts); - Arrays.fill(windowContexts, null); + SafeCloseableArray.close(windowContexts); isDirty = false; } @@ -388,6 +397,16 @@ public UpdateByBucketHelperListener(@Nullable String description, public void onUpdate(TableUpdate upstream) { prepareForUpdate(upstream, false); + UpdateByBucketHelper.this.committer = + new UpdateCommitter(UpdateByBucketHelper.this, (bucket) -> { + // ensure that the item has been cleaned up + Assert.eqFalse(bucket.isDirty, "bucket.isDirty"); + for (UpdateByWindow.UpdateByWindowBucketContext ctx : bucket.windowContexts) { + Assert.eqNull(ctx, "bucket.windowContexts[]"); + } + }); + UpdateByBucketHelper.this.committer.maybeActivate(); + // pass the update unchanged, just increment the ref count TableUpdate downstream = upstream.acquire(); result.notifyListeners(downstream); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java index c562cf06dcb..08f0bc4ae84 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java @@ -59,6 +59,7 @@ public void accumulate(RowSequence inputKeys, handleBadData(this, isNull); } else if (isNullTime) { // no change to curVal and lastStamp + continue; } else { if (curVal == null) { curVal = input; @@ -97,13 +98,14 @@ public void push(long key, int pos, int count) { /** * An operator that computes an EMA from a BigDecimal column using an exponential decay function. * - * @param pair the {@link MatchPair} that defines the input/output for this operation - * @param affectingColumns the names of the columns that affect this ema - * @param control defines how to handle {@code null} input values. + * @param pair the {@link MatchPair} that defines the input/output for this operation + * @param affectingColumns the names of the columns that affect this ema + * @param control defines how to handle {@code null} input values. * @param timestampColumnName the name of the column containing timestamps for time-based calcuations - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is measured in ticks, otherwise it is measured in nanoseconds - * @param rowRedirection the {@link RowRedirection} to use for dense output sources - * @param valueSource a reference to the input column source for this operation + * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is + * measured in ticks, otherwise it is measured in nanoseconds + * @param rowRedirection the {@link RowRedirection} to use for dense output sources + * @param valueSource a reference to the input column source for this operation */ public BigDecimalEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java index f11b9c56016..f204a1a0845 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java @@ -27,7 +27,6 @@ public abstract class BigNumberEMAOperator extends BaseObjectUpdateByOperator public abstract class Context extends BaseObjectUpdateByOperator.Context { - public LongChunk timestampValueChunk; public ObjectChunk objectValueChunk; long lastStamp = NULL_LONG; @@ -56,21 +55,22 @@ public void reset() { /** * An operator that computes an EMA from a big number column using an exponential decay function. * - * @param pair the {@link MatchPair} that defines the input/output for this operation - * @param affectingColumns the names of the columns that affect this ema - * @param control defines how to handle {@code null} input values. + * @param pair the {@link MatchPair} that defines the input/output for this operation + * @param affectingColumns the names of the columns that affect this ema + * @param control defines how to handle {@code null} input values. * @param timestampColumnName the name of the column containing timestamps for time-based calcuations - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is measured in ticks, otherwise it is measured in nanoseconds - * @param rowRedirection the {@link RowRedirection} to use for dense output sources - * @param valueSource a reference to the input column source for this operation + * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is + * measured in ticks, otherwise it is measured in nanoseconds + * @param rowRedirection the {@link RowRedirection} to use for dense output sources + * @param valueSource a reference to the input column source for this operation */ public BigNumberEMAOperator(@NotNull final MatchPair pair, - @NotNull final String[] affectingColumns, - @NotNull final OperationControl control, - @Nullable final String timestampColumnName, - final long timeScaleUnits, - @Nullable final RowRedirection rowRedirection, - final ColumnSource valueSource) { + @NotNull final String[] affectingColumns, + @NotNull final OperationControl control, + @Nullable final String timestampColumnName, + final long timeScaleUnits, + @Nullable final RowRedirection rowRedirection, + final ColumnSource valueSource) { super(pair, affectingColumns, rowRedirection, timestampColumnName, timeScaleUnits, BigDecimal.class); this.control = control; @@ -83,8 +83,8 @@ public BigNumberEMAOperator(@NotNull final MatchPair pair, @Override public void initializeUpdate(@NotNull final UpdateContext updateContext, - final long firstUnmodifiedKey, - final long firstUnmodifiedTimestamp) { + final long firstUnmodifiedKey, + final long firstUnmodifiedTimestamp) { super.initializeUpdate(updateContext, firstUnmodifiedKey, firstUnmodifiedTimestamp); final Context ctx = (Context) updateContext; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java index a5e8ebedd0e..df37553e6ad 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java @@ -156,7 +156,8 @@ public void run() { * * @param executionContext the execution context for this task * @param description the description to use for logging - * @param taskThreadContextFactory the factory that supplies {@link JobContext contexts} for the tasks + * @param taskThreadContextFactory the factory that supplies {@link JobContext contexts} for the threads handling + * the sub-tasks * @param start the integer value from which to start iterating * @param count the number of times this task should be called * @param action the task to perform, the current iteration index is provided as a parameter From 03db0c548a1f7e237673510cb1cc739738a94fda Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 27 Jan 2023 15:54:01 -0800 Subject: [PATCH 103/123] Re-cleaning the operators. --- .../minmax/ByteCumMinMaxOperator.java | 65 +--------------- .../updateby/minmax/IntCumMinMaxOperator.java | 65 +--------------- .../minmax/LongCumMinMaxOperator.java | 76 ++++--------------- .../minmax/ShortCumMinMaxOperator.java | 65 +--------------- .../rollingsum/ByteRollingSumOperator.java | 3 +- .../rollingsum/IntRollingSumOperator.java | 3 +- .../rollingsum/LongRollingSumOperator.java | 3 +- .../rollingsum/ShortRollingSumOperator.java | 3 +- 8 files changed, 27 insertions(+), 256 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java index d730f958de2..b95a161abf2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java @@ -1,68 +1,8 @@ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ByteCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ /* * --------------------------------------------------------------------------------------------------------------------- * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ShortCumMinMaxOperator and regenerate * --------------------------------------------------------------------------------------------------------------------- */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ByteCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ByteCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ByteCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ByteCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ByteCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ByteCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ByteCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ByteCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ByteCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ByteCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ByteCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ package io.deephaven.engine.table.impl.updateby.minmax; import io.deephaven.base.verify.Assert; @@ -75,8 +15,7 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.NULL_LONG; -import static io.deephaven.util.QueryConstants.NULL_BYTE; +import static io.deephaven.util.QueryConstants.*; public class ByteCumMinMaxOperator extends BaseByteUpdateByOperator { private final boolean isMax; @@ -124,6 +63,8 @@ public ByteCumMinMaxOperator(@NotNull final MatchPair pair, // region constructor // endregion constructor } + // region extra-methods + // endregion extra-methods @NotNull @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java index 8ddd854fc20..421626094f1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java @@ -1,68 +1,8 @@ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit IntegerCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ /* * --------------------------------------------------------------------------------------------------------------------- * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ShortCumMinMaxOperator and regenerate * --------------------------------------------------------------------------------------------------------------------- */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit IntegerCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit IntegerCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit IntegerCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit IntegerCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit IntegerCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit IntegerCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit IntegerCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit IntegerCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit IntegerCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit IntegerCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit IntegerCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ package io.deephaven.engine.table.impl.updateby.minmax; import io.deephaven.base.verify.Assert; @@ -75,8 +15,7 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.NULL_LONG; -import static io.deephaven.util.QueryConstants.NULL_INT; +import static io.deephaven.util.QueryConstants.*; public class IntCumMinMaxOperator extends BaseIntUpdateByOperator { private final boolean isMax; @@ -124,6 +63,8 @@ public IntCumMinMaxOperator(@NotNull final MatchPair pair, // region constructor // endregion constructor } + // region extra-methods + // endregion extra-methods @NotNull @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java index 42a4ae56b57..93fd5aa88c3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java @@ -1,68 +1,8 @@ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit LongCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ /* * --------------------------------------------------------------------------------------------------------------------- * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ShortCumMinMaxOperator and regenerate * --------------------------------------------------------------------------------------------------------------------- */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit LongCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit LongCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit LongCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit LongCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit LongCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit LongCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit LongCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit LongCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit LongCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit LongCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit LongCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ package io.deephaven.engine.table.impl.updateby.minmax; import io.deephaven.engine.table.ColumnSource; @@ -82,8 +22,7 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.NULL_LONG; -import static io.deephaven.util.QueryConstants.NULL_LONG; +import static io.deephaven.util.QueryConstants.*; public class LongCumMinMaxOperator extends BaseLongUpdateByOperator { private final boolean isMax; @@ -134,6 +73,19 @@ public LongCumMinMaxOperator(@NotNull final MatchPair pair, this.type = type; // endregion constructor } + // region extra-methods + @NotNull + @Override + public Map> getOutputColumns() { + final ColumnSource actualOutput; + if(type == DateTime.class) { + actualOutput = ReinterpretUtils.longToDateTimeSource(outputSource); + } else { + actualOutput = outputSource; + } + return Collections.singletonMap(pair.leftColumn, actualOutput); + } + // endregion extra-methods @NotNull @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java index 1f250b8bf0c..81913428872 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java @@ -1,63 +1,3 @@ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ShortCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ShortCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ShortCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ShortCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ShortCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ShortCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ShortCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ShortCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ShortCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ShortCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ShortCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ShortCumMinMaxOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ package io.deephaven.engine.table.impl.updateby.minmax; import io.deephaven.base.verify.Assert; @@ -70,8 +10,7 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.NULL_LONG; -import static io.deephaven.util.QueryConstants.NULL_SHORT; +import static io.deephaven.util.QueryConstants.*; public class ShortCumMinMaxOperator extends BaseShortUpdateByOperator { private final boolean isMax; @@ -119,6 +58,8 @@ public ShortCumMinMaxOperator(@NotNull final MatchPair pair, // region constructor // endregion constructor } + // region extra-methods + // endregion extra-methods @NotNull @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java index 97887c64d01..f84feeb5316 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java @@ -16,8 +16,7 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.NULL_LONG; -import static io.deephaven.util.QueryConstants.NULL_BYTE; +import static io.deephaven.util.QueryConstants.*; public class ByteRollingSumOperator extends BaseWindowedLongUpdateByOperator { private static final int RING_BUFFER_INITIAL_CAPACITY = 512; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java index a8a6ea1fe16..68d930a92b8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java @@ -16,8 +16,7 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.NULL_LONG; -import static io.deephaven.util.QueryConstants.NULL_INT; +import static io.deephaven.util.QueryConstants.*; public class IntRollingSumOperator extends BaseWindowedLongUpdateByOperator { private static final int RING_BUFFER_INITIAL_CAPACITY = 512; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java index 0f4075ff23b..3d326c0d9c5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java @@ -16,8 +16,7 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.NULL_LONG; -import static io.deephaven.util.QueryConstants.NULL_LONG; +import static io.deephaven.util.QueryConstants.*; public class LongRollingSumOperator extends BaseWindowedLongUpdateByOperator { private static final int RING_BUFFER_INITIAL_CAPACITY = 512; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java index 7d474f1ec29..f6a8682c3e5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java @@ -11,8 +11,7 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.NULL_LONG; -import static io.deephaven.util.QueryConstants.NULL_SHORT; +import static io.deephaven.util.QueryConstants.*; public class ShortRollingSumOperator extends BaseWindowedLongUpdateByOperator { private static final int RING_BUFFER_INITIAL_CAPACITY = 512; From 2856868815c9587ce228abdacdf859aada81fc1a Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 27 Jan 2023 16:16:01 -0800 Subject: [PATCH 104/123] Fixed a race condition, can't seem to repro the failure. --- .../engine/table/impl/updateby/UpdateBy.java | 65 +++++++++++-------- .../impl/updateby/UpdateByBucketHelper.java | 3 +- 2 files changed, 41 insertions(+), 27 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java index dc5dc954b71..136f92f7fee 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java @@ -35,6 +35,7 @@ import java.util.*; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicReferenceArray; import java.util.stream.IntStream; /** @@ -205,7 +206,7 @@ protected UpdateBy( /** Release the input sources that will not be needed for the rest of this update */ private void releaseInputSources(int winIdx, ColumnSource[] maybeCachedInputSources, - WritableRowSet[] inputSourceRowSets, int[] inputSourceReferenceCounts) { + AtomicReferenceArray inputSourceRowSets, int[] inputSourceReferenceCounts) { final UpdateByWindow win = windows[winIdx]; final int[] uniqueWindowSources = win.getUniqueSourceIndices(); @@ -218,22 +219,21 @@ private void releaseInputSources(int winIdx, ColumnSource[] maybeCachedInputS } if (--inputSourceReferenceCounts[srcIdx] == 0) { - // release any objects we are holding in the cache - if (maybeCachedInputSources[srcIdx] instanceof ObjectArraySource) { - final long targetCapacity = inputSourceRowSets[srcIdx].size(); - for (long positionToNull = 0; positionToNull < targetCapacity; positionToNull += - backingChunk.size()) { - ((ObjectArraySource) maybeCachedInputSources[srcIdx]) - .resetWritableChunkToBackingStore(backingChunk, positionToNull); - backingChunk.fillWithNullValue(0, backingChunk.size()); + // Last use of this set, let's clean up + try (final RowSet rows = inputSourceRowSets.get(srcIdx)) { + // release any objects we are holding in the cache + if (maybeCachedInputSources[srcIdx] instanceof ObjectArraySource) { + final long targetCapacity = rows.size(); + for (long positionToNull = 0; positionToNull < targetCapacity; positionToNull += + backingChunk.size()) { + ((ObjectArraySource) maybeCachedInputSources[srcIdx]) + .resetWritableChunkToBackingStore(backingChunk, positionToNull); + backingChunk.fillWithNullValue(0, backingChunk.size()); + } } + inputSourceRowSets.set(srcIdx, null); + maybeCachedInputSources[srcIdx] = null; } - - // release the row set - inputSourceRowSets[srcIdx].close(); - inputSourceRowSets[srcIdx] = null; - - maybeCachedInputSources[srcIdx] = null; } } } @@ -270,7 +270,7 @@ class PhasedUpdateProcessor implements LogOutputAppendable { /** The active set of sources to use for processing, each source may be cached or original */ final ColumnSource[] maybeCachedInputSources; /** For cacheable sources, the minimal rowset to cache (union of bucket influencer rows) */ - final WritableRowSet[] inputSourceRowSets; + final AtomicReferenceArray inputSourceRowSets; /** For cacheable sources, track how many windows require this source */ final int[] inputSourceReferenceCounts; @@ -282,6 +282,7 @@ class PhasedUpdateProcessor implements LogOutputAppendable { * {@code prepareForParallelPopulation()} calls */ WritableRowSet changedRows; + /*** * These rows will be unused after this cycle and Object columns should NULL these keys */ @@ -291,6 +292,10 @@ class PhasedUpdateProcessor implements LogOutputAppendable { this.upstream = upstream; this.initialStep = initialStep; + // TODO: remove this + for (UpdateByBucketHelper bucket : buckets) { + bucket.pup = this; + } // determine which buckets we'll examine during this update dirtyBuckets = buckets.stream().filter(UpdateByBucketHelper::isDirty).toArray(UpdateByBucketHelper[]::new); // which windows are dirty and need to be computed this cycle @@ -298,7 +303,7 @@ class PhasedUpdateProcessor implements LogOutputAppendable { if (inputCacheNeeded) { maybeCachedInputSources = new ColumnSource[inputSources.length]; - inputSourceRowSets = new WritableRowSet[inputSources.length]; + inputSourceRowSets = new AtomicReferenceArray<>(inputSources.length); inputSourceReferenceCounts = new int[inputSources.length]; // set the uncacheable columns into the array @@ -361,7 +366,7 @@ private void computeCachedColumnRowsets(final Runnable resumeAction) { for (int srcIdx : cacheableSourceIndices) { if (inputSourceCacheNeeded[srcIdx]) { // create a RowSet to be used by `InverseWrappedRowSetWritableRowRedirection` - inputSourceRowSets[srcIdx] = source.getRowSet().copy(); + inputSourceRowSets.set(srcIdx, source.getRowSet().copy()); // record how many windows require this input source inputSourceReferenceCounts[srcIdx] = @@ -385,12 +390,19 @@ private void computeCachedColumnRowsets(final Runnable resumeAction) { UpdateByWindow.UpdateByWindowBucketContext winCtx = bucket.windowContexts[winIdx]; if (win.isWindowDirty(winCtx)) { - // add this rowset to the running total for this input source - if (inputSourceRowSets[srcIdx] == null) { - inputSourceRowSets[srcIdx] = - win.getInfluencerRows(winCtx).copy(); - } else { - inputSourceRowSets[srcIdx].insert(win.getInfluencerRows(winCtx)); + WritableRowSet rows = inputSourceRowSets.get(srcIdx); + if (rows == null) { + final WritableRowSet influencerCopy = win.getInfluencerRows(winCtx).copy(); + if (!inputSourceRowSets.compareAndSet(srcIdx, null, influencerCopy)) { + influencerCopy.close(); + rows = inputSourceRowSets.get(srcIdx); + } + } + if (rows != null) { + // if not null, then insert this window's rowset + synchronized (rows) { + rows.insert(win.getInfluencerRows(winCtx)); + } } // at least one dirty bucket will need this source srcNeeded = true; @@ -413,14 +425,15 @@ private void computeCachedColumnRowsets(final Runnable resumeAction) { * when the work is complete */ private void createCachedColumnSource(int srcIdx, final Runnable resumeAction) { - if (maybeCachedInputSources[srcIdx] != null || inputSourceRowSets[srcIdx] == null) { + final WritableRowSet inputRowSet = inputSourceRowSets.get(srcIdx); + + if (maybeCachedInputSources[srcIdx] != null || inputRowSet == null) { // already cached from another operator (or caching not needed) resumeAction.run(); return; } final ColumnSource inputSource = inputSources[srcIdx]; - final WritableRowSet inputRowSet = inputSourceRowSets[srcIdx]; // re-use the dense column cache if it still exists WritableColumnSource innerSource; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java index dd74e62890f..f6e62e8525b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java @@ -64,6 +64,7 @@ class UpdateByBucketHelper extends IntrusiveDoublyLinkedNode.Impl committer; public UpdateBy parentUpdateBy; public long createdStep; + public UpdateBy.PhasedUpdateProcessor pup; /** * Perform updateBy operations on a single bucket of data (either zero-key or already limited through partitioning) @@ -398,7 +399,7 @@ public void onUpdate(TableUpdate upstream) { prepareForUpdate(upstream, false); UpdateByBucketHelper.this.committer = - new UpdateCommitter(UpdateByBucketHelper.this, (bucket) -> { + new UpdateCommitter<>(UpdateByBucketHelper.this, (bucket) -> { // ensure that the item has been cleaned up Assert.eqFalse(bucket.isDirty, "bucket.isDirty"); for (UpdateByWindow.UpdateByWindowBucketContext ctx : bucket.windowContexts) { From 9421854fbab1625ebeca29eef4c2eb3133ad14cc Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Mon, 30 Jan 2023 13:36:33 -0800 Subject: [PATCH 105/123] Addressing simple PR comments. --- .../BucketedPartitionedUpdateByManager.java | 3 +- .../engine/table/impl/updateby/UpdateBy.java | 118 ++++++++------- .../impl/updateby/UpdateByBucketHelper.java | 9 +- .../updateby/UpdateByOperatorFactory.java | 28 ++-- .../table/impl/updateby/UpdateByWindow.java | 28 ++-- .../impl/updateby/UpdateByWindowTicks.java | 11 +- .../impl/updateby/UpdateByWindowTime.java | 14 +- .../impl/updateby/ZeroKeyUpdateByManager.java | 3 +- .../updateby/ema/BigDecimalEMAOperator.java | 15 +- .../updateby/ema/BigIntegerEMAOperator.java | 13 +- .../updateby/ema/BigNumberEMAOperator.java | 17 ++- .../impl/updateby/ema/ByteEMAOperator.java | 2 - .../impl/updateby/ema/IntEMAOperator.java | 2 - .../impl/updateby/ema/LongEMAOperator.java | 2 - .../impl/updateby/ema/ShortEMAOperator.java | 2 - .../minmax/DoubleCumMinMaxOperator.java | 14 +- .../minmax/FloatCumMinMaxOperator.java | 17 ++- .../prod/BigIntegerCumProdOperator.java | 3 - .../engine/table/impl/util/JobScheduler.java | 25 ++-- .../impl/updateby/TestUpdateByGeneral.java | 47 +++++- .../engine/util/TestJobScheduler.java | 42 +++--- .../replicators/ReplicateUpdateBy.java | 18 ++- .../replication/ReplicatePrimitiveCode.java | 23 +-- .../api/updateby/UpdateByOperation.java | 140 +++++++++--------- .../api/updateby/spec/CumMinMaxSpec.java | 9 +- .../api/updateby/spec/CumProdSpec.java | 10 +- .../api/updateby/spec/CumSumSpec.java | 9 +- .../deephaven/api/updateby/spec/EmaSpec.java | 10 +- .../api/updateby/spec/RollingSumSpec.java | 59 ++++---- .../api/updateby/spec/UpdateBySpecBase.java | 15 ++ 30 files changed, 388 insertions(+), 320 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java index 5b36ac87a6b..b2773722dd7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java @@ -115,7 +115,8 @@ protected BucketedPartitionedUpdateByManager( } // make a dummy update to generate the initial row keys - final TableUpdateImpl fakeUpdate = new TableUpdateImpl(source.getRowSet(), + final TableUpdateImpl fakeUpdate = new TableUpdateImpl( + source.getRowSet().copy(), // send a copy since this will be closed by release() RowSetFactory.empty(), RowSetFactory.empty(), RowSetShiftData.EMPTY, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java index 136f92f7fee..e5dcb0b3b81 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java @@ -14,6 +14,7 @@ import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.exceptions.CancellationException; import io.deephaven.engine.exceptions.UncheckedTableException; +import io.deephaven.engine.liveness.LivenessScopeStack; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.*; @@ -21,6 +22,7 @@ import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.sources.sparse.SparseConstants; import io.deephaven.engine.table.impl.util.*; +import io.deephaven.engine.updategraph.DynamicNode; import io.deephaven.engine.updategraph.UpdateGraphProcessor; import io.deephaven.hash.KeyedObjectHashMap; import io.deephaven.hash.KeyedObjectKey; @@ -346,6 +348,7 @@ private void onError(Exception error) { waitForResult.completeExceptionally(error); } else { cleanUpAfterError(); + result().forceReferenceCountToZero(); // this is part of an update, need to notify downstream result().notifyListenersOnError(error, null); } @@ -378,8 +381,8 @@ private void computeCachedColumnRowsets(final Runnable resumeAction) { return; } - jobScheduler.iterateParallel(ExecutionContext.getContextToRecord(), this, JobScheduler.JobContext::new, - 0, cacheableSourceIndices.length, + jobScheduler.iterateParallel(ExecutionContext.getContextToRecord(), this, + JobScheduler.DEFAULT_CONTEXT_FACTORY, 0, cacheableSourceIndices.length, (context, idx) -> { final int srcIdx = cacheableSourceIndices[idx]; for (int winIdx = 0; winIdx < windows.length; winIdx++) { @@ -457,7 +460,7 @@ private void createCachedColumnSource(int srcIdx, final Runnable resumeAction) { final int taskCount = Math.toIntExact((inputRowSet.size() + PARALLEL_CACHE_BATCH_SIZE - 1) / PARALLEL_CACHE_BATCH_SIZE); - final class BatchContext extends JobScheduler.JobContext { + final class BatchThreadContext implements JobScheduler.JobThreadContext { final RowSequence.Iterator rsIt = inputRowSet.getRowSequenceIterator(); final ChunkSink.FillFromContext ffc = outputSource.makeFillFromContext(PARALLEL_CACHE_CHUNK_SIZE); @@ -470,7 +473,7 @@ public void close() { } } - jobScheduler.iterateParallel(ExecutionContext.getContextToRecord(), this, BatchContext::new, + jobScheduler.iterateParallel(ExecutionContext.getContextToRecord(), this, BatchThreadContext::new, 0, taskCount, (ctx, idx) -> { // advance to the first key of this block @@ -499,8 +502,8 @@ private void cacheInputSources(final int winIdx, final Runnable resumeAction) { final UpdateByWindow win = windows[winIdx]; final int[] uniqueWindowSources = win.getUniqueSourceIndices(); - jobScheduler.iterateParallel(ExecutionContext.getContextToRecord(), this, JobScheduler.JobContext::new, - 0, uniqueWindowSources.length, + jobScheduler.iterateParallel(ExecutionContext.getContextToRecord(), this, + JobScheduler.DEFAULT_CONTEXT_FACTORY, 0, uniqueWindowSources.length, (context, idx, sourceComplete) -> createCachedColumnSource(uniqueWindowSources[idx], sourceComplete), resumeAction, this::onError); @@ -517,7 +520,8 @@ private void cacheInputSources(final int winIdx, final Runnable resumeAction) { private void processWindowBuckets(int winIdx, final Runnable resumeAction) { if (jobScheduler.threadCount() > 1 && dirtyBuckets.length > 1) { // process the buckets in parallel - jobScheduler.iterateParallel(ExecutionContext.getContextToRecord(), this, JobScheduler.JobContext::new, + jobScheduler.iterateParallel(ExecutionContext.getContextToRecord(), this, + JobScheduler.DEFAULT_CONTEXT_FACTORY, 0, dirtyBuckets.length, (context, bucketIdx) -> { UpdateByBucketHelper bucket = dirtyBuckets[bucketIdx]; @@ -540,7 +544,8 @@ private void processWindowBuckets(int winIdx, final Runnable resumeAction) { * cached columns before starting the next window. Calls {@code completedAction} when the work is complete */ private void processWindows(final Runnable resumeAction) { - jobScheduler.iterateSerial(ExecutionContext.getContextToRecord(), this, JobScheduler.JobContext::new, 0, + jobScheduler.iterateSerial(ExecutionContext.getContextToRecord(), this, + JobScheduler.DEFAULT_CONTEXT_FACTORY, 0, windows.length, (context, winIdx, windowComplete) -> { UpdateByWindow win = windows[winIdx]; @@ -636,10 +641,10 @@ private void cleanUpAndNotify(final Runnable resumeAction) { } } - try (final RowSet ignoredRs = changedRows; - final RowSet ignoredRs2 = toClear) { - // auto close these resources - } + SafeCloseable.closeArray(changedRows, toClear); + + upstream.release(); + resumeAction.run(); } @@ -647,15 +652,14 @@ private void cleanUpAndNotify(final Runnable resumeAction) { * Clean up the resources created during this update. */ private void cleanUpAfterError() { - try (final RowSet ignoredRs = changedRows; - final RowSet ignoredRs2 = toClear) { - // auto close these resources - } - // allow the helpers to release their resources for (UpdateByBucketHelper bucket : dirtyBuckets) { bucket.finalizeUpdate(); } + + SafeCloseable.closeArray(changedRows, toClear); + + upstream.release(); } /** @@ -982,28 +986,30 @@ public boolean equalKey(UpdateByOperator updateByOperator, final Map> resultSources = new LinkedHashMap<>(source.getColumnSourceMap()); resultSources.putAll(opResultSources); - + final String fTimestampColumnName = timestampColumnName; if (pairs.length == 0) { descriptionBuilder.append(")"); - final ZeroKeyUpdateByManager zkm = new ZeroKeyUpdateByManager( - descriptionBuilder.toString(), - opArr, - windowArr, - inputSourceArr, - source, - resultSources, - timestampColumnName, - rowRedirection, - control); - - if (source.isRefreshing()) { - // start tracking previous values - if (rowRedirection != null) { - rowRedirection.startTrackingPrevValues(); + return LivenessScopeStack.computeEnclosed(() -> { + final ZeroKeyUpdateByManager zkm = new ZeroKeyUpdateByManager( + descriptionBuilder.toString(), + opArr, + windowArr, + inputSourceArr, + source, + resultSources, + fTimestampColumnName, + rowRedirection, + control); + + if (source.isRefreshing()) { + // start tracking previous values + if (rowRedirection != null) { + rowRedirection.startTrackingPrevValues(); + } + ops.forEach(UpdateByOperator::startTrackingPrev); } - ops.forEach(UpdateByOperator::startTrackingPrev); - } - return zkm.result; + return zkm.result; + }, source::isRefreshing, DynamicNode::isRefreshing); } descriptionBuilder.append(", pairs={").append(MatchPair.matchString(pairs)).append("})"); @@ -1019,26 +1025,28 @@ public boolean equalKey(UpdateByOperator updateByOperator, String.join(", ", problems) + "}"); } - final BucketedPartitionedUpdateByManager bm = new BucketedPartitionedUpdateByManager( - descriptionBuilder.toString(), - opArr, - windowArr, - inputSourceArr, - source, - resultSources, - byColumns, - timestampColumnName, - rowRedirection, - control); - - if (source.isRefreshing()) { - // start tracking previous values - if (rowRedirection != null) { - rowRedirection.startTrackingPrevValues(); + return LivenessScopeStack.computeEnclosed(() -> { + final BucketedPartitionedUpdateByManager bm = new BucketedPartitionedUpdateByManager( + descriptionBuilder.toString(), + opArr, + windowArr, + inputSourceArr, + source, + resultSources, + byColumns, + fTimestampColumnName, + rowRedirection, + control); + + if (source.isRefreshing()) { + // start tracking previous values + if (rowRedirection != null) { + rowRedirection.startTrackingPrevValues(); + } + ops.forEach(UpdateByOperator::startTrackingPrev); } - ops.forEach(UpdateByOperator::startTrackingPrev); - } - return bm.result; + return bm.result; + }, source::isRefreshing, DynamicNode::isRefreshing); } // endregion } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java index f6e62e8525b..140de6a06eb 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java @@ -116,7 +116,8 @@ protected UpdateByBucketHelper(@NotNull final String description, this.windowContexts = new UpdateByWindow.UpdateByWindowBucketContext[windows.length]; // make a fake update with the initial rows of the table - final TableUpdateImpl initialUpdate = new TableUpdateImpl(source.getRowSet(), + final TableUpdateImpl initialUpdate = new TableUpdateImpl( + source.getRowSet().copy(), // send a copy since this will be closed by release() RowSetFactory.empty(), RowSetFactory.empty(), RowSetShiftData.EMPTY, @@ -124,6 +125,8 @@ protected UpdateByBucketHelper(@NotNull final String description, prepareForUpdate(initialUpdate, true); + initialUpdate.release(); + if (source.isRefreshing()) { final UpdateByBucketHelperListener listener = newListener(description); source.addUpdateListener(listener); @@ -407,10 +410,6 @@ public void onUpdate(TableUpdate upstream) { } }); UpdateByBucketHelper.this.committer.maybeActivate(); - - // pass the update unchanged, just increment the ref count - TableUpdate downstream = upstream.acquire(); - result.notifyListeners(downstream); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperatorFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperatorFactory.java index 2fd99dd11a9..78dff2734ce 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperatorFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperatorFactory.java @@ -198,8 +198,8 @@ public Void visit(CumProdSpec p) { @Override public Void visit(@NotNull final RollingSumSpec rs) { - final boolean isTimeBased = rs.prevTimeScale().isTimeBased(); - final String timestampCol = rs.prevTimeScale().timestampCol(); + final boolean isTimeBased = rs.revTimeScale().isTimeBased(); + final String timestampCol = rs.revTimeScale().timestampCol(); Arrays.stream(pairs) .filter(p -> !isTimeBased || !p.rightColumn().equals(timestampCol)) @@ -360,41 +360,41 @@ private UpdateByOperator makeRollingSumOperator(@NotNull final MatchPair pair, final Class csType = columnSource.getType(); final String[] affectingColumns; - if (rs.prevTimeScale().timestampCol() == null) { + if (rs.revTimeScale().timestampCol() == null) { affectingColumns = new String[] {pair.rightColumn}; } else { - affectingColumns = new String[] {rs.prevTimeScale().timestampCol(), pair.rightColumn}; + affectingColumns = new String[] {rs.revTimeScale().timestampCol(), pair.rightColumn}; } - final long prevTimeScaleUnits = rs.prevTimeScale().timescaleUnits(); + final long prevTimeScaleUnits = rs.revTimeScale().timescaleUnits(); final long fwdTimeScaleUnits = rs.fwdTimeScale().timescaleUnits(); if (csType == Boolean.class || csType == boolean.class) { - return new ByteRollingSumOperator(pair, affectingColumns, rs.prevTimeScale().timestampCol(), + return new ByteRollingSumOperator(pair, affectingColumns, rs.revTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, rowRedirection, NULL_BOOLEAN_AS_BYTE); } else if (csType == byte.class || csType == Byte.class) { - return new ByteRollingSumOperator(pair, affectingColumns, rs.prevTimeScale().timestampCol(), + return new ByteRollingSumOperator(pair, affectingColumns, rs.revTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, rowRedirection, NULL_BYTE); } else if (csType == short.class || csType == Short.class) { - return new ShortRollingSumOperator(pair, affectingColumns, rs.prevTimeScale().timestampCol(), + return new ShortRollingSumOperator(pair, affectingColumns, rs.revTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, rowRedirection); } else if (csType == int.class || csType == Integer.class) { - return new IntRollingSumOperator(pair, affectingColumns, rs.prevTimeScale().timestampCol(), + return new IntRollingSumOperator(pair, affectingColumns, rs.revTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, rowRedirection); } else if (csType == long.class || csType == Long.class) { - return new LongRollingSumOperator(pair, affectingColumns, rs.prevTimeScale().timestampCol(), + return new LongRollingSumOperator(pair, affectingColumns, rs.revTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, rowRedirection); } else if (csType == float.class || csType == Float.class) { - return new FloatRollingSumOperator(pair, affectingColumns, rs.prevTimeScale().timestampCol(), + return new FloatRollingSumOperator(pair, affectingColumns, rs.revTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, rowRedirection); } else if (csType == double.class || csType == Double.class) { - return new DoubleRollingSumOperator(pair, affectingColumns, rs.prevTimeScale().timestampCol(), + return new DoubleRollingSumOperator(pair, affectingColumns, rs.revTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, rowRedirection); } else if (csType == BigDecimal.class) { - return new BigDecimalRollingSumOperator(pair, affectingColumns, rs.prevTimeScale().timestampCol(), + return new BigDecimalRollingSumOperator(pair, affectingColumns, rs.revTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, rowRedirection, control.mathContextOrDefault()); } else if (csType == BigInteger.class) { - return new BigIntegerRollingSumOperator(pair, affectingColumns, rs.prevTimeScale().timestampCol(), + return new BigIntegerRollingSumOperator(pair, affectingColumns, rs.revTimeScale().timestampCol(), prevTimeScaleUnits, fwdTimeScaleUnits, rowRedirection); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java index 6e37b6302b9..a864c906572 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java @@ -12,6 +12,7 @@ import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; import io.deephaven.util.SafeCloseable; +import io.deephaven.util.SafeCloseableArray; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -100,12 +101,10 @@ public void close() { influencerRows == affectedRows || influencerRows == timestampValidRowSet ? null : influencerRows) { } - SafeCloseable.closeArray(opContexts); - Arrays.fill(opContexts, null); + SafeCloseableArray.close(opContexts); if (inputSourceGetContexts != null) { - SafeCloseable.closeArray(inputSourceGetContexts); - Arrays.fill(inputSourceGetContexts, null); + SafeCloseableArray.close(inputSourceGetContexts); } } } @@ -308,42 +307,37 @@ RowSet getInfluencerRows(final UpdateByWindowBucketContext context) { */ void processUpdateForContext(UpdateByWindowBucketContext context, @NotNull TableUpdate upstream) { boolean addsOrRemoves = upstream.added().isNonempty() || upstream.removed().isNonempty(); - if (addsOrRemoves) { // mark all operators as affected by this update context.dirtyOperatorIndices = IntStream.range(0, operators.length).toArray(); context.dirtySourceIndices = getUniqueSourceIndices(); context.isDirty = true; - // still need to compute whether any input columns were modified + if (upstream.modifiedColumnSet().empty()) { + return; + } for (UpdateByOperator op : operators) { - final boolean opInputModified = upstream.modifiedColumnSet().nonempty() && - (op.getInputModifiedColumnSet() == null || - upstream.modifiedColumnSet().containsAny(op.getInputModifiedColumnSet())); - + final boolean opInputModified = op.getInputModifiedColumnSet() == null || + upstream.modifiedColumnSet().containsAny(op.getInputModifiedColumnSet()); if (opInputModified) { context.inputModified = true; break; } } - } else { + } else if (upstream.modifiedColumnSet().nonempty()) { // determine which operators are affected by this update and whether any input columns were modified BitSet dirtyOperators = new BitSet(); BitSet dirtySourceIndices = new BitSet(); - for (int opIdx = 0; opIdx < operators.length; opIdx++) { UpdateByOperator op = operators[opIdx]; - final boolean opInputModified = upstream.modifiedColumnSet().nonempty() && - (op.getInputModifiedColumnSet() == null || - upstream.modifiedColumnSet().containsAny(op.getInputModifiedColumnSet())); - + final boolean opInputModified = op.getInputModifiedColumnSet() == null || + upstream.modifiedColumnSet().containsAny(op.getInputModifiedColumnSet()); if (opInputModified) { dirtyOperators.set(opIdx); Arrays.stream(operatorInputSourceSlots[opIdx]).forEach(dirtySourceIndices::set); context.inputModified = true; } } - context.isDirty = !dirtyOperators.isEmpty(); context.dirtyOperatorIndices = dirtyOperators.stream().toArray(); context.dirtySourceIndices = dirtySourceIndices.stream().toArray(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java index f62271f07ef..c457f49c738 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java @@ -1,5 +1,6 @@ package io.deephaven.engine.table.impl.updateby; +import io.deephaven.base.ArrayUtil; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableIntChunk; @@ -55,12 +56,10 @@ public void close() { this.prevUnits = prevUnits; this.fwdUnits = fwdUnits; - // We would like to use jdk.internal.util.ArraysSupport.MAX_ARRAY_LENGTH, but it is not exported - final int MAX_ARRAY_SIZE = Integer.MAX_VALUE - 8; - - if (prevUnits + fwdUnits > MAX_ARRAY_SIZE) { - throw (new IllegalArgumentException( - "UpdateBy window size may not exceed MAX_ARRAY_SIZE (" + MAX_ARRAY_SIZE + ")")); + // this is also checked at RollingSumSpec creation against a hard-coded value (Integer.MAX_VALUE - 8) + if (prevUnits + fwdUnits > ArrayUtil.MAX_ARRAY_SIZE) { + throw new IllegalArgumentException( + "UpdateBy window size may not exceed MAX_ARRAY_SIZE (" + ArrayUtil.MAX_ARRAY_SIZE + ")"); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java index 85a0129f426..8eaf55da6c1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java @@ -29,7 +29,9 @@ * a buffer of `influencer` values to add to the rolling window as the current row changes. */ class UpdateByWindowTime extends UpdateByWindow { - /** growth rate after the contexts have exceeded the poolable chunk size */ + /** + * growth rate after the contexts have exceeded the poolable chunk size + */ private static final double CONTEXT_GROWTH_PERCENTAGE = 0.25; private static final int WINDOW_CHUNK_SIZE = 4096; private static final int RING_BUFFER_INITIAL_SIZE = 512; @@ -335,8 +337,7 @@ public void processRows(UpdateByWindowBucketContext context, boolean initialStep Assert.eqTrue(affectedTsIt.hasNext(), "affectedTsIt.hasNext()"); final long EXHAUSTED = -1L; - // long currentTimestamp = affectedTsIt.nextLong(); - long affectedTs = affectedTsIt.hasNext() ? affectedTsIt.nextLong() : EXHAUSTED; + long affectedTs = affectedTsIt.nextLong(); while (affectedRowsIt.hasMore()) { // NOTE: we did not put null values into our SSA and our influencer rowset is built using the @@ -346,8 +347,11 @@ public void processRows(UpdateByWindowBucketContext context, boolean initialStep long totalPushCount = 0; long skipCount = 0; - for (affectedRowIndex = 0; affectedRowIndex < ctx.workingChunkSize - && affectedTs != EXHAUSTED; affectedRowIndex++) { + // @formatter:off + for (affectedRowIndex = 0; + affectedRowIndex < ctx.workingChunkSize && affectedTs != EXHAUSTED; + affectedRowIndex++) { + // @formatter:on if (affectedTs == NULL_LONG) { // this signifies that does not belong to a time window popChunk.set(affectedRowIndex, NULL_INT); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java index 095fd20115a..aa7f910fd90 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java @@ -78,7 +78,8 @@ protected ZeroKeyUpdateByManager( } // make a dummy update to generate the initial row keys - final TableUpdateImpl fakeUpdate = new TableUpdateImpl(source.getRowSet(), + final TableUpdateImpl fakeUpdate = new TableUpdateImpl( + source.getRowSet().copy(), // send a copy since this will be closed by release() RowSetFactory.empty(), RowSetFactory.empty(), RowSetShiftData.EMPTY, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java index 08f0bc4ae84..fa5904a1c22 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java @@ -40,8 +40,8 @@ public void accumulate(RowSequence inputKeys, if (curVal == null) { curVal = input; } else { - curVal = curVal.multiply(alpha, control.bigValueContextOrDefault()) - .add(input.multiply(oneMinusAlpha, control.bigValueContextOrDefault()), + curVal = curVal.multiply(opAlpha, control.bigValueContextOrDefault()) + .add(input.multiply(opOneMinusAlpha, control.bigValueContextOrDefault()), control.bigValueContextOrDefault()); } } @@ -68,11 +68,14 @@ public void accumulate(RowSequence inputKeys, } else { final long dt = timestamp - lastStamp; if (dt != 0) { - + // alpha is dynamic based on time, but only recalculated when needed + if (dt != lastDt) { + alpha = BigDecimal.valueOf(Math.exp(-dt / (double) reverseWindowScaleUnits)); + oneMinusAlpha = + BigDecimal.ONE.subtract(alpha, control.bigValueContextOrDefault()); + lastDt = dt; + } // alpha is dynamic, based on time - BigDecimal alpha = BigDecimal.valueOf(Math.exp(-dt / (double) reverseWindowScaleUnits)); - BigDecimal oneMinusAlpha = - BigDecimal.ONE.subtract(alpha, control.bigValueContextOrDefault()); curVal = curVal.multiply(alpha, control.bigValueContextOrDefault()) .add(input.multiply(oneMinusAlpha, control.bigValueContextOrDefault()), diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java index 115451acc43..aaddc148f07 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java @@ -42,8 +42,8 @@ public void accumulate(RowSequence inputKeys, if (curVal == null) { curVal = decimalInput; } else { - curVal = curVal.multiply(alpha, control.bigValueContextOrDefault()) - .add(decimalInput.multiply(oneMinusAlpha, control.bigValueContextOrDefault()), + curVal = curVal.multiply(opAlpha, control.bigValueContextOrDefault()) + .add(decimalInput.multiply(opOneMinusAlpha, control.bigValueContextOrDefault()), control.bigValueContextOrDefault()); } } @@ -69,9 +69,12 @@ public void accumulate(RowSequence inputKeys, } else { final long dt = timestamp - lastStamp; if (dt != 0) { - // alpha is dynamic, based on time - BigDecimal alpha = BigDecimal.valueOf(Math.exp(-dt / (double) reverseWindowScaleUnits)); - BigDecimal oneMinusAlpha = BigDecimal.ONE.subtract(alpha, control.bigValueContextOrDefault()); + // alpha is dynamic based on time, but only recalculated when needed + if (dt != lastDt) { + alpha = BigDecimal.valueOf(Math.exp(-dt / (double) reverseWindowScaleUnits)); + oneMinusAlpha = BigDecimal.ONE.subtract(alpha, control.bigValueContextOrDefault()); + lastDt = dt; + } curVal = curVal.multiply(alpha, control.bigValueContextOrDefault()) .add(decimalInput.multiply(oneMinusAlpha, control.bigValueContextOrDefault()), diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java index f204a1a0845..c87a7d2347c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java @@ -3,7 +3,6 @@ import io.deephaven.api.updateby.BadDataBehavior; import io.deephaven.api.updateby.OperationControl; import io.deephaven.chunk.Chunk; -import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.ColumnSource; @@ -22,14 +21,17 @@ public abstract class BigNumberEMAOperator extends BaseObjectUpdateByOperator { protected final ColumnSource valueSource; protected final OperationControl control; - protected final BigDecimal alpha; - protected final BigDecimal oneMinusAlpha; + protected final BigDecimal opAlpha; + protected final BigDecimal opOneMinusAlpha; public abstract class Context extends BaseObjectUpdateByOperator.Context { public ObjectChunk objectValueChunk; - long lastStamp = NULL_LONG; + protected BigDecimal alpha; + protected BigDecimal oneMinusAlpha; + protected long lastDt = NULL_LONG; + protected long lastStamp = NULL_LONG; protected Context(final int chunkSize, final int chunkCount) { super(chunkSize, chunkCount); @@ -76,9 +78,10 @@ public BigNumberEMAOperator(@NotNull final MatchPair pair, this.control = control; this.valueSource = valueSource; - alpha = BigDecimal.valueOf(Math.exp(-1.0 / (double) timeScaleUnits)); - oneMinusAlpha = - timestampColumnName == null ? BigDecimal.ONE.subtract(alpha, control.bigValueContextOrDefault()) : null; + opAlpha = BigDecimal.valueOf(Math.exp(-1.0 / (double) timeScaleUnits)); + opOneMinusAlpha = + timestampColumnName == null ? BigDecimal.ONE.subtract(opAlpha, control.bigValueContextOrDefault()) + : null; } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java index c99a15935dd..03173f74a64 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java @@ -17,8 +17,6 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.math.BigDecimal; - import static io.deephaven.util.QueryConstants.*; public class ByteEMAOperator extends BasePrimitiveEMAOperator { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java index 52c2483594b..48297efe9c7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java @@ -17,8 +17,6 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.math.BigDecimal; - import static io.deephaven.util.QueryConstants.*; public class IntEMAOperator extends BasePrimitiveEMAOperator { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java index 9446e79dd73..b25e4879f72 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java @@ -17,8 +17,6 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.math.BigDecimal; - import static io.deephaven.util.QueryConstants.*; public class LongEMAOperator extends BasePrimitiveEMAOperator { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java index 87e7ed52a3d..97e59ddc16d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java @@ -12,8 +12,6 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.math.BigDecimal; - import static io.deephaven.util.QueryConstants.*; public class ShortEMAOperator extends BasePrimitiveEMAOperator { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java index 1b5d79c0cf8..67d072469e6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java @@ -1,6 +1,6 @@ /* * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit FloatCumMinMaxOperator and regenerate + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ShortCumMinMaxOperator and regenerate * --------------------------------------------------------------------------------------------------------------------- */ package io.deephaven.engine.table.impl.updateby.minmax; @@ -15,7 +15,7 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.NULL_DOUBLE; +import static io.deephaven.util.QueryConstants.*; public class DoubleCumMinMaxOperator extends BaseDoubleUpdateByOperator { private final boolean isMax; @@ -41,9 +41,9 @@ public void push(long key, int pos, int count) { final double val = doubleValueChunk.get(pos); - if(curVal == NULL_DOUBLE) { + if (curVal == NULL_DOUBLE) { curVal = val; - } else if(val != NULL_DOUBLE) { + } else if (val != NULL_DOUBLE) { if ((isMax && val > curVal) || (!isMax && val < curVal)) { curVal = val; @@ -55,14 +55,16 @@ public void push(long key, int pos, int count) { public DoubleCumMinMaxOperator(@NotNull final MatchPair pair, final boolean isMax, @Nullable final RowRedirection rowRedirection - // region extra-constructor-args - // endregion extra-constructor-args + // region extra-constructor-args + // endregion extra-constructor-args ) { super(pair, new String[] { pair.rightColumn }, rowRedirection); this.isMax = isMax; // region constructor // endregion constructor } + // region extra-methods + // endregion extra-methods @NotNull @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java index dc46e11a1d4..177d84582d8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java @@ -1,3 +1,8 @@ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit ShortCumMinMaxOperator and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ package io.deephaven.engine.table.impl.updateby.minmax; import io.deephaven.base.verify.Assert; @@ -10,7 +15,7 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.util.QueryConstants.NULL_FLOAT; +import static io.deephaven.util.QueryConstants.*; public class FloatCumMinMaxOperator extends BaseFloatUpdateByOperator { private final boolean isMax; @@ -36,9 +41,9 @@ public void push(long key, int pos, int count) { final float val = floatValueChunk.get(pos); - if(curVal == NULL_FLOAT) { + if (curVal == NULL_FLOAT) { curVal = val; - } else if(val != NULL_FLOAT) { + } else if (val != NULL_FLOAT) { if ((isMax && val > curVal) || (!isMax && val < curVal)) { curVal = val; @@ -50,14 +55,16 @@ public void push(long key, int pos, int count) { public FloatCumMinMaxOperator(@NotNull final MatchPair pair, final boolean isMax, @Nullable final RowRedirection rowRedirection - // region extra-constructor-args - // endregion extra-constructor-args + // region extra-constructor-args + // endregion extra-constructor-args ) { super(pair, new String[] { pair.rightColumn }, rowRedirection); this.isMax = isMax; // region constructor // endregion constructor } + // region extra-methods + // endregion extra-methods @NotNull @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java index 2bdd5545e0b..f9f28d5a964 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java @@ -12,9 +12,6 @@ import java.math.BigInteger; -import static io.deephaven.util.QueryConstants.NULL_LONG; -import static io.deephaven.util.QueryConstants.NULL_SHORT; - public final class BigIntegerCumProdOperator extends BaseObjectUpdateByOperator { protected class Context extends BaseObjectUpdateByOperator.Context { public ObjectChunk objectValueChunk; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java index df37553e6ad..d7e398126c6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/JobScheduler.java @@ -23,9 +23,12 @@ public interface JobScheduler { * A default context for the scheduled job actions. Override this to provide reusable resources for the serial and * parallel iterate actions. */ - class JobContext implements Context { + interface JobThreadContext extends Context { } + JobThreadContext DEFAULT_CONTEXT = new JobThreadContext() {}; + Supplier DEFAULT_CONTEXT_FACTORY = () -> DEFAULT_CONTEXT; + /** * Cause runnable to be executed. * @@ -58,7 +61,7 @@ void submit( * schedule the next iteration. */ @FunctionalInterface - interface IterateAction { + interface IterateAction { void run(CONTEXT_TYPE taskThreadContext, int index); } @@ -72,11 +75,11 @@ interface IterateAction { * will not block the scheduler, but the {@code completeAction} {@link Runnable} will never be called. */ @FunctionalInterface - interface IterateResumeAction { + interface IterateResumeAction { void run(CONTEXT_TYPE taskThreadContext, int index, Runnable resume); } - class ErrorAccounter extends ReferenceCounted + class ErrorAccounter extends ReferenceCounted implements Consumer, Runnable { private final Supplier taskThreadContextFactory; private final int start; @@ -156,8 +159,8 @@ public void run() { * * @param executionContext the execution context for this task * @param description the description to use for logging - * @param taskThreadContextFactory the factory that supplies {@link JobContext contexts} for the threads handling - * the sub-tasks + * @param taskThreadContextFactory the factory that supplies {@link JobThreadContext contexts} for the threads + * handling the sub-tasks * @param start the integer value from which to start iterating * @param count the number of times this task should be called * @param action the task to perform, the current iteration index is provided as a parameter @@ -165,7 +168,7 @@ public void run() { * @param onError error handler for the scheduler to use while iterating */ @FinalDefault - default void iterateParallel( + default void iterateParallel( ExecutionContext executionContext, LogOutputAppendable description, Supplier taskThreadContextFactory, @@ -190,7 +193,7 @@ default void iterateParallel( * * @param executionContext the execution context for this task * @param description the description to use for logging - * @param taskThreadContextFactory the factory that supplies {@link JobContext contexts} for the tasks + * @param taskThreadContextFactory the factory that supplies {@link JobThreadContext contexts} for the tasks * @param start the integer value from which to start iterating * @param count the number of times this task should be called * @param action the task to perform, the current iteration index and a resume Runnable are parameters @@ -198,7 +201,7 @@ default void iterateParallel( * @param onError error handler for the scheduler to use while iterating */ @FinalDefault - default void iterateParallel( + default void iterateParallel( ExecutionContext executionContext, LogOutputAppendable description, Supplier taskThreadContextFactory, @@ -233,7 +236,7 @@ default void iterateParallel( * * @param executionContext the execution context for this task * @param description the description to use for logging - * @param taskThreadContextFactory the factory that supplies {@link JobContext contexts} for the tasks + * @param taskThreadContextFactory the factory that supplies {@link JobThreadContext contexts} for the tasks * @param start the integer value from which to start iterating * @param count the number of times this task should be called * @param action the task to perform, the current iteration index and a resume Runnable are parameters @@ -241,7 +244,7 @@ default void iterateParallel( * @param onError error handler for the scheduler to use while iterating */ @FinalDefault - default void iterateSerial(ExecutionContext executionContext, + default void iterateSerial(ExecutionContext executionContext, LogOutputAppendable description, Supplier taskThreadContextFactory, int start, diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java index 077ce922bd2..c323122076d 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java @@ -5,13 +5,16 @@ import io.deephaven.api.updateby.OperationControl; import io.deephaven.api.updateby.UpdateByOperation; import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.impl.QueryTable; import io.deephaven.engine.testutil.EvalNugget; import io.deephaven.engine.table.impl.TableDefaults; import io.deephaven.api.updateby.UpdateByControl; +import io.deephaven.engine.testutil.TstUtils; import io.deephaven.engine.testutil.generator.TestDataGenerator; import io.deephaven.engine.testutil.generator.SortedDateTimeGenerator; import io.deephaven.engine.updategraph.UpdateGraphProcessor; -import io.deephaven.test.types.ParallelTest; +import io.deephaven.engine.util.TableTools; +import io.deephaven.test.types.OutOfBandTest; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -19,18 +22,21 @@ import java.util.*; import static io.deephaven.engine.testutil.GenerateTableUpdates.generateAppends; +import static io.deephaven.engine.testutil.TstUtils.*; import static io.deephaven.engine.testutil.testcase.RefreshingTableTestCase.simulateShiftAwareStep; -import static io.deephaven.engine.testutil.TstUtils.validate; +import static io.deephaven.engine.util.TableTools.col; +import static io.deephaven.engine.util.TableTools.intCol; import static io.deephaven.time.DateTimeUtils.MINUTE; import static io.deephaven.time.DateTimeUtils.convertDateTime; -@Category(ParallelTest.class) +@Category(OutOfBandTest.class) public class TestUpdateByGeneral extends BaseUpdateByTest { @Test public void testMixedAppendOnlyZeroKey() { for (int size = 10; size <= 10000; size *= 10) { for (int seed = 10; seed < 20; seed++) { + System.out.println("Stating test: size " + size + ", seed " + seed); doTestTicking(seed > 15, false, true, 20, size, seed); } } @@ -40,6 +46,7 @@ public void testMixedAppendOnlyZeroKey() { public void testMixedAppendOnlyBucketed() { for (int size = 10; size <= 10000; size *= 10) { for (int seed = 10; seed < 20; seed++) { + System.out.println("Stating test: size " + size + ", seed " + seed); doTestTicking(seed > 15, true, true, 20, size, seed); } } @@ -49,6 +56,7 @@ public void testMixedAppendOnlyBucketed() { public void testMixedGeneralZeroKey() { for (int size = 10; size <= 10000; size *= 10) { for (int seed = 10; seed < 20; seed++) { + System.out.println("Stating test: size " + size + ", seed " + seed); doTestTicking(seed > 15, false, false, 20, size, seed); } } @@ -58,6 +66,7 @@ public void testMixedGeneralZeroKey() { public void testMixedGeneralBucketed() { for (int size = 10; size <= 10000; size *= 10) { for (int seed = 10; seed < 20; seed++) { + System.out.println("Stating test: size " + size + ", seed " + seed); doTestTicking(seed > 15, true, false, 20, size, seed); } } @@ -125,6 +134,7 @@ protected Table e() { }; for (int step = 0; step < steps; step++) { + System.out.println(" beginning step " + step); try { if (appendOnly) { UpdateGraphProcessor.DEFAULT.runWithinUnitTestCycle(() -> { @@ -149,4 +159,35 @@ private String[] makeOpColNames(String[] colNames, String suffix, String... toOm .map(cn -> cn + suffix + "=" + cn) .toArray(String[]::new); } + + @Test + public void testNewBuckets() { + final QueryTable table = TstUtils.testRefreshingTable( + i(2, 4, 6).toTracking(), + col("Key", "A", "B", "A"), + intCol("Int", 2, 4, 6)); + final QueryTable result = (QueryTable) table.updateBy(List.of(UpdateByOperation.Fill("Filled=Int"), UpdateByOperation.RollingSum(2, "Sum=Int")), "Key"); + + UpdateGraphProcessor.DEFAULT.runWithinUnitTestCycle(() -> { + TstUtils.addToTable(table, i(8), col("Key", "B"), intCol("Int", 8)); // Add to "B" bucket + table.notifyListeners(i(8), i(), i()); + }); + + UpdateGraphProcessor.DEFAULT.runWithinUnitTestCycle(() -> { + TstUtils.addToTable(table, i(9), col("Key", "C"), intCol("Int", 10)); // New "C" bucket in isolation + table.notifyListeners(i(9), i(), i()); + }); + + UpdateGraphProcessor.DEFAULT.runWithinUnitTestCycle(() -> { + TstUtils.addToTable(table, i(8), col("Key", "C"), intCol("Int", 11)); // Row from "B" bucket to "C" bucket + table.notifyListeners(i(), i(), i(8)); + }); + + UpdateGraphProcessor.DEFAULT.runWithinUnitTestCycle(() -> { + TstUtils.addToTable(table, i(10, 11), col("Key", "D", "C"), intCol("Int", 10, 11)); // New "D" bucket + table.notifyListeners(i(10, 11), i(), i()); + }); + + TableTools.show(result); + } } diff --git a/engine/table/src/test/java/io/deephaven/engine/util/TestJobScheduler.java b/engine/table/src/test/java/io/deephaven/engine/util/TestJobScheduler.java index 0161c10bf43..3db49abf975 100644 --- a/engine/table/src/test/java/io/deephaven/engine/util/TestJobScheduler.java +++ b/engine/table/src/test/java/io/deephaven/engine/util/TestJobScheduler.java @@ -32,7 +32,7 @@ public void TestParallel() { scheduler.iterateParallel( ExecutionContext.getContext(), null, - JobScheduler.JobContext::new, + JobScheduler.DEFAULT_CONTEXT_FACTORY, 0, 50, (context, idx) -> { @@ -81,7 +81,7 @@ public void TestParallelWithResume() { scheduler.iterateParallel( ExecutionContext.getContext(), null, - JobScheduler.JobContext::new, + JobScheduler.DEFAULT_CONTEXT_FACTORY, 0, 50, (context, idx, resume) -> { @@ -127,8 +127,8 @@ public void TestParallelWithContext() { UpdateGraphProcessor.DEFAULT.resetForUnitTests(false, true, 0, 4, 10, 5); UpdateGraphProcessor.DEFAULT.runWithinUnitTestCycle(() -> { - class TestJobContext extends JobScheduler.JobContext { - TestJobContext() { + class TestJobThreadContext implements JobScheduler.JobThreadContext { + TestJobThreadContext() { openCount.incrementAndGet(); } @@ -144,12 +144,12 @@ public void close() { scheduler.iterateParallel( ExecutionContext.getContext(), null, - TestJobContext::new, + TestJobThreadContext::new, 0, 50, (context, idx, resume) -> { // verify the type is correct - Assert.instanceOf(context, "context", TestJobContext.class); + Assert.instanceOf(context, "context", TestJobThreadContext.class); completed[idx] = true; resume.run(); @@ -200,7 +200,7 @@ public void TestSerialWithResume() { scheduler.iterateSerial( ExecutionContext.getContext(), null, - JobScheduler.JobContext::new, + JobScheduler.DEFAULT_CONTEXT_FACTORY, 0, 50, (context, idx, resume) -> { @@ -247,8 +247,8 @@ public void TestSerialWithContext() { UpdateGraphProcessor.DEFAULT.resetForUnitTests(false, true, 0, 4, 10, 5); UpdateGraphProcessor.DEFAULT.runWithinUnitTestCycle(() -> { - class TestJobContext extends JobScheduler.JobContext { - TestJobContext() { + class TestJobThreadContext implements JobScheduler.JobThreadContext { + TestJobThreadContext() { openCount.incrementAndGet(); } @@ -264,12 +264,12 @@ public void close() { scheduler.iterateSerial( ExecutionContext.getContext(), null, - TestJobContext::new, + TestJobThreadContext::new, 0, 50, (context, idx, resume) -> { // verify the type is correct - Assert.instanceOf(context, "context", TestJobContext.class); + Assert.instanceOf(context, "context", TestJobThreadContext.class); completed[idx] = true; resume.run(); @@ -319,7 +319,7 @@ public void TestSerialEmpty() { scheduler.iterateSerial( ExecutionContext.getContext(), null, - JobScheduler.JobContext::new, + JobScheduler.DEFAULT_CONTEXT_FACTORY, 0, 0, (context, idx, resume) -> { @@ -360,7 +360,7 @@ public void TestParallelEmpty() { scheduler.iterateParallel( ExecutionContext.getContext(), null, - JobScheduler.JobContext::new, + JobScheduler.DEFAULT_CONTEXT_FACTORY, 0, 0, (context, idx, resume) -> { @@ -399,8 +399,8 @@ public void TestParallelError() { UpdateGraphProcessor.DEFAULT.runWithinUnitTestCycle(() -> { final boolean[] completed = new boolean[100]; - class TestJobContext extends JobScheduler.JobContext { - TestJobContext() { + class TestJobThreadContext implements JobScheduler.JobThreadContext { + TestJobThreadContext() { openCount.incrementAndGet(); } @@ -414,12 +414,12 @@ public void close() { scheduler.iterateParallel( ExecutionContext.getContext(), null, - TestJobContext::new, + TestJobThreadContext::new, 0, 50, (context, idx) -> { // verify the type is correct - Assert.instanceOf(context, "context", TestJobContext.class); + Assert.instanceOf(context, "context", TestJobThreadContext.class); completed[idx] = true; // throw after this is set to make verification easy @@ -476,8 +476,8 @@ public void TestSerialError() { UpdateGraphProcessor.DEFAULT.runWithinUnitTestCycle(() -> { final boolean[] completed = new boolean[100]; - class TestJobContext extends JobScheduler.JobContext { - TestJobContext() { + class TestJobThreadContext implements JobScheduler.JobThreadContext { + TestJobThreadContext() { openCount.incrementAndGet(); } @@ -491,12 +491,12 @@ public void close() { scheduler.iterateSerial( ExecutionContext.getContext(), null, - TestJobContext::new, + TestJobThreadContext::new, 0, 50, (context, idx, resume) -> { // verify the type is correct - Assert.instanceOf(context, "context", TestJobContext.class); + Assert.instanceOf(context, "context", TestJobThreadContext.class); completed[idx] = true; diff --git a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java index 342d95c3a19..b8629c860cd 100644 --- a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java +++ b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java @@ -71,9 +71,23 @@ public static void main(String[] args) throws IOException { replicateNumericOperator( "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java", "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java"); - replicateNumericOperator( + + files = ReplicatePrimitiveCode.shortToAllNumericals( "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java", - "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java"); + null); + for (final String f : files) { + if (f.contains("Integer")) { + fixupInteger(f); + } + + if (f.contains("Byte") && (f.contains("CumSum") || f.contains("RollingSum"))) { + fixupByte(f); + } + + if (f.contains("Long") && f.contains("MinMax")) { + augmentLongWithReinterps(f); + } + } replicateNumericOperator( "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java", diff --git a/replication/util/src/main/java/io/deephaven/replication/ReplicatePrimitiveCode.java b/replication/util/src/main/java/io/deephaven/replication/ReplicatePrimitiveCode.java index 3280b19e1f4..1716a6aa594 100644 --- a/replication/util/src/main/java/io/deephaven/replication/ReplicatePrimitiveCode.java +++ b/replication/util/src/main/java/io/deephaven/replication/ReplicatePrimitiveCode.java @@ -239,14 +239,15 @@ private static String shortToByte(String sourceClassJavaPath, Map return replicateCodeBasedOnShort(sourceClassJavaPath, serialVersionUIDs, exemptions, "Byte", "byte", "BYTE"); } - private static void shortToDouble(String sourceClassJavaPath, Map serialVersionUIDs, + private static String shortToDouble(String sourceClassJavaPath, Map serialVersionUIDs, String... exemptions) throws IOException { - replicateCodeBasedOnShort(sourceClassJavaPath, serialVersionUIDs, exemptions, "Double", "double", "DOUBLE"); + return replicateCodeBasedOnShort(sourceClassJavaPath, serialVersionUIDs, exemptions, "Double", "double", + "DOUBLE"); } - private static void shortToFloat(String sourceClassJavaPath, Map serialVersionUIDs, + private static String shortToFloat(String sourceClassJavaPath, Map serialVersionUIDs, String... exemptions) throws IOException { - replicateCodeBasedOnShort(sourceClassJavaPath, serialVersionUIDs, exemptions, "Float", "float", "FLOAT"); + return replicateCodeBasedOnShort(sourceClassJavaPath, serialVersionUIDs, exemptions, "Float", "float", "FLOAT"); } private static String shortToInteger(String sourceClassJavaPath, Map serialVersionUIDs, @@ -386,13 +387,15 @@ public static List charToAllButBooleanAndFloats(String sourceClass, Map< return files; } - public static void shortToAllNumericals(String sourceClassJavaPath, Map serialVersionUIDs, + public static List shortToAllNumericals(String sourceClassJavaPath, Map serialVersionUIDs, String... exemptions) throws IOException { - shortToByte(sourceClassJavaPath, serialVersionUIDs, exemptions); - shortToDouble(sourceClassJavaPath, serialVersionUIDs, exemptions); - shortToFloat(sourceClassJavaPath, serialVersionUIDs, exemptions); - shortToInteger(sourceClassJavaPath, serialVersionUIDs, exemptions); - shortToLong(sourceClassJavaPath, serialVersionUIDs, exemptions); + final List results = new ArrayList<>(); + results.add(shortToByte(sourceClassJavaPath, serialVersionUIDs, exemptions)); + results.add(shortToDouble(sourceClassJavaPath, serialVersionUIDs, exemptions)); + results.add(shortToFloat(sourceClassJavaPath, serialVersionUIDs, exemptions)); + results.add(shortToInteger(sourceClassJavaPath, serialVersionUIDs, exemptions)); + results.add(shortToLong(sourceClassJavaPath, serialVersionUIDs, exemptions)); + return results; } public static void intToAllNumericals(String sourceClassJavaPath, Map serialVersionUIDs, diff --git a/table-api/src/main/java/io/deephaven/api/updateby/UpdateByOperation.java b/table-api/src/main/java/io/deephaven/api/updateby/UpdateByOperation.java index 822c28d5349..978dc2ae8b0 100644 --- a/table-api/src/main/java/io/deephaven/api/updateby/UpdateByOperation.java +++ b/table-api/src/main/java/io/deephaven/api/updateby/UpdateByOperation.java @@ -91,16 +91,16 @@ static UpdateByOperation Fill(String... pairs) { *

              * *
              -     *     a = e^(-1 / timeScaleTicks)
              +     *     a = e^(-1 / tickDecay)
                    *     ema_next = a * ema_last + (1 - a) * value
                    * 
              * - * @param timeScaleTicks the decay rate in ticks + * @param tickDecay the decay rate in ticks * @param pairs The input/output column name pairs * @return The aggregation */ - static UpdateByOperation Ema(long timeScaleTicks, String... pairs) { - return EmaSpec.ofTicks(timeScaleTicks).clause(pairs); + static UpdateByOperation Ema(long tickDecay, String... pairs) { + return EmaSpec.ofTicks(tickDecay).clause(pairs); } /** @@ -111,18 +111,18 @@ static UpdateByOperation Ema(long timeScaleTicks, String... pairs) { *

              * *
              -     *     a = e^(-1 / timeScaleTicks)
              +     *     a = e^(-1 / tickDecay)
                    *     ema_next = a * ema_last + (1 - a) * value
                    * 
              * * @param control a {@link OperationControl control} object that defines how special cases should behave. See * {@link OperationControl} for further details. - * @param timeScaleTicks the decay rate in ticks + * @param tickDecay the decay rate in ticks * @param pairs The input/output column name pairs * @return The aggregation */ - static UpdateByOperation Ema(final OperationControl control, long timeScaleTicks, String... pairs) { - return EmaSpec.ofTicks(control, timeScaleTicks).clause(pairs); + static UpdateByOperation Ema(final OperationControl control, long tickDecay, String... pairs) { + return EmaSpec.ofTicks(control, tickDecay).clause(pairs); } /** @@ -133,17 +133,17 @@ static UpdateByOperation Ema(final OperationControl control, long timeScaleTicks *

              * *
              -     *     a = e^(-dt / timeScaleNanos)
              +     *     a = e^(-dt / timeDecay)
                    *     ema_next = a * ema_last + (1 - a) * value
                    * 
              * * @param timestampColumn the column in the source table to use for timestamps - * @param timeScaleNanos the decay rate in nanoseconds + * @param timeDecay the decay rate in nanoseconds * @param pairs The input/output column name pairs * @return The aggregation */ - static UpdateByOperation Ema(String timestampColumn, long timeScaleNanos, String... pairs) { - return EmaSpec.ofTime(timestampColumn, timeScaleNanos).clause(pairs); + static UpdateByOperation Ema(String timestampColumn, long timeDecay, String... pairs) { + return EmaSpec.ofTime(timestampColumn, timeDecay).clause(pairs); } /** @@ -154,20 +154,19 @@ static UpdateByOperation Ema(String timestampColumn, long timeScaleNanos, String *

              * *
              -     *     a = e^(-dt / timeScaleNanos)
              +     *     a = e^(-dt / timeDecay)
                    *     ema_next = a * ema_last + (1 - a) * value
                    * 
              * * @param control a {@link OperationControl control} object that defines how special cases should behave. See * {@link OperationControl} for further details. * @param timestampColumn the column in the source table to use for timestamps - * @param timeScaleNanos the decay rate in nanoseconds + * @param timeDecay the decay rate in nanoseconds * @param pairs The input/output column name pairs * @return The aggregation */ - static UpdateByOperation Ema(OperationControl control, String timestampColumn, long timeScaleNanos, - String... pairs) { - return EmaSpec.ofTime(control, timestampColumn, timeScaleNanos).clause(pairs); + static UpdateByOperation Ema(OperationControl control, String timestampColumn, long timeDecay, String... pairs) { + return EmaSpec.ofTime(control, timestampColumn, timeDecay).clause(pairs); } /** @@ -178,17 +177,17 @@ static UpdateByOperation Ema(OperationControl control, String timestampColumn, l *

              * *
              -     *     a = e^(-dt / timeScaleNanos)
              +     *     a = e^(-dt / durationDecay)
                    *     ema_next = a * ema_last + (1 - a) * value
                    * 
              * * @param timestampColumn the column in the source table to use for timestamps - * @param emaDuration the decay rate as {@Link Duration duration} + * @param durationDecay the decay rate as {@Link Duration duration} * @param pairs The input/output column name pairs * @return The aggregation */ - static UpdateByOperation Ema(String timestampColumn, Duration emaDuration, String... pairs) { - return EmaSpec.ofTime(timestampColumn, emaDuration).clause(pairs); + static UpdateByOperation Ema(String timestampColumn, Duration durationDecay, String... pairs) { + return EmaSpec.ofTime(timestampColumn, durationDecay).clause(pairs); } /** @@ -199,151 +198,150 @@ static UpdateByOperation Ema(String timestampColumn, Duration emaDuration, Strin *

              * *
              -     *     a = e^(-dt / timeScaleNanos)
              +     *     a = e^(-dt / durationDecay)
                    *     ema_next = a * ema_last + (1 - a) * value
                    * 
              * * @param control a {@link OperationControl control} object that defines how special cases should behave. See * {@link OperationControl} for further details. * @param timestampColumn the column in the source table to use for timestamps - * @param emaDuration the decay rate as {@Link Duration duration} + * @param durationDecay the decay rate as {@Link Duration duration} * @param pairs The input/output column name pairs * @return The aggregation */ - static UpdateByOperation Ema(OperationControl control, String timestampColumn, Duration emaDuration, + static UpdateByOperation Ema(OperationControl control, String timestampColumn, Duration durationDecay, String... pairs) { - return EmaSpec.ofTime(control, timestampColumn, emaDuration).clause(pairs); + return EmaSpec.ofTime(control, timestampColumn, durationDecay).clause(pairs); } /** * Create a {@link RollingSumSpec rolling sum} for the supplied column name pairs, using ticks as the windowing * unit. Ticks are row counts and you may specify the previous window in number of rows to include. The current row - * is considered to belong to the reverse window, so calling this with {@code prevTicks = 1} will simply return the - * current row. Specifying {@code prevTicks = 10} will include the previous 9 rows to this one and this row for a + * is considered to belong to the reverse window, so calling this with {@code revTicks = 1} will simply return the + * current row. Specifying {@code revTicks = 10} will include the previous 9 rows to this one and this row for a * total of 10 rows. * - * @param prevTicks the look-behind window size (in rows/ticks) + * @param revTicks the look-behind window size (in rows/ticks) * @param pairs The input/output column name pairs * @return The aggregation */ - static UpdateByOperation RollingSum(long prevTicks, String... pairs) { - return RollingSumSpec.ofTicks(prevTicks).clause(pairs); + static UpdateByOperation RollingSum(long revTicks, String... pairs) { + return RollingSumSpec.ofTicks(revTicks).clause(pairs); } /** * Create a {@link RollingSumSpec rolling sum} for the supplied column name pairs, using ticks as the windowing - * unit. Ticks are row counts and you may specify the previous and forward window in number of rows to include. The + * unit. Ticks are row counts and you may specify the reverse and forward window in number of rows to include. The * current row is considered to belong to the reverse window but not the forward window. Also, negative values are * allowed and can be used to generate completely forward or completely reverse windows. Here are some examples of * window values: *
                - *
              • {@code prevTicks = 1, fwdTicks = 0} - contains only the current row
              • - *
              • {@code prevTicks = 10, fwdTicks = 0} - contains 9 previous rows and the current row
              • - *
              • {@code prevTicks = 0, fwdTicks = 10} - contains the following 10 rows, excludes the current row
              • - *
              • {@code prevTicks = 10, fwdTicks = 10} - contains the previous 9 rows, the current row and the 10 rows + *
              • {@code revTicks = 1, fwdTicks = 0} - contains only the current row
              • + *
              • {@code revTicks = 10, fwdTicks = 0} - contains 9 previous rows and the current row
              • + *
              • {@code revTicks = 0, fwdTicks = 10} - contains the following 10 rows, excludes the current row
              • + *
              • {@code revTicks = 10, fwdTicks = 10} - contains the previous 9 rows, the current row and the 10 rows * following
              • - *
              • {@code prevTicks = 10, fwdTicks = -5} - contains 5 rows, beginning at 9 rows before, ending at 5 rows before + *
              • {@code revTicks = 10, fwdTicks = -5} - contains 5 rows, beginning at 9 rows before, ending at 5 rows before * the current row (inclusive)
              • - *
              • {@code prevTicks = 11, fwdTicks = -1} - contains 10 rows, beginning at 10 rows before, ending at 1 row before + *
              • {@code revTicks = 11, fwdTicks = -1} - contains 10 rows, beginning at 10 rows before, ending at 1 row before * the current row (inclusive)
              • - *
              • {@code prevTicks = -5, fwdTicks = 10} - contains 5 rows, beginning 5 rows following, ending at 10 rows + *
              • {@code revTicks = -5, fwdTicks = 10} - contains 5 rows, beginning 5 rows following, ending at 10 rows * following the current row (inclusive)
              • *
              * - * @param prevTicks the look-behind window size (in rows/ticks) + * @param revTicks the look-behind window size (in rows/ticks) * @param fwdTicks the look-ahead window size (in rows/ticks) * @param pairs The input/output column name pairs * @return The aggregation */ - static UpdateByOperation RollingSum(long prevTicks, long fwdTicks, String... pairs) { - return RollingSumSpec.ofTicks(prevTicks, fwdTicks).clause(pairs); + static UpdateByOperation RollingSum(long revTicks, long fwdTicks, String... pairs) { + return RollingSumSpec.ofTicks(revTicks, fwdTicks).clause(pairs); } /** * Create a {@link RollingSumSpec rolling sum} for the supplied column name pairs, using time as the windowing unit. - * This function accepts {@link Duration duration} as the prev window parameter. A row that contains a {@code null} + * This function accepts {@link Duration duration} as the reverse window parameter. A row containing a {@code null} * in the timestamp column belongs to no window and will not have a value computed or be considered in the windows * of other rows. * * Here are some examples of window values: *
                - *
              • {@code prevDuration = 0m} - contains rows that exactly match the current row timestamp
              • - *
              • {@code prevDuration = 10m} - contains rows from 10m earlier through the current row timestamp - * (inclusive)
              • + *
              • {@code revDuration = 0m} - contains rows that exactly match the current row timestamp
              • + *
              • {@code revDuration = 10m} - contains rows from 10m earlier through the current row timestamp (inclusive)
              • *
              * * @param timestampCol the name of the timestamp column - * @param prevDuration the look-behind window size (in Duration) + * @param revDuration the look-behind window size (in Duration) * @param pairs The input/output column name pairs * @return The aggregation */ - static UpdateByOperation RollingSum(String timestampCol, Duration prevDuration, String... pairs) { - return RollingSumSpec.ofTime(timestampCol, prevDuration).clause(pairs); + static UpdateByOperation RollingSum(String timestampCol, Duration revDuration, String... pairs) { + return RollingSumSpec.ofTime(timestampCol, revDuration).clause(pairs); } /** * Create a {@link RollingSumSpec rolling sum} for the supplied column name pairs, using time as the windowing unit. - * This function accepts {@link Duration durations} as the prev and forward window parameters. Negative values are - * allowed and can be used to generate completely forward or completely reverse windows. A row that contains a + * This function accepts {@link Duration durations} as the reverse and forward window parameters. Negative values + * are allowed and can be used to generate completely forward or completely reverse windows. A row containing a * {@code null} in the timestamp column belongs to no window and will not have a value computed or be considered in * the windows of other rows. * * Here are some examples of window values: *
                - *
              • {@code prevDuration = 0m, fwdDuration = 0m} - contains rows that exactly match the current row timestamp
              • - *
              • {@code prevDuration = 10m, fwdDuration = 0m} - contains rows from 10m earlier through the current row + *
              • {@code revDuration = 0m, fwdDuration = 0m} - contains rows that exactly match the current row timestamp
              • + *
              • {@code revDuration = 10m, fwdDuration = 0m} - contains rows from 10m earlier through the current row * timestamp (inclusive)
              • - *
              • {@code prevDuration = 0m, fwdDuration = 10m} - contains rows from the current row through 10m following the + *
              • {@code revDuration = 0m, fwdDuration = 10m} - contains rows from the current row through 10m following the * current row timestamp (inclusive)
              • - *
              • {@code prevDuration = 10m, fwdDuration = 10m} - contains rows from 10m earlier through 10m following the + *
              • {@code revDuration = 10m, fwdDuration = 10m} - contains rows from 10m earlier through 10m following the * current row timestamp (inclusive)
              • - *
              • {@code prevDuration = 10m, fwdDuration = -5m} - contains rows from 10m earlier through 5m before the current + *
              • {@code revDuration = 10m, fwdDuration = -5m} - contains rows from 10m earlier through 5m before the current * row timestamp (inclusive), this is a purely backwards looking window
              • - *
              • {@code prevDuration = -5m, fwdDuration = 10m} - contains rows from 5m following through 10m following the + *
              • {@code revDuration = -5m, fwdDuration = 10m} - contains rows from 5m following through 10m following the * current row timestamp (inclusive), this is a purely forwards looking window
              • *
              * * @param timestampCol the name of the timestamp column - * @param prevDuration the look-behind window size (in Duration) + * @param revDuration the look-behind window size (in Duration) * @param fwdDuration the look-ahead window size (in Duration) * @param pairs The input/output column name pairs * @return The aggregation */ - static UpdateByOperation RollingSum(String timestampCol, Duration prevDuration, Duration fwdDuration, + static UpdateByOperation RollingSum(String timestampCol, Duration revDuration, Duration fwdDuration, String... pairs) { - return RollingSumSpec.ofTime(timestampCol, prevDuration, fwdDuration).clause(pairs); + return RollingSumSpec.ofTime(timestampCol, revDuration, fwdDuration).clause(pairs); } /** * Create a {@link RollingSumSpec rolling sum} for the supplied column name pairs, using time as the windowing unit. - * This function accepts {@code nanoseconds} as the prev window parameters. A row that contains a {@code null} in + * This function accepts {@code nanoseconds} as the reverse window parameters. A row containing a {@code null} in * the timestamp column belongs to no window and will not have a value computed or be considered in the windows of * other rows. * * @param timestampCol the name of the timestamp column - * @param prevNanos the look-behind window size (in nanoseconds) + * @param revTime the look-behind window size (in nanoseconds) * @param pairs The input/output column name pairs * @return The aggregation */ - static UpdateByOperation RollingSum(String timestampCol, long prevNanos, String... pairs) { - return RollingSumSpec.ofTime(timestampCol, prevNanos).clause(pairs); + static UpdateByOperation RollingSum(String timestampCol, long revTime, String... pairs) { + return RollingSumSpec.ofTime(timestampCol, revTime).clause(pairs); } /** * Create a {@link RollingSumSpec rolling sum} for the supplied column name pairs, using time as the windowing unit. - * This function accepts {@code nanoseconds} as the prev and forward window parameters. Negative values are allowed - * and can be used to generate completely forward or completely reverse windows. A row that contains a {@code null} - * in the timestamp column belongs to no window and will not have a value computed or be considered in the windows - * of other rows. + * This function accepts {@code nanoseconds} as the reverse and forward window parameters. Negative values are + * allowed and can be used to generate completely forward or completely reverse windows. A row containing a + * {@code null} in the timestamp column belongs to no window and will not have a value computed or be considered in + * the windows of other rows. * * @param timestampCol the name of the timestamp column - * @param prevNanos the look-behind window size (in nanoseconds) - * @param fwdNanos the look-ahead window size (in nanoseconds) + * @param revTime the look-behind window size (in nanoseconds) + * @param fwdTime the look-ahead window size (in nanoseconds) * @param pairs The input/output column name pairs * @return The aggregation */ - static UpdateByOperation RollingSum(String timestampCol, long prevNanos, long fwdNanos, String... pairs) { - return RollingSumSpec.ofTime(timestampCol, prevNanos, fwdNanos).clause(pairs); + static UpdateByOperation RollingSum(String timestampCol, long revTime, long fwdTime, String... pairs) { + return RollingSumSpec.ofTime(timestampCol, revTime, fwdTime).clause(pairs); } T walk(Visitor visitor); diff --git a/table-api/src/main/java/io/deephaven/api/updateby/spec/CumMinMaxSpec.java b/table-api/src/main/java/io/deephaven/api/updateby/spec/CumMinMaxSpec.java index daa648858d7..92a36cc81f7 100644 --- a/table-api/src/main/java/io/deephaven/api/updateby/spec/CumMinMaxSpec.java +++ b/table-api/src/main/java/io/deephaven/api/updateby/spec/CumMinMaxSpec.java @@ -20,13 +20,8 @@ public static CumMinMaxSpec of(boolean isMax) { @Override public final boolean applicableTo(Class inputType) { return - // is primitive numeric? - inputType == double.class || inputType == float.class - || inputType == int.class || inputType == long.class || inputType == short.class - || inputType == byte.class - - // is boxed numeric? - || Number.class.isAssignableFrom(inputType) + // is primitive or boxed numeric? + applicableToNumeric(inputType) // is comparable? || (Comparable.class.isAssignableFrom(inputType) && inputType != Boolean.class); diff --git a/table-api/src/main/java/io/deephaven/api/updateby/spec/CumProdSpec.java b/table-api/src/main/java/io/deephaven/api/updateby/spec/CumProdSpec.java index bce90dc9d17..3b3f76608bc 100644 --- a/table-api/src/main/java/io/deephaven/api/updateby/spec/CumProdSpec.java +++ b/table-api/src/main/java/io/deephaven/api/updateby/spec/CumProdSpec.java @@ -15,14 +15,8 @@ public static CumProdSpec of() { @Override public final boolean applicableTo(Class inputType) { - return - // is primitive numeric? - inputType == double.class || inputType == float.class - || inputType == int.class || inputType == long.class || inputType == short.class - || inputType == byte.class - - // is boxed numeric? - || Number.class.isAssignableFrom(inputType); + // is primitive or boxed numeric? + return applicableToNumeric(inputType); } @Override diff --git a/table-api/src/main/java/io/deephaven/api/updateby/spec/CumSumSpec.java b/table-api/src/main/java/io/deephaven/api/updateby/spec/CumSumSpec.java index cecffe68fc2..fc89105f711 100644 --- a/table-api/src/main/java/io/deephaven/api/updateby/spec/CumSumSpec.java +++ b/table-api/src/main/java/io/deephaven/api/updateby/spec/CumSumSpec.java @@ -16,13 +16,8 @@ public static CumSumSpec of() { @Override public final boolean applicableTo(Class inputType) { return - // is primitive numeric? - inputType == double.class || inputType == float.class - || inputType == int.class || inputType == long.class || inputType == short.class - || inputType == byte.class - - // is boxed numeric? - || Number.class.isAssignableFrom(inputType) + // is primitive or boxed numeric? + applicableToNumeric(inputType) // is boolean? || inputType == boolean.class || inputType == Boolean.class; diff --git a/table-api/src/main/java/io/deephaven/api/updateby/spec/EmaSpec.java b/table-api/src/main/java/io/deephaven/api/updateby/spec/EmaSpec.java index 4a6f4c1d197..bb403c99567 100644 --- a/table-api/src/main/java/io/deephaven/api/updateby/spec/EmaSpec.java +++ b/table-api/src/main/java/io/deephaven/api/updateby/spec/EmaSpec.java @@ -60,14 +60,8 @@ public final OperationControl controlOrDefault() { @Override public final boolean applicableTo(Class inputType) { - return - // is primitive numeric? - inputType == double.class || inputType == float.class - || inputType == int.class || inputType == long.class || inputType == short.class - || inputType == byte.class - - // is boxed numeric? - || Number.class.isAssignableFrom(inputType); + // is primitive or boxed numeric? + return applicableToNumeric(inputType); } @Override diff --git a/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java b/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java index 7c012fac83d..562e39a60e6 100644 --- a/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java +++ b/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java @@ -15,47 +15,56 @@ @BuildableStyle public abstract class RollingSumSpec extends UpdateBySpecBase { - // most common usages first, will complete the list later - - public static RollingSumSpec ofTicks(long prevTicks) { - return of(WindowScale.ofTicks(prevTicks)); + public static RollingSumSpec ofTicks(long revTicks) { + return of(WindowScale.ofTicks(revTicks)); } - public static RollingSumSpec ofTicks(long prevTicks, long fwdTicks) { - return of(WindowScale.ofTicks(prevTicks), WindowScale.ofTicks(fwdTicks)); + public static RollingSumSpec ofTicks(long revTicks, long fwdTicks) { + return of(WindowScale.ofTicks(revTicks), WindowScale.ofTicks(fwdTicks)); } - public static RollingSumSpec ofTime(final String timestampCol, Duration prevDuration) { - return of(WindowScale.ofTime(timestampCol, prevDuration)); + public static RollingSumSpec ofTime(final String timestampCol, Duration revDuration) { + return of(WindowScale.ofTime(timestampCol, revDuration)); } - public static RollingSumSpec ofTime(final String timestampCol, Duration prevDuration, Duration fwdDuration) { - return of(WindowScale.ofTime(timestampCol, prevDuration), + public static RollingSumSpec ofTime(final String timestampCol, Duration revDuration, Duration fwdDuration) { + return of(WindowScale.ofTime(timestampCol, revDuration), WindowScale.ofTime(timestampCol, fwdDuration)); } - public static RollingSumSpec ofTime(final String timestampCol, long prevDuration) { - return of(WindowScale.ofTime(timestampCol, prevDuration)); + public static RollingSumSpec ofTime(final String timestampCol, long revDuration) { + return of(WindowScale.ofTime(timestampCol, revDuration)); } - public static RollingSumSpec ofTime(final String timestampCol, long prevDuration, long fwdDuration) { - return of(WindowScale.ofTime(timestampCol, prevDuration), + public static RollingSumSpec ofTime(final String timestampCol, long revDuration, long fwdDuration) { + return of(WindowScale.ofTime(timestampCol, revDuration), WindowScale.ofTime(timestampCol, fwdDuration)); } - // general use constructors + // internal use constructors - public static RollingSumSpec of(WindowScale prevWindowScale) { - return ImmutableRollingSumSpec.builder().prevTimeScale(prevWindowScale).build(); + private static RollingSumSpec of(WindowScale revWindowScale) { + return ImmutableRollingSumSpec.builder().revTimeScale(revWindowScale).build(); } - public static RollingSumSpec of(WindowScale prevWindowScale, WindowScale fwdWindowScale) { - return ImmutableRollingSumSpec.builder().prevTimeScale(prevWindowScale).fwdTimeScale(fwdWindowScale).build(); + private static RollingSumSpec of(WindowScale revWindowScale, WindowScale fwdWindowScale) { + // We would like to use jdk.internal.util.ArraysSupport.MAX_ARRAY_LENGTH, but it is not exported + final int MAX_ARRAY_SIZE = Integer.MAX_VALUE - 8; + + // assert some rational constraints + final long size = revWindowScale.timescaleUnits() + fwdWindowScale.timescaleUnits(); + if (size < 0) { + throw new IllegalArgumentException("UpdateBy rolling window size must be non-negative"); + } else if (!revWindowScale.isTimeBased() && size >= MAX_ARRAY_SIZE) { + throw new IllegalArgumentException( + "UpdateBy rolling window size may not exceed MAX_ARRAY_SIZE (" + MAX_ARRAY_SIZE + ")"); + } + return ImmutableRollingSumSpec.builder().revTimeScale(revWindowScale).fwdTimeScale(fwdWindowScale).build(); } public abstract Optional control(); - public abstract WindowScale prevTimeScale(); + public abstract WindowScale revTimeScale(); /** * provide a default forward-looking timescale @@ -68,14 +77,8 @@ public WindowScale fwdTimeScale() { @Override public final boolean applicableTo(Class inputType) { return - // is primitive numeric? - inputType == double.class || inputType == float.class - || inputType == int.class || inputType == long.class || inputType == short.class - || inputType == byte.class - - // is boxed numeric? - || Number.class.isAssignableFrom(inputType) - + // is primitive or boxed numeric + applicableToNumeric(inputType) // is boolean? || inputType == boolean.class || inputType == Boolean.class; } diff --git a/table-api/src/main/java/io/deephaven/api/updateby/spec/UpdateBySpecBase.java b/table-api/src/main/java/io/deephaven/api/updateby/spec/UpdateBySpecBase.java index 6c3d54127ae..3af135fe857 100644 --- a/table-api/src/main/java/io/deephaven/api/updateby/spec/UpdateBySpecBase.java +++ b/table-api/src/main/java/io/deephaven/api/updateby/spec/UpdateBySpecBase.java @@ -44,4 +44,19 @@ public final ColumnUpdateOperation clause(Collection pairs) { .addAllColumns(pairs) .build(); } + + /** + * Returns {@code true} if the input class is a primitive or boxed numeric type + * + * @param inputType the input class to test + */ + static boolean applicableToNumeric(Class inputType) { + return + // is primitive numeric? + inputType == double.class || inputType == float.class + || inputType == int.class || inputType == long.class || inputType == short.class + || inputType == byte.class + // is boxed numeric? + || Number.class.isAssignableFrom(inputType); + } } From 6d046fabac35a09b3d50d6ff68923976fd412b92 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Mon, 30 Jan 2023 14:38:10 -0800 Subject: [PATCH 106/123] Spotless and ZeroKey bug fix. --- .../engine/table/impl/updateby/ZeroKeyUpdateByManager.java | 2 +- .../engine/table/impl/updateby/TestUpdateByGeneral.java | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java index aa7f910fd90..c7a01eca6fb 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java @@ -69,7 +69,7 @@ protected ZeroKeyUpdateByManager( transformer = source.newModifiedColumnSetTransformer(result, source.getDefinition().getColumnNamesArray()); // result will depend on zeroKeyUpdateBy - result.addParentReference(zeroKeyUpdateBy); + result.addParentReference(zeroKeyUpdateBy.result); } else { zeroKeyUpdateBy = new UpdateByBucketHelper(description, source, operators, windows, inputSources, resultSources, timestampColumnName, rowRedirection, control); diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java index c323122076d..0f1ae43af5f 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java @@ -166,7 +166,8 @@ public void testNewBuckets() { i(2, 4, 6).toTracking(), col("Key", "A", "B", "A"), intCol("Int", 2, 4, 6)); - final QueryTable result = (QueryTable) table.updateBy(List.of(UpdateByOperation.Fill("Filled=Int"), UpdateByOperation.RollingSum(2, "Sum=Int")), "Key"); + final QueryTable result = (QueryTable) table.updateBy( + List.of(UpdateByOperation.Fill("Filled=Int"), UpdateByOperation.RollingSum(2, "Sum=Int")), "Key"); UpdateGraphProcessor.DEFAULT.runWithinUnitTestCycle(() -> { TstUtils.addToTable(table, i(8), col("Key", "B"), intCol("Int", 8)); // Add to "B" bucket From 7ac5dbd594df26555e1b6afe777e2d8b36a75a54 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Mon, 30 Jan 2023 18:55:06 -0800 Subject: [PATCH 107/123] More updates, but resource leak is back --- .../BucketedPartitionedUpdateByManager.java | 5 +-- .../engine/table/impl/updateby/UpdateBy.java | 11 +++++++ .../impl/updateby/ZeroKeyUpdateByManager.java | 3 +- .../updateby/ema/BigDecimalEMAOperator.java | 9 ++---- .../updateby/ema/BigIntegerEMAOperator.java | 5 ++- .../updateby/ema/BigNumberEMAOperator.java | 32 ++++++++++++------- .../api/updateby/spec/RollingSumSpec.java | 2 +- 7 files changed, 41 insertions(+), 26 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java index b2773722dd7..91a59160eb0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java @@ -49,6 +49,7 @@ protected BucketedPartitionedUpdateByManager( @NotNull final UpdateByWindow[] windows, @NotNull final ColumnSource[] inputSources, @NotNull final QueryTable source, + final String[] persistentColumns, @NotNull final Map> resultSources, @NotNull final Collection byColumns, @Nullable final String timestampColumnName, @@ -76,8 +77,8 @@ protected BucketedPartitionedUpdateByManager( } pt = source.partitionedAggBy(List.of(), true, null, byColumnNames); - // make the source->result transformer - transformer = source.newModifiedColumnSetTransformer(result, source.getDefinition().getColumnNamesArray()); + // make the source->result transformer from only the columns in the source that are present in result + transformer = source.newModifiedColumnSetTransformer(result, persistentColumns); } else { pt = source.partitionedAggBy(List.of(), true, null, byColumnNames); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java index e5dcb0b3b81..c7a064a134a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java @@ -885,6 +885,10 @@ public static Table updateBy(@NotNull final QueryTable source, .append("}"); String timestampColumnName = null; + // create an initial set of all source columns + final Set persistentColumnSet = new LinkedHashSet<>(); + persistentColumnSet.addAll(source.getColumnSourceMap().keySet()); + final Set problems = new LinkedHashSet<>(); final Map> opResultSources = new LinkedHashMap<>(); for (final UpdateByOperator op : opArr) { @@ -892,6 +896,8 @@ public static Table updateBy(@NotNull final QueryTable source, if (opResultSources.putIfAbsent(name, col) != null) { problems.add(name); } + // remove overridden source columns + persistentColumnSet.remove(name); }); // verify zero or one timestamp column names if (op.getTimestampColumnName() != null) { @@ -912,6 +918,9 @@ public static Table updateBy(@NotNull final QueryTable source, String.join(", ", problems) + "}"); } + // These are the source columns that exist unchanged in the result + final String[] persistentColumns = persistentColumnSet.toArray(String[]::new); + // We will divide the operators into similar windows for efficient processing. final KeyedObjectHashMap> windowMap = new KeyedObjectHashMap<>(new KeyedObjectKey<>() { @@ -996,6 +1005,7 @@ public boolean equalKey(UpdateByOperator updateByOperator, windowArr, inputSourceArr, source, + persistentColumns, resultSources, fTimestampColumnName, rowRedirection, @@ -1032,6 +1042,7 @@ public boolean equalKey(UpdateByOperator updateByOperator, windowArr, inputSourceArr, source, + persistentColumns, resultSources, byColumns, fTimestampColumnName, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java index c7a01eca6fb..ccb6870ef08 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java @@ -39,6 +39,7 @@ protected ZeroKeyUpdateByManager( @NotNull UpdateByWindow[] windows, @NotNull ColumnSource[] inputSources, @NotNull QueryTable source, + final String[] persistentColumns, @NotNull final Map> resultSources, @Nullable String timestampColumnName, @Nullable WritableRowRedirection rowRedirection, @@ -66,7 +67,7 @@ protected ZeroKeyUpdateByManager( buckets.offer(zeroKeyUpdateBy); // make the source->result transformer - transformer = source.newModifiedColumnSetTransformer(result, source.getDefinition().getColumnNamesArray()); + transformer = source.newModifiedColumnSetTransformer(result, persistentColumns); // result will depend on zeroKeyUpdateBy result.addParentReference(zeroKeyUpdateBy.result); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java index fa5904a1c22..39b456e36ff 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java @@ -59,24 +59,19 @@ public void accumulate(RowSequence inputKeys, handleBadData(this, isNull); } else if (isNullTime) { // no change to curVal and lastStamp - continue; } else { if (curVal == null) { curVal = input; lastStamp = timestamp; - } else { final long dt = timestamp - lastStamp; if (dt != 0) { // alpha is dynamic based on time, but only recalculated when needed if (dt != lastDt) { - alpha = BigDecimal.valueOf(Math.exp(-dt / (double) reverseWindowScaleUnits)); - oneMinusAlpha = - BigDecimal.ONE.subtract(alpha, control.bigValueContextOrDefault()); + alpha = computeAlpha(-dt, reverseWindowScaleUnits); + oneMinusAlpha = computeOneMinusAlpha(alpha); lastDt = dt; } - // alpha is dynamic, based on time - curVal = curVal.multiply(alpha, control.bigValueContextOrDefault()) .add(input.multiply(oneMinusAlpha, control.bigValueContextOrDefault()), control.bigValueContextOrDefault()); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java index aaddc148f07..3f3d7044947 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java @@ -71,11 +71,10 @@ public void accumulate(RowSequence inputKeys, if (dt != 0) { // alpha is dynamic based on time, but only recalculated when needed if (dt != lastDt) { - alpha = BigDecimal.valueOf(Math.exp(-dt / (double) reverseWindowScaleUnits)); - oneMinusAlpha = BigDecimal.ONE.subtract(alpha, control.bigValueContextOrDefault()); + alpha = computeAlpha(-dt, reverseWindowScaleUnits); + oneMinusAlpha = computeOneMinusAlpha(alpha); lastDt = dt; } - curVal = curVal.multiply(alpha, control.bigValueContextOrDefault()) .add(decimalInput.multiply(oneMinusAlpha, control.bigValueContextOrDefault()), control.bigValueContextOrDefault()); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java index c87a7d2347c..d27b7b51afa 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java @@ -51,6 +51,7 @@ public boolean isValueValid(long atKey) { public void reset() { curVal = null; lastStamp = NULL_LONG; + lastDt = NULL_LONG; } } @@ -78,10 +79,15 @@ public BigNumberEMAOperator(@NotNull final MatchPair pair, this.control = control; this.valueSource = valueSource; - opAlpha = BigDecimal.valueOf(Math.exp(-1.0 / (double) timeScaleUnits)); - opOneMinusAlpha = - timestampColumnName == null ? BigDecimal.ONE.subtract(opAlpha, control.bigValueContextOrDefault()) - : null; + if (timestampColumnName == null) { + // tick-based, pre-compute alpha and oneMinusAlpha + opAlpha = computeAlpha(-1, timeScaleUnits); + opOneMinusAlpha = computeOneMinusAlpha(opAlpha); + } else { + // time-based, must compute alpha and oneMinusAlpha for each time delta + opAlpha = null; + opOneMinusAlpha = null; + } } @Override @@ -91,14 +97,8 @@ public void initializeUpdate(@NotNull final UpdateContext updateContext, super.initializeUpdate(updateContext, firstUnmodifiedKey, firstUnmodifiedTimestamp); final Context ctx = (Context) updateContext; - // If we set the last state to null, then we know it was a reset state and the timestamp must also - // have been reset. - if (ctx.curVal == null || (firstUnmodifiedKey == NULL_ROW_KEY)) { - ctx.lastStamp = NULL_LONG; - } else { - // rely on the caller to validate this is a valid timestamp (or NULL_LONG when appropriate) - ctx.lastStamp = firstUnmodifiedTimestamp; - } + // rely on the caller to validate this is a valid timestamp (or NULL_LONG when appropriate) + ctx.lastStamp = firstUnmodifiedTimestamp; } void handleBadData(@NotNull final Context ctx, final boolean isNull) { @@ -114,4 +114,12 @@ void handleBadData(@NotNull final Context ctx, final boolean isNull) { ctx.reset(); } } + + BigDecimal computeAlpha(final long dt, final long timeScaleUnits) { + return BigDecimal.valueOf(Math.exp(dt / (double) timeScaleUnits)); + } + + BigDecimal computeOneMinusAlpha(final BigDecimal alpha) { + return BigDecimal.ONE.subtract(alpha, control.bigValueContextOrDefault()); + } } diff --git a/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java b/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java index 562e39a60e6..7d1df58b040 100644 --- a/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java +++ b/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingSumSpec.java @@ -55,7 +55,7 @@ private static RollingSumSpec of(WindowScale revWindowScale, WindowScale fwdWind final long size = revWindowScale.timescaleUnits() + fwdWindowScale.timescaleUnits(); if (size < 0) { throw new IllegalArgumentException("UpdateBy rolling window size must be non-negative"); - } else if (!revWindowScale.isTimeBased() && size >= MAX_ARRAY_SIZE) { + } else if (!revWindowScale.isTimeBased() && size > MAX_ARRAY_SIZE) { throw new IllegalArgumentException( "UpdateBy rolling window size may not exceed MAX_ARRAY_SIZE (" + MAX_ARRAY_SIZE + ")"); } From f5ed59168da347eb4196d597c13dd580b26a0e7d Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Tue, 31 Jan 2023 13:16:35 -0500 Subject: [PATCH 108/123] 1. Improve bucket and listener descriptions 2. Listen for transformation failures when bucketed 3. Ensure that source, transformation, and bucket failures are propagated, exactly once 4. Ensure that parallel execution costs are properly accumulated for performance logging 5. Fix some messages and warnings 6. Fix missing long cast --- .../referencecounting/ReferenceCounted.java | 8 +- .../engine/table/impl/BaseTable.java | 6 +- .../impl/InstrumentedTableListenerBase.java | 6 +- .../impl/sources/UnionSourceManager.java | 3 +- .../BucketedPartitionedUpdateByManager.java | 69 +++++++---- .../engine/table/impl/updateby/UpdateBy.java | 107 ++++++++++++++---- .../impl/updateby/UpdateByBucketHelper.java | 68 ++++++----- .../impl/updateby/ZeroKeyUpdateByManager.java | 15 +-- 8 files changed, 188 insertions(+), 94 deletions(-) diff --git a/Util/src/main/java/io/deephaven/util/referencecounting/ReferenceCounted.java b/Util/src/main/java/io/deephaven/util/referencecounting/ReferenceCounted.java index 235b10d2b8f..4969609d802 100644 --- a/Util/src/main/java/io/deephaven/util/referencecounting/ReferenceCounted.java +++ b/Util/src/main/java/io/deephaven/util/referencecounting/ReferenceCounted.java @@ -190,14 +190,20 @@ public final boolean tryDecrementReferenceCount() { * normally, but subsequent invocations of {@link #decrementReferenceCount()} and * {@link #tryDecrementReferenceCount()} will act as if the reference count was successfully decremented until * {@link #resetReferenceCount()} is invoked. + * + * @return Whether this invocation actually forced the reference count to zero (and invoked + * {@link #onReferenceCountAtZero()}. {@code false} means that this ReferenceCounted reached a zero through + * other means. */ - public final void forceReferenceCountToZero() { + public final boolean forceReferenceCountToZero() { int currentReferenceCount; while (!isZero(currentReferenceCount = getCurrentReferenceCount())) { if (tryUpdateReferenceCount(currentReferenceCount, FORCED_TERMINAL_ZERO_VALUE)) { onReferenceCountAtZero(); + return true; } } + return false; } /** diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/BaseTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/BaseTable.java index 8d0658d0091..d1ab17976f9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/BaseTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/BaseTable.java @@ -820,10 +820,10 @@ public static class ListenerImpl extends InstrumentedTableUpdateListener { @ReferentialIntegrity private final Table parent; - private final BaseTable dependent; + private final BaseTable dependent; private final boolean canReuseModifiedColumnSet; - public ListenerImpl(String description, Table parent, BaseTable dependent) { + public ListenerImpl(String description, Table parent, BaseTable dependent) { super(description); this.parent = parent; this.dependent = dependent; @@ -876,7 +876,7 @@ protected Table getParent() { return parent; } - protected BaseTable getDependent() { + protected BaseTable getDependent() { return dependent; } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/InstrumentedTableListenerBase.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/InstrumentedTableListenerBase.java index f30d3b3de08..dd4b43585fa 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/InstrumentedTableListenerBase.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/InstrumentedTableListenerBase.java @@ -158,7 +158,7 @@ public void onFailure(Throwable originalException, Entry sourceEntry) { protected abstract void onFailureInternal(Throwable originalException, Entry sourceEntry); - protected final void onFailureInternalWithDependent(final BaseTable dependent, final Throwable originalException, + protected final void onFailureInternalWithDependent(final BaseTable dependent, final Throwable originalException, final Entry sourceEntry) { dependent.notifyListenersOnError(originalException, sourceEntry); @@ -168,7 +168,9 @@ protected final void onFailureInternalWithDependent(final BaseTable dependent, f AsyncClientErrorNotifier.reportError(originalException); } } catch (IOException e) { - throw new UncheckedTableException("Exception in " + sourceEntry.toString(), originalException); + throw new UncheckedTableException( + "Exception while delivering async client error notification for " + sourceEntry.toString(), + originalException); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/UnionSourceManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/UnionSourceManager.java index 9906802b151..8f477767a2a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/UnionSourceManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/UnionSourceManager.java @@ -78,10 +78,9 @@ public UnionSourceManager(@NotNull final PartitionedTable partitionedTable) { // noinspection resource resultRows = RowSetFactory.empty().toTracking(); unionRedirection = new UnionRedirection(initialNumSlots, refreshing); - // noinspection unchecked resultColumnSources = partitionedTable.constituentDefinition().getColumnStream() .map(cd -> new UnionColumnSource<>(cd.getDataType(), cd.getComponentType(), this, unionRedirection, - new TableSourceLookup(cd.getName()))) + new TableSourceLookup<>(cd.getName()))) .toArray(UnionColumnSource[]::new); resultTable = new QueryTable(resultRows, getColumnSources()); modifiedColumnSet = resultTable.getModifiedColumnSetForUpdates(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java index 91a59160eb0..40214d3617e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java @@ -2,12 +2,11 @@ import io.deephaven.api.ColumnName; import io.deephaven.api.updateby.UpdateByControl; +import io.deephaven.base.verify.Assert; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.rowset.RowSetShiftData; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.ModifiedColumnSet; -import io.deephaven.engine.table.PartitionedTable; -import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.*; +import io.deephaven.engine.table.impl.InstrumentedTableUpdateListenerAdapter; import io.deephaven.engine.table.impl.QueryTable; import io.deephaven.engine.table.impl.TableUpdateImpl; import io.deephaven.engine.table.impl.util.WritableRowRedirection; @@ -15,19 +14,19 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.util.Collection; -import java.util.List; -import java.util.Map; +import java.util.*; +import java.util.stream.Collectors; /** * An implementation of {@link UpdateBy} dedicated to bucketed computation. */ class BucketedPartitionedUpdateByManager extends UpdateBy { + /** The output table for this UpdateBy operation */ final QueryTable result; - /** The partitioned table used for identifying buckets */ - final Table transformedTable; + /** Listener to the partitioned table used for identifying buckets */ + final TransformFailureListener transformFailureListener; /** * Perform a bucketed updateBy using {@code byColumns} as the keys @@ -60,12 +59,12 @@ protected BucketedPartitionedUpdateByManager( // this table will always have the rowset of the source result = new QueryTable(source.getRowSet(), resultSources); - String[] byColumnNames = byColumns.stream().map(ColumnName::name).toArray(String[]::new); + final String[] byColumnNames = byColumns.stream().map(ColumnName::name).toArray(String[]::new); final PartitionedTable pt; if (source.isRefreshing()) { // this is a refreshing source, we will need a listener - listener = newUpdateByListener(description); + listener = newUpdateByListener(); source.addUpdateListener(listener); // result will depend on listener result.addParentReference(listener); @@ -84,16 +83,20 @@ protected BucketedPartitionedUpdateByManager( } final PartitionedTable transformed = pt.transform(t -> { + final long firstSourceRowKey = t.getRowSet().firstRowKey(); + final String bucketDescription = BucketedPartitionedUpdateByManager.this + "-bucket-" + + Arrays.stream(byColumnNames) + .map(bcn -> Objects.toString(t.getColumnSource(bcn).get(firstSourceRowKey))) + .collect(Collectors.joining(", ", "[", "]")); UpdateByBucketHelper bucket = new UpdateByBucketHelper( - description, + bucketDescription, (QueryTable) t, - operators, windows, inputSources, resultSources, timestampColumnName, - rowRedirection, - control); + control, + (oe, se) -> deliverUpdateError(oe, se, true)); bucket.parentUpdateBy = this; bucket.createdStep = LogicalClock.DEFAULT.currentStep(); @@ -107,12 +110,12 @@ protected BucketedPartitionedUpdateByManager( }); if (source.isRefreshing()) { - transformedTable = transformed.table(); - - // result also depends on the transformedTable - result.addParentReference(transformedTable); + final Table transformedTable = transformed.table(); + transformFailureListener = new TransformFailureListener(transformedTable); + transformedTable.addUpdateListener(transformFailureListener); + result.addParentReference(transformFailureListener); } else { - transformedTable = null; + transformFailureListener = null; } // make a dummy update to generate the initial row keys @@ -135,7 +138,29 @@ protected QueryTable result() { @Override protected boolean upstreamSatisfied(final long step) { - // For bucketed, need to verify the source and the transformed table is satisfied. - return source.satisfied(step) && transformedTable.satisfied(step); + // For bucketed, need to verify the source and the transformed table listener are satisfied. + return source.satisfied(step) && transformFailureListener.satisfied(step); + } + + private final class TransformFailureListener extends InstrumentedTableUpdateListenerAdapter { + + private TransformFailureListener(@NotNull final Table transformed) { + super(BucketedPartitionedUpdateByManager.this + "-TransformFailureListener", transformed, false); + } + + @Override + public void onUpdate(@NotNull final TableUpdate upstream) { + // No-op: We react to bucket creation inside the transform function, no need to do anything here. + // Validation: We expect only adds, because the partitioned table was created by partitionedAggBy with + // preserveEmpty==true + Assert.assertion(upstream.removed().isEmpty(), "upstream.removed().isEmpty()"); + Assert.assertion(upstream.modified().isEmpty(), "upstream.modified().isEmpty()"); + Assert.assertion(upstream.shifted().empty(), "upstream.shifted().empty()"); + } + + @Override + public void onFailureInternal(@NotNull final Throwable originalException, @Nullable final Entry sourceEntry) { + deliverUpdateError(originalException, sourceEntry, true); + } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java index c7a064a134a..aad5d22dd4b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java @@ -19,11 +19,16 @@ import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.*; +import io.deephaven.engine.table.impl.perf.BasePerformanceEntry; +import io.deephaven.engine.table.impl.perf.QueryPerformanceNugget; +import io.deephaven.engine.table.impl.perf.QueryPerformanceRecorder; import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.sources.sparse.SparseConstants; import io.deephaven.engine.table.impl.util.*; import io.deephaven.engine.updategraph.DynamicNode; +import io.deephaven.engine.updategraph.TerminalNotification; import io.deephaven.engine.updategraph.UpdateGraphProcessor; +import io.deephaven.engine.util.systemicmarking.SystemicObjectTracker; import io.deephaven.hash.KeyedObjectHashMap; import io.deephaven.hash.KeyedObjectKey; import io.deephaven.util.SafeCloseable; @@ -33,6 +38,7 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import java.io.IOException; import java.lang.ref.SoftReference; import java.util.*; import java.util.concurrent.CompletableFuture; @@ -199,6 +205,7 @@ protected UpdateBy( .toArray(); inputCacheNeeded = cacheableSourceIndices.length > 0; + //noinspection unchecked inputSourceCaches = new SoftReference[inputSources.length]; buckets = @@ -342,15 +349,15 @@ public LogOutput append(LogOutput logOutput) { return logOutput.append("UpdateBy.PhasedUpdateProcessor"); } - private void onError(Exception error) { + private void onError(@NotNull final Exception error) { if (waitForResult != null) { - // signal to the future that an exception has occurred + // Use the Future to signal that an exception has occurred. Cleanup will be done by the waiting thread. waitForResult.completeExceptionally(error); } else { + // This error was delivered as part of update processing, we need to ensure that cleanup happens and + // a notification is dispatched downstream. cleanUpAfterError(); - result().forceReferenceCountToZero(); - // this is part of an update, need to notify downstream - result().notifyListenersOnError(error, null); + deliverUpdateError(error, listener.getEntry(), false); } } @@ -477,7 +484,7 @@ public void close() { 0, taskCount, (ctx, idx) -> { // advance to the first key of this block - ctx.rsIt.advance(inputRowSet.get(idx * PARALLEL_CACHE_BATCH_SIZE)); + ctx.rsIt.advance(inputRowSet.get((long)idx * PARALLEL_CACHE_BATCH_SIZE)); int remaining = PARALLEL_CACHE_BATCH_SIZE; while (ctx.rsIt.hasMore() && remaining > 0) { final RowSequence chunkOk = ctx.rsIt @@ -617,12 +624,7 @@ private void processWindows(final Runnable resumeAction) { */ private void cleanUpAndNotify(final Runnable resumeAction) { // create the downstream before calling finalize() on the buckets (which releases resources) - final TableUpdate downstream; - if (!initialStep) { - downstream = computeDownstreamUpdate(); - } else { - downstream = null; - } + final TableUpdate downstream = initialStep ? null : computeDownstreamUpdate(); // allow the helpers to release their resources for (UpdateByBucketHelper bucket : dirtyBuckets) { @@ -641,10 +643,31 @@ private void cleanUpAndNotify(final Runnable resumeAction) { } } + // release remaining resources SafeCloseable.closeArray(changedRows, toClear); - upstream.release(); + // accumulate performance data + final BasePerformanceEntry accumulated = jobScheduler.getAccumulatedPerformance(); + if (accumulated != null) { + if (initialStep) { + final QueryPerformanceNugget outerNugget = QueryPerformanceRecorder.getInstance().getOuterNugget(); + if (outerNugget != null) { + outerNugget.addBaseEntry(accumulated); + } + } else { + UpdateGraphProcessor.DEFAULT.addNotification(new TerminalNotification() { + @Override + public void run() { + synchronized (accumulated) { + listener.getEntry().accumulate(accumulated); + } + } + }); + } + } + + // continue resumeAction.run(); } @@ -793,29 +816,71 @@ public void processUpdate() { } } + /** + * Disconnect result from the {@link UpdateGraphProcessor}, deliver downstream failure notifications, and + * cleanup if needed. + * + * @param error The {@link Throwable} to deliver, either from upstream or update processing + * @param sourceEntry The {@link TableListener.Entry} to associate with failure messages + * @param bucketCleanupNeeded Whether to clean up the buckets; unnecessary if the caller has already done this + */ + void deliverUpdateError( + @NotNull final Throwable error, + @Nullable final TableListener.Entry sourceEntry, + final boolean bucketCleanupNeeded) { + + final QueryTable result = result(); + if (!result.forceReferenceCountToZero()) { + // No work to do here, another invocation is responsible for delivering failures. + return; + } + + if (bucketCleanupNeeded) { + buckets.stream().filter(UpdateByBucketHelper::isDirty).forEach(UpdateByBucketHelper::finalizeUpdate); + } + + result.notifyListenersOnError(error, sourceEntry); + + // Secondary notification to client error monitoring + try { + if (SystemicObjectTracker.isSystemic(result)) { + AsyncClientErrorNotifier.reportError(error); + } + } catch (IOException e) { + throw new UncheckedTableException( + "Exception while delivering async client error notification for " + sourceEntry, error); + } + } + /** * The Listener that is called when all input tables (source and constituent) are satisfied. This listener will * initiate UpdateBy operator processing in parallel by bucket */ - class UpdateByListener extends InstrumentedTableUpdateListenerAdapter { - public UpdateByListener(@Nullable String description) { - super(description, UpdateBy.this.source, false); + protected class UpdateByListener extends InstrumentedTableUpdateListenerAdapter { + + private UpdateByListener() { + super(UpdateBy.this + "-SourceListener", UpdateBy.this.source, false); } @Override - public void onUpdate(final TableUpdate upstream) { + public void onUpdate(@NotNull final TableUpdate upstream) { final PhasedUpdateProcessor sm = new PhasedUpdateProcessor(upstream.acquire(), false); sm.processUpdate(); } + @Override + public void onFailureInternal(@NotNull final Throwable originalException, @Nullable final Entry sourceEntry) { + deliverUpdateError(originalException, sourceEntry, true); + } + @Override public boolean canExecute(final long step) { return upstreamSatisfied(step); } } - public UpdateByListener newUpdateByListener(@NotNull final String description) { - return new UpdateByListener(description); + public UpdateByListener newUpdateByListener() { + return new UpdateByListener(); } protected abstract QueryTable result(); @@ -886,8 +951,7 @@ public static Table updateBy(@NotNull final QueryTable source, String timestampColumnName = null; // create an initial set of all source columns - final Set persistentColumnSet = new LinkedHashSet<>(); - persistentColumnSet.addAll(source.getColumnSourceMap().keySet()); + final Set persistentColumnSet = new LinkedHashSet<>(source.getColumnSourceMap().keySet()); final Set problems = new LinkedHashSet<>(); final Map> opResultSources = new LinkedHashMap<>(); @@ -1000,7 +1064,6 @@ public boolean equalKey(UpdateByOperator updateByOperator, descriptionBuilder.append(")"); return LivenessScopeStack.computeEnclosed(() -> { final ZeroKeyUpdateByManager zkm = new ZeroKeyUpdateByManager( - descriptionBuilder.toString(), opArr, windowArr, inputSourceArr, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java index 140de6a06eb..9ab3cc981b8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java @@ -7,17 +7,13 @@ import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.table.ChunkSource; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.ModifiedColumnSet; -import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.InstrumentedTableUpdateListenerAdapter; import io.deephaven.engine.table.impl.QueryTable; import io.deephaven.engine.table.impl.TableUpdateImpl; import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.sources.ReinterpretUtils; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; -import io.deephaven.engine.table.impl.util.RowRedirection; import io.deephaven.engine.updategraph.UpdateCommitter; import io.deephaven.util.SafeCloseableArray; import io.deephaven.util.datastructures.linked.IntrusiveDoublyLinkedNode; @@ -26,6 +22,7 @@ import org.jetbrains.annotations.Nullable; import java.util.Map; +import java.util.function.BiConsumer; import static io.deephaven.util.QueryConstants.NULL_LONG; @@ -35,30 +32,28 @@ */ class UpdateByBucketHelper extends IntrusiveDoublyLinkedNode.Impl { private static final int SSA_LEAF_SIZE = 4096; - protected final ColumnSource[] inputSources; - // some columns will have multiple inputs, such as time-based and Weighted computations - final UpdateByOperator[] operators; - final UpdateByWindow[] windows; - final QueryTable source; - final RowRedirection rowRedirection; - final UpdateByControl control; + private final ColumnSource[] inputSources; + private final UpdateByWindow[] windows; + private final QueryTable source; + private final UpdateByControl control; + private final BiConsumer failureNotifier; final QueryTable result; /** An array of {@link UpdateByWindow.UpdateByWindowBucketContext}s for each window */ final UpdateByWindow.UpdateByWindowBucketContext[] windowContexts; /** store timestamp data in an SSA (if needed) */ - final String timestampColumnName; - final LongSegmentedSortedArray timestampSsa; - final ColumnSource timestampColumnSource; - final ModifiedColumnSet timestampColumnSet; + private final String timestampColumnName; + private final LongSegmentedSortedArray timestampSsa; + private final ColumnSource timestampColumnSource; + private final ModifiedColumnSet timestampColumnSet; /** Indicates this bucket needs to be processed (at least one window and operator are dirty) */ - boolean isDirty; + private boolean isDirty; /** This rowset will store row keys where the timestamp is not null (will mirror the SSA contents) */ - TrackingRowSet timestampValidRowSet; + private TrackingRowSet timestampValidRowSet; /** Track how many rows in this bucket have NULL timestamps */ - long nullTimestampCount; + private long nullTimestampCount; // TODO: remove these data collection entries when bug-hunt complete public UpdateCommitter committer; @@ -69,32 +64,30 @@ class UpdateByBucketHelper extends IntrusiveDoublyLinkedNode.Impl[] inputSources, @NotNull final Map> resultSources, @Nullable String timestampColumnName, - @Nullable final RowRedirection rowRedirection, - @NotNull final UpdateByControl control) { + @NotNull final UpdateByControl control, + @NotNull final BiConsumer failureNotifier) { this.source = source; - this.operators = operators; + // some columns will have multiple inputs, such as time-based and Weighted computations this.windows = windows; this.inputSources = inputSources; - this.rowRedirection = rowRedirection; this.control = control; + this.failureNotifier = failureNotifier; result = new QueryTable(source.getRowSet(), resultSources); @@ -391,9 +384,9 @@ public void finalizeUpdate() { * prepares this bucket for processing. This includes determination of `isDirty` status and the computation of * `affected` and `influencer` row sets for this processing cycle. */ - class UpdateByBucketHelperListener extends InstrumentedTableUpdateListenerAdapter { - public UpdateByBucketHelperListener(@Nullable String description, - @NotNull final QueryTable source) { + private class UpdateByBucketHelperListener extends InstrumentedTableUpdateListenerAdapter { + + private UpdateByBucketHelperListener(@Nullable final String description, @NotNull final QueryTable source) { super(description, source, false); } @@ -411,5 +404,10 @@ public void onUpdate(TableUpdate upstream) { }); UpdateByBucketHelper.this.committer.maybeActivate(); } + + @Override + public void onFailure(@NotNull final Throwable originalException, @Nullable final Entry sourceEntry) { + failureNotifier.accept(originalException, sourceEntry); + } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java index ccb6870ef08..d926df94eb8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java @@ -23,7 +23,6 @@ public class ZeroKeyUpdateByManager extends UpdateBy { /** * Perform an updateBy without any key columns. * - * @param description the operation description * @param operators the operations to perform * @param windows the unique windows for this UpdateBy * @param inputSources the primitive input sources @@ -34,7 +33,6 @@ public class ZeroKeyUpdateByManager extends UpdateBy { * @param control the control object. */ protected ZeroKeyUpdateByManager( - @NotNull final String description, @NotNull UpdateByOperator[] operators, @NotNull UpdateByWindow[] windows, @NotNull ColumnSource[] inputSources, @@ -45,12 +43,13 @@ protected ZeroKeyUpdateByManager( @Nullable WritableRowRedirection rowRedirection, @NotNull UpdateByControl control) { super(source, operators, windows, inputSources, timestampColumnName, rowRedirection, control); + final String bucketDescription = this + "-bucket-[]"; if (source.isRefreshing()) { result = new QueryTable(source.getRowSet(), resultSources); // this is a refreshing source, we will need a listener - listener = newUpdateByListener(description); + listener = newUpdateByListener(); source.addUpdateListener(listener); // result will depend on listener result.addParentReference(listener); @@ -62,8 +61,8 @@ protected ZeroKeyUpdateByManager( } // create an updateby bucket instance directly from the source table - zeroKeyUpdateBy = new UpdateByBucketHelper(description, source, operators, windows, inputSources, - resultSources, timestampColumnName, rowRedirection, control); + zeroKeyUpdateBy = new UpdateByBucketHelper(bucketDescription, source, windows, inputSources, + resultSources, timestampColumnName, control, (oe, se) -> deliverUpdateError(oe, se, true)); buckets.offer(zeroKeyUpdateBy); // make the source->result transformer @@ -72,8 +71,10 @@ protected ZeroKeyUpdateByManager( // result will depend on zeroKeyUpdateBy result.addParentReference(zeroKeyUpdateBy.result); } else { - zeroKeyUpdateBy = new UpdateByBucketHelper(description, source, operators, windows, inputSources, - resultSources, timestampColumnName, rowRedirection, control); + zeroKeyUpdateBy = new UpdateByBucketHelper(bucketDescription, source, windows, inputSources, + resultSources, timestampColumnName, control, (oe, se) -> { + throw new IllegalStateException("Update failure from static zero key updateBy"); + }); result = zeroKeyUpdateBy.result; buckets.offer(zeroKeyUpdateBy); } From dfe50f00e94587bb68decc4808aa61f9bd7e7ce5 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 31 Jan 2023 12:03:01 -0800 Subject: [PATCH 109/123] UpdateBy Operator unification --- .../updateby/UpdateByCumulativeOperator.java | 64 ------ .../table/impl/updateby/UpdateByOperator.java | 96 ++++++-- .../updateby/UpdateByOperatorFactory.java | 79 ++++--- .../table/impl/updateby/UpdateByWindow.java | 16 +- .../updateby/UpdateByWindowCumulative.java | 18 +- .../impl/updateby/UpdateByWindowTicks.java | 9 +- .../impl/updateby/UpdateByWindowTime.java | 9 +- .../updateby/UpdateByWindowedOperator.java | 57 ----- .../ema/BasePrimitiveEMAOperator.java | 21 +- .../updateby/ema/BigDecimalEMAOperator.java | 15 +- .../updateby/ema/BigIntegerEMAOperator.java | 21 +- .../updateby/ema/BigNumberEMAOperator.java | 19 +- .../impl/updateby/ema/ByteEMAOperator.java | 21 +- .../impl/updateby/ema/DoubleEMAOperator.java | 21 +- .../impl/updateby/ema/FloatEMAOperator.java | 21 +- .../impl/updateby/ema/IntEMAOperator.java | 21 +- .../impl/updateby/ema/LongEMAOperator.java | 21 +- .../impl/updateby/ema/ShortEMAOperator.java | 21 +- .../updateby/fill/BooleanFillByOperator.java | 5 +- .../updateby/fill/ByteFillByOperator.java | 5 +- .../updateby/fill/CharFillByOperator.java | 5 +- .../updateby/fill/DoubleFillByOperator.java | 5 +- .../updateby/fill/FloatFillByOperator.java | 5 +- .../impl/updateby/fill/IntFillByOperator.java | 5 +- .../updateby/fill/LongFillByOperator.java | 5 +- .../updateby/fill/ObjectFillByOperator.java | 5 +- .../updateby/fill/ShortFillByOperator.java | 5 +- .../internal/BaseByteUpdateByOperator.java | 81 +++++-- .../internal/BaseCharUpdateByOperator.java | 81 +++++-- .../internal/BaseDoubleUpdateByOperator.java | 81 +++++-- .../internal/BaseFloatUpdateByOperator.java | 81 +++++-- .../internal/BaseIntUpdateByOperator.java | 81 +++++-- .../internal/BaseLongUpdateByOperator.java | 81 +++++-- .../internal/BaseObjectBinaryOperator.java | 12 +- .../internal/BaseObjectUpdateByOperator.java | 81 +++++-- .../internal/BaseShortUpdateByOperator.java | 81 +++++-- .../BaseWindowedByteUpdateByOperator.java | 208 ------------------ .../BaseWindowedCharUpdateByOperator.java | 180 --------------- .../BaseWindowedDoubleUpdateByOperator.java | 185 ---------------- .../BaseWindowedFloatUpdateByOperator.java | 185 ---------------- .../BaseWindowedIntUpdateByOperator.java | 185 ---------------- .../BaseWindowedLongUpdateByOperator.java | 185 ---------------- .../BaseWindowedObjectUpdateByOperator.java | 195 ---------------- .../BaseWindowedShortUpdateByOperator.java | 185 ---------------- .../minmax/ByteCumMinMaxOperator.java | 5 +- .../minmax/ComparableCumMinMaxOperator.java | 8 +- .../minmax/DoubleCumMinMaxOperator.java | 5 +- .../minmax/FloatCumMinMaxOperator.java | 5 +- .../updateby/minmax/IntCumMinMaxOperator.java | 5 +- .../minmax/LongCumMinMaxOperator.java | 5 +- .../minmax/ShortCumMinMaxOperator.java | 5 +- .../prod/BigDecimalCumProdOperator.java | 5 +- .../prod/BigIntegerCumProdOperator.java | 6 +- .../updateby/prod/ByteCumProdOperator.java | 5 +- .../updateby/prod/DoubleCumProdOperator.java | 7 +- .../updateby/prod/FloatCumProdOperator.java | 7 +- .../updateby/prod/IntCumProdOperator.java | 5 +- .../updateby/prod/LongCumProdOperator.java | 5 +- .../updateby/prod/ShortCumProdOperator.java | 5 +- .../BigDecimalRollingSumOperator.java | 15 +- .../BigIntegerRollingSumOperator.java | 15 +- .../rollingsum/ByteRollingSumOperator.java | 15 +- .../rollingsum/DoubleRollingSumOperator.java | 15 +- .../rollingsum/FloatRollingSumOperator.java | 15 +- .../rollingsum/IntRollingSumOperator.java | 15 +- .../rollingsum/LongRollingSumOperator.java | 15 +- .../rollingsum/ShortRollingSumOperator.java | 15 +- .../sum/BigDecimalCumSumOperator.java | 6 +- .../sum/BigIntegerCumSumOperator.java | 5 +- .../impl/updateby/sum/ByteCumSumOperator.java | 5 +- .../updateby/sum/DoubleCumSumOperator.java | 5 +- .../updateby/sum/FloatCumSumOperator.java | 5 +- .../impl/updateby/sum/IntCumSumOperator.java | 5 +- .../impl/updateby/sum/LongCumSumOperator.java | 5 +- .../updateby/sum/ShortCumSumOperator.java | 5 +- .../replicators/ReplicateUpdateBy.java | 25 +-- 76 files changed, 920 insertions(+), 2112 deletions(-) delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByCumulativeOperator.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowedOperator.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByCumulativeOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByCumulativeOperator.java deleted file mode 100644 index 97f235f0e2a..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByCumulativeOperator.java +++ /dev/null @@ -1,64 +0,0 @@ -package io.deephaven.engine.table.impl.updateby; - -import io.deephaven.chunk.Chunk; -import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.util.RowRedirection; -import io.deephaven.util.annotations.FinalDefault; -import org.jetbrains.annotations.NotNull; - -import javax.annotation.Nullable; - -public abstract class UpdateByCumulativeOperator extends UpdateByOperator { - public abstract static class Context implements UpdateContext { - /** Holds the chunks of input data for use by the accumulate call */ - public final Chunk[] chunkArr; - - public Context(int chunkCount) { - chunkArr = new Chunk[chunkCount]; - } - - public boolean isValueValid(long atKey) { - throw new UnsupportedOperationException( - "isValueValid() must be overridden by time-aware cumulative operators"); - } - - @Override - public void close() {} - - @FinalDefault - public void pop(int count) { - throw new UnsupportedOperationException("Cumulative operators should never call pop()"); - } - - public abstract void accumulate(RowSequence inputKeys, - Chunk[] valueChunkArr, - LongChunk tsChunk, - int len); - } - - /** - * An operator that computes a cumulative operation from a column. The operation may be time or ticks aware (e.g. - * EMA) and timestamp column name and time units (ticks or nanoseconds) may optionally be provided - * - * @param pair the {@link MatchPair} that defines the input/output for this operation - * @param affectingColumns the names of the columns that affect this operation - * @param rowRedirection the row redirection context to use for the operation - */ - protected UpdateByCumulativeOperator(@NotNull final MatchPair pair, - @NotNull final String[] affectingColumns, - @Nullable final RowRedirection rowRedirection, - @Nullable final String timestampColumnName, - final long reverseWindowScaleUnits) { - super(pair, affectingColumns, timestampColumnName, reverseWindowScaleUnits, 0L, rowRedirection); - } - - - /** - * Initialize the bucket context for this cumulative operator - */ - public void initializeUpdate(@NotNull final UpdateContext context, final long firstUnmodifiedKey, - long firstUnmodifiedTimestamp) {} -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperator.java index 45f145cdfac..50423c36d2c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperator.java @@ -24,12 +24,12 @@ * interface, the pattern of calls will be as follows. * *
                - *
              1. {@link UpdateByCumulativeOperator#initializeUpdate(UpdateContext, long, long)} for cumulative operators or - * {@link UpdateByWindowedOperator#initializeUpdate(UpdateContext)} for windowed operators
              2. - *
              3. {@link UpdateByCumulativeOperator.Context#accumulate(RowSequence, Chunk[], LongChunk, int)} for cumulative - * operators or {@link UpdateByWindowedOperator.Context#accumulate(RowSequence, Chunk[], IntChunk, IntChunk, int)} for + *
              4. {@link UpdateByOperator#initializeCumulative(Context, long, long)} for cumulative operators or + * {@link UpdateByOperator#initializeRolling(Context)} (Context)} for windowed operators
              5. + *
              6. {@link UpdateByOperator.Context#accumulateCumulative(RowSequence, Chunk[], LongChunk, int)} for cumulative + * operators or {@link UpdateByOperator.Context#accumulateRolling(RowSequence, Chunk[], IntChunk, IntChunk, int)} for * windowed operators
              7. - *
              8. {@link #finishUpdate(UpdateContext)}
              9. + *
              10. {@link #finishUpdate(UpdateByOperator.Context)}
              11. *
              */ public abstract class UpdateByOperator { @@ -43,6 +43,8 @@ public abstract class UpdateByOperator { protected final long forwardWindowScaleUnits; protected final String timestampColumnName; + protected final boolean isWindowed; + /** * The input modifiedColumnSet for this operator */ @@ -55,9 +57,23 @@ public abstract class UpdateByOperator { /** * A context item for use with updateBy operators */ - public interface UpdateContext extends SafeCloseable { + public abstract class Context implements SafeCloseable { + protected final Chunk[] chunkArr; + protected int nullCount = 0; + + public Context(int chunkCount) { + chunkArr = new Chunk[chunkCount]; + } + + public boolean isValueValid(long atKey) { + throw new UnsupportedOperationException( + "isValueValid() must be overridden by time-aware cumulative operators"); + } + + @Override + public void close() {} - void setValuesChunk(@NotNull Chunk valuesChunk); + protected abstract void setValuesChunk(@NotNull Chunk valuesChunk); /** * Add values to the operators current data set @@ -68,7 +84,7 @@ public interface UpdateContext extends SafeCloseable { * of the operator to pull the data from the chunk and use it properly * @param count the number of items to push from the chunk */ - void push(long key, int pos, int count); + protected abstract void push(long key, int pos, int count); /** * Remove values from the operators current data set. This is only valid for windowed operators as cumulative @@ -76,42 +92,68 @@ public interface UpdateContext extends SafeCloseable { * * @param count the number of items to pop from the data set */ - void pop(int count); + protected void pop(int count) { + throw new UnsupportedOperationException("pop() must be overriden by rolling operators"); + } + + public abstract void accumulateCumulative(RowSequence inputKeys, + Chunk[] valueChunkArr, + LongChunk tsChunk, + int len); + + public abstract void accumulateRolling(RowSequence inputKeys, + Chunk influencerValueChunkArr[], + IntChunk pushChunk, + IntChunk popChunk, + int len); /** * Write the current value for this row to the output chunk */ - void writeToOutputChunk(int outIdx); + protected abstract void writeToOutputChunk(int outIdx); + /** - * Reset the operator data values to a known state. This may occur during initialization or when a windowed - * operator has an empty window + * Write the output chunk to the output column */ - - @OverridingMethodsMustInvokeSuper - void reset(); + protected abstract void writeToOutputColumn(@NotNull final RowSequence inputKeys); /** - * Write the output chunk to the output column + * Reset the operator data values to a known state. This may occur during initialization or when a windowed + * operator has an empty window */ - void writeToOutputColumn(@NotNull final RowSequence inputKeys); + @OverridingMethodsMustInvokeSuper + protected abstract void reset(); } protected UpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, + @Nullable final RowRedirection rowRedirection, @Nullable final String timestampColumnName, final long reverseWindowScaleUnits, final long forwardWindowScaleUnits, - @Nullable final RowRedirection rowRedirection) { + final boolean isWindowed) { this.pair = pair; this.affectingColumns = affectingColumns; this.rowRedirection = rowRedirection; this.timestampColumnName = timestampColumnName; this.reverseWindowScaleUnits = reverseWindowScaleUnits; this.forwardWindowScaleUnits = forwardWindowScaleUnits; + this.isWindowed = isWindowed; } + /** + * Initialize the bucket context for a cumulative operator + */ + public void initializeCumulative(@NotNull final Context context, final long firstUnmodifiedKey, + long firstUnmodifiedTimestamp) {} + + /** + * Initialize the bucket context for s windowed operator + */ + public void initializeRolling(@NotNull final Context context) {} + /** * Get the names of the input column(s) for this operator. * @@ -174,28 +216,38 @@ protected String[] getOutputColumnNames() { @NotNull protected abstract Map> getOutputColumns(); + /** + * Whether this operator supports windows (is rolling operator) + * + * @return true if the operator is windowed, false if cumulative + */ + @NotNull + protected boolean getIsWindowed() { + return isWindowed; + } + /** * Indicate that the operation should start tracking previous values for ticking updates. */ protected abstract void startTrackingPrev(); /** - * Make an {@link UpdateContext} suitable for use with updates. + * Make an {@link Context} suitable for use with updates. * * @param chunkSize The expected size of chunks that will be provided during the update, * @param chunkCount The number of chunks that will be provided during the update, * @return a new context */ @NotNull - public abstract UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount); + public abstract Context makeUpdateContext(final int chunkSize, final int chunkCount); /** * Perform any bookkeeping required at the end of a single part of the update. This is always preceded with a call - * to {@code #initializeUpdate(UpdateContext)} (specialized for each type of operator) + * to {@code #initializeUpdate(Context)} (specialized for each type of operator) * * @param context the context object */ - protected void finishUpdate(@NotNull final UpdateContext context) {} + protected void finishUpdate(@NotNull final Context context) {} /** * Apply a shift to the operation. diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperatorFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperatorFactory.java index 78dff2734ce..f301638d88f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperatorFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperatorFactory.java @@ -229,29 +229,29 @@ private UpdateByOperator makeEmaOperator(@NotNull final MatchPair pair, final OperationControl control = ema.controlOrDefault(); if (csType == byte.class || csType == Byte.class) { - return new ByteEMAOperator(pair, affectingColumns, control, - ema.timeScale().timestampCol(), timeScaleUnits, rowRedirection, columnSource); + return new ByteEMAOperator(pair, affectingColumns, rowRedirection, control, + ema.timeScale().timestampCol(), timeScaleUnits, columnSource); } else if (csType == short.class || csType == Short.class) { - return new ShortEMAOperator(pair, affectingColumns, control, - ema.timeScale().timestampCol(), timeScaleUnits, rowRedirection, columnSource); + return new ShortEMAOperator(pair, affectingColumns, rowRedirection, control, + ema.timeScale().timestampCol(), timeScaleUnits, columnSource); } else if (csType == int.class || csType == Integer.class) { - return new IntEMAOperator(pair, affectingColumns, control, - ema.timeScale().timestampCol(), timeScaleUnits, rowRedirection, columnSource); + return new IntEMAOperator(pair, affectingColumns, rowRedirection, control, + ema.timeScale().timestampCol(), timeScaleUnits, columnSource); } else if (csType == long.class || csType == Long.class) { - return new LongEMAOperator(pair, affectingColumns, control, - ema.timeScale().timestampCol(), timeScaleUnits, rowRedirection, columnSource); + return new LongEMAOperator(pair, affectingColumns, rowRedirection, control, + ema.timeScale().timestampCol(), timeScaleUnits, columnSource); } else if (csType == float.class || csType == Float.class) { - return new FloatEMAOperator(pair, affectingColumns, control, - ema.timeScale().timestampCol(), timeScaleUnits, rowRedirection, columnSource); + return new FloatEMAOperator(pair, affectingColumns, rowRedirection, control, + ema.timeScale().timestampCol(), timeScaleUnits, columnSource); } else if (csType == double.class || csType == Double.class) { - return new DoubleEMAOperator(pair, affectingColumns, control, - ema.timeScale().timestampCol(), timeScaleUnits, rowRedirection, columnSource); + return new DoubleEMAOperator(pair, affectingColumns, rowRedirection, control, + ema.timeScale().timestampCol(), timeScaleUnits, columnSource); } else if (csType == BigDecimal.class) { - return new BigDecimalEMAOperator(pair, affectingColumns, control, - ema.timeScale().timestampCol(), timeScaleUnits, rowRedirection, columnSource); + return new BigDecimalEMAOperator(pair, affectingColumns, rowRedirection, control, + ema.timeScale().timestampCol(), timeScaleUnits, columnSource); } else if (csType == BigInteger.class) { - return new BigIntegerEMAOperator(pair, affectingColumns, control, - ema.timeScale().timestampCol(), timeScaleUnits, rowRedirection, columnSource); + return new BigIntegerEMAOperator(pair, affectingColumns, rowRedirection, control, + ema.timeScale().timestampCol(), timeScaleUnits, columnSource); } throw new IllegalArgumentException("Can not perform EMA on type " + csType); @@ -297,7 +297,7 @@ private UpdateByOperator makeCumMinMaxOperator(MatchPair fc, TableDefaults sourc return new DoubleCumMinMaxOperator(fc, isMax, rowRedirection); } else if (Comparable.class.isAssignableFrom(csType)) { // noinspection rawtypes - return new ComparableCumMinMaxOperator(csType, fc, isMax, rowRedirection); + return new ComparableCumMinMaxOperator(fc, isMax, rowRedirection, csType); } throw new IllegalArgumentException("Can not perform Cumulative Min/Max on type " + csType); @@ -370,32 +370,41 @@ private UpdateByOperator makeRollingSumOperator(@NotNull final MatchPair pair, final long fwdTimeScaleUnits = rs.fwdTimeScale().timescaleUnits(); if (csType == Boolean.class || csType == boolean.class) { - return new ByteRollingSumOperator(pair, affectingColumns, rs.revTimeScale().timestampCol(), - prevTimeScaleUnits, fwdTimeScaleUnits, rowRedirection, NULL_BOOLEAN_AS_BYTE); + return new ByteRollingSumOperator(pair, affectingColumns, rowRedirection, + rs.revTimeScale().timestampCol(), + prevTimeScaleUnits, fwdTimeScaleUnits, NULL_BOOLEAN_AS_BYTE); } else if (csType == byte.class || csType == Byte.class) { - return new ByteRollingSumOperator(pair, affectingColumns, rs.revTimeScale().timestampCol(), - prevTimeScaleUnits, fwdTimeScaleUnits, rowRedirection, NULL_BYTE); + return new ByteRollingSumOperator(pair, affectingColumns, rowRedirection, + rs.revTimeScale().timestampCol(), + prevTimeScaleUnits, fwdTimeScaleUnits, NULL_BYTE); } else if (csType == short.class || csType == Short.class) { - return new ShortRollingSumOperator(pair, affectingColumns, rs.revTimeScale().timestampCol(), - prevTimeScaleUnits, fwdTimeScaleUnits, rowRedirection); + return new ShortRollingSumOperator(pair, affectingColumns, rowRedirection, + rs.revTimeScale().timestampCol(), + prevTimeScaleUnits, fwdTimeScaleUnits); } else if (csType == int.class || csType == Integer.class) { - return new IntRollingSumOperator(pair, affectingColumns, rs.revTimeScale().timestampCol(), - prevTimeScaleUnits, fwdTimeScaleUnits, rowRedirection); + return new IntRollingSumOperator(pair, affectingColumns, rowRedirection, + rs.revTimeScale().timestampCol(), + prevTimeScaleUnits, fwdTimeScaleUnits); } else if (csType == long.class || csType == Long.class) { - return new LongRollingSumOperator(pair, affectingColumns, rs.revTimeScale().timestampCol(), - prevTimeScaleUnits, fwdTimeScaleUnits, rowRedirection); + return new LongRollingSumOperator(pair, affectingColumns, rowRedirection, + rs.revTimeScale().timestampCol(), + prevTimeScaleUnits, fwdTimeScaleUnits); } else if (csType == float.class || csType == Float.class) { - return new FloatRollingSumOperator(pair, affectingColumns, rs.revTimeScale().timestampCol(), - prevTimeScaleUnits, fwdTimeScaleUnits, rowRedirection); + return new FloatRollingSumOperator(pair, affectingColumns, rowRedirection, + rs.revTimeScale().timestampCol(), + prevTimeScaleUnits, fwdTimeScaleUnits); } else if (csType == double.class || csType == Double.class) { - return new DoubleRollingSumOperator(pair, affectingColumns, rs.revTimeScale().timestampCol(), - prevTimeScaleUnits, fwdTimeScaleUnits, rowRedirection); + return new DoubleRollingSumOperator(pair, affectingColumns, rowRedirection, + rs.revTimeScale().timestampCol(), + prevTimeScaleUnits, fwdTimeScaleUnits); } else if (csType == BigDecimal.class) { - return new BigDecimalRollingSumOperator(pair, affectingColumns, rs.revTimeScale().timestampCol(), - prevTimeScaleUnits, fwdTimeScaleUnits, rowRedirection, control.mathContextOrDefault()); + return new BigDecimalRollingSumOperator(pair, affectingColumns, rowRedirection, + rs.revTimeScale().timestampCol(), + prevTimeScaleUnits, fwdTimeScaleUnits, control.mathContextOrDefault()); } else if (csType == BigInteger.class) { - return new BigIntegerRollingSumOperator(pair, affectingColumns, rs.revTimeScale().timestampCol(), - prevTimeScaleUnits, fwdTimeScaleUnits, rowRedirection); + return new BigIntegerRollingSumOperator(pair, affectingColumns, rowRedirection, + rs.revTimeScale().timestampCol(), + prevTimeScaleUnits, fwdTimeScaleUnits); } throw new IllegalArgumentException("Can not perform RollingSum on type " + csType); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java index a864c906572..b2f50012659 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindow.java @@ -49,7 +49,7 @@ class UpdateByWindowBucketContext implements SafeCloseable { /** Were any timestamps modified in the current update? */ protected final boolean timestampsModified; /** An array of context objects for each underlying operator */ - protected final UpdateByOperator.UpdateContext[] opContexts; + protected final UpdateByOperator.Context[] opContexts; /** Whether this is the creation phase of this window */ protected final boolean initialStep; @@ -87,7 +87,7 @@ class UpdateByWindowBucketContext implements SafeCloseable { this.timestampValidRowSet = timestampValidRowSet; this.timestampsModified = timestampsModified; - this.opContexts = new UpdateByOperator.UpdateContext[operators.length]; + this.opContexts = new UpdateByOperator.Context[operators.length]; this.workingChunkSize = chunkSize; this.initialStep = initialStep; @@ -145,8 +145,7 @@ static UpdateByWindow createFromOperatorArray(final UpdateByOperator[] operators } // return the correct type of UpdateByWindow - final boolean windowed = operators[0] instanceof UpdateByWindowedOperator; - if (!windowed) { + if (!operators[0].isWindowed) { return new UpdateByWindowCumulative(operators, operatorSourceSlots, timestampColumnName); @@ -377,7 +376,7 @@ private static int hashCode(boolean windowed, @NotNull String[] inputColumnNames * Returns a hash code given a particular operator */ static int hashCodeFromOperator(final UpdateByOperator op) { - return hashCode(op instanceof UpdateByWindowedOperator, + return hashCode(op.isWindowed, op.getInputColumnNames(), op.getTimestampColumnName(), op.getPrevWindowUnits(), @@ -393,13 +392,10 @@ static boolean isEquivalentWindow(final UpdateByOperator opA, final UpdateByOper return false; } - final boolean aWindowed = opA instanceof UpdateByWindowedOperator; - final boolean bWindowed = opB instanceof UpdateByWindowedOperator; - // equivalent if both are cumulative, not equivalent if only one is cumulative - if (!aWindowed && !bWindowed) { + if (!opA.isWindowed && !opB.isWindowed) { return true; - } else if (aWindowed != bWindowed) { + } else if (opA.isWindowed != opB.isWindowed) { return false; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java index 7d0bcb56951..64c5ec14127 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowCumulative.java @@ -105,8 +105,8 @@ void processRows(UpdateByWindowBucketContext context, final boolean initialStep) if (initialStep) { // always at the beginning of the RowSet at creation phase for (int opIdx : context.dirtyOperatorIndices) { - UpdateByCumulativeOperator cumOp = (UpdateByCumulativeOperator) operators[opIdx]; - cumOp.initializeUpdate(context.opContexts[opIdx], NULL_ROW_KEY, NULL_LONG); + UpdateByOperator cumOp = operators[opIdx]; + cumOp.initializeCumulative(context.opContexts[opIdx], NULL_ROW_KEY, NULL_LONG); } } else { // find the key before the first affected row @@ -115,15 +115,14 @@ void processRows(UpdateByWindowBucketContext context, final boolean initialStep) // and preload that data for these operators for (int opIdx : context.dirtyOperatorIndices) { - UpdateByCumulativeOperator cumOp = (UpdateByCumulativeOperator) operators[opIdx]; + UpdateByOperator cumOp = operators[opIdx]; if (cumOp.getTimestampColumnName() == null || keyBefore == NULL_ROW_KEY) { // this operator doesn't care about timestamps or we know we are at the beginning of the rowset - cumOp.initializeUpdate(context.opContexts[opIdx], keyBefore, NULL_LONG); + cumOp.initializeCumulative(context.opContexts[opIdx], keyBefore, NULL_LONG); } else { // this operator cares about timestamps, so make sure it is starting from a valid value and // valid timestamp by looking backward until the conditions are met - UpdateByCumulativeOperator.Context cumOpContext = - (UpdateByCumulativeOperator.Context) context.opContexts[opIdx]; + UpdateByOperator.Context cumOpContext = context.opContexts[opIdx]; long potentialResetTimestamp = context.timestampColumnSource.getLong(keyBefore); if (potentialResetTimestamp == NULL_LONG || !cumOpContext.isValueValid(keyBefore)) { @@ -141,7 +140,7 @@ void processRows(UpdateByWindowBucketContext context, final boolean initialStep) } } // call the specialized version of `intializeUpdate()` for these operators - cumOp.initializeUpdate(context.opContexts[opIdx], keyBefore, potentialResetTimestamp); + cumOp.initializeCumulative(context.opContexts[opIdx], keyBefore, potentialResetTimestamp); } } } @@ -160,8 +159,7 @@ void processRows(UpdateByWindowBucketContext context, final boolean initialStep) : context.timestampColumnSource.getChunk(tsGetCtx, rs).asLongChunk(); for (int opIdx : context.dirtyOperatorIndices) { - UpdateByCumulativeOperator.Context opCtx = - (UpdateByCumulativeOperator.Context) context.opContexts[opIdx]; + UpdateByOperator.Context opCtx = context.opContexts[opIdx]; // prep the chunk array needed by the accumulate call final int[] srcIndices = operatorInputSourceSlots[opIdx]; for (int ii = 0; ii < srcIndices.length; ii++) { @@ -172,7 +170,7 @@ void processRows(UpdateByWindowBucketContext context, final boolean initialStep) } // make the specialized call for cumulative operators - ((UpdateByCumulativeOperator.Context) context.opContexts[opIdx]).accumulate( + context.opContexts[opIdx].accumulateCumulative( rs, opCtx.chunkArr, tsChunk, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java index c457f49c738..9d0347a7001 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTicks.java @@ -283,9 +283,9 @@ void processRows(UpdateByWindowBucketContext context, boolean initialStep) { Assert.neqNull(context.inputSources, "assignInputSources() must be called before processRow()"); for (int opIdx : context.dirtyOperatorIndices) { - UpdateByWindowedOperator winOp = (UpdateByWindowedOperator) operators[opIdx]; + UpdateByOperator winOp = operators[opIdx]; // call the specialized version of `intializeUpdate()` for these operators - winOp.initializeUpdate(ctx.opContexts[opIdx]); + winOp.initializeRolling(ctx.opContexts[opIdx]); } try (final RowSequence.Iterator it = ctx.affectedRows.getRowSequenceIterator(); @@ -338,8 +338,7 @@ void processRows(UpdateByWindowBucketContext context, boolean initialStep) { Arrays.fill(ctx.inputSourceChunks, null); for (int opIdx : context.dirtyOperatorIndices) { - UpdateByWindowedOperator.Context opCtx = - (UpdateByWindowedOperator.Context) context.opContexts[opIdx]; + UpdateByOperator.Context opCtx = context.opContexts[opIdx]; // prep the chunk array needed by the accumulate call final int[] srcIndices = operatorInputSourceSlots[opIdx]; for (int ii = 0; ii < srcIndices.length; ii++) { @@ -350,7 +349,7 @@ void processRows(UpdateByWindowBucketContext context, boolean initialStep) { } // make the specialized call for windowed operators - ((UpdateByWindowedOperator.Context) ctx.opContexts[opIdx]).accumulate( + ctx.opContexts[opIdx].accumulateRolling( chunkRs, opCtx.chunkArr, pushChunk, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java index 8eaf55da6c1..71d050493e3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowTime.java @@ -316,9 +316,9 @@ public void processRows(UpdateByWindowBucketContext context, boolean initialStep UpdateByWindowTimeBucketContext ctx = (UpdateByWindowTimeBucketContext) context; for (int opIdx : context.dirtyOperatorIndices) { - UpdateByWindowedOperator winOp = (UpdateByWindowedOperator) operators[opIdx]; + UpdateByOperator winOp = operators[opIdx]; // call the specialized version of `intializeUpdate()` for these operators - winOp.initializeUpdate(ctx.opContexts[opIdx]); + winOp.initializeRolling(ctx.opContexts[opIdx]); } try (final RowSequence.Iterator affectedRowsIt = ctx.affectedRows.getRowSequenceIterator(); @@ -407,8 +407,7 @@ public void processRows(UpdateByWindowBucketContext context, boolean initialStep Arrays.fill(ctx.inputSourceChunks, null); for (int opIdx : context.dirtyOperatorIndices) { - UpdateByWindowedOperator.Context opCtx = - (UpdateByWindowedOperator.Context) context.opContexts[opIdx]; + UpdateByOperator.Context opCtx = context.opContexts[opIdx]; // prep the chunk array needed by the accumulate call final int[] srcIndices = operatorInputSourceSlots[opIdx]; for (int ii = 0; ii < srcIndices.length; ii++) { @@ -419,7 +418,7 @@ public void processRows(UpdateByWindowBucketContext context, boolean initialStep } // make the specialized call for windowed operators - ((UpdateByWindowedOperator.Context) ctx.opContexts[opIdx]).accumulate( + ctx.opContexts[opIdx].accumulateRolling( chunkAffectedRows, opCtx.chunkArr, pushChunk, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowedOperator.java deleted file mode 100644 index a61aa82de21..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowedOperator.java +++ /dev/null @@ -1,57 +0,0 @@ -package io.deephaven.engine.table.impl.updateby; - -import io.deephaven.chunk.Chunk; -import io.deephaven.chunk.IntChunk; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.util.RowRedirection; -import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; - -public abstract class UpdateByWindowedOperator extends UpdateByOperator { - - public abstract static class Context implements UpdateContext { - /** Holds the chunks of input data for use by the accumulate call */ - protected final Chunk[] chunkArr; - protected int nullCount = 0; - - public Context(int chunkCount) { - chunkArr = new Chunk[chunkCount]; - } - - @Override - public void close() {} - - public abstract void accumulate(RowSequence inputKeys, - Chunk influencerValueChunkArr[], - IntChunk pushChunk, - IntChunk popChunk, - int len); - } - - /** - * An operator that computes a windowed operation from a column - * - * @param pair the {@link MatchPair} that defines the input/output for this operation - * @param affectingColumns the names of the columns that affect this operation - * @param timestampColumnName the optional time stamp column for windowing (uses ticks if not provided) - * @param reverseWindowScaleUnits the time (us) or ticks to extend the window backwards - * @param forwardWindowScaleUnits the time (us) or ticks to extend the window forwards - * @param rowRedirection the row redirection to use for the operator output columns - */ - protected UpdateByWindowedOperator(@NotNull final MatchPair pair, - @NotNull final String[] affectingColumns, - @Nullable final String timestampColumnName, - final long reverseWindowScaleUnits, - final long forwardWindowScaleUnits, - @Nullable final RowRedirection rowRedirection) { - super(pair, affectingColumns, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, - rowRedirection); - } - - /** - * Initialize the bucket context for this windowed operator - */ - public void initializeUpdate(@NotNull final UpdateContext context) {} -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java index bd0bc50c609..5cecd6fdaeb 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BasePrimitiveEMAOperator.java @@ -6,13 +6,12 @@ import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.locations.TableDataException; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseDoubleUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; -import static io.deephaven.util.QueryConstants.NULL_DOUBLE; import static io.deephaven.util.QueryConstants.NULL_LONG; public abstract class BasePrimitiveEMAOperator extends BaseDoubleUpdateByOperator { @@ -21,8 +20,6 @@ public abstract class BasePrimitiveEMAOperator extends BaseDoubleUpdateByOperato protected double oneMinusAlpha; public abstract class Context extends BaseDoubleUpdateByOperator.Context { - public LongChunk timestampValueChunk; - long lastStamp = NULL_LONG; Context(final int chunkSize, final int chunkCount) { @@ -41,32 +38,32 @@ public void reset() { * * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this ema + * @param rowRedirection the row redirection to use for the EMA output columns * @param control the control parameters for EMA * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is + * @param windowScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is * measured in ticks, otherwise it is measured in nanoseconds. - * @param rowRedirection the row redirection to use for the EMA output columns */ public BasePrimitiveEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, + @Nullable final RowRedirection rowRedirection, @NotNull final OperationControl control, @Nullable final String timestampColumnName, - final long timeScaleUnits, - @Nullable final RowRedirection rowRedirection) { - super(pair, affectingColumns, rowRedirection, timestampColumnName, timeScaleUnits); + final long windowScaleUnits) { + super(pair, affectingColumns, rowRedirection, timestampColumnName, windowScaleUnits, 0, false); this.control = control; - alpha = Math.exp(-1.0 / (double) timeScaleUnits); + alpha = Math.exp(-1.0 / (double) windowScaleUnits); oneMinusAlpha = 1 - alpha; } @Override - public void initializeUpdate(@NotNull final UpdateContext updateContext, + public void initializeCumulative(@NotNull final UpdateByOperator.Context updateContext, final long firstUnmodifiedKey, final long firstUnmodifiedTimestamp) { - super.initializeUpdate(updateContext, firstUnmodifiedKey, firstUnmodifiedTimestamp); + super.initializeCumulative(updateContext, firstUnmodifiedKey, firstUnmodifiedTimestamp); final Context ctx = (Context) updateContext; // rely on the caller to validate this is a valid timestamp (or NULL_LONG when appropriate) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java index 39b456e36ff..2998214ed75 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigDecimalEMAOperator.java @@ -7,6 +7,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -22,7 +23,7 @@ protected Context(final int chunkSize, final int chunkCount) { } @Override - public void accumulate(RowSequence inputKeys, + public void accumulateCumulative(RowSequence inputKeys, Chunk[] valueChunkArr, LongChunk tsChunk, int len) { @@ -98,26 +99,26 @@ public void push(long key, int pos, int count) { * * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this ema + * @param rowRedirection the {@link RowRedirection} to use for dense output sources * @param control defines how to handle {@code null} input values. * @param timestampColumnName the name of the column containing timestamps for time-based calcuations - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is + * @param windowScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is * measured in ticks, otherwise it is measured in nanoseconds - * @param rowRedirection the {@link RowRedirection} to use for dense output sources * @param valueSource a reference to the input column source for this operation */ public BigDecimalEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, + @Nullable final RowRedirection rowRedirection, @NotNull final OperationControl control, @Nullable final String timestampColumnName, - final long timeScaleUnits, - @Nullable final RowRedirection rowRedirection, + final long windowScaleUnits, final ColumnSource valueSource) { - super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, rowRedirection, valueSource); + super(pair, affectingColumns, rowRedirection, control, timestampColumnName, windowScaleUnits, valueSource); } @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java index 3f3d7044947..aec9fa3525a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigIntegerEMAOperator.java @@ -7,6 +7,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -23,10 +24,10 @@ protected Context(final int chunkSize, final int chunkCount) { } @Override - public void accumulate(RowSequence inputKeys, - Chunk[] valueChunkArr, - LongChunk tsChunk, - int len) { + public void accumulateCumulative(RowSequence inputKeys, + Chunk[] valueChunkArr, + LongChunk tsChunk, + int len) { setValuesChunk(valueChunkArr[0]); // chunk processing @@ -101,25 +102,25 @@ public void push(long key, int pos, int count) { * * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this ema + * @param rowRedirection the {@link RowRedirection} to use for dense output sources * @param control defines how to handle {@code null} input values. * @param timestampColumnName the name of the column containing timestamps for time-based calcuations - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is measured in ticks, otherwise it is measured in nanoseconds - * @param rowRedirection the {@link RowRedirection} to use for dense output sources + * @param windowScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is measured in ticks, otherwise it is measured in nanoseconds * @param valueSource a reference to the input column source for this operation */ public BigIntegerEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, + @Nullable final RowRedirection rowRedirection, @NotNull final OperationControl control, @Nullable final String timestampColumnName, - final long timeScaleUnits, - @Nullable final RowRedirection rowRedirection, + final long windowScaleUnits, final ColumnSource valueSource) { - super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, rowRedirection, valueSource); + super(pair, affectingColumns, rowRedirection, control, timestampColumnName, windowScaleUnits, valueSource); } @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java index d27b7b51afa..e3d2c793e60 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java @@ -8,6 +8,7 @@ import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.locations.TableDataException; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -15,7 +16,6 @@ import java.math.BigDecimal; -import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; import static io.deephaven.util.QueryConstants.NULL_LONG; public abstract class BigNumberEMAOperator extends BaseObjectUpdateByOperator { @@ -60,28 +60,29 @@ public void reset() { * * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this ema + * @param rowRedirection the {@link RowRedirection} to use for dense output sources * @param control defines how to handle {@code null} input values. * @param timestampColumnName the name of the column containing timestamps for time-based calcuations - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is + * @param windowScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is * measured in ticks, otherwise it is measured in nanoseconds - * @param rowRedirection the {@link RowRedirection} to use for dense output sources * @param valueSource a reference to the input column source for this operation */ public BigNumberEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, + @Nullable final RowRedirection rowRedirection, @NotNull final OperationControl control, @Nullable final String timestampColumnName, - final long timeScaleUnits, - @Nullable final RowRedirection rowRedirection, + final long windowScaleUnits, final ColumnSource valueSource) { - super(pair, affectingColumns, rowRedirection, timestampColumnName, timeScaleUnits, BigDecimal.class); + super(pair, affectingColumns, rowRedirection, timestampColumnName, windowScaleUnits, 0, false, + BigDecimal.class); this.control = control; this.valueSource = valueSource; if (timestampColumnName == null) { // tick-based, pre-compute alpha and oneMinusAlpha - opAlpha = computeAlpha(-1, timeScaleUnits); + opAlpha = computeAlpha(-1, windowScaleUnits); opOneMinusAlpha = computeOneMinusAlpha(opAlpha); } else { // time-based, must compute alpha and oneMinusAlpha for each time delta @@ -91,10 +92,10 @@ public BigNumberEMAOperator(@NotNull final MatchPair pair, } @Override - public void initializeUpdate(@NotNull final UpdateContext updateContext, + public void initializeCumulative(@NotNull final UpdateByOperator.Context updateContext, final long firstUnmodifiedKey, final long firstUnmodifiedTimestamp) { - super.initializeUpdate(updateContext, firstUnmodifiedKey, firstUnmodifiedTimestamp); + super.initializeCumulative(updateContext, firstUnmodifiedKey, firstUnmodifiedTimestamp); final Context ctx = (Context) updateContext; // rely on the caller to validate this is a valid timestamp (or NULL_LONG when appropriate) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java index 03173f74a64..0d92001ef81 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ByteEMAOperator.java @@ -13,6 +13,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -31,10 +32,10 @@ protected Context(final int chunkSize, final int chunkCount) { } @Override - public void accumulate(RowSequence inputKeys, - Chunk[] valueChunkArr, - LongChunk tsChunk, - int len) { + public void accumulateCumulative(RowSequence inputKeys, + Chunk[] valueChunkArr, + LongChunk tsChunk, + int len) { setValuesChunk(valueChunkArr[0]); // chunk processing @@ -109,23 +110,23 @@ public void push(long key, int pos, int count) { * * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this ema + * @param rowRedirection the {@link RowRedirection} to use for dense output sources * @param control defines how to handle {@code null} input values. * @param timestampColumnName the name of the column containing timestamps for time-based calcuations - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is measured in ticks, otherwise it is measured in nanoseconds - * @param rowRedirection the {@link RowRedirection} to use for dense output sources + * @param windowScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is measured in ticks, otherwise it is measured in nanoseconds * @param valueSource a reference to the input column source for this operation */ public ByteEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, + @Nullable final RowRedirection rowRedirection, @NotNull final OperationControl control, @Nullable final String timestampColumnName, - final long timeScaleUnits, - @Nullable final RowRedirection rowRedirection, + final long windowScaleUnits, final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, rowRedirection); + super(pair, affectingColumns, rowRedirection, control, timestampColumnName, windowScaleUnits); this.valueSource = valueSource; // region constructor // endregion constructor @@ -133,7 +134,7 @@ public ByteEMAOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java index e96d8068050..5c65b4d7d07 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/DoubleEMAOperator.java @@ -14,6 +14,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -31,10 +32,10 @@ protected Context(final int chunkSize, final int chunkCount) { } @Override - public void accumulate(RowSequence inputKeys, - Chunk[] valueChunkArr, - LongChunk tsChunk, - int len) { + public void accumulateCumulative(RowSequence inputKeys, + Chunk[] valueChunkArr, + LongChunk tsChunk, + int len) { setValuesChunk(valueChunkArr[0]); // chunk processing @@ -117,23 +118,23 @@ public void push(long key, int pos, int count) { * * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this ema + * @param rowRedirection the {@link RowRedirection} to use for dense output sources * @param control defines how to handle {@code null} input values. * @param timestampColumnName the name of the column containing timestamps for time-based calcuations - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is measured in ticks, otherwise it is measured in nanoseconds - * @param rowRedirection the {@link RowRedirection} to use for dense output sources + * @param windowScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is measured in ticks, otherwise it is measured in nanoseconds * @param valueSource a reference to the input column source for this operation */ public DoubleEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, + @Nullable final RowRedirection rowRedirection, @NotNull final OperationControl control, @Nullable final String timestampColumnName, - final long timeScaleUnits, - @Nullable final RowRedirection rowRedirection, + final long windowScaleUnits, final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, rowRedirection); + super(pair, affectingColumns, rowRedirection, control, timestampColumnName, windowScaleUnits); this.valueSource = valueSource; // region constructor // endregion constructor @@ -141,7 +142,7 @@ public DoubleEMAOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java index 56728214d19..24bf958a214 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/FloatEMAOperator.java @@ -9,6 +9,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -26,10 +27,10 @@ protected Context(final int chunkSize, final int chunkCount) { } @Override - public void accumulate(RowSequence inputKeys, - Chunk[] valueChunkArr, - LongChunk tsChunk, - int len) { + public void accumulateCumulative(RowSequence inputKeys, + Chunk[] valueChunkArr, + LongChunk tsChunk, + int len) { setValuesChunk(valueChunkArr[0]); // chunk processing @@ -112,23 +113,23 @@ public void push(long key, int pos, int count) { * * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this ema + * @param rowRedirection the {@link RowRedirection} to use for dense output sources * @param control defines how to handle {@code null} input values. * @param timestampColumnName the name of the column containing timestamps for time-based calcuations - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is measured in ticks, otherwise it is measured in nanoseconds - * @param rowRedirection the {@link RowRedirection} to use for dense output sources + * @param windowScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is measured in ticks, otherwise it is measured in nanoseconds * @param valueSource a reference to the input column source for this operation */ public FloatEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, + @Nullable final RowRedirection rowRedirection, @NotNull final OperationControl control, @Nullable final String timestampColumnName, - final long timeScaleUnits, - @Nullable final RowRedirection rowRedirection, + final long windowScaleUnits, final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, rowRedirection); + super(pair, affectingColumns, rowRedirection, control, timestampColumnName, windowScaleUnits); this.valueSource = valueSource; // region constructor // endregion constructor @@ -136,7 +137,7 @@ public FloatEMAOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java index 48297efe9c7..1751b7535fc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/IntEMAOperator.java @@ -13,6 +13,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -31,10 +32,10 @@ protected Context(final int chunkSize, final int chunkCount) { } @Override - public void accumulate(RowSequence inputKeys, - Chunk[] valueChunkArr, - LongChunk tsChunk, - int len) { + public void accumulateCumulative(RowSequence inputKeys, + Chunk[] valueChunkArr, + LongChunk tsChunk, + int len) { setValuesChunk(valueChunkArr[0]); // chunk processing @@ -109,23 +110,23 @@ public void push(long key, int pos, int count) { * * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this ema + * @param rowRedirection the {@link RowRedirection} to use for dense output sources * @param control defines how to handle {@code null} input values. * @param timestampColumnName the name of the column containing timestamps for time-based calcuations - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is measured in ticks, otherwise it is measured in nanoseconds - * @param rowRedirection the {@link RowRedirection} to use for dense output sources + * @param windowScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is measured in ticks, otherwise it is measured in nanoseconds * @param valueSource a reference to the input column source for this operation */ public IntEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, + @Nullable final RowRedirection rowRedirection, @NotNull final OperationControl control, @Nullable final String timestampColumnName, - final long timeScaleUnits, - @Nullable final RowRedirection rowRedirection, + final long windowScaleUnits, final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, rowRedirection); + super(pair, affectingColumns, rowRedirection, control, timestampColumnName, windowScaleUnits); this.valueSource = valueSource; // region constructor // endregion constructor @@ -133,7 +134,7 @@ public IntEMAOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java index b25e4879f72..45f5190b68c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/LongEMAOperator.java @@ -13,6 +13,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -31,10 +32,10 @@ protected Context(final int chunkSize, final int chunkCount) { } @Override - public void accumulate(RowSequence inputKeys, - Chunk[] valueChunkArr, - LongChunk tsChunk, - int len) { + public void accumulateCumulative(RowSequence inputKeys, + Chunk[] valueChunkArr, + LongChunk tsChunk, + int len) { setValuesChunk(valueChunkArr[0]); // chunk processing @@ -109,23 +110,23 @@ public void push(long key, int pos, int count) { * * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this ema + * @param rowRedirection the {@link RowRedirection} to use for dense output sources * @param control defines how to handle {@code null} input values. * @param timestampColumnName the name of the column containing timestamps for time-based calcuations - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is measured in ticks, otherwise it is measured in nanoseconds - * @param rowRedirection the {@link RowRedirection} to use for dense output sources + * @param windowScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is measured in ticks, otherwise it is measured in nanoseconds * @param valueSource a reference to the input column source for this operation */ public LongEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, + @Nullable final RowRedirection rowRedirection, @NotNull final OperationControl control, @Nullable final String timestampColumnName, - final long timeScaleUnits, - @Nullable final RowRedirection rowRedirection, + final long windowScaleUnits, final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, rowRedirection); + super(pair, affectingColumns, rowRedirection, control, timestampColumnName, windowScaleUnits); this.valueSource = valueSource; // region constructor // endregion constructor @@ -133,7 +134,7 @@ public LongEMAOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java index 97e59ddc16d..aeb5683e308 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/ShortEMAOperator.java @@ -8,6 +8,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -26,10 +27,10 @@ protected Context(final int chunkSize, final int chunkCount) { } @Override - public void accumulate(RowSequence inputKeys, - Chunk[] valueChunkArr, - LongChunk tsChunk, - int len) { + public void accumulateCumulative(RowSequence inputKeys, + Chunk[] valueChunkArr, + LongChunk tsChunk, + int len) { setValuesChunk(valueChunkArr[0]); // chunk processing @@ -104,23 +105,23 @@ public void push(long key, int pos, int count) { * * @param pair the {@link MatchPair} that defines the input/output for this operation * @param affectingColumns the names of the columns that affect this ema + * @param rowRedirection the {@link RowRedirection} to use for dense output sources * @param control defines how to handle {@code null} input values. * @param timestampColumnName the name of the column containing timestamps for time-based calcuations - * @param timeScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is measured in ticks, otherwise it is measured in nanoseconds - * @param rowRedirection the {@link RowRedirection} to use for dense output sources + * @param windowScaleUnits the smoothing window for the EMA. If no {@code timestampColumnName} is provided, this is measured in ticks, otherwise it is measured in nanoseconds * @param valueSource a reference to the input column source for this operation */ public ShortEMAOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, + @Nullable final RowRedirection rowRedirection, @NotNull final OperationControl control, @Nullable final String timestampColumnName, - final long timeScaleUnits, - @Nullable final RowRedirection rowRedirection, + final long windowScaleUnits, final ColumnSource valueSource // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, control, timestampColumnName, timeScaleUnits, rowRedirection); + super(pair, affectingColumns, rowRedirection, control, timestampColumnName, windowScaleUnits); this.valueSource = valueSource; // region constructor // endregion constructor @@ -128,7 +129,7 @@ public ShortEMAOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java index 2a3c1bc929f..8bc564f0ac6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java @@ -17,6 +17,7 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseByteUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -56,14 +57,14 @@ public BooleanFillByOperator(@NotNull final MatchPair fillPair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection); + super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection, false); // region constructor // endregion constructor } @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java index 713ce87dcdc..d9b64c53afd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java @@ -10,6 +10,7 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseByteUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -49,14 +50,14 @@ public ByteFillByOperator(@NotNull final MatchPair fillPair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection); + super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection, false); // region constructor // endregion constructor } @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java index 73a1ed15b51..5a2f2b56864 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java @@ -5,6 +5,7 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseCharUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -44,14 +45,14 @@ public CharFillByOperator(@NotNull final MatchPair fillPair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection); + super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection, false); // region constructor // endregion constructor } @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java index d6ceb593c2b..ed2ff529cbc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java @@ -10,6 +10,7 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseDoubleUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -49,14 +50,14 @@ public DoubleFillByOperator(@NotNull final MatchPair fillPair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection); + super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection, false); // region constructor // endregion constructor } @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java index 369caab1d2f..7610c7b1877 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java @@ -10,6 +10,7 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseFloatUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -49,14 +50,14 @@ public FloatFillByOperator(@NotNull final MatchPair fillPair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection); + super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection, false); // region constructor // endregion constructor } @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java index f37d2b6ba58..018823d839e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java @@ -10,6 +10,7 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseIntUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -49,14 +50,14 @@ public IntFillByOperator(@NotNull final MatchPair fillPair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection); + super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection, false); // region constructor // endregion constructor } @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java index d5a0ee5b78d..9b50252377e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java @@ -17,6 +17,7 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -58,7 +59,7 @@ public LongFillByOperator(@NotNull final MatchPair fillPair, ,@NotNull final Class type // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection); + super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection, false); // region constructor this.type = type; // endregion constructor @@ -66,7 +67,7 @@ public LongFillByOperator(@NotNull final MatchPair fillPair, @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java index ca9046d23a0..ebd54d0441b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java @@ -12,6 +12,7 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -51,14 +52,14 @@ public ObjectFillByOperator(@NotNull final MatchPair fillPair, , final Class colType // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection, colType); + super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection, false, colType); // region constructor // endregion constructor } @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java index 4f20e07da4a..6e0d28c0e96 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java @@ -10,6 +10,7 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseShortUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -49,14 +50,14 @@ public ShortFillByOperator(@NotNull final MatchPair fillPair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection); + super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection, false); // region constructor // endregion constructor } @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java index f73da17ebe0..eae12354867 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java @@ -11,6 +11,7 @@ import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableByteChunk; import io.deephaven.chunk.attributes.Values; @@ -18,7 +19,6 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.updateby.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -30,7 +30,7 @@ import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; import static io.deephaven.util.QueryConstants.*; -public abstract class BaseByteUpdateByOperator extends UpdateByCumulativeOperator { +public abstract class BaseByteUpdateByOperator extends UpdateByOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; @@ -38,7 +38,7 @@ public abstract class BaseByteUpdateByOperator extends UpdateByCumulativeOperato final byte nullValue; // endregion extra-fields - protected abstract class Context extends UpdateByCumulativeOperator.Context { + protected abstract class Context extends UpdateByOperator.Context { public final ChunkSink.FillFromContext outputFillContext; public final WritableByteChunk outputValues; @@ -51,10 +51,10 @@ protected Context(final int chunkSize, final int chunkCount) { } @Override - public void accumulate(RowSequence inputKeys, - Chunk[] valueChunkArr, - LongChunk tsChunk, - int len) { + public void accumulateCumulative(RowSequence inputKeys, + Chunk[] valueChunkArr, + LongChunk tsChunk, + int len) { setValuesChunk(valueChunkArr[0]); @@ -69,21 +69,56 @@ public void accumulate(RowSequence inputKeys, } @Override - public void close() { - super.close(); - outputValues.close(); - outputFillContext.close(); + public void accumulateRolling(RowSequence inputKeys, + Chunk[] influencerValueChunkArr, + IntChunk pushChunk, + IntChunk popChunk, + int len) { + + setValuesChunk(influencerValueChunkArr[0]); + int pushIndex = 0; + + // chunk processing + for (int ii = 0; ii < len; ii++) { + final int pushCount = pushChunk.get(ii); + final int popCount = popChunk.get(ii); + + if (pushCount == NULL_INT) { + writeNullToOutputChunk(ii); + continue; + } + + // pop for this row + if (popCount > 0) { + pop(popCount); + } + + // push for this row + if (pushCount > 0) { + push(NULL_ROW_KEY, pushIndex, pushCount); + pushIndex += pushCount; + } + + // write the results to the output chunk + writeToOutputChunk(ii); + } + + // chunk output to column + writeToOutputColumn(inputKeys); } @Override public void setValuesChunk(@NotNull final Chunk valuesChunk) {} - @Override public void writeToOutputChunk(int outIdx) { outputValues.set(outIdx, curVal); } + void writeNullToOutputChunk(int outIdx) { + outputValues.set(outIdx, NULL_BYTE); + } + @Override public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); @@ -93,6 +128,13 @@ public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { public void reset() { curVal = NULL_BYTE; } + + @Override + public void close() { + super.close(); + outputValues.close(); + outputFillContext.close(); + } } /** @@ -105,11 +147,12 @@ public void reset() { */ public BaseByteUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final RowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection, + final boolean isWindowed // region extra-constructor-args // endregion extra-constructor-args ) { - this(pair, affectingColumns, rowRedirection, null, 0); + this(pair, affectingColumns, rowRedirection, null, 0, 0, isWindowed); } /** @@ -121,18 +164,20 @@ public BaseByteUpdateByOperator(@NotNull final MatchPair pair, * @param rowRedirection the {@link RowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. - * @param timeScaleUnits the smoothing window for the operator. If no {@code timestampColumnName} is provided, this + * @param reverseWindowScaleUnits the smoothing window for the operator. If no {@code timestampColumnName} is provided, this * is measured in ticks, otherwise it is measured in nanoseconds. */ public BaseByteUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @Nullable final RowRedirection rowRedirection, @Nullable final String timestampColumnName, - final long timeScaleUnits + final long reverseWindowScaleUnits, + final long forwardWindowScaleUnits, + final boolean isWindowed // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, rowRedirection, timestampColumnName, timeScaleUnits); + super(pair, affectingColumns, rowRedirection, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, isWindowed); if(rowRedirection != null) { // region create-dense this.maybeInnerSource = makeDenseSource(); @@ -167,7 +212,7 @@ protected WritableColumnSource makeDenseSource() { // endregion extra-methods @Override - public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifiedKey, long firstUnmodifiedTimestamp) { + public void initializeCumulative(@NotNull UpdateByOperator.Context context, long firstUnmodifiedKey, long firstUnmodifiedTimestamp) { Context ctx = (Context) context; if (firstUnmodifiedKey != NULL_ROW_KEY) { ctx.curVal = outputSource.getByte(firstUnmodifiedKey); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java index fd18745dd7e..48fa2dcb883 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java @@ -1,6 +1,7 @@ package io.deephaven.engine.table.impl.updateby.internal; import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableCharChunk; import io.deephaven.chunk.attributes.Values; @@ -8,7 +9,6 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.updateby.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -20,14 +20,14 @@ import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; import static io.deephaven.util.QueryConstants.*; -public abstract class BaseCharUpdateByOperator extends UpdateByCumulativeOperator { +public abstract class BaseCharUpdateByOperator extends UpdateByOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; // region extra-fields // endregion extra-fields - protected abstract class Context extends UpdateByCumulativeOperator.Context { + protected abstract class Context extends UpdateByOperator.Context { public final ChunkSink.FillFromContext outputFillContext; public final WritableCharChunk outputValues; @@ -40,10 +40,10 @@ protected Context(final int chunkSize, final int chunkCount) { } @Override - public void accumulate(RowSequence inputKeys, - Chunk[] valueChunkArr, - LongChunk tsChunk, - int len) { + public void accumulateCumulative(RowSequence inputKeys, + Chunk[] valueChunkArr, + LongChunk tsChunk, + int len) { setValuesChunk(valueChunkArr[0]); @@ -58,21 +58,56 @@ public void accumulate(RowSequence inputKeys, } @Override - public void close() { - super.close(); - outputValues.close(); - outputFillContext.close(); + public void accumulateRolling(RowSequence inputKeys, + Chunk[] influencerValueChunkArr, + IntChunk pushChunk, + IntChunk popChunk, + int len) { + + setValuesChunk(influencerValueChunkArr[0]); + int pushIndex = 0; + + // chunk processing + for (int ii = 0; ii < len; ii++) { + final int pushCount = pushChunk.get(ii); + final int popCount = popChunk.get(ii); + + if (pushCount == NULL_INT) { + writeNullToOutputChunk(ii); + continue; + } + + // pop for this row + if (popCount > 0) { + pop(popCount); + } + + // push for this row + if (pushCount > 0) { + push(NULL_ROW_KEY, pushIndex, pushCount); + pushIndex += pushCount; + } + + // write the results to the output chunk + writeToOutputChunk(ii); + } + + // chunk output to column + writeToOutputColumn(inputKeys); } @Override public void setValuesChunk(@NotNull final Chunk valuesChunk) {} - @Override public void writeToOutputChunk(int outIdx) { outputValues.set(outIdx, curVal); } + void writeNullToOutputChunk(int outIdx) { + outputValues.set(outIdx, NULL_CHAR); + } + @Override public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); @@ -82,6 +117,13 @@ public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { public void reset() { curVal = NULL_CHAR; } + + @Override + public void close() { + super.close(); + outputValues.close(); + outputFillContext.close(); + } } /** @@ -94,11 +136,12 @@ public void reset() { */ public BaseCharUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final RowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection, + final boolean isWindowed // region extra-constructor-args // endregion extra-constructor-args ) { - this(pair, affectingColumns, rowRedirection, null, 0); + this(pair, affectingColumns, rowRedirection, null, 0, 0, isWindowed); } /** @@ -110,18 +153,20 @@ public BaseCharUpdateByOperator(@NotNull final MatchPair pair, * @param rowRedirection the {@link RowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. - * @param timeScaleUnits the smoothing window for the operator. If no {@code timestampColumnName} is provided, this + * @param reverseWindowScaleUnits the smoothing window for the operator. If no {@code timestampColumnName} is provided, this * is measured in ticks, otherwise it is measured in nanoseconds. */ public BaseCharUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @Nullable final RowRedirection rowRedirection, @Nullable final String timestampColumnName, - final long timeScaleUnits + final long reverseWindowScaleUnits, + final long forwardWindowScaleUnits, + final boolean isWindowed // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, rowRedirection, timestampColumnName, timeScaleUnits); + super(pair, affectingColumns, rowRedirection, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, isWindowed); if(rowRedirection != null) { // region create-dense this.maybeInnerSource = new CharacterArraySource(); @@ -143,7 +188,7 @@ public BaseCharUpdateByOperator(@NotNull final MatchPair pair, // endregion extra-methods @Override - public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifiedKey, long firstUnmodifiedTimestamp) { + public void initializeCumulative(@NotNull UpdateByOperator.Context context, long firstUnmodifiedKey, long firstUnmodifiedTimestamp) { Context ctx = (Context) context; if (firstUnmodifiedKey != NULL_ROW_KEY) { ctx.curVal = outputSource.getChar(firstUnmodifiedKey); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java index 1b5a93f872a..b56dcf7dc65 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java @@ -6,6 +6,7 @@ package io.deephaven.engine.table.impl.updateby.internal; import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableDoubleChunk; import io.deephaven.chunk.attributes.Values; @@ -13,7 +14,6 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.updateby.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -25,14 +25,14 @@ import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; import static io.deephaven.util.QueryConstants.*; -public abstract class BaseDoubleUpdateByOperator extends UpdateByCumulativeOperator { +public abstract class BaseDoubleUpdateByOperator extends UpdateByOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; // region extra-fields // endregion extra-fields - protected abstract class Context extends UpdateByCumulativeOperator.Context { + protected abstract class Context extends UpdateByOperator.Context { public final ChunkSink.FillFromContext outputFillContext; public final WritableDoubleChunk outputValues; @@ -45,10 +45,10 @@ protected Context(final int chunkSize, final int chunkCount) { } @Override - public void accumulate(RowSequence inputKeys, - Chunk[] valueChunkArr, - LongChunk tsChunk, - int len) { + public void accumulateCumulative(RowSequence inputKeys, + Chunk[] valueChunkArr, + LongChunk tsChunk, + int len) { setValuesChunk(valueChunkArr[0]); @@ -63,21 +63,56 @@ public void accumulate(RowSequence inputKeys, } @Override - public void close() { - super.close(); - outputValues.close(); - outputFillContext.close(); + public void accumulateRolling(RowSequence inputKeys, + Chunk[] influencerValueChunkArr, + IntChunk pushChunk, + IntChunk popChunk, + int len) { + + setValuesChunk(influencerValueChunkArr[0]); + int pushIndex = 0; + + // chunk processing + for (int ii = 0; ii < len; ii++) { + final int pushCount = pushChunk.get(ii); + final int popCount = popChunk.get(ii); + + if (pushCount == NULL_INT) { + writeNullToOutputChunk(ii); + continue; + } + + // pop for this row + if (popCount > 0) { + pop(popCount); + } + + // push for this row + if (pushCount > 0) { + push(NULL_ROW_KEY, pushIndex, pushCount); + pushIndex += pushCount; + } + + // write the results to the output chunk + writeToOutputChunk(ii); + } + + // chunk output to column + writeToOutputColumn(inputKeys); } @Override public void setValuesChunk(@NotNull final Chunk valuesChunk) {} - @Override public void writeToOutputChunk(int outIdx) { outputValues.set(outIdx, curVal); } + void writeNullToOutputChunk(int outIdx) { + outputValues.set(outIdx, NULL_DOUBLE); + } + @Override public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); @@ -87,6 +122,13 @@ public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { public void reset() { curVal = NULL_DOUBLE; } + + @Override + public void close() { + super.close(); + outputValues.close(); + outputFillContext.close(); + } } /** @@ -99,11 +141,12 @@ public void reset() { */ public BaseDoubleUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final RowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection, + final boolean isWindowed // region extra-constructor-args // endregion extra-constructor-args ) { - this(pair, affectingColumns, rowRedirection, null, 0); + this(pair, affectingColumns, rowRedirection, null, 0, 0, isWindowed); } /** @@ -115,18 +158,20 @@ public BaseDoubleUpdateByOperator(@NotNull final MatchPair pair, * @param rowRedirection the {@link RowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. - * @param timeScaleUnits the smoothing window for the operator. If no {@code timestampColumnName} is provided, this + * @param reverseWindowScaleUnits the smoothing window for the operator. If no {@code timestampColumnName} is provided, this * is measured in ticks, otherwise it is measured in nanoseconds. */ public BaseDoubleUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @Nullable final RowRedirection rowRedirection, @Nullable final String timestampColumnName, - final long timeScaleUnits + final long reverseWindowScaleUnits, + final long forwardWindowScaleUnits, + final boolean isWindowed // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, rowRedirection, timestampColumnName, timeScaleUnits); + super(pair, affectingColumns, rowRedirection, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, isWindowed); if(rowRedirection != null) { // region create-dense this.maybeInnerSource = new DoubleArraySource(); @@ -148,7 +193,7 @@ public BaseDoubleUpdateByOperator(@NotNull final MatchPair pair, // endregion extra-methods @Override - public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifiedKey, long firstUnmodifiedTimestamp) { + public void initializeCumulative(@NotNull UpdateByOperator.Context context, long firstUnmodifiedKey, long firstUnmodifiedTimestamp) { Context ctx = (Context) context; if (firstUnmodifiedKey != NULL_ROW_KEY) { ctx.curVal = outputSource.getDouble(firstUnmodifiedKey); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java index be608bae468..76667ba6be2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java @@ -6,6 +6,7 @@ package io.deephaven.engine.table.impl.updateby.internal; import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableFloatChunk; import io.deephaven.chunk.attributes.Values; @@ -13,7 +14,6 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.updateby.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -25,14 +25,14 @@ import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; import static io.deephaven.util.QueryConstants.*; -public abstract class BaseFloatUpdateByOperator extends UpdateByCumulativeOperator { +public abstract class BaseFloatUpdateByOperator extends UpdateByOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; // region extra-fields // endregion extra-fields - protected abstract class Context extends UpdateByCumulativeOperator.Context { + protected abstract class Context extends UpdateByOperator.Context { public final ChunkSink.FillFromContext outputFillContext; public final WritableFloatChunk outputValues; @@ -45,10 +45,10 @@ protected Context(final int chunkSize, final int chunkCount) { } @Override - public void accumulate(RowSequence inputKeys, - Chunk[] valueChunkArr, - LongChunk tsChunk, - int len) { + public void accumulateCumulative(RowSequence inputKeys, + Chunk[] valueChunkArr, + LongChunk tsChunk, + int len) { setValuesChunk(valueChunkArr[0]); @@ -63,21 +63,56 @@ public void accumulate(RowSequence inputKeys, } @Override - public void close() { - super.close(); - outputValues.close(); - outputFillContext.close(); + public void accumulateRolling(RowSequence inputKeys, + Chunk[] influencerValueChunkArr, + IntChunk pushChunk, + IntChunk popChunk, + int len) { + + setValuesChunk(influencerValueChunkArr[0]); + int pushIndex = 0; + + // chunk processing + for (int ii = 0; ii < len; ii++) { + final int pushCount = pushChunk.get(ii); + final int popCount = popChunk.get(ii); + + if (pushCount == NULL_INT) { + writeNullToOutputChunk(ii); + continue; + } + + // pop for this row + if (popCount > 0) { + pop(popCount); + } + + // push for this row + if (pushCount > 0) { + push(NULL_ROW_KEY, pushIndex, pushCount); + pushIndex += pushCount; + } + + // write the results to the output chunk + writeToOutputChunk(ii); + } + + // chunk output to column + writeToOutputColumn(inputKeys); } @Override public void setValuesChunk(@NotNull final Chunk valuesChunk) {} - @Override public void writeToOutputChunk(int outIdx) { outputValues.set(outIdx, curVal); } + void writeNullToOutputChunk(int outIdx) { + outputValues.set(outIdx, NULL_FLOAT); + } + @Override public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); @@ -87,6 +122,13 @@ public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { public void reset() { curVal = NULL_FLOAT; } + + @Override + public void close() { + super.close(); + outputValues.close(); + outputFillContext.close(); + } } /** @@ -99,11 +141,12 @@ public void reset() { */ public BaseFloatUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final RowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection, + final boolean isWindowed // region extra-constructor-args // endregion extra-constructor-args ) { - this(pair, affectingColumns, rowRedirection, null, 0); + this(pair, affectingColumns, rowRedirection, null, 0, 0, isWindowed); } /** @@ -115,18 +158,20 @@ public BaseFloatUpdateByOperator(@NotNull final MatchPair pair, * @param rowRedirection the {@link RowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. - * @param timeScaleUnits the smoothing window for the operator. If no {@code timestampColumnName} is provided, this + * @param reverseWindowScaleUnits the smoothing window for the operator. If no {@code timestampColumnName} is provided, this * is measured in ticks, otherwise it is measured in nanoseconds. */ public BaseFloatUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @Nullable final RowRedirection rowRedirection, @Nullable final String timestampColumnName, - final long timeScaleUnits + final long reverseWindowScaleUnits, + final long forwardWindowScaleUnits, + final boolean isWindowed // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, rowRedirection, timestampColumnName, timeScaleUnits); + super(pair, affectingColumns, rowRedirection, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, isWindowed); if(rowRedirection != null) { // region create-dense this.maybeInnerSource = new FloatArraySource(); @@ -148,7 +193,7 @@ public BaseFloatUpdateByOperator(@NotNull final MatchPair pair, // endregion extra-methods @Override - public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifiedKey, long firstUnmodifiedTimestamp) { + public void initializeCumulative(@NotNull UpdateByOperator.Context context, long firstUnmodifiedKey, long firstUnmodifiedTimestamp) { Context ctx = (Context) context; if (firstUnmodifiedKey != NULL_ROW_KEY) { ctx.curVal = outputSource.getFloat(firstUnmodifiedKey); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java index 09d61b49281..99458bbbfc3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java @@ -6,6 +6,7 @@ package io.deephaven.engine.table.impl.updateby.internal; import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.attributes.Values; @@ -13,7 +14,6 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.updateby.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -25,14 +25,14 @@ import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; import static io.deephaven.util.QueryConstants.*; -public abstract class BaseIntUpdateByOperator extends UpdateByCumulativeOperator { +public abstract class BaseIntUpdateByOperator extends UpdateByOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; // region extra-fields // endregion extra-fields - protected abstract class Context extends UpdateByCumulativeOperator.Context { + protected abstract class Context extends UpdateByOperator.Context { public final ChunkSink.FillFromContext outputFillContext; public final WritableIntChunk outputValues; @@ -45,10 +45,10 @@ protected Context(final int chunkSize, final int chunkCount) { } @Override - public void accumulate(RowSequence inputKeys, - Chunk[] valueChunkArr, - LongChunk tsChunk, - int len) { + public void accumulateCumulative(RowSequence inputKeys, + Chunk[] valueChunkArr, + LongChunk tsChunk, + int len) { setValuesChunk(valueChunkArr[0]); @@ -63,21 +63,56 @@ public void accumulate(RowSequence inputKeys, } @Override - public void close() { - super.close(); - outputValues.close(); - outputFillContext.close(); + public void accumulateRolling(RowSequence inputKeys, + Chunk[] influencerValueChunkArr, + IntChunk pushChunk, + IntChunk popChunk, + int len) { + + setValuesChunk(influencerValueChunkArr[0]); + int pushIndex = 0; + + // chunk processing + for (int ii = 0; ii < len; ii++) { + final int pushCount = pushChunk.get(ii); + final int popCount = popChunk.get(ii); + + if (pushCount == NULL_INT) { + writeNullToOutputChunk(ii); + continue; + } + + // pop for this row + if (popCount > 0) { + pop(popCount); + } + + // push for this row + if (pushCount > 0) { + push(NULL_ROW_KEY, pushIndex, pushCount); + pushIndex += pushCount; + } + + // write the results to the output chunk + writeToOutputChunk(ii); + } + + // chunk output to column + writeToOutputColumn(inputKeys); } @Override public void setValuesChunk(@NotNull final Chunk valuesChunk) {} - @Override public void writeToOutputChunk(int outIdx) { outputValues.set(outIdx, curVal); } + void writeNullToOutputChunk(int outIdx) { + outputValues.set(outIdx, NULL_INT); + } + @Override public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); @@ -87,6 +122,13 @@ public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { public void reset() { curVal = NULL_INT; } + + @Override + public void close() { + super.close(); + outputValues.close(); + outputFillContext.close(); + } } /** @@ -99,11 +141,12 @@ public void reset() { */ public BaseIntUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final RowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection, + final boolean isWindowed // region extra-constructor-args // endregion extra-constructor-args ) { - this(pair, affectingColumns, rowRedirection, null, 0); + this(pair, affectingColumns, rowRedirection, null, 0, 0, isWindowed); } /** @@ -115,18 +158,20 @@ public BaseIntUpdateByOperator(@NotNull final MatchPair pair, * @param rowRedirection the {@link RowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. - * @param timeScaleUnits the smoothing window for the operator. If no {@code timestampColumnName} is provided, this + * @param reverseWindowScaleUnits the smoothing window for the operator. If no {@code timestampColumnName} is provided, this * is measured in ticks, otherwise it is measured in nanoseconds. */ public BaseIntUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @Nullable final RowRedirection rowRedirection, @Nullable final String timestampColumnName, - final long timeScaleUnits + final long reverseWindowScaleUnits, + final long forwardWindowScaleUnits, + final boolean isWindowed // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, rowRedirection, timestampColumnName, timeScaleUnits); + super(pair, affectingColumns, rowRedirection, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, isWindowed); if(rowRedirection != null) { // region create-dense this.maybeInnerSource = new IntegerArraySource(); @@ -148,7 +193,7 @@ public BaseIntUpdateByOperator(@NotNull final MatchPair pair, // endregion extra-methods @Override - public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifiedKey, long firstUnmodifiedTimestamp) { + public void initializeCumulative(@NotNull UpdateByOperator.Context context, long firstUnmodifiedKey, long firstUnmodifiedTimestamp) { Context ctx = (Context) context; if (firstUnmodifiedKey != NULL_ROW_KEY) { ctx.curVal = outputSource.getInt(firstUnmodifiedKey); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java index 51f588cb336..4de5666546b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java @@ -6,6 +6,7 @@ package io.deephaven.engine.table.impl.updateby.internal; import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableLongChunk; import io.deephaven.chunk.attributes.Values; @@ -13,7 +14,6 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.updateby.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -25,14 +25,14 @@ import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; import static io.deephaven.util.QueryConstants.*; -public abstract class BaseLongUpdateByOperator extends UpdateByCumulativeOperator { +public abstract class BaseLongUpdateByOperator extends UpdateByOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; // region extra-fields // endregion extra-fields - protected abstract class Context extends UpdateByCumulativeOperator.Context { + protected abstract class Context extends UpdateByOperator.Context { public final ChunkSink.FillFromContext outputFillContext; public final WritableLongChunk outputValues; @@ -45,10 +45,10 @@ protected Context(final int chunkSize, final int chunkCount) { } @Override - public void accumulate(RowSequence inputKeys, - Chunk[] valueChunkArr, - LongChunk tsChunk, - int len) { + public void accumulateCumulative(RowSequence inputKeys, + Chunk[] valueChunkArr, + LongChunk tsChunk, + int len) { setValuesChunk(valueChunkArr[0]); @@ -63,21 +63,56 @@ public void accumulate(RowSequence inputKeys, } @Override - public void close() { - super.close(); - outputValues.close(); - outputFillContext.close(); + public void accumulateRolling(RowSequence inputKeys, + Chunk[] influencerValueChunkArr, + IntChunk pushChunk, + IntChunk popChunk, + int len) { + + setValuesChunk(influencerValueChunkArr[0]); + int pushIndex = 0; + + // chunk processing + for (int ii = 0; ii < len; ii++) { + final int pushCount = pushChunk.get(ii); + final int popCount = popChunk.get(ii); + + if (pushCount == NULL_INT) { + writeNullToOutputChunk(ii); + continue; + } + + // pop for this row + if (popCount > 0) { + pop(popCount); + } + + // push for this row + if (pushCount > 0) { + push(NULL_ROW_KEY, pushIndex, pushCount); + pushIndex += pushCount; + } + + // write the results to the output chunk + writeToOutputChunk(ii); + } + + // chunk output to column + writeToOutputColumn(inputKeys); } @Override public void setValuesChunk(@NotNull final Chunk valuesChunk) {} - @Override public void writeToOutputChunk(int outIdx) { outputValues.set(outIdx, curVal); } + void writeNullToOutputChunk(int outIdx) { + outputValues.set(outIdx, NULL_LONG); + } + @Override public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); @@ -87,6 +122,13 @@ public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { public void reset() { curVal = NULL_LONG; } + + @Override + public void close() { + super.close(); + outputValues.close(); + outputFillContext.close(); + } } /** @@ -99,11 +141,12 @@ public void reset() { */ public BaseLongUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final RowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection, + final boolean isWindowed // region extra-constructor-args // endregion extra-constructor-args ) { - this(pair, affectingColumns, rowRedirection, null, 0); + this(pair, affectingColumns, rowRedirection, null, 0, 0, isWindowed); } /** @@ -115,18 +158,20 @@ public BaseLongUpdateByOperator(@NotNull final MatchPair pair, * @param rowRedirection the {@link RowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. - * @param timeScaleUnits the smoothing window for the operator. If no {@code timestampColumnName} is provided, this + * @param reverseWindowScaleUnits the smoothing window for the operator. If no {@code timestampColumnName} is provided, this * is measured in ticks, otherwise it is measured in nanoseconds. */ public BaseLongUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @Nullable final RowRedirection rowRedirection, @Nullable final String timestampColumnName, - final long timeScaleUnits + final long reverseWindowScaleUnits, + final long forwardWindowScaleUnits, + final boolean isWindowed // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, rowRedirection, timestampColumnName, timeScaleUnits); + super(pair, affectingColumns, rowRedirection, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, isWindowed); if(rowRedirection != null) { // region create-dense this.maybeInnerSource = new LongArraySource(); @@ -148,7 +193,7 @@ public BaseLongUpdateByOperator(@NotNull final MatchPair pair, // endregion extra-methods @Override - public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifiedKey, long firstUnmodifiedTimestamp) { + public void initializeCumulative(@NotNull UpdateByOperator.Context context, long firstUnmodifiedKey, long firstUnmodifiedTimestamp) { Context ctx = (Context) context; if (firstUnmodifiedKey != NULL_ROW_KEY) { ctx.curVal = outputSource.getLong(firstUnmodifiedKey); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java index 62d38714e26..036fdf5bd00 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java @@ -4,6 +4,7 @@ import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -40,18 +41,19 @@ public void reset() { } } - public BaseObjectBinaryOperator(@NotNull final Class type, - @NotNull final MatchPair pair, + public BaseObjectBinaryOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final RowRedirection rowRedirection) { - super(pair, affectingColumns, rowRedirection, type); + @Nullable final RowRedirection rowRedirection, + final boolean isWindowed, + @NotNull final Class type) { + super(pair, affectingColumns, rowRedirection, isWindowed, type); } protected abstract T doOperation(T bucketCurVal, T chunkCurVal); @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java index 38658a457ad..f7d6338e6d4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java @@ -8,6 +8,7 @@ import io.deephaven.engine.table.impl.util.ChunkUtils; import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.attributes.Values; @@ -15,7 +16,6 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.updateby.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -27,7 +27,7 @@ import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; import static io.deephaven.util.QueryConstants.*; -public abstract class BaseObjectUpdateByOperator extends UpdateByCumulativeOperator { +public abstract class BaseObjectUpdateByOperator extends UpdateByOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; @@ -35,7 +35,7 @@ public abstract class BaseObjectUpdateByOperator extends UpdateByCumulativeOp private final Class colType; // endregion extra-fields - protected abstract class Context extends UpdateByCumulativeOperator.Context { + protected abstract class Context extends UpdateByOperator.Context { public final ChunkSink.FillFromContext outputFillContext; public final WritableObjectChunk outputValues; @@ -48,10 +48,10 @@ protected Context(final int chunkSize, final int chunkCount) { } @Override - public void accumulate(RowSequence inputKeys, - Chunk[] valueChunkArr, - LongChunk tsChunk, - int len) { + public void accumulateCumulative(RowSequence inputKeys, + Chunk[] valueChunkArr, + LongChunk tsChunk, + int len) { setValuesChunk(valueChunkArr[0]); @@ -66,21 +66,56 @@ public void accumulate(RowSequence inputKeys, } @Override - public void close() { - super.close(); - outputValues.close(); - outputFillContext.close(); + public void accumulateRolling(RowSequence inputKeys, + Chunk[] influencerValueChunkArr, + IntChunk pushChunk, + IntChunk popChunk, + int len) { + + setValuesChunk(influencerValueChunkArr[0]); + int pushIndex = 0; + + // chunk processing + for (int ii = 0; ii < len; ii++) { + final int pushCount = pushChunk.get(ii); + final int popCount = popChunk.get(ii); + + if (pushCount == NULL_INT) { + writeNullToOutputChunk(ii); + continue; + } + + // pop for this row + if (popCount > 0) { + pop(popCount); + } + + // push for this row + if (pushCount > 0) { + push(NULL_ROW_KEY, pushIndex, pushCount); + pushIndex += pushCount; + } + + // write the results to the output chunk + writeToOutputChunk(ii); + } + + // chunk output to column + writeToOutputColumn(inputKeys); } @Override public void setValuesChunk(@NotNull final Chunk valuesChunk) {} - @Override public void writeToOutputChunk(int outIdx) { outputValues.set(outIdx, curVal); } + void writeNullToOutputChunk(int outIdx) { + outputValues.set(outIdx, null); + } + @Override public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); @@ -90,6 +125,13 @@ public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { public void reset() { curVal = null; } + + @Override + public void close() { + super.close(); + outputValues.close(); + outputFillContext.close(); + } } /** @@ -102,12 +144,13 @@ public void reset() { */ public BaseObjectUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final RowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection, + final boolean isWindowed // region extra-constructor-args , final Class colType // endregion extra-constructor-args ) { - this(pair, affectingColumns, rowRedirection, null, 0, colType); + this(pair, affectingColumns, rowRedirection, null, 0, 0, isWindowed, colType); } /** @@ -119,19 +162,21 @@ public BaseObjectUpdateByOperator(@NotNull final MatchPair pair, * @param rowRedirection the {@link RowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. - * @param timeScaleUnits the smoothing window for the operator. If no {@code timestampColumnName} is provided, this + * @param reverseWindowScaleUnits the smoothing window for the operator. If no {@code timestampColumnName} is provided, this * is measured in ticks, otherwise it is measured in nanoseconds. */ public BaseObjectUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @Nullable final RowRedirection rowRedirection, @Nullable final String timestampColumnName, - final long timeScaleUnits + final long reverseWindowScaleUnits, + final long forwardWindowScaleUnits, + final boolean isWindowed // region extra-constructor-args , final Class colType // endregion extra-constructor-args ) { - super(pair, affectingColumns, rowRedirection, timestampColumnName, timeScaleUnits); + super(pair, affectingColumns, rowRedirection, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, isWindowed); if(rowRedirection != null) { // region create-dense this.maybeInnerSource = new ObjectArraySource<>(colType); @@ -154,7 +199,7 @@ public BaseObjectUpdateByOperator(@NotNull final MatchPair pair, // endregion extra-methods @Override - public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifiedKey, long firstUnmodifiedTimestamp) { + public void initializeCumulative(@NotNull UpdateByOperator.Context context, long firstUnmodifiedKey, long firstUnmodifiedTimestamp) { Context ctx = (Context) context; if (firstUnmodifiedKey != NULL_ROW_KEY) { ctx.curVal = outputSource.get(firstUnmodifiedKey); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java index 439d7e5c7d6..e8337859382 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java @@ -6,6 +6,7 @@ package io.deephaven.engine.table.impl.updateby.internal; import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableShortChunk; import io.deephaven.chunk.attributes.Values; @@ -13,7 +14,6 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.updateby.UpdateByCumulativeOperator; import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -25,14 +25,14 @@ import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; import static io.deephaven.util.QueryConstants.*; -public abstract class BaseShortUpdateByOperator extends UpdateByCumulativeOperator { +public abstract class BaseShortUpdateByOperator extends UpdateByOperator { protected final WritableColumnSource outputSource; protected final WritableColumnSource maybeInnerSource; // region extra-fields // endregion extra-fields - protected abstract class Context extends UpdateByCumulativeOperator.Context { + protected abstract class Context extends UpdateByOperator.Context { public final ChunkSink.FillFromContext outputFillContext; public final WritableShortChunk outputValues; @@ -45,10 +45,10 @@ protected Context(final int chunkSize, final int chunkCount) { } @Override - public void accumulate(RowSequence inputKeys, - Chunk[] valueChunkArr, - LongChunk tsChunk, - int len) { + public void accumulateCumulative(RowSequence inputKeys, + Chunk[] valueChunkArr, + LongChunk tsChunk, + int len) { setValuesChunk(valueChunkArr[0]); @@ -63,21 +63,56 @@ public void accumulate(RowSequence inputKeys, } @Override - public void close() { - super.close(); - outputValues.close(); - outputFillContext.close(); + public void accumulateRolling(RowSequence inputKeys, + Chunk[] influencerValueChunkArr, + IntChunk pushChunk, + IntChunk popChunk, + int len) { + + setValuesChunk(influencerValueChunkArr[0]); + int pushIndex = 0; + + // chunk processing + for (int ii = 0; ii < len; ii++) { + final int pushCount = pushChunk.get(ii); + final int popCount = popChunk.get(ii); + + if (pushCount == NULL_INT) { + writeNullToOutputChunk(ii); + continue; + } + + // pop for this row + if (popCount > 0) { + pop(popCount); + } + + // push for this row + if (pushCount > 0) { + push(NULL_ROW_KEY, pushIndex, pushCount); + pushIndex += pushCount; + } + + // write the results to the output chunk + writeToOutputChunk(ii); + } + + // chunk output to column + writeToOutputColumn(inputKeys); } @Override public void setValuesChunk(@NotNull final Chunk valuesChunk) {} - @Override public void writeToOutputChunk(int outIdx) { outputValues.set(outIdx, curVal); } + void writeNullToOutputChunk(int outIdx) { + outputValues.set(outIdx, NULL_SHORT); + } + @Override public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); @@ -87,6 +122,13 @@ public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { public void reset() { curVal = NULL_SHORT; } + + @Override + public void close() { + super.close(); + outputValues.close(); + outputFillContext.close(); + } } /** @@ -99,11 +141,12 @@ public void reset() { */ public BaseShortUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final RowRedirection rowRedirection + @Nullable final RowRedirection rowRedirection, + final boolean isWindowed // region extra-constructor-args // endregion extra-constructor-args ) { - this(pair, affectingColumns, rowRedirection, null, 0); + this(pair, affectingColumns, rowRedirection, null, 0, 0, isWindowed); } /** @@ -115,18 +158,20 @@ public BaseShortUpdateByOperator(@NotNull final MatchPair pair, * @param rowRedirection the {@link RowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. - * @param timeScaleUnits the smoothing window for the operator. If no {@code timestampColumnName} is provided, this + * @param reverseWindowScaleUnits the smoothing window for the operator. If no {@code timestampColumnName} is provided, this * is measured in ticks, otherwise it is measured in nanoseconds. */ public BaseShortUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @Nullable final RowRedirection rowRedirection, @Nullable final String timestampColumnName, - final long timeScaleUnits + final long reverseWindowScaleUnits, + final long forwardWindowScaleUnits, + final boolean isWindowed // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, rowRedirection, timestampColumnName, timeScaleUnits); + super(pair, affectingColumns, rowRedirection, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, isWindowed); if(rowRedirection != null) { // region create-dense this.maybeInnerSource = new ShortArraySource(); @@ -148,7 +193,7 @@ public BaseShortUpdateByOperator(@NotNull final MatchPair pair, // endregion extra-methods @Override - public void initializeUpdate(@NotNull UpdateContext context, long firstUnmodifiedKey, long firstUnmodifiedTimestamp) { + public void initializeCumulative(@NotNull UpdateByOperator.Context context, long firstUnmodifiedKey, long firstUnmodifiedTimestamp) { Context ctx = (Context) context; if (firstUnmodifiedKey != NULL_ROW_KEY) { ctx.curVal = outputSource.getShort(firstUnmodifiedKey); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java deleted file mode 100644 index 1e11f9ed583..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedByteUpdateByOperator.java +++ /dev/null @@ -1,208 +0,0 @@ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit BaseWindowedCharUpdateByOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -package io.deephaven.engine.table.impl.updateby.internal; - -import io.deephaven.util.QueryConstants; -import io.deephaven.engine.table.impl.sources.ByteArraySource; -import io.deephaven.engine.table.impl.sources.ByteSparseArraySource; -import io.deephaven.engine.table.WritableColumnSource; - -import io.deephaven.api.updateby.OperationControl; -import io.deephaven.chunk.Chunk; -import io.deephaven.chunk.IntChunk; -import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.WritableByteChunk; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.updateby.UpdateByWindowedOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; -import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; - -import java.util.Collections; -import java.util.Map; - -import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; -import static io.deephaven.util.QueryConstants.*; - -public abstract class BaseWindowedByteUpdateByOperator extends UpdateByWindowedOperator { - protected final WritableColumnSource outputSource; - protected final WritableColumnSource maybeInnerSource; - - // region extra-fields - final byte nullValue; - // endregion extra-fields - - protected abstract class Context extends UpdateByWindowedOperator.Context { - public final ChunkSink.FillFromContext outputFillContext; - public final WritableByteChunk outputValues; - - public byte curVal = NULL_BYTE; - - protected Context(final int chunkSize, final int chunkCount) { - super(chunkCount); - this.outputFillContext = outputSource.makeFillFromContext(chunkSize); - this.outputValues = WritableByteChunk.makeWritableChunk(chunkSize); - } - - @Override - public void accumulate(RowSequence inputKeys, - Chunk[] influencerValueChunkArr, - IntChunk pushChunk, - IntChunk popChunk, - int len) { - - setValuesChunk(influencerValueChunkArr[0]); - int pushIndex = 0; - - // chunk processing - for (int ii = 0; ii < len; ii++) { - final int pushCount = pushChunk.get(ii); - final int popCount = popChunk.get(ii); - - if (pushCount == NULL_INT) { - writeNullToOutputChunk(ii); - continue; - } - - // pop for this row - if (popCount > 0) { - pop(popCount); - } - - // push for this row - if (pushCount > 0) { - push(NULL_ROW_KEY, pushIndex, pushCount); - pushIndex += pushCount; - } - - // write the results to the output chunk - writeToOutputChunk(ii); - } - - // chunk output to column - writeToOutputColumn(inputKeys); - } - - @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) {} - - @Override - public void writeToOutputChunk(int outIdx) { - outputValues.set(outIdx, curVal); - } - - void writeNullToOutputChunk(int outIdx) { - outputValues.set(outIdx, NULL_BYTE); - } - - @Override - public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { - outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); - } - - @Override - public void close() { - super.close(); - outputValues.close(); - outputFillContext.close(); - } - - @Override - public void reset() { - curVal = NULL_BYTE; - nullCount = 0; - } - } - - public BaseWindowedByteUpdateByOperator(@NotNull final MatchPair pair, - @NotNull final String[] affectingColumns, - @Nullable final String timestampColumnName, - final long reverseWindowScaleUnits, - final long forwardWindowScaleUnits, - @Nullable final RowRedirection rowRedirection - // region extra-constructor-args - // endregion extra-constructor-args - ) { - super(pair, affectingColumns, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, rowRedirection); - if (rowRedirection != null) { - // region create-dense - this.maybeInnerSource = makeDenseSource(); - // endregion create-dense - this.outputSource = WritableRedirectedColumnSource.maybeRedirect(rowRedirection, maybeInnerSource, 0); - } else { - this.maybeInnerSource = null; - // region create-sparse - this.outputSource = makeSparseSource(); - // endregion create-sparse - } - - // region constructor - this.nullValue = getNullValue(); - // endregion constructor - } - - // region extra-methods - protected byte getNullValue() { - return QueryConstants.NULL_BYTE; - } - - // region extra-methods - protected WritableColumnSource makeSparseSource() { - return new ByteSparseArraySource(); - } - - protected WritableColumnSource makeDenseSource() { - return new ByteArraySource(); - } - // endregion extra-methods - - @Override - public void startTrackingPrev() { - outputSource.startTrackingPrevValues(); - if (rowRedirection != null) { - assert maybeInnerSource != null; - maybeInnerSource.startTrackingPrevValues(); - } - } - - // region Shifts - @Override - public void applyOutputShift(@NotNull final RowSet subIndexToShift, final long delta) { - if (outputSource instanceof BooleanSparseArraySource.ReinterpretedAsByte) { - ((BooleanSparseArraySource.ReinterpretedAsByte)outputSource).shift(subIndexToShift, delta); - } else { - ((ByteSparseArraySource)outputSource).shift(subIndexToShift, delta); - } - } - // endregion Shifts - - @Override - public void prepareForParallelPopulation(final RowSet changedRows) { - if (rowRedirection != null) { - assert maybeInnerSource != null; - ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); - } else { - ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); - } - } - - @NotNull - @Override - public Map> getOutputColumns() { - return Collections.singletonMap(pair.leftColumn, outputSource); - } - - // region clear-output - @Override - public void clearOutputRows(final RowSet toClear) { - // NOP for primitive types - } - // endregion clear-output -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java deleted file mode 100644 index efc26a4eff3..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedCharUpdateByOperator.java +++ /dev/null @@ -1,180 +0,0 @@ -package io.deephaven.engine.table.impl.updateby.internal; - -import io.deephaven.api.updateby.OperationControl; -import io.deephaven.chunk.Chunk; -import io.deephaven.chunk.IntChunk; -import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.WritableCharChunk; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.updateby.UpdateByWindowedOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; -import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; - -import java.util.Collections; -import java.util.Map; - -import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; -import static io.deephaven.util.QueryConstants.*; - -public abstract class BaseWindowedCharUpdateByOperator extends UpdateByWindowedOperator { - protected final WritableColumnSource outputSource; - protected final WritableColumnSource maybeInnerSource; - - // region extra-fields - // endregion extra-fields - - protected abstract class Context extends UpdateByWindowedOperator.Context { - public final ChunkSink.FillFromContext outputFillContext; - public final WritableCharChunk outputValues; - - public char curVal = NULL_CHAR; - - protected Context(final int chunkSize, final int chunkCount) { - super(chunkCount); - this.outputFillContext = outputSource.makeFillFromContext(chunkSize); - this.outputValues = WritableCharChunk.makeWritableChunk(chunkSize); - } - - @Override - public void accumulate(RowSequence inputKeys, - Chunk[] influencerValueChunkArr, - IntChunk pushChunk, - IntChunk popChunk, - int len) { - - setValuesChunk(influencerValueChunkArr[0]); - int pushIndex = 0; - - // chunk processing - for (int ii = 0; ii < len; ii++) { - final int pushCount = pushChunk.get(ii); - final int popCount = popChunk.get(ii); - - if (pushCount == NULL_INT) { - writeNullToOutputChunk(ii); - continue; - } - - // pop for this row - if (popCount > 0) { - pop(popCount); - } - - // push for this row - if (pushCount > 0) { - push(NULL_ROW_KEY, pushIndex, pushCount); - pushIndex += pushCount; - } - - // write the results to the output chunk - writeToOutputChunk(ii); - } - - // chunk output to column - writeToOutputColumn(inputKeys); - } - - @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) {} - - @Override - public void writeToOutputChunk(int outIdx) { - outputValues.set(outIdx, curVal); - } - - void writeNullToOutputChunk(int outIdx) { - outputValues.set(outIdx, NULL_CHAR); - } - - @Override - public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { - outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); - } - - @Override - public void close() { - super.close(); - outputValues.close(); - outputFillContext.close(); - } - - @Override - public void reset() { - curVal = NULL_CHAR; - nullCount = 0; - } - } - - public BaseWindowedCharUpdateByOperator(@NotNull final MatchPair pair, - @NotNull final String[] affectingColumns, - @Nullable final String timestampColumnName, - final long reverseWindowScaleUnits, - final long forwardWindowScaleUnits, - @Nullable final RowRedirection rowRedirection - // region extra-constructor-args - // endregion extra-constructor-args - ) { - super(pair, affectingColumns, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, rowRedirection); - if (rowRedirection != null) { - // region create-dense - this.maybeInnerSource = new CharacterArraySource(); - // endregion create-dense - this.outputSource = WritableRedirectedColumnSource.maybeRedirect(rowRedirection, maybeInnerSource, 0); - } else { - this.maybeInnerSource = null; - // region create-sparse - this.outputSource = new CharacterSparseArraySource(); - // endregion create-sparse - } - - // region constructor - // endregion constructor - } - - // region extra-methods - // endregion extra-methods - - @Override - public void startTrackingPrev() { - outputSource.startTrackingPrevValues(); - if (rowRedirection != null) { - assert maybeInnerSource != null; - maybeInnerSource.startTrackingPrevValues(); - } - } - - // region Shifts - @Override - public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long delta) { - ((CharacterSparseArraySource)outputSource).shift(subRowSetToShift, delta); - } - // endregion Shifts - - @Override - public void prepareForParallelPopulation(final RowSet changedRows) { - if (rowRedirection != null) { - assert maybeInnerSource != null; - ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); - } else { - ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); - } - } - - @NotNull - @Override - public Map> getOutputColumns() { - return Collections.singletonMap(pair.leftColumn, outputSource); - } - - // region clear-output - @Override - public void clearOutputRows(final RowSet toClear) { - // NOP for primitive types - } - // endregion clear-output -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java deleted file mode 100644 index d6c65c816dd..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedDoubleUpdateByOperator.java +++ /dev/null @@ -1,185 +0,0 @@ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit BaseWindowedCharUpdateByOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -package io.deephaven.engine.table.impl.updateby.internal; - -import io.deephaven.api.updateby.OperationControl; -import io.deephaven.chunk.Chunk; -import io.deephaven.chunk.IntChunk; -import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.WritableDoubleChunk; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.updateby.UpdateByWindowedOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; -import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; - -import java.util.Collections; -import java.util.Map; - -import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; -import static io.deephaven.util.QueryConstants.*; - -public abstract class BaseWindowedDoubleUpdateByOperator extends UpdateByWindowedOperator { - protected final WritableColumnSource outputSource; - protected final WritableColumnSource maybeInnerSource; - - // region extra-fields - // endregion extra-fields - - protected abstract class Context extends UpdateByWindowedOperator.Context { - public final ChunkSink.FillFromContext outputFillContext; - public final WritableDoubleChunk outputValues; - - public double curVal = NULL_DOUBLE; - - protected Context(final int chunkSize, final int chunkCount) { - super(chunkCount); - this.outputFillContext = outputSource.makeFillFromContext(chunkSize); - this.outputValues = WritableDoubleChunk.makeWritableChunk(chunkSize); - } - - @Override - public void accumulate(RowSequence inputKeys, - Chunk[] influencerValueChunkArr, - IntChunk pushChunk, - IntChunk popChunk, - int len) { - - setValuesChunk(influencerValueChunkArr[0]); - int pushIndex = 0; - - // chunk processing - for (int ii = 0; ii < len; ii++) { - final int pushCount = pushChunk.get(ii); - final int popCount = popChunk.get(ii); - - if (pushCount == NULL_INT) { - writeNullToOutputChunk(ii); - continue; - } - - // pop for this row - if (popCount > 0) { - pop(popCount); - } - - // push for this row - if (pushCount > 0) { - push(NULL_ROW_KEY, pushIndex, pushCount); - pushIndex += pushCount; - } - - // write the results to the output chunk - writeToOutputChunk(ii); - } - - // chunk output to column - writeToOutputColumn(inputKeys); - } - - @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) {} - - @Override - public void writeToOutputChunk(int outIdx) { - outputValues.set(outIdx, curVal); - } - - void writeNullToOutputChunk(int outIdx) { - outputValues.set(outIdx, NULL_DOUBLE); - } - - @Override - public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { - outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); - } - - @Override - public void close() { - super.close(); - outputValues.close(); - outputFillContext.close(); - } - - @Override - public void reset() { - curVal = NULL_DOUBLE; - nullCount = 0; - } - } - - public BaseWindowedDoubleUpdateByOperator(@NotNull final MatchPair pair, - @NotNull final String[] affectingColumns, - @Nullable final String timestampColumnName, - final long reverseWindowScaleUnits, - final long forwardWindowScaleUnits, - @Nullable final RowRedirection rowRedirection - // region extra-constructor-args - // endregion extra-constructor-args - ) { - super(pair, affectingColumns, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, rowRedirection); - if (rowRedirection != null) { - // region create-dense - this.maybeInnerSource = new DoubleArraySource(); - // endregion create-dense - this.outputSource = WritableRedirectedColumnSource.maybeRedirect(rowRedirection, maybeInnerSource, 0); - } else { - this.maybeInnerSource = null; - // region create-sparse - this.outputSource = new DoubleSparseArraySource(); - // endregion create-sparse - } - - // region constructor - // endregion constructor - } - - // region extra-methods - // endregion extra-methods - - @Override - public void startTrackingPrev() { - outputSource.startTrackingPrevValues(); - if (rowRedirection != null) { - assert maybeInnerSource != null; - maybeInnerSource.startTrackingPrevValues(); - } - } - - // region Shifts - @Override - public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long delta) { - ((DoubleSparseArraySource)outputSource).shift(subRowSetToShift, delta); - } - // endregion Shifts - - @Override - public void prepareForParallelPopulation(final RowSet changedRows) { - if (rowRedirection != null) { - assert maybeInnerSource != null; - ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); - } else { - ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); - } - } - - @NotNull - @Override - public Map> getOutputColumns() { - return Collections.singletonMap(pair.leftColumn, outputSource); - } - - // region clear-output - @Override - public void clearOutputRows(final RowSet toClear) { - // NOP for primitive types - } - // endregion clear-output -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java deleted file mode 100644 index ce7e159669f..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedFloatUpdateByOperator.java +++ /dev/null @@ -1,185 +0,0 @@ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit BaseWindowedCharUpdateByOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -package io.deephaven.engine.table.impl.updateby.internal; - -import io.deephaven.api.updateby.OperationControl; -import io.deephaven.chunk.Chunk; -import io.deephaven.chunk.IntChunk; -import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.WritableFloatChunk; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.updateby.UpdateByWindowedOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; -import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; - -import java.util.Collections; -import java.util.Map; - -import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; -import static io.deephaven.util.QueryConstants.*; - -public abstract class BaseWindowedFloatUpdateByOperator extends UpdateByWindowedOperator { - protected final WritableColumnSource outputSource; - protected final WritableColumnSource maybeInnerSource; - - // region extra-fields - // endregion extra-fields - - protected abstract class Context extends UpdateByWindowedOperator.Context { - public final ChunkSink.FillFromContext outputFillContext; - public final WritableFloatChunk outputValues; - - public float curVal = NULL_FLOAT; - - protected Context(final int chunkSize, final int chunkCount) { - super(chunkCount); - this.outputFillContext = outputSource.makeFillFromContext(chunkSize); - this.outputValues = WritableFloatChunk.makeWritableChunk(chunkSize); - } - - @Override - public void accumulate(RowSequence inputKeys, - Chunk[] influencerValueChunkArr, - IntChunk pushChunk, - IntChunk popChunk, - int len) { - - setValuesChunk(influencerValueChunkArr[0]); - int pushIndex = 0; - - // chunk processing - for (int ii = 0; ii < len; ii++) { - final int pushCount = pushChunk.get(ii); - final int popCount = popChunk.get(ii); - - if (pushCount == NULL_INT) { - writeNullToOutputChunk(ii); - continue; - } - - // pop for this row - if (popCount > 0) { - pop(popCount); - } - - // push for this row - if (pushCount > 0) { - push(NULL_ROW_KEY, pushIndex, pushCount); - pushIndex += pushCount; - } - - // write the results to the output chunk - writeToOutputChunk(ii); - } - - // chunk output to column - writeToOutputColumn(inputKeys); - } - - @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) {} - - @Override - public void writeToOutputChunk(int outIdx) { - outputValues.set(outIdx, curVal); - } - - void writeNullToOutputChunk(int outIdx) { - outputValues.set(outIdx, NULL_FLOAT); - } - - @Override - public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { - outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); - } - - @Override - public void close() { - super.close(); - outputValues.close(); - outputFillContext.close(); - } - - @Override - public void reset() { - curVal = NULL_FLOAT; - nullCount = 0; - } - } - - public BaseWindowedFloatUpdateByOperator(@NotNull final MatchPair pair, - @NotNull final String[] affectingColumns, - @Nullable final String timestampColumnName, - final long reverseWindowScaleUnits, - final long forwardWindowScaleUnits, - @Nullable final RowRedirection rowRedirection - // region extra-constructor-args - // endregion extra-constructor-args - ) { - super(pair, affectingColumns, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, rowRedirection); - if (rowRedirection != null) { - // region create-dense - this.maybeInnerSource = new FloatArraySource(); - // endregion create-dense - this.outputSource = WritableRedirectedColumnSource.maybeRedirect(rowRedirection, maybeInnerSource, 0); - } else { - this.maybeInnerSource = null; - // region create-sparse - this.outputSource = new FloatSparseArraySource(); - // endregion create-sparse - } - - // region constructor - // endregion constructor - } - - // region extra-methods - // endregion extra-methods - - @Override - public void startTrackingPrev() { - outputSource.startTrackingPrevValues(); - if (rowRedirection != null) { - assert maybeInnerSource != null; - maybeInnerSource.startTrackingPrevValues(); - } - } - - // region Shifts - @Override - public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long delta) { - ((FloatSparseArraySource)outputSource).shift(subRowSetToShift, delta); - } - // endregion Shifts - - @Override - public void prepareForParallelPopulation(final RowSet changedRows) { - if (rowRedirection != null) { - assert maybeInnerSource != null; - ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); - } else { - ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); - } - } - - @NotNull - @Override - public Map> getOutputColumns() { - return Collections.singletonMap(pair.leftColumn, outputSource); - } - - // region clear-output - @Override - public void clearOutputRows(final RowSet toClear) { - // NOP for primitive types - } - // endregion clear-output -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java deleted file mode 100644 index 07e211de8bf..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedIntUpdateByOperator.java +++ /dev/null @@ -1,185 +0,0 @@ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit BaseWindowedCharUpdateByOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -package io.deephaven.engine.table.impl.updateby.internal; - -import io.deephaven.api.updateby.OperationControl; -import io.deephaven.chunk.Chunk; -import io.deephaven.chunk.IntChunk; -import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.WritableIntChunk; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.updateby.UpdateByWindowedOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; -import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; - -import java.util.Collections; -import java.util.Map; - -import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; -import static io.deephaven.util.QueryConstants.*; - -public abstract class BaseWindowedIntUpdateByOperator extends UpdateByWindowedOperator { - protected final WritableColumnSource outputSource; - protected final WritableColumnSource maybeInnerSource; - - // region extra-fields - // endregion extra-fields - - protected abstract class Context extends UpdateByWindowedOperator.Context { - public final ChunkSink.FillFromContext outputFillContext; - public final WritableIntChunk outputValues; - - public int curVal = NULL_INT; - - protected Context(final int chunkSize, final int chunkCount) { - super(chunkCount); - this.outputFillContext = outputSource.makeFillFromContext(chunkSize); - this.outputValues = WritableIntChunk.makeWritableChunk(chunkSize); - } - - @Override - public void accumulate(RowSequence inputKeys, - Chunk[] influencerValueChunkArr, - IntChunk pushChunk, - IntChunk popChunk, - int len) { - - setValuesChunk(influencerValueChunkArr[0]); - int pushIndex = 0; - - // chunk processing - for (int ii = 0; ii < len; ii++) { - final int pushCount = pushChunk.get(ii); - final int popCount = popChunk.get(ii); - - if (pushCount == NULL_INT) { - writeNullToOutputChunk(ii); - continue; - } - - // pop for this row - if (popCount > 0) { - pop(popCount); - } - - // push for this row - if (pushCount > 0) { - push(NULL_ROW_KEY, pushIndex, pushCount); - pushIndex += pushCount; - } - - // write the results to the output chunk - writeToOutputChunk(ii); - } - - // chunk output to column - writeToOutputColumn(inputKeys); - } - - @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) {} - - @Override - public void writeToOutputChunk(int outIdx) { - outputValues.set(outIdx, curVal); - } - - void writeNullToOutputChunk(int outIdx) { - outputValues.set(outIdx, NULL_INT); - } - - @Override - public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { - outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); - } - - @Override - public void close() { - super.close(); - outputValues.close(); - outputFillContext.close(); - } - - @Override - public void reset() { - curVal = NULL_INT; - nullCount = 0; - } - } - - public BaseWindowedIntUpdateByOperator(@NotNull final MatchPair pair, - @NotNull final String[] affectingColumns, - @Nullable final String timestampColumnName, - final long reverseWindowScaleUnits, - final long forwardWindowScaleUnits, - @Nullable final RowRedirection rowRedirection - // region extra-constructor-args - // endregion extra-constructor-args - ) { - super(pair, affectingColumns, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, rowRedirection); - if (rowRedirection != null) { - // region create-dense - this.maybeInnerSource = new IntegerArraySource(); - // endregion create-dense - this.outputSource = WritableRedirectedColumnSource.maybeRedirect(rowRedirection, maybeInnerSource, 0); - } else { - this.maybeInnerSource = null; - // region create-sparse - this.outputSource = new IntegerSparseArraySource(); - // endregion create-sparse - } - - // region constructor - // endregion constructor - } - - // region extra-methods - // endregion extra-methods - - @Override - public void startTrackingPrev() { - outputSource.startTrackingPrevValues(); - if (rowRedirection != null) { - assert maybeInnerSource != null; - maybeInnerSource.startTrackingPrevValues(); - } - } - - // region Shifts - @Override - public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long delta) { - ((IntegerSparseArraySource)outputSource).shift(subRowSetToShift, delta); - } - // endregion Shifts - - @Override - public void prepareForParallelPopulation(final RowSet changedRows) { - if (rowRedirection != null) { - assert maybeInnerSource != null; - ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); - } else { - ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); - } - } - - @NotNull - @Override - public Map> getOutputColumns() { - return Collections.singletonMap(pair.leftColumn, outputSource); - } - - // region clear-output - @Override - public void clearOutputRows(final RowSet toClear) { - // NOP for primitive types - } - // endregion clear-output -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java deleted file mode 100644 index 34622e8aac1..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedLongUpdateByOperator.java +++ /dev/null @@ -1,185 +0,0 @@ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit BaseWindowedCharUpdateByOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -package io.deephaven.engine.table.impl.updateby.internal; - -import io.deephaven.api.updateby.OperationControl; -import io.deephaven.chunk.Chunk; -import io.deephaven.chunk.IntChunk; -import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.WritableLongChunk; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.updateby.UpdateByWindowedOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; -import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; - -import java.util.Collections; -import java.util.Map; - -import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; -import static io.deephaven.util.QueryConstants.*; - -public abstract class BaseWindowedLongUpdateByOperator extends UpdateByWindowedOperator { - protected final WritableColumnSource outputSource; - protected final WritableColumnSource maybeInnerSource; - - // region extra-fields - // endregion extra-fields - - protected abstract class Context extends UpdateByWindowedOperator.Context { - public final ChunkSink.FillFromContext outputFillContext; - public final WritableLongChunk outputValues; - - public long curVal = NULL_LONG; - - protected Context(final int chunkSize, final int chunkCount) { - super(chunkCount); - this.outputFillContext = outputSource.makeFillFromContext(chunkSize); - this.outputValues = WritableLongChunk.makeWritableChunk(chunkSize); - } - - @Override - public void accumulate(RowSequence inputKeys, - Chunk[] influencerValueChunkArr, - IntChunk pushChunk, - IntChunk popChunk, - int len) { - - setValuesChunk(influencerValueChunkArr[0]); - int pushIndex = 0; - - // chunk processing - for (int ii = 0; ii < len; ii++) { - final int pushCount = pushChunk.get(ii); - final int popCount = popChunk.get(ii); - - if (pushCount == NULL_INT) { - writeNullToOutputChunk(ii); - continue; - } - - // pop for this row - if (popCount > 0) { - pop(popCount); - } - - // push for this row - if (pushCount > 0) { - push(NULL_ROW_KEY, pushIndex, pushCount); - pushIndex += pushCount; - } - - // write the results to the output chunk - writeToOutputChunk(ii); - } - - // chunk output to column - writeToOutputColumn(inputKeys); - } - - @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) {} - - @Override - public void writeToOutputChunk(int outIdx) { - outputValues.set(outIdx, curVal); - } - - void writeNullToOutputChunk(int outIdx) { - outputValues.set(outIdx, NULL_LONG); - } - - @Override - public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { - outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); - } - - @Override - public void close() { - super.close(); - outputValues.close(); - outputFillContext.close(); - } - - @Override - public void reset() { - curVal = NULL_LONG; - nullCount = 0; - } - } - - public BaseWindowedLongUpdateByOperator(@NotNull final MatchPair pair, - @NotNull final String[] affectingColumns, - @Nullable final String timestampColumnName, - final long reverseWindowScaleUnits, - final long forwardWindowScaleUnits, - @Nullable final RowRedirection rowRedirection - // region extra-constructor-args - // endregion extra-constructor-args - ) { - super(pair, affectingColumns, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, rowRedirection); - if (rowRedirection != null) { - // region create-dense - this.maybeInnerSource = new LongArraySource(); - // endregion create-dense - this.outputSource = WritableRedirectedColumnSource.maybeRedirect(rowRedirection, maybeInnerSource, 0); - } else { - this.maybeInnerSource = null; - // region create-sparse - this.outputSource = new LongSparseArraySource(); - // endregion create-sparse - } - - // region constructor - // endregion constructor - } - - // region extra-methods - // endregion extra-methods - - @Override - public void startTrackingPrev() { - outputSource.startTrackingPrevValues(); - if (rowRedirection != null) { - assert maybeInnerSource != null; - maybeInnerSource.startTrackingPrevValues(); - } - } - - // region Shifts - @Override - public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long delta) { - ((LongSparseArraySource)outputSource).shift(subRowSetToShift, delta); - } - // endregion Shifts - - @Override - public void prepareForParallelPopulation(final RowSet changedRows) { - if (rowRedirection != null) { - assert maybeInnerSource != null; - ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); - } else { - ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); - } - } - - @NotNull - @Override - public Map> getOutputColumns() { - return Collections.singletonMap(pair.leftColumn, outputSource); - } - - // region clear-output - @Override - public void clearOutputRows(final RowSet toClear) { - // NOP for primitive types - } - // endregion clear-output -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java deleted file mode 100644 index bbaee240cbe..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedObjectUpdateByOperator.java +++ /dev/null @@ -1,195 +0,0 @@ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit BaseWindowedCharUpdateByOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -package io.deephaven.engine.table.impl.updateby.internal; - -import io.deephaven.engine.table.impl.util.ChunkUtils; - -import io.deephaven.api.updateby.OperationControl; -import io.deephaven.chunk.Chunk; -import io.deephaven.chunk.IntChunk; -import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.WritableObjectChunk; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.updateby.UpdateByWindowedOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; -import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; - -import java.util.Collections; -import java.util.Map; - -import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; -import static io.deephaven.util.QueryConstants.*; - -public abstract class BaseWindowedObjectUpdateByOperator extends UpdateByWindowedOperator { - protected final WritableColumnSource outputSource; - protected final WritableColumnSource maybeInnerSource; - - // region extra-fields - private final Class colType; - // endregion extra-fields - - protected abstract class Context extends UpdateByWindowedOperator.Context { - public final ChunkSink.FillFromContext outputFillContext; - public final WritableObjectChunk outputValues; - - public T curVal = null; - - protected Context(final int chunkSize, final int chunkCount) { - super(chunkCount); - this.outputFillContext = outputSource.makeFillFromContext(chunkSize); - this.outputValues = WritableObjectChunk.makeWritableChunk(chunkSize); - } - - @Override - public void accumulate(RowSequence inputKeys, - Chunk[] influencerValueChunkArr, - IntChunk pushChunk, - IntChunk popChunk, - int len) { - - setValuesChunk(influencerValueChunkArr[0]); - int pushIndex = 0; - - // chunk processing - for (int ii = 0; ii < len; ii++) { - final int pushCount = pushChunk.get(ii); - final int popCount = popChunk.get(ii); - - if (pushCount == NULL_INT) { - writeNullToOutputChunk(ii); - continue; - } - - // pop for this row - if (popCount > 0) { - pop(popCount); - } - - // push for this row - if (pushCount > 0) { - push(NULL_ROW_KEY, pushIndex, pushCount); - pushIndex += pushCount; - } - - // write the results to the output chunk - writeToOutputChunk(ii); - } - - // chunk output to column - writeToOutputColumn(inputKeys); - } - - @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) {} - - @Override - public void writeToOutputChunk(int outIdx) { - outputValues.set(outIdx, curVal); - } - - void writeNullToOutputChunk(int outIdx) { - outputValues.set(outIdx, null); - } - - @Override - public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { - outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); - } - - @Override - public void close() { - super.close(); - outputValues.close(); - outputFillContext.close(); - } - - @Override - public void reset() { - curVal = null; - nullCount = 0; - } - } - - public BaseWindowedObjectUpdateByOperator(@NotNull final MatchPair pair, - @NotNull final String[] affectingColumns, - @Nullable final String timestampColumnName, - final long reverseWindowScaleUnits, - final long forwardWindowScaleUnits, - @Nullable final RowRedirection rowRedirection - // region extra-constructor-args - , final Class colType - // endregion extra-constructor-args - ) { - super(pair, affectingColumns, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, rowRedirection); - if (rowRedirection != null) { - // region create-dense - this.maybeInnerSource = new ObjectArraySource<>(colType); - // endregion create-dense - this.outputSource = WritableRedirectedColumnSource.maybeRedirect(rowRedirection, maybeInnerSource, 0); - } else { - this.maybeInnerSource = null; - // region create-sparse - this.outputSource = new ObjectSparseArraySource<>(colType); - // endregion create-sparse - } - - // region constructor - this.colType = colType; - // endregion constructor - } - - // region extra-methods - // endregion extra-methods - - @Override - public void startTrackingPrev() { - outputSource.startTrackingPrevValues(); - if (rowRedirection != null) { - assert maybeInnerSource != null; - maybeInnerSource.startTrackingPrevValues(); - } - } - - // region Shifts - @Override - public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long delta) { - ((ObjectSparseArraySource)outputSource).shift(subRowSetToShift, delta); - } - // endregion Shifts - - @Override - public void prepareForParallelPopulation(final RowSet changedRows) { - if (rowRedirection != null) { - assert maybeInnerSource != null; - ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); - } else { - ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); - } - } - - @NotNull - @Override - public Map> getOutputColumns() { - return Collections.singletonMap(pair.leftColumn, outputSource); - } - - // region clear-output - @Override - public void clearOutputRows(final RowSet toClear) { - // if we are redirected, clear the inner source - if (rowRedirection != null) { - ChunkUtils.fillWithNullValue(maybeInnerSource, toClear); - } else { - ChunkUtils.fillWithNullValue(outputSource, toClear); - } - } - // endregion clear-output -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java deleted file mode 100644 index 08d6f1be9d9..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseWindowedShortUpdateByOperator.java +++ /dev/null @@ -1,185 +0,0 @@ -/* - * --------------------------------------------------------------------------------------------------------------------- - * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit BaseWindowedCharUpdateByOperator and regenerate - * --------------------------------------------------------------------------------------------------------------------- - */ -package io.deephaven.engine.table.impl.updateby.internal; - -import io.deephaven.api.updateby.OperationControl; -import io.deephaven.chunk.Chunk; -import io.deephaven.chunk.IntChunk; -import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.WritableShortChunk; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.updateby.UpdateByWindowedOperator; -import io.deephaven.engine.table.impl.util.RowRedirection; -import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; - -import java.util.Collections; -import java.util.Map; - -import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; -import static io.deephaven.util.QueryConstants.*; - -public abstract class BaseWindowedShortUpdateByOperator extends UpdateByWindowedOperator { - protected final WritableColumnSource outputSource; - protected final WritableColumnSource maybeInnerSource; - - // region extra-fields - // endregion extra-fields - - protected abstract class Context extends UpdateByWindowedOperator.Context { - public final ChunkSink.FillFromContext outputFillContext; - public final WritableShortChunk outputValues; - - public short curVal = NULL_SHORT; - - protected Context(final int chunkSize, final int chunkCount) { - super(chunkCount); - this.outputFillContext = outputSource.makeFillFromContext(chunkSize); - this.outputValues = WritableShortChunk.makeWritableChunk(chunkSize); - } - - @Override - public void accumulate(RowSequence inputKeys, - Chunk[] influencerValueChunkArr, - IntChunk pushChunk, - IntChunk popChunk, - int len) { - - setValuesChunk(influencerValueChunkArr[0]); - int pushIndex = 0; - - // chunk processing - for (int ii = 0; ii < len; ii++) { - final int pushCount = pushChunk.get(ii); - final int popCount = popChunk.get(ii); - - if (pushCount == NULL_INT) { - writeNullToOutputChunk(ii); - continue; - } - - // pop for this row - if (popCount > 0) { - pop(popCount); - } - - // push for this row - if (pushCount > 0) { - push(NULL_ROW_KEY, pushIndex, pushCount); - pushIndex += pushCount; - } - - // write the results to the output chunk - writeToOutputChunk(ii); - } - - // chunk output to column - writeToOutputColumn(inputKeys); - } - - @Override - public void setValuesChunk(@NotNull final Chunk valuesChunk) {} - - @Override - public void writeToOutputChunk(int outIdx) { - outputValues.set(outIdx, curVal); - } - - void writeNullToOutputChunk(int outIdx) { - outputValues.set(outIdx, NULL_SHORT); - } - - @Override - public void writeToOutputColumn(@NotNull final RowSequence inputKeys) { - outputSource.fillFromChunk(outputFillContext, outputValues, inputKeys); - } - - @Override - public void close() { - super.close(); - outputValues.close(); - outputFillContext.close(); - } - - @Override - public void reset() { - curVal = NULL_SHORT; - nullCount = 0; - } - } - - public BaseWindowedShortUpdateByOperator(@NotNull final MatchPair pair, - @NotNull final String[] affectingColumns, - @Nullable final String timestampColumnName, - final long reverseWindowScaleUnits, - final long forwardWindowScaleUnits, - @Nullable final RowRedirection rowRedirection - // region extra-constructor-args - // endregion extra-constructor-args - ) { - super(pair, affectingColumns, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, rowRedirection); - if (rowRedirection != null) { - // region create-dense - this.maybeInnerSource = new ShortArraySource(); - // endregion create-dense - this.outputSource = WritableRedirectedColumnSource.maybeRedirect(rowRedirection, maybeInnerSource, 0); - } else { - this.maybeInnerSource = null; - // region create-sparse - this.outputSource = new ShortSparseArraySource(); - // endregion create-sparse - } - - // region constructor - // endregion constructor - } - - // region extra-methods - // endregion extra-methods - - @Override - public void startTrackingPrev() { - outputSource.startTrackingPrevValues(); - if (rowRedirection != null) { - assert maybeInnerSource != null; - maybeInnerSource.startTrackingPrevValues(); - } - } - - // region Shifts - @Override - public void applyOutputShift(@NotNull final RowSet subRowSetToShift, final long delta) { - ((ShortSparseArraySource)outputSource).shift(subRowSetToShift, delta); - } - // endregion Shifts - - @Override - public void prepareForParallelPopulation(final RowSet changedRows) { - if (rowRedirection != null) { - assert maybeInnerSource != null; - ((WritableSourceWithPrepareForParallelPopulation) maybeInnerSource).prepareForParallelPopulation(changedRows); - } else { - ((WritableSourceWithPrepareForParallelPopulation) outputSource).prepareForParallelPopulation(changedRows); - } - } - - @NotNull - @Override - public Map> getOutputColumns() { - return Collections.singletonMap(pair.leftColumn, outputSource); - } - - // region clear-output - @Override - public void clearOutputRows(final RowSet toClear) { - // NOP for primitive types - } - // endregion clear-output -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java index b95a161abf2..aaac21ecb23 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java @@ -10,6 +10,7 @@ import io.deephaven.chunk.ByteChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseByteUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -58,7 +59,7 @@ public ByteCumMinMaxOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection); + super(pair, new String[] { pair.rightColumn }, rowRedirection, false); this.isMax = isMax; // region constructor // endregion constructor @@ -68,7 +69,7 @@ public ByteCumMinMaxOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ComparableCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ComparableCumMinMaxOperator.java index 754d93a8509..1a1e0b1c139 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ComparableCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ComparableCumMinMaxOperator.java @@ -9,11 +9,11 @@ public final class ComparableCumMinMaxOperator> extends BaseObjectBinaryOperator { private final boolean isMax; - public ComparableCumMinMaxOperator(final Class colType, - @NotNull final MatchPair inputPair, + public ComparableCumMinMaxOperator(@NotNull final MatchPair inputPair, final boolean isMax, - @Nullable final RowRedirection rowRedirection) { - super(colType, inputPair, new String[] {inputPair.rightColumn}, rowRedirection); + @Nullable final RowRedirection rowRedirection, + final Class colType) { + super(inputPair, new String[] {inputPair.rightColumn}, rowRedirection, false, colType); this.isMax = isMax; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java index 67d072469e6..ab9492de8e9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java @@ -10,6 +10,7 @@ import io.deephaven.chunk.DoubleChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseDoubleUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -58,7 +59,7 @@ public DoubleCumMinMaxOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection); + super(pair, new String[] { pair.rightColumn }, rowRedirection, false); this.isMax = isMax; // region constructor // endregion constructor @@ -68,7 +69,7 @@ public DoubleCumMinMaxOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java index 177d84582d8..be0f0fb0eb2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java @@ -10,6 +10,7 @@ import io.deephaven.chunk.FloatChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseFloatUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -58,7 +59,7 @@ public FloatCumMinMaxOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection); + super(pair, new String[] { pair.rightColumn }, rowRedirection, false); this.isMax = isMax; // region constructor // endregion constructor @@ -68,7 +69,7 @@ public FloatCumMinMaxOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java index 421626094f1..edec0a79ee6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java @@ -10,6 +10,7 @@ import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseIntUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -58,7 +59,7 @@ public IntCumMinMaxOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection); + super(pair, new String[] { pair.rightColumn }, rowRedirection, false); this.isMax = isMax; // region constructor // endregion constructor @@ -68,7 +69,7 @@ public IntCumMinMaxOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java index 93fd5aa88c3..c195d2b339a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java @@ -17,6 +17,7 @@ import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -67,7 +68,7 @@ public LongCumMinMaxOperator(@NotNull final MatchPair pair, ,@NotNull final Class type // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection); + super(pair, new String[] { pair.rightColumn }, rowRedirection, false); this.isMax = isMax; // region constructor this.type = type; @@ -89,7 +90,7 @@ public Map> getOutputColumns() { @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java index 81913428872..ae789326766 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java @@ -5,6 +5,7 @@ import io.deephaven.chunk.ShortChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseShortUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -53,7 +54,7 @@ public ShortCumMinMaxOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection); + super(pair, new String[] { pair.rightColumn }, rowRedirection, false); this.isMax = isMax; // region constructor // endregion constructor @@ -63,7 +64,7 @@ public ShortCumMinMaxOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java index 4fe9a2d3228..6f8586d6be8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java @@ -5,6 +5,7 @@ import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -49,13 +50,13 @@ public void reset() { public BigDecimalCumProdOperator(@NotNull final MatchPair inputPair, @Nullable final RowRedirection rowRedirection, @NotNull final MathContext mathContext) { - super(inputPair, new String[] {inputPair.rightColumn}, rowRedirection, BigDecimal.class); + super(inputPair, new String[] {inputPair.rightColumn}, rowRedirection, false, BigDecimal.class); this.mathContext = mathContext; } @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java index f9f28d5a964..e3593a9c0dc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java @@ -5,6 +5,7 @@ import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -44,12 +45,13 @@ public void reset() { public BigIntegerCumProdOperator(@NotNull final MatchPair inputPair, @Nullable final RowRedirection rowRedirection) { - super(inputPair, new String[] {inputPair.rightColumn}, rowRedirection, BigInteger.class); + super(inputPair, new String[] {inputPair.rightColumn}, rowRedirection, false, BigInteger.class); } @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } + } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java index db92c73412c..2d7bfa3a497 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java @@ -10,6 +10,7 @@ import io.deephaven.chunk.ByteChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -51,14 +52,14 @@ public ByteCumProdOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection); + super(pair, new String[] { pair.rightColumn }, rowRedirection, false); // region constructor // endregion constructor } @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java index 3955755b76a..0536746c630 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java @@ -10,6 +10,7 @@ import io.deephaven.chunk.DoubleChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseDoubleUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -50,16 +51,14 @@ public DoubleCumProdOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection); + super(pair, new String[] { pair.rightColumn }, rowRedirection, false); // region constructor // endregion constructor } @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } - - } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java index e14c83c938c..7ed159d88e3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java @@ -5,6 +5,7 @@ import io.deephaven.chunk.FloatChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseFloatUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -45,16 +46,14 @@ public FloatCumProdOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection); + super(pair, new String[] { pair.rightColumn }, rowRedirection, false); // region constructor // endregion constructor } @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } - - } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java index aad1b3898fc..1d817163813 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java @@ -10,6 +10,7 @@ import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -51,14 +52,14 @@ public IntCumProdOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection); + super(pair, new String[] { pair.rightColumn }, rowRedirection, false); // region constructor // endregion constructor } @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java index d31cef26e5d..9bb7bb15345 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java @@ -10,6 +10,7 @@ import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -51,14 +52,14 @@ public LongCumProdOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection); + super(pair, new String[] { pair.rightColumn }, rowRedirection, false); // region constructor // endregion constructor } @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java index bc0a351e45c..1e952539ad5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java @@ -5,6 +5,7 @@ import io.deephaven.chunk.ShortChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -46,14 +47,14 @@ public ShortCumProdOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection); + super(pair, new String[] { pair.rightColumn }, rowRedirection, false); // region constructor // endregion constructor } @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java index 3f666356e67..07500ac13c2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigDecimalRollingSumOperator.java @@ -1,13 +1,12 @@ package io.deephaven.engine.table.impl.updateby.rollingsum; -import io.deephaven.api.updateby.OperationControl; import io.deephaven.base.RingBuffer; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.updateby.UpdateByOperator; -import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedObjectUpdateByOperator; +import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -15,12 +14,12 @@ import java.math.BigDecimal; import java.math.MathContext; -public final class BigDecimalRollingSumOperator extends BaseWindowedObjectUpdateByOperator { +public final class BigDecimalRollingSumOperator extends BaseObjectUpdateByOperator { private static final int RING_BUFFER_INITIAL_CAPACITY = 512; @NotNull private final MathContext mathContext; - protected class Context extends BaseWindowedObjectUpdateByOperator.Context { + protected class Context extends BaseObjectUpdateByOperator.Context { protected ObjectChunk objectInfluencerValuesChunk; protected RingBuffer objectWindowValues; @@ -88,19 +87,19 @@ public void writeToOutputChunk(int outIdx) { @NotNull @Override - public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } public BigDecimalRollingSumOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, + @Nullable final RowRedirection rowRedirection, @Nullable final String timestampColumnName, final long reverseWindowScaleUnits, final long forwardWindowScaleUnits, - @Nullable final RowRedirection rowRedirection, @NotNull final MathContext mathContext) { - super(pair, affectingColumns, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, - rowRedirection, BigDecimal.class); + super(pair, affectingColumns, rowRedirection, timestampColumnName, reverseWindowScaleUnits, + forwardWindowScaleUnits, true, BigDecimal.class); this.mathContext = mathContext; } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java index c1fe24d7816..f35a70f5654 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/BigIntegerRollingSumOperator.java @@ -1,24 +1,23 @@ package io.deephaven.engine.table.impl.updateby.rollingsum; -import io.deephaven.api.updateby.OperationControl; import io.deephaven.base.RingBuffer; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.updateby.UpdateByOperator; -import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedObjectUpdateByOperator; +import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import java.math.BigInteger; -public final class BigIntegerRollingSumOperator extends BaseWindowedObjectUpdateByOperator { +public final class BigIntegerRollingSumOperator extends BaseObjectUpdateByOperator { private static final int RING_BUFFER_INITIAL_CAPACITY = 512; - protected class Context extends BaseWindowedObjectUpdateByOperator.Context { + protected class Context extends BaseObjectUpdateByOperator.Context { protected ObjectChunk objectInfluencerValuesChunk; protected RingBuffer objectWindowValues; @@ -85,20 +84,20 @@ public void writeToOutputChunk(int outIdx) { @NotNull @Override - public UpdateByOperator.UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } public BigIntegerRollingSumOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, + @Nullable final RowRedirection rowRedirection, @Nullable final String timestampColumnName, final long reverseWindowScaleUnits, - final long forwardWindowScaleUnits, - @Nullable final RowRedirection rowRedirection + final long forwardWindowScaleUnits // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, rowRedirection, BigInteger.class); + super(pair, affectingColumns, rowRedirection, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, true, BigInteger.class); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java index f84feeb5316..f20e6bc3d24 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ByteRollingSumOperator.java @@ -11,20 +11,21 @@ import io.deephaven.chunk.ByteChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedLongUpdateByOperator; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; +import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.*; -public class ByteRollingSumOperator extends BaseWindowedLongUpdateByOperator { +public class ByteRollingSumOperator extends BaseLongUpdateByOperator { private static final int RING_BUFFER_INITIAL_CAPACITY = 512; // region extra-fields final byte nullValue; // endregion extra-fields - protected class Context extends BaseWindowedLongUpdateByOperator.Context { + protected class Context extends BaseLongUpdateByOperator.Context { protected ByteChunk byteInfluencerValuesChunk; protected ByteRingBuffer byteWindowValues; @@ -95,21 +96,21 @@ public void writeToOutputChunk(int outIdx) { @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } public ByteRollingSumOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, + @Nullable final RowRedirection rowRedirection, @Nullable final String timestampColumnName, final long reverseWindowScaleUnits, - final long forwardWindowScaleUnits, - @Nullable final RowRedirection rowRedirection + final long forwardWindowScaleUnits // region extra-constructor-args ,final byte nullValue // endregion extra-constructor-args ) { - super(pair, affectingColumns, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, rowRedirection); + super(pair, affectingColumns, rowRedirection, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, true); // region constructor this.nullValue = nullValue; // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java index 6a546d847b3..bee966f9047 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/DoubleRollingSumOperator.java @@ -10,7 +10,8 @@ import io.deephaven.chunk.DoubleChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedDoubleUpdateByOperator; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; +import io.deephaven.engine.table.impl.updateby.internal.BaseDoubleUpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.PairwiseDoubleRingBuffer; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -18,12 +19,12 @@ import static io.deephaven.util.QueryConstants.NULL_DOUBLE; -public class DoubleRollingSumOperator extends BaseWindowedDoubleUpdateByOperator { +public class DoubleRollingSumOperator extends BaseDoubleUpdateByOperator { private static final int PAIRWISE_BUFFER_INITIAL_SIZE = 64; // region extra-fields // endregion extra-fields - protected class Context extends BaseWindowedDoubleUpdateByOperator.Context { + protected class Context extends BaseDoubleUpdateByOperator.Context { protected DoubleChunk doubleInfluencerValuesChunk; protected PairwiseDoubleRingBuffer doublePairwiseSum; @@ -89,20 +90,20 @@ public void writeToOutputChunk(int outIdx) { @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } public DoubleRollingSumOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, + @Nullable final RowRedirection rowRedirection, @Nullable final String timestampColumnName, final long reverseWindowScaleUnits, - final long forwardWindowScaleUnits, - @Nullable final RowRedirection rowRedirection + final long forwardWindowScaleUnits // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, rowRedirection); + super(pair, affectingColumns, rowRedirection, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, true); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java index 7d3d4976732..2c0a579e56c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/FloatRollingSumOperator.java @@ -5,7 +5,8 @@ import io.deephaven.chunk.FloatChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedFloatUpdateByOperator; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; +import io.deephaven.engine.table.impl.updateby.internal.BaseFloatUpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.PairwiseFloatRingBuffer; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -13,12 +14,12 @@ import static io.deephaven.util.QueryConstants.NULL_FLOAT; -public class FloatRollingSumOperator extends BaseWindowedFloatUpdateByOperator { +public class FloatRollingSumOperator extends BaseFloatUpdateByOperator { private static final int PAIRWISE_BUFFER_INITIAL_SIZE = 64; // region extra-fields // endregion extra-fields - protected class Context extends BaseWindowedFloatUpdateByOperator.Context { + protected class Context extends BaseFloatUpdateByOperator.Context { protected FloatChunk floatInfluencerValuesChunk; protected PairwiseFloatRingBuffer floatPairwiseSum; @@ -84,20 +85,20 @@ public void writeToOutputChunk(int outIdx) { @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } public FloatRollingSumOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, + @Nullable final RowRedirection rowRedirection, @Nullable final String timestampColumnName, final long reverseWindowScaleUnits, - final long forwardWindowScaleUnits, - @Nullable final RowRedirection rowRedirection + final long forwardWindowScaleUnits // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, rowRedirection); + super(pair, affectingColumns, rowRedirection, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, true); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java index 68d930a92b8..895244100d6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/IntRollingSumOperator.java @@ -11,19 +11,20 @@ import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedLongUpdateByOperator; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; +import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.*; -public class IntRollingSumOperator extends BaseWindowedLongUpdateByOperator { +public class IntRollingSumOperator extends BaseLongUpdateByOperator { private static final int RING_BUFFER_INITIAL_CAPACITY = 512; // region extra-fields // endregion extra-fields - protected class Context extends BaseWindowedLongUpdateByOperator.Context { + protected class Context extends BaseLongUpdateByOperator.Context { protected IntChunk intInfluencerValuesChunk; protected IntRingBuffer intWindowValues; @@ -94,20 +95,20 @@ public void writeToOutputChunk(int outIdx) { @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } public IntRollingSumOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, + @Nullable final RowRedirection rowRedirection, @Nullable final String timestampColumnName, final long reverseWindowScaleUnits, - final long forwardWindowScaleUnits, - @Nullable final RowRedirection rowRedirection + final long forwardWindowScaleUnits // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, rowRedirection); + super(pair, affectingColumns, rowRedirection, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, true); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java index 3d326c0d9c5..554b5bf5ea7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/LongRollingSumOperator.java @@ -11,19 +11,20 @@ import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedLongUpdateByOperator; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; +import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.*; -public class LongRollingSumOperator extends BaseWindowedLongUpdateByOperator { +public class LongRollingSumOperator extends BaseLongUpdateByOperator { private static final int RING_BUFFER_INITIAL_CAPACITY = 512; // region extra-fields // endregion extra-fields - protected class Context extends BaseWindowedLongUpdateByOperator.Context { + protected class Context extends BaseLongUpdateByOperator.Context { protected LongChunk longInfluencerValuesChunk; protected LongRingBuffer longWindowValues; @@ -94,20 +95,20 @@ public void writeToOutputChunk(int outIdx) { @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } public LongRollingSumOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, + @Nullable final RowRedirection rowRedirection, @Nullable final String timestampColumnName, final long reverseWindowScaleUnits, - final long forwardWindowScaleUnits, - @Nullable final RowRedirection rowRedirection + final long forwardWindowScaleUnits // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, rowRedirection); + super(pair, affectingColumns, rowRedirection, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, true); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java index f6a8682c3e5..637254a1785 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingsum/ShortRollingSumOperator.java @@ -6,19 +6,20 @@ import io.deephaven.chunk.ShortChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.updateby.internal.BaseWindowedLongUpdateByOperator; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; +import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import static io.deephaven.util.QueryConstants.*; -public class ShortRollingSumOperator extends BaseWindowedLongUpdateByOperator { +public class ShortRollingSumOperator extends BaseLongUpdateByOperator { private static final int RING_BUFFER_INITIAL_CAPACITY = 512; // region extra-fields // endregion extra-fields - protected class Context extends BaseWindowedLongUpdateByOperator.Context { + protected class Context extends BaseLongUpdateByOperator.Context { protected ShortChunk shortInfluencerValuesChunk; protected ShortRingBuffer shortWindowValues; @@ -89,20 +90,20 @@ public void writeToOutputChunk(int outIdx) { @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } public ShortRollingSumOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, + @Nullable final RowRedirection rowRedirection, @Nullable final String timestampColumnName, final long reverseWindowScaleUnits, - final long forwardWindowScaleUnits, - @Nullable final RowRedirection rowRedirection + final long forwardWindowScaleUnits // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, affectingColumns, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, rowRedirection); + super(pair, affectingColumns, rowRedirection, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, true); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java index 6e6275c13bf..93131b8e368 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java @@ -5,6 +5,7 @@ import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -51,13 +52,14 @@ public void push(long key, int pos, int count) { public BigDecimalCumSumOperator(@NotNull final MatchPair inputPair, @Nullable final RowRedirection rowRedirection, @NotNull final MathContext mathContext) { - super(inputPair, new String[] {inputPair.rightColumn}, rowRedirection, BigDecimal.class); + super(inputPair, new String[] {inputPair.rightColumn}, rowRedirection, false, BigDecimal.class); this.mathContext = mathContext; } @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } + } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java index adfacc5a552..8f993cabeec 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java @@ -5,6 +5,7 @@ import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseObjectUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -45,12 +46,12 @@ public void push(long key, int pos, int count) { public BigIntegerCumSumOperator(@NotNull final MatchPair inputPair, @Nullable final RowRedirection rowRedirection) { - super(inputPair, new String[] {inputPair.rightColumn}, rowRedirection, BigInteger.class); + super(inputPair, new String[] {inputPair.rightColumn}, rowRedirection, false, BigInteger.class); } @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java index a972d555f51..4fe201c173c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java @@ -10,6 +10,7 @@ import io.deephaven.chunk.ByteChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -56,7 +57,7 @@ public ByteCumSumOperator(@NotNull final MatchPair pair, ,final byte nullValue // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection); + super(pair, new String[] { pair.rightColumn }, rowRedirection, false); // region constructor this.nullValue = nullValue; // endregion constructor @@ -64,7 +65,7 @@ public ByteCumSumOperator(@NotNull final MatchPair pair, @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java index a510eccf0ab..65e0d2ea78c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java @@ -10,6 +10,7 @@ import io.deephaven.chunk.DoubleChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseDoubleUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -53,14 +54,14 @@ public DoubleCumSumOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection); + super(pair, new String[] { pair.rightColumn }, rowRedirection, false); // region constructor // endregion constructor } @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java index 1596472e5df..e11752bedce 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java @@ -5,6 +5,7 @@ import io.deephaven.chunk.FloatChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseFloatUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -48,14 +49,14 @@ public FloatCumSumOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection); + super(pair, new String[] { pair.rightColumn }, rowRedirection, false); // region constructor // endregion constructor } @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java index 83e3b04d42d..df9d84c24eb 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java @@ -10,6 +10,7 @@ import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -54,14 +55,14 @@ public IntCumSumOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection); + super(pair, new String[] { pair.rightColumn }, rowRedirection, false); // region constructor // endregion constructor } @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java index eda98e9c230..84d2a8dd9f4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java @@ -10,6 +10,7 @@ import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -54,14 +55,14 @@ public LongCumSumOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection); + super(pair, new String[] { pair.rightColumn }, rowRedirection, false); // region constructor // endregion constructor } @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java index 8d92fd4e186..c17d9e17d12 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java @@ -5,6 +5,7 @@ import io.deephaven.chunk.ShortChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.impl.updateby.UpdateByOperator; import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator; import io.deephaven.engine.table.impl.util.RowRedirection; import org.jetbrains.annotations.NotNull; @@ -49,14 +50,14 @@ public ShortCumSumOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection); + super(pair, new String[] { pair.rightColumn }, rowRedirection, false); // region constructor // endregion constructor } @NotNull @Override - public UpdateContext makeUpdateContext(final int chunkSize, final int chunkCount) { + public UpdateByOperator.Context makeUpdateContext(final int chunkSize, final int chunkCount) { return new Context(chunkSize, chunkCount); } } diff --git a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java index b8629c860cd..860dcbbe7c8 100644 --- a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java +++ b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java @@ -34,8 +34,8 @@ public static void main(String[] args) throws IOException { String objectResult = ReplicatePrimitiveCode.charToObject( "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java"); fixupStandardObject(objectResult, "ObjectFillByOperator", false, - "super\\(fillPair, new String\\[\\] \\{ fillPair.rightColumn \\}, rowRedirection\\);", - "super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection, colType);", + "super\\(fillPair, new String\\[\\] \\{ fillPair.rightColumn \\}, rowRedirection, false\\);", + "super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection, false, colType);", " BaseObjectUpdateByOperator", " BaseObjectUpdateByOperator", "public ObjectChunk Date: Tue, 31 Jan 2023 12:36:34 -0800 Subject: [PATCH 110/123] Reconciled with error handler. Still has debug code! --- .../BucketedPartitionedUpdateByManager.java | 1 - .../engine/table/impl/updateby/UpdateBy.java | 8 ++-- .../impl/updateby/UpdateByBucketHelper.java | 44 ++++++++++++++----- .../impl/updateby/ZeroKeyUpdateByManager.java | 8 ++-- 4 files changed, 41 insertions(+), 20 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java index 40214d3617e..09bfc507efc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java @@ -92,7 +92,6 @@ protected BucketedPartitionedUpdateByManager( bucketDescription, (QueryTable) t, windows, - inputSources, resultSources, timestampColumnName, control, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java index aad5d22dd4b..a8df8dc1125 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java @@ -205,7 +205,7 @@ protected UpdateBy( .toArray(); inputCacheNeeded = cacheableSourceIndices.length > 0; - //noinspection unchecked + // noinspection unchecked inputSourceCaches = new SoftReference[inputSources.length]; buckets = @@ -484,7 +484,7 @@ public void close() { 0, taskCount, (ctx, idx) -> { // advance to the first key of this block - ctx.rsIt.advance(inputRowSet.get((long)idx * PARALLEL_CACHE_BATCH_SIZE)); + ctx.rsIt.advance(inputRowSet.get((long) idx * PARALLEL_CACHE_BATCH_SIZE)); int remaining = PARALLEL_CACHE_BATCH_SIZE; while (ctx.rsIt.hasMore() && remaining > 0) { final RowSequence chunkOk = ctx.rsIt @@ -817,8 +817,8 @@ public void processUpdate() { } /** - * Disconnect result from the {@link UpdateGraphProcessor}, deliver downstream failure notifications, and - * cleanup if needed. + * Disconnect result from the {@link UpdateGraphProcessor}, deliver downstream failure notifications, and cleanup if + * needed. * * @param error The {@link Throwable} to deliver, either from upstream or update processing * @param sourceEntry The {@link TableListener.Entry} to associate with failure messages diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java index 9ab3cc981b8..2d8684c90bc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java @@ -14,7 +14,9 @@ import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.sources.ReinterpretUtils; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; +import io.deephaven.engine.updategraph.LogicalClock; import io.deephaven.engine.updategraph.UpdateCommitter; +import io.deephaven.engine.updategraph.UpdateGraphProcessor; import io.deephaven.util.SafeCloseableArray; import io.deephaven.util.datastructures.linked.IntrusiveDoublyLinkedNode; import org.apache.commons.lang3.mutable.MutableLong; @@ -32,8 +34,8 @@ */ class UpdateByBucketHelper extends IntrusiveDoublyLinkedNode.Impl { private static final int SSA_LEAF_SIZE = 4096; - private final ColumnSource[] inputSources; private final UpdateByWindow[] windows; + private final String description; private final QueryTable source; private final UpdateByControl control; private final BiConsumer failureNotifier; @@ -64,28 +66,26 @@ class UpdateByBucketHelper extends IntrusiveDoublyLinkedNode.Impl[] inputSources, @NotNull final Map> resultSources, @Nullable String timestampColumnName, @NotNull final UpdateByControl control, @NotNull final BiConsumer failureNotifier) { + this.description = description; this.source = source; // some columns will have multiple inputs, such as time-based and Weighted computations this.windows = windows; - this.inputSources = inputSources; this.control = control; this.failureNotifier = failureNotifier; @@ -118,6 +118,19 @@ protected UpdateByBucketHelper( prepareForUpdate(initialUpdate, true); + if (UpdateGraphProcessor.DEFAULT.isRefreshThread()) { + committer = new UpdateCommitter<>(UpdateByBucketHelper.this, (bucket) -> { + // ensure that the item has been cleaned up + if (bucket.isDirty) { + System.out.printf("%d-Failing bucket %s, refreshThread=%b, clockValue=%d%n", + UpdateByBucketHelper.this.hashCode(), description, + UpdateGraphProcessor.DEFAULT.isRefreshThread(), + LogicalClock.DEFAULT.currentValue()); + } + }); + committer.maybeActivate(); + } + initialUpdate.release(); if (source.isRefreshing()) { @@ -301,6 +314,10 @@ private int fillChunkWithNonNull(LongChunk keysChunk, LongChunk< public void prepareForUpdate(final TableUpdate upstream, final boolean initialStep) { Assert.eqFalse(isDirty, "UpdateBy bucket was marked dirty before processing an update"); + System.out.printf("%d-Preparing bucket %s, initial=%b, refreshThread=%b, clockValue=%d%n", hashCode(), + description, initialStep, UpdateGraphProcessor.DEFAULT.isRefreshThread(), + LogicalClock.DEFAULT.currentValue()); + final boolean timestampsModified; // add all the SSA data @@ -375,6 +392,9 @@ public void processWindow(final int winIdx, final boolean initialStep) { * Close the window contexts and release resources for this bucket */ public void finalizeUpdate() { + System.out.printf("%d-Finalizing bucket %s, clockValue=%d%n", hashCode(), description, + LogicalClock.DEFAULT.currentValue()); + SafeCloseableArray.close(windowContexts); isDirty = false; } @@ -397,9 +417,11 @@ public void onUpdate(TableUpdate upstream) { UpdateByBucketHelper.this.committer = new UpdateCommitter<>(UpdateByBucketHelper.this, (bucket) -> { // ensure that the item has been cleaned up - Assert.eqFalse(bucket.isDirty, "bucket.isDirty"); - for (UpdateByWindow.UpdateByWindowBucketContext ctx : bucket.windowContexts) { - Assert.eqNull(ctx, "bucket.windowContexts[]"); + if (bucket.isDirty) { + System.out.printf("%d-Failing bucket %s, refreshThread=%b, clockValue=%d%n", + UpdateByBucketHelper.this.hashCode(), description, + UpdateGraphProcessor.DEFAULT.isRefreshThread(), + LogicalClock.DEFAULT.currentValue()); } }); UpdateByBucketHelper.this.committer.maybeActivate(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java index d926df94eb8..3ba50aca088 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java @@ -61,8 +61,8 @@ protected ZeroKeyUpdateByManager( } // create an updateby bucket instance directly from the source table - zeroKeyUpdateBy = new UpdateByBucketHelper(bucketDescription, source, windows, inputSources, - resultSources, timestampColumnName, control, (oe, se) -> deliverUpdateError(oe, se, true)); + zeroKeyUpdateBy = new UpdateByBucketHelper(bucketDescription, source, windows, resultSources, + timestampColumnName, control, (oe, se) -> deliverUpdateError(oe, se, true)); buckets.offer(zeroKeyUpdateBy); // make the source->result transformer @@ -71,8 +71,8 @@ protected ZeroKeyUpdateByManager( // result will depend on zeroKeyUpdateBy result.addParentReference(zeroKeyUpdateBy.result); } else { - zeroKeyUpdateBy = new UpdateByBucketHelper(bucketDescription, source, windows, inputSources, - resultSources, timestampColumnName, control, (oe, se) -> { + zeroKeyUpdateBy = new UpdateByBucketHelper(bucketDescription, source, windows, resultSources, + timestampColumnName, control, (oe, se) -> { throw new IllegalStateException("Update failure from static zero key updateBy"); }); result = zeroKeyUpdateBy.result; From da2853e46756b2e98effa01531469953f6994baf Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 31 Jan 2023 13:37:59 -0800 Subject: [PATCH 111/123] Final PR comments addressed, still bug-finding code present. --- .../BucketedPartitionedUpdateByManager.java | 5 +++-- .../engine/table/impl/updateby/UpdateBy.java | 10 +++++----- .../impl/updateby/ZeroKeyUpdateByManager.java | 19 ++++++++++--------- .../updateby/ema/BigNumberEMAOperator.java | 2 ++ 4 files changed, 20 insertions(+), 16 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java index 09bfc507efc..984a5c3fa4b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java @@ -36,6 +36,7 @@ class BucketedPartitionedUpdateByManager extends UpdateBy { * @param windows the unique windows for this UpdateBy * @param inputSources the primitive input sources * @param source the source table + * @param preservedColumns columns from the source table that are unchanged in the result table * @param resultSources the result sources * @param byColumns the columns to use for the bucket keys * @param timestampColumnName the column to use for all time-aware operators @@ -48,7 +49,7 @@ protected BucketedPartitionedUpdateByManager( @NotNull final UpdateByWindow[] windows, @NotNull final ColumnSource[] inputSources, @NotNull final QueryTable source, - final String[] persistentColumns, + @NotNull final String[] preservedColumns, @NotNull final Map> resultSources, @NotNull final Collection byColumns, @Nullable final String timestampColumnName, @@ -77,7 +78,7 @@ protected BucketedPartitionedUpdateByManager( pt = source.partitionedAggBy(List.of(), true, null, byColumnNames); // make the source->result transformer from only the columns in the source that are present in result - transformer = source.newModifiedColumnSetTransformer(result, persistentColumns); + transformer = source.newModifiedColumnSetTransformer(result, preservedColumns); } else { pt = source.partitionedAggBy(List.of(), true, null, byColumnNames); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java index a8df8dc1125..9fb83e81aad 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java @@ -951,7 +951,7 @@ public static Table updateBy(@NotNull final QueryTable source, String timestampColumnName = null; // create an initial set of all source columns - final Set persistentColumnSet = new LinkedHashSet<>(source.getColumnSourceMap().keySet()); + final Set preservedColumnSet = new LinkedHashSet<>(source.getColumnSourceMap().keySet()); final Set problems = new LinkedHashSet<>(); final Map> opResultSources = new LinkedHashMap<>(); @@ -961,7 +961,7 @@ public static Table updateBy(@NotNull final QueryTable source, problems.add(name); } // remove overridden source columns - persistentColumnSet.remove(name); + preservedColumnSet.remove(name); }); // verify zero or one timestamp column names if (op.getTimestampColumnName() != null) { @@ -983,7 +983,7 @@ public static Table updateBy(@NotNull final QueryTable source, } // These are the source columns that exist unchanged in the result - final String[] persistentColumns = persistentColumnSet.toArray(String[]::new); + final String[] preservedColumns = preservedColumnSet.toArray(String[]::new); // We will divide the operators into similar windows for efficient processing. final KeyedObjectHashMap> windowMap = @@ -1068,7 +1068,7 @@ public boolean equalKey(UpdateByOperator updateByOperator, windowArr, inputSourceArr, source, - persistentColumns, + preservedColumns, resultSources, fTimestampColumnName, rowRedirection, @@ -1105,7 +1105,7 @@ public boolean equalKey(UpdateByOperator updateByOperator, windowArr, inputSourceArr, source, - persistentColumns, + preservedColumns, resultSources, byColumns, fTimestampColumnName, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java index 3ba50aca088..b59bd137d48 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java @@ -26,6 +26,7 @@ public class ZeroKeyUpdateByManager extends UpdateBy { * @param operators the operations to perform * @param windows the unique windows for this UpdateBy * @param inputSources the primitive input sources + * @param preservedColumns columns from the source table that are unchanged in the result table * @param source the source table * @param resultSources the result sources * @param timestampColumnName the column to use for all time-aware operators @@ -33,15 +34,15 @@ public class ZeroKeyUpdateByManager extends UpdateBy { * @param control the control object. */ protected ZeroKeyUpdateByManager( - @NotNull UpdateByOperator[] operators, - @NotNull UpdateByWindow[] windows, - @NotNull ColumnSource[] inputSources, - @NotNull QueryTable source, - final String[] persistentColumns, + @NotNull final UpdateByOperator[] operators, + @NotNull final UpdateByWindow[] windows, + @NotNull final ColumnSource[] inputSources, + @NotNull final QueryTable source, + @NotNull final String[] preservedColumns, @NotNull final Map> resultSources, - @Nullable String timestampColumnName, - @Nullable WritableRowRedirection rowRedirection, - @NotNull UpdateByControl control) { + @Nullable final String timestampColumnName, + @Nullable final WritableRowRedirection rowRedirection, + @NotNull final UpdateByControl control) { super(source, operators, windows, inputSources, timestampColumnName, rowRedirection, control); final String bucketDescription = this + "-bucket-[]"; @@ -66,7 +67,7 @@ protected ZeroKeyUpdateByManager( buckets.offer(zeroKeyUpdateBy); // make the source->result transformer - transformer = source.newModifiedColumnSetTransformer(result, persistentColumns); + transformer = source.newModifiedColumnSetTransformer(result, preservedColumns); // result will depend on zeroKeyUpdateBy result.addParentReference(zeroKeyUpdateBy.result); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java index e3d2c793e60..3aa99f96881 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ema/BigNumberEMAOperator.java @@ -52,6 +52,8 @@ public void reset() { curVal = null; lastStamp = NULL_LONG; lastDt = NULL_LONG; + alpha = null; + oneMinusAlpha = null; } } From d17ce5ac93347f009620320fba9dcea3aee8ceb8 Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Tue, 31 Jan 2023 17:24:39 -0500 Subject: [PATCH 112/123] 1. Rip out most of the debug logging/instrumentation 2. Defer async error notification to a terminal listener for TestUpdateByGeneral 3. Fix races in error delivery 4. Refactoring to make some fields final and owned by the subclasses --- .../BucketedPartitionedUpdateByManager.java | 70 +++++++++++++++---- .../engine/table/impl/updateby/UpdateBy.java | 43 +++++++----- .../impl/updateby/UpdateByBucketHelper.java | 47 +------------ .../impl/updateby/ZeroKeyUpdateByManager.java | 34 +++++++-- .../impl/updateby/TestUpdateByGeneral.java | 33 ++++++++- 5 files changed, 145 insertions(+), 82 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java index 984a5c3fa4b..a1aa60b68e3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java @@ -10,7 +10,6 @@ import io.deephaven.engine.table.impl.QueryTable; import io.deephaven.engine.table.impl.TableUpdateImpl; import io.deephaven.engine.table.impl.util.WritableRowRedirection; -import io.deephaven.engine.updategraph.LogicalClock; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -23,15 +22,26 @@ class BucketedPartitionedUpdateByManager extends UpdateBy { /** The output table for this UpdateBy operation */ - final QueryTable result; + private final QueryTable result; + + /** Listener to react to upstream changes to refreshing source tables */ + private final UpdateByListener sourceListener; + + /** ColumnSet transformer from source to downstream */ + private final ModifiedColumnSet.Transformer mcsTransformer; + + /** Pending failure encountered in a bucket update. */ + private volatile Throwable bucketFailureThrowable; + + /** Entry associated with {@link #bucketFailureThrowable}. */ + private TableListener.Entry bucketFailureSourceEntry; /** Listener to the partitioned table used for identifying buckets */ - final TransformFailureListener transformFailureListener; + private final TransformFailureListener transformFailureListener; /** * Perform a bucketed updateBy using {@code byColumns} as the keys * - * @param description the operation description * @param operators the operations to perform * @param windows the unique windows for this UpdateBy * @param inputSources the primitive input sources @@ -44,7 +54,6 @@ class BucketedPartitionedUpdateByManager extends UpdateBy { * @param control the control object. */ protected BucketedPartitionedUpdateByManager( - @NotNull final String description, @NotNull final UpdateByOperator[] operators, @NotNull final UpdateByWindow[] windows, @NotNull final ColumnSource[] inputSources, @@ -65,10 +74,10 @@ protected BucketedPartitionedUpdateByManager( final PartitionedTable pt; if (source.isRefreshing()) { // this is a refreshing source, we will need a listener - listener = newUpdateByListener(); - source.addUpdateListener(listener); + sourceListener = newUpdateByListener(); + source.addUpdateListener(sourceListener); // result will depend on listener - result.addParentReference(listener); + result.addParentReference(sourceListener); // create input and output modified column sets for (UpdateByOperator op : operators) { @@ -78,8 +87,10 @@ protected BucketedPartitionedUpdateByManager( pt = source.partitionedAggBy(List.of(), true, null, byColumnNames); // make the source->result transformer from only the columns in the source that are present in result - transformer = source.newModifiedColumnSetTransformer(result, preservedColumns); + mcsTransformer = source.newModifiedColumnSetTransformer(result, preservedColumns); } else { + sourceListener = null; + mcsTransformer = null; pt = source.partitionedAggBy(List.of(), true, null, byColumnNames); } @@ -96,10 +107,7 @@ protected BucketedPartitionedUpdateByManager( resultSources, timestampColumnName, control, - (oe, se) -> deliverUpdateError(oe, se, true)); - - bucket.parentUpdateBy = this; - bucket.createdStep = LogicalClock.DEFAULT.currentStep(); + this::onBucketFailure); // add this to the bucket list synchronized (buckets) { @@ -136,12 +144,46 @@ protected QueryTable result() { return result; } + @Override + protected UpdateByListener sourceListener() { + return sourceListener; + } + + @Override + protected ModifiedColumnSet.Transformer mcsTransformer() { + return mcsTransformer; + } + @Override protected boolean upstreamSatisfied(final long step) { // For bucketed, need to verify the source and the transformed table listener are satisfied. return source.satisfied(step) && transformFailureListener.satisfied(step); } + private void onBucketFailure(@NotNull final Throwable originalException, + @Nullable final TableListener.Entry sourceEntry) { + if (bucketFailureThrowable != null) { + return; + } + synchronized (this) { + if (bucketFailureThrowable != null) { + return; + } + bucketFailureThrowable = originalException; + bucketFailureSourceEntry = sourceEntry; + } + } + + @Override + protected boolean maybeDeliverPendingFailure() { + final Throwable localBucketFailureThrowable = bucketFailureThrowable; + if (localBucketFailureThrowable != null) { + deliverUpdateError(localBucketFailureThrowable, bucketFailureSourceEntry, true); + return true; + } + return false; + } + private final class TransformFailureListener extends InstrumentedTableUpdateListenerAdapter { private TransformFailureListener(@NotNull final Table transformed) { @@ -152,7 +194,7 @@ private TransformFailureListener(@NotNull final Table transformed) { public void onUpdate(@NotNull final TableUpdate upstream) { // No-op: We react to bucket creation inside the transform function, no need to do anything here. // Validation: We expect only adds, because the partitioned table was created by partitionedAggBy with - // preserveEmpty==true + // preserveEmpty==true. Assert.assertion(upstream.removed().isEmpty(), "upstream.removed().isEmpty()"); Assert.assertion(upstream.modified().isEmpty(), "upstream.modified().isEmpty()"); Assert.assertion(upstream.shifted().empty(), "upstream.shifted().empty()"); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java index 9fb83e81aad..1c079a6f821 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java @@ -7,6 +7,7 @@ import io.deephaven.api.updateby.UpdateByOperation; import io.deephaven.base.log.LogOutput; import io.deephaven.base.log.LogOutputAppendable; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ResettableWritableObjectChunk; import io.deephaven.chunk.attributes.Values; @@ -26,6 +27,7 @@ import io.deephaven.engine.table.impl.sources.sparse.SparseConstants; import io.deephaven.engine.table.impl.util.*; import io.deephaven.engine.updategraph.DynamicNode; +import io.deephaven.engine.updategraph.LogicalClock; import io.deephaven.engine.updategraph.TerminalNotification; import io.deephaven.engine.updategraph.UpdateGraphProcessor; import io.deephaven.engine.util.systemicmarking.SystemicObjectTracker; @@ -86,12 +88,6 @@ public abstract class UpdateBy { /** For easy iteration, create a list of the source indices that need to be cached */ protected final int[] cacheableSourceIndices; - /** ColumnSet transformer from source to downstream */ - protected ModifiedColumnSet.Transformer transformer; - - /** Listener to react to upstream changes to refreshing source tables */ - protected UpdateByListener listener; - /** Store every bucket in this list for processing */ protected final IntrusiveDoublyLinkedQueue buckets; @@ -301,10 +297,6 @@ class PhasedUpdateProcessor implements LogOutputAppendable { this.upstream = upstream; this.initialStep = initialStep; - // TODO: remove this - for (UpdateByBucketHelper bucket : buckets) { - bucket.pup = this; - } // determine which buckets we'll examine during this update dirtyBuckets = buckets.stream().filter(UpdateByBucketHelper::isDirty).toArray(UpdateByBucketHelper[]::new); // which windows are dirty and need to be computed this cycle @@ -357,7 +349,7 @@ private void onError(@NotNull final Exception error) { // This error was delivered as part of update processing, we need to ensure that cleanup happens and // a notification is dispatched downstream. cleanUpAfterError(); - deliverUpdateError(error, listener.getEntry(), false); + deliverUpdateError(error, sourceListener().getEntry(), false); } } @@ -410,6 +402,7 @@ private void computeCachedColumnRowsets(final Runnable resumeAction) { } if (rows != null) { // if not null, then insert this window's rowset + // noinspection SynchronizationOnLocalVariableOrMethodParameter synchronized (rows) { rows.insert(win.getInfluencerRows(winCtx)); } @@ -660,7 +653,7 @@ private void cleanUpAndNotify(final Runnable resumeAction) { @Override public void run() { synchronized (accumulated) { - listener.getEntry().accumulate(accumulated); + sourceListener().getEntry().accumulate(accumulated); } } }); @@ -705,7 +698,7 @@ private TableUpdate computeDownstreamUpdate() { downstream.modified = modifiedRowSet; if (upstream.modified().isNonempty()) { - transformer.transform(upstream.modifiedColumnSet(), downstream.modifiedColumnSet); + mcsTransformer().transform(upstream.modifiedColumnSet(), downstream.modifiedColumnSet); } for (UpdateByBucketHelper bucket : dirtyBuckets) { @@ -864,6 +857,19 @@ private UpdateByListener() { @Override public void onUpdate(@NotNull final TableUpdate upstream) { + // If we have a bucket update failure to deliver, deliver it + if (maybeDeliverPendingFailure()) { + return; + } + + // If we delivered a failure in bucketing or bucket creation, short-circuit update delivery + final QueryTable result = result(); + if (result.isFailed()) { + Assert.eq(result.getLastNotificationStep(), "result.getLastNotificationStep()", + LogicalClock.DEFAULT.currentStep(), "LogicalClock.DEFAULT.currentStep()"); + return; + } + final PhasedUpdateProcessor sm = new PhasedUpdateProcessor(upstream.acquire(), false); sm.processUpdate(); } @@ -885,8 +891,14 @@ public UpdateByListener newUpdateByListener() { protected abstract QueryTable result(); + protected abstract UpdateByListener sourceListener(); + + protected abstract ModifiedColumnSet.Transformer mcsTransformer(); + protected abstract boolean upstreamSatisfied(final long step); + protected abstract boolean maybeDeliverPendingFailure(); + // region UpdateBy implementation /** @@ -1081,7 +1093,7 @@ public boolean equalKey(UpdateByOperator updateByOperator, } ops.forEach(UpdateByOperator::startTrackingPrev); } - return zkm.result; + return zkm.result(); }, source::isRefreshing, DynamicNode::isRefreshing); } @@ -1100,7 +1112,6 @@ public boolean equalKey(UpdateByOperator updateByOperator, return LivenessScopeStack.computeEnclosed(() -> { final BucketedPartitionedUpdateByManager bm = new BucketedPartitionedUpdateByManager( - descriptionBuilder.toString(), opArr, windowArr, inputSourceArr, @@ -1119,7 +1130,7 @@ public boolean equalKey(UpdateByOperator updateByOperator, } ops.forEach(UpdateByOperator::startTrackingPrev); } - return bm.result; + return bm.result(); }, source::isRefreshing, DynamicNode::isRefreshing); } // endregion diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java index 2d8684c90bc..8f8a5576c21 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java @@ -14,9 +14,6 @@ import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.sources.ReinterpretUtils; import io.deephaven.engine.table.impl.ssa.LongSegmentedSortedArray; -import io.deephaven.engine.updategraph.LogicalClock; -import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.updategraph.UpdateGraphProcessor; import io.deephaven.util.SafeCloseableArray; import io.deephaven.util.datastructures.linked.IntrusiveDoublyLinkedNode; import org.apache.commons.lang3.mutable.MutableLong; @@ -35,7 +32,6 @@ class UpdateByBucketHelper extends IntrusiveDoublyLinkedNode.Impl { private static final int SSA_LEAF_SIZE = 4096; private final UpdateByWindow[] windows; - private final String description; private final QueryTable source; private final UpdateByControl control; private final BiConsumer failureNotifier; @@ -57,12 +53,6 @@ class UpdateByBucketHelper extends IntrusiveDoublyLinkedNode.Impl committer; - public UpdateBy parentUpdateBy; - public long createdStep; - public UpdateBy.PhasedUpdateProcessor pup; - /** * Perform updateBy operations on a single bucket of data (either zero-key or already limited through partitioning) * @@ -81,8 +71,6 @@ protected UpdateByBucketHelper( @Nullable String timestampColumnName, @NotNull final UpdateByControl control, @NotNull final BiConsumer failureNotifier) { - - this.description = description; this.source = source; // some columns will have multiple inputs, such as time-based and Weighted computations this.windows = windows; @@ -118,19 +106,6 @@ protected UpdateByBucketHelper( prepareForUpdate(initialUpdate, true); - if (UpdateGraphProcessor.DEFAULT.isRefreshThread()) { - committer = new UpdateCommitter<>(UpdateByBucketHelper.this, (bucket) -> { - // ensure that the item has been cleaned up - if (bucket.isDirty) { - System.out.printf("%d-Failing bucket %s, refreshThread=%b, clockValue=%d%n", - UpdateByBucketHelper.this.hashCode(), description, - UpdateGraphProcessor.DEFAULT.isRefreshThread(), - LogicalClock.DEFAULT.currentValue()); - } - }); - committer.maybeActivate(); - } - initialUpdate.release(); if (source.isRefreshing()) { @@ -313,11 +288,6 @@ private int fillChunkWithNonNull(LongChunk keysChunk, LongChunk< */ public void prepareForUpdate(final TableUpdate upstream, final boolean initialStep) { Assert.eqFalse(isDirty, "UpdateBy bucket was marked dirty before processing an update"); - - System.out.printf("%d-Preparing bucket %s, initial=%b, refreshThread=%b, clockValue=%d%n", hashCode(), - description, initialStep, UpdateGraphProcessor.DEFAULT.isRefreshThread(), - LogicalClock.DEFAULT.currentValue()); - final boolean timestampsModified; // add all the SSA data @@ -392,9 +362,6 @@ public void processWindow(final int winIdx, final boolean initialStep) { * Close the window contexts and release resources for this bucket */ public void finalizeUpdate() { - System.out.printf("%d-Finalizing bucket %s, clockValue=%d%n", hashCode(), description, - LogicalClock.DEFAULT.currentValue()); - SafeCloseableArray.close(windowContexts); isDirty = false; } @@ -413,22 +380,10 @@ private UpdateByBucketHelperListener(@Nullable final String description, @NotNul @Override public void onUpdate(TableUpdate upstream) { prepareForUpdate(upstream, false); - - UpdateByBucketHelper.this.committer = - new UpdateCommitter<>(UpdateByBucketHelper.this, (bucket) -> { - // ensure that the item has been cleaned up - if (bucket.isDirty) { - System.out.printf("%d-Failing bucket %s, refreshThread=%b, clockValue=%d%n", - UpdateByBucketHelper.this.hashCode(), description, - UpdateGraphProcessor.DEFAULT.isRefreshThread(), - LogicalClock.DEFAULT.currentValue()); - } - }); - UpdateByBucketHelper.this.committer.maybeActivate(); } @Override - public void onFailure(@NotNull final Throwable originalException, @Nullable final Entry sourceEntry) { + public void onFailureInternal(@NotNull final Throwable originalException, @Nullable final Entry sourceEntry) { failureNotifier.accept(originalException, sourceEntry); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java index b59bd137d48..762ab9bc17b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/ZeroKeyUpdateByManager.java @@ -14,8 +14,15 @@ import java.util.Map; public class ZeroKeyUpdateByManager extends UpdateBy { + /** The output table for this UpdateBy operation */ - final QueryTable result; + private final QueryTable result; + + /** Listener to react to upstream changes to refreshing source tables */ + private final UpdateByListener sourceListener; + + /** ColumnSet transformer from source to downstream */ + private final ModifiedColumnSet.Transformer mcsTransformer; // this manager has only one bucket, managed by this object final UpdateByBucketHelper zeroKeyUpdateBy; @@ -50,10 +57,10 @@ protected ZeroKeyUpdateByManager( result = new QueryTable(source.getRowSet(), resultSources); // this is a refreshing source, we will need a listener - listener = newUpdateByListener(); - source.addUpdateListener(listener); + sourceListener = newUpdateByListener(); + source.addUpdateListener(sourceListener); // result will depend on listener - result.addParentReference(listener); + result.addParentReference(sourceListener); // create input and output modified column sets for (UpdateByOperator op : operators) { @@ -67,7 +74,7 @@ protected ZeroKeyUpdateByManager( buckets.offer(zeroKeyUpdateBy); // make the source->result transformer - transformer = source.newModifiedColumnSetTransformer(result, preservedColumns); + mcsTransformer = source.newModifiedColumnSetTransformer(result, preservedColumns); // result will depend on zeroKeyUpdateBy result.addParentReference(zeroKeyUpdateBy.result); @@ -78,6 +85,8 @@ protected ZeroKeyUpdateByManager( }); result = zeroKeyUpdateBy.result; buckets.offer(zeroKeyUpdateBy); + sourceListener = null; + mcsTransformer = null; } // make a dummy update to generate the initial row keys @@ -98,9 +107,24 @@ protected QueryTable result() { return result; } + @Override + protected UpdateByListener sourceListener() { + return sourceListener; + } + + @Override + protected ModifiedColumnSet.Transformer mcsTransformer() { + return mcsTransformer; + } + @Override protected boolean upstreamSatisfied(final long step) { // for Zero-Key, verify the source and the single bucket are satisfied return source.satisfied(step) && zeroKeyUpdateBy.result.satisfied(step); } + + @Override + protected boolean maybeDeliverPendingFailure() { + return false; + } } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java index 0f1ae43af5f..e4ffb574391 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java @@ -6,15 +6,22 @@ import io.deephaven.api.updateby.UpdateByOperation; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.impl.QueryTable; +import io.deephaven.engine.table.impl.UpdateErrorReporter; +import io.deephaven.engine.table.impl.util.AsyncClientErrorNotifier; import io.deephaven.engine.testutil.EvalNugget; import io.deephaven.engine.table.impl.TableDefaults; import io.deephaven.api.updateby.UpdateByControl; import io.deephaven.engine.testutil.TstUtils; import io.deephaven.engine.testutil.generator.TestDataGenerator; import io.deephaven.engine.testutil.generator.SortedDateTimeGenerator; +import io.deephaven.engine.updategraph.TerminalNotification; import io.deephaven.engine.updategraph.UpdateGraphProcessor; import io.deephaven.engine.util.TableTools; import io.deephaven.test.types.OutOfBandTest; +import io.deephaven.util.ExceptionDetails; +import junit.framework.TestCase; +import org.junit.After; +import org.junit.Before; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -30,7 +37,20 @@ import static io.deephaven.time.DateTimeUtils.convertDateTime; @Category(OutOfBandTest.class) -public class TestUpdateByGeneral extends BaseUpdateByTest { +public class TestUpdateByGeneral extends BaseUpdateByTest implements UpdateErrorReporter { + + private UpdateErrorReporter oldReporter; + + @Before + public void setUp() throws Exception { + oldReporter = AsyncClientErrorNotifier.setReporter(this); + } + + @After + public void tearDown() throws Exception { + AsyncClientErrorNotifier.setReporter(oldReporter); + oldReporter = null; + } @Test public void testMixedAppendOnlyZeroKey() { @@ -191,4 +211,15 @@ public void testNewBuckets() { TableTools.show(result); } + + @Override + public void reportUpdateError(Throwable t) { + UpdateGraphProcessor.DEFAULT.addNotification(new TerminalNotification() { + @Override + public void run() { + System.err.println("Received error notification: " + new ExceptionDetails(t).getFullStackTrace()); + TestCase.fail(t.getMessage()); + } + }); + } } From 536c32c947bbff5a360db2019a825d5d2b58aa02 Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Tue, 31 Jan 2023 18:19:41 -0500 Subject: [PATCH 113/123] Minor cleanup to visibility --- .../impl/updateby/BucketedPartitionedUpdateByManager.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java index a1aa60b68e3..9c3298f86df 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java @@ -160,7 +160,8 @@ protected boolean upstreamSatisfied(final long step) { return source.satisfied(step) && transformFailureListener.satisfied(step); } - private void onBucketFailure(@NotNull final Throwable originalException, + private void onBucketFailure( + @NotNull final Throwable originalException, @Nullable final TableListener.Entry sourceEntry) { if (bucketFailureThrowable != null) { return; @@ -169,8 +170,8 @@ private void onBucketFailure(@NotNull final Throwable originalException, if (bucketFailureThrowable != null) { return; } - bucketFailureThrowable = originalException; bucketFailureSourceEntry = sourceEntry; + bucketFailureThrowable = originalException; } } From e868d94225a4ccf4ea34428c39550e91f7aa79f7 Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Tue, 31 Jan 2023 18:51:47 -0500 Subject: [PATCH 114/123] EvalNugget "improvement" ? --- .../deephaven/engine/testutil/EvalNugget.java | 13 +++- .../engine/testutil/EvalNuggetInterface.java | 2 + .../engine/testutil/EvalNuggetSet.java | 63 ++++++++++--------- .../deephaven/engine/testutil/TstUtils.java | 1 + 4 files changed, 48 insertions(+), 31 deletions(-) diff --git a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/EvalNugget.java b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/EvalNugget.java index 16fee60d9a9..b2aca26a2eb 100644 --- a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/EvalNugget.java +++ b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/EvalNugget.java @@ -116,7 +116,6 @@ public void validate(final String msg) { recomputedTable = e(); } checkDifferences(msg, recomputedTable); - recomputedTable = null; } public void showResult(String label, Table e) { @@ -163,7 +162,6 @@ public void show() { if (recomputedForComparison != recomputedTable) { showResult("Recomputed Table (unmodified):", recomputedTable); - e(); } if (originalForComparison != originalValue) { showResult("Incremental Table (unmodified):", originalValue); @@ -171,6 +169,17 @@ public void show() { } } + @Override + public void releaseRecomputed() { + if (recomputedTable != null) { + if (recomputedTable.tryRetainReference()) { + recomputedTable.dropReference(); + throw new IllegalStateException("Recomputed table " + recomputedTable + " is still live upon release"); + } + } + recomputedTable = null; + } + public abstract static class Sorted extends EvalNugget { private final String[] sortColumns; diff --git a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/EvalNuggetInterface.java b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/EvalNuggetInterface.java index 62807230b8e..dd0ba42adec 100644 --- a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/EvalNuggetInterface.java +++ b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/EvalNuggetInterface.java @@ -9,4 +9,6 @@ public interface EvalNuggetInterface { void validate(final String msg); void show() throws IOException; + + default void releaseRecomputed() {} } diff --git a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/EvalNuggetSet.java b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/EvalNuggetSet.java index 4344ed919d9..3eaba18cc53 100644 --- a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/EvalNuggetSet.java +++ b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/EvalNuggetSet.java @@ -3,11 +3,13 @@ */ package io.deephaven.engine.testutil; +import io.deephaven.engine.liveness.LivenessScopeStack; import io.deephaven.engine.table.Table; import io.deephaven.engine.util.TableTools; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.rowset.RowSet; import io.deephaven.tuple.ArrayTuple; +import io.deephaven.util.SafeCloseable; import org.junit.Assert; import java.util.*; @@ -19,38 +21,41 @@ public EvalNuggetSet(String description) { @Override public void validate(final String msg) { - final Table expected = e(); - try { - TableTools.show(expected); - TableTools.show(originalValue); - } catch (Exception e) { - e.printStackTrace(); - } - Collection sources = originalValue.getColumnSources(); - // TODO create a key for each row and go from there - Map originalSet = new HashMap<>(); - Assert.assertEquals(expected.size(), originalValue.size()); - for (RowSet.Iterator iterator = originalValue.getRowSet().iterator(); iterator.hasNext();) { - long next = iterator.nextLong(); - Object key[] = new Object[sources.size()]; - int i = 0; - for (ColumnSource source : sources) { - key[i++] = source.get(next); + try (final SafeCloseable ignored = LivenessScopeStack.open()) { + final Table expected = e(); + try { + TableTools.show(expected); + TableTools.show(originalValue); + } catch (Exception e) { + e.printStackTrace(); } - final ArrayTuple k = new ArrayTuple(key); + Collection> sources = originalValue.getColumnSources(); + // TODO create a key for each row and go from there + final Map originalSet = new HashMap<>(); + Assert.assertEquals(expected.size(), originalValue.size()); + try (final RowSet.Iterator iterator = originalValue.getRowSet().iterator()) { + while (iterator.hasNext()) { + final long next = iterator.nextLong(); + final Object[] key = new Object[sources.size()]; + int i = 0; + for (ColumnSource source : sources) { + key[i++] = source.get(next); + } + final ArrayTuple k = new ArrayTuple(key); - Assert.assertEquals(msg + " k = " + k, originalSet.put(k, next), null); - } - sources = expected.getColumnSources(); - for (RowSet.Iterator iterator = expected.getRowSet().iterator(); iterator.hasNext();) { - long next = iterator.nextLong(); - Object key[] = new Object[sources.size()]; - int i = 0; - for (ColumnSource source : sources) { - key[i++] = source.get(next); + Assert.assertNull(msg + " k = " + k, originalSet.put(k, next)); + } + } + sources = expected.getColumnSources(); + for (final RowSet.Iterator iterator = expected.getRowSet().iterator(); iterator.hasNext();) { + final long next = iterator.nextLong(); + final Object[] key = new Object[sources.size()]; + int i = 0; + for (final ColumnSource source : sources) { + key[i++] = source.get(next); + } + Assert.assertNotSame(msg, originalSet.remove(new ArrayTuple(key)), null); } - Assert.assertNotSame(msg, originalSet.remove(new ArrayTuple(key)), null); } - } } diff --git a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/TstUtils.java b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/TstUtils.java index 6ed295003aa..4e2b9f9658a 100644 --- a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/TstUtils.java +++ b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/TstUtils.java @@ -428,6 +428,7 @@ public static void validate(final String ctxt, final EvalNuggetInterface[] en) { } en[i].validate(ctxt + " en_i = " + i); } + en[i].releaseRecomputed(); } } From 9165143c672ed65af4cd7dd5b8f0505b4f33d08c Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 31 Jan 2023 16:24:29 -0800 Subject: [PATCH 115/123] Minor comments --- .../BucketedPartitionedUpdateByManager.java | 3 -- .../impl/updateby/UpdateByBucketHelper.java | 42 +------------------ .../table/impl/updateby/UpdateByOperator.java | 18 ++------ .../updateby/fill/BooleanFillByOperator.java | 2 +- .../updateby/fill/ByteFillByOperator.java | 2 +- .../updateby/fill/CharFillByOperator.java | 2 +- .../updateby/fill/DoubleFillByOperator.java | 2 +- .../updateby/fill/FloatFillByOperator.java | 2 +- .../impl/updateby/fill/IntFillByOperator.java | 2 +- .../updateby/fill/LongFillByOperator.java | 2 +- .../updateby/fill/ObjectFillByOperator.java | 2 +- .../updateby/fill/ShortFillByOperator.java | 2 +- .../internal/BaseByteUpdateByOperator.java | 9 ++-- .../internal/BaseCharUpdateByOperator.java | 9 ++-- .../internal/BaseDoubleUpdateByOperator.java | 9 ++-- .../internal/BaseFloatUpdateByOperator.java | 9 ++-- .../internal/BaseIntUpdateByOperator.java | 9 ++-- .../internal/BaseLongUpdateByOperator.java | 9 ++-- .../internal/BaseObjectBinaryOperator.java | 3 +- .../internal/BaseObjectUpdateByOperator.java | 9 ++-- .../internal/BaseShortUpdateByOperator.java | 9 ++-- .../minmax/ByteCumMinMaxOperator.java | 2 +- .../minmax/ComparableCumMinMaxOperator.java | 2 +- .../minmax/DoubleCumMinMaxOperator.java | 2 +- .../minmax/FloatCumMinMaxOperator.java | 2 +- .../updateby/minmax/IntCumMinMaxOperator.java | 2 +- .../minmax/LongCumMinMaxOperator.java | 2 +- .../minmax/ShortCumMinMaxOperator.java | 2 +- .../prod/BigDecimalCumProdOperator.java | 2 +- .../prod/BigIntegerCumProdOperator.java | 2 +- .../updateby/prod/ByteCumProdOperator.java | 2 +- .../updateby/prod/DoubleCumProdOperator.java | 2 +- .../updateby/prod/FloatCumProdOperator.java | 2 +- .../updateby/prod/IntCumProdOperator.java | 2 +- .../updateby/prod/LongCumProdOperator.java | 2 +- .../updateby/prod/ShortCumProdOperator.java | 2 +- .../sum/BigDecimalCumSumOperator.java | 2 +- .../sum/BigIntegerCumSumOperator.java | 2 +- .../impl/updateby/sum/ByteCumSumOperator.java | 2 +- .../updateby/sum/DoubleCumSumOperator.java | 2 +- .../updateby/sum/FloatCumSumOperator.java | 2 +- .../impl/updateby/sum/IntCumSumOperator.java | 2 +- .../impl/updateby/sum/LongCumSumOperator.java | 2 +- .../updateby/sum/ShortCumSumOperator.java | 2 +- .../impl/updateby/TestUpdateByGeneral.java | 5 --- .../replicators/ReplicateUpdateBy.java | 8 ++-- 46 files changed, 83 insertions(+), 132 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java index 984a5c3fa4b..139257e0fd8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java @@ -98,9 +98,6 @@ protected BucketedPartitionedUpdateByManager( control, (oe, se) -> deliverUpdateError(oe, se, true)); - bucket.parentUpdateBy = this; - bucket.createdStep = LogicalClock.DEFAULT.currentStep(); - // add this to the bucket list synchronized (buckets) { buckets.offer(bucket); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java index 2d8684c90bc..21dbb92818e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByBucketHelper.java @@ -57,12 +57,6 @@ class UpdateByBucketHelper extends IntrusiveDoublyLinkedNode.Impl committer; - public UpdateBy parentUpdateBy; - public long createdStep; - public UpdateBy.PhasedUpdateProcessor pup; - /** * Perform updateBy operations on a single bucket of data (either zero-key or already limited through partitioning) * @@ -78,7 +72,7 @@ protected UpdateByBucketHelper( @NotNull final QueryTable source, @NotNull final UpdateByWindow[] windows, @NotNull final Map> resultSources, - @Nullable String timestampColumnName, + @Nullable final String timestampColumnName, @NotNull final UpdateByControl control, @NotNull final BiConsumer failureNotifier) { @@ -118,19 +112,6 @@ protected UpdateByBucketHelper( prepareForUpdate(initialUpdate, true); - if (UpdateGraphProcessor.DEFAULT.isRefreshThread()) { - committer = new UpdateCommitter<>(UpdateByBucketHelper.this, (bucket) -> { - // ensure that the item has been cleaned up - if (bucket.isDirty) { - System.out.printf("%d-Failing bucket %s, refreshThread=%b, clockValue=%d%n", - UpdateByBucketHelper.this.hashCode(), description, - UpdateGraphProcessor.DEFAULT.isRefreshThread(), - LogicalClock.DEFAULT.currentValue()); - } - }); - committer.maybeActivate(); - } - initialUpdate.release(); if (source.isRefreshing()) { @@ -312,11 +293,7 @@ private int fillChunkWithNonNull(LongChunk keysChunk, LongChunk< * @param initialStep Whether this update is part of the initial creation of the bucket */ public void prepareForUpdate(final TableUpdate upstream, final boolean initialStep) { - Assert.eqFalse(isDirty, "UpdateBy bucket was marked dirty before processing an update"); - - System.out.printf("%d-Preparing bucket %s, initial=%b, refreshThread=%b, clockValue=%d%n", hashCode(), - description, initialStep, UpdateGraphProcessor.DEFAULT.isRefreshThread(), - LogicalClock.DEFAULT.currentValue()); + Assert.eqFalse(isDirty, "UpdateByBucketHelper.isDirty"); final boolean timestampsModified; @@ -392,9 +369,6 @@ public void processWindow(final int winIdx, final boolean initialStep) { * Close the window contexts and release resources for this bucket */ public void finalizeUpdate() { - System.out.printf("%d-Finalizing bucket %s, clockValue=%d%n", hashCode(), description, - LogicalClock.DEFAULT.currentValue()); - SafeCloseableArray.close(windowContexts); isDirty = false; } @@ -413,18 +387,6 @@ private UpdateByBucketHelperListener(@Nullable final String description, @NotNul @Override public void onUpdate(TableUpdate upstream) { prepareForUpdate(upstream, false); - - UpdateByBucketHelper.this.committer = - new UpdateCommitter<>(UpdateByBucketHelper.this, (bucket) -> { - // ensure that the item has been cleaned up - if (bucket.isDirty) { - System.out.printf("%d-Failing bucket %s, refreshThread=%b, clockValue=%d%n", - UpdateByBucketHelper.this.hashCode(), description, - UpdateGraphProcessor.DEFAULT.isRefreshThread(), - LogicalClock.DEFAULT.currentValue()); - } - }); - UpdateByBucketHelper.this.committer.maybeActivate(); } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperator.java index 50423c36d2c..cb157b597f0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperator.java @@ -43,7 +43,7 @@ public abstract class UpdateByOperator { protected final long forwardWindowScaleUnits; protected final String timestampColumnName; - protected final boolean isWindowed; + final boolean isWindowed; /** * The input modifiedColumnSet for this operator @@ -57,7 +57,7 @@ public abstract class UpdateByOperator { /** * A context item for use with updateBy operators */ - public abstract class Context implements SafeCloseable { + public abstract static class Context implements SafeCloseable { protected final Chunk[] chunkArr; protected int nullCount = 0; @@ -102,7 +102,7 @@ public abstract void accumulateCumulative(RowSequence inputKeys, int len); public abstract void accumulateRolling(RowSequence inputKeys, - Chunk influencerValueChunkArr[], + Chunk[] influencerValueChunkArr, IntChunk pushChunk, IntChunk popChunk, int len); @@ -150,7 +150,7 @@ public void initializeCumulative(@NotNull final Context context, final long firs long firstUnmodifiedTimestamp) {} /** - * Initialize the bucket context for s windowed operator + * Initialize the bucket context for a windowed operator */ public void initializeRolling(@NotNull final Context context) {} @@ -216,16 +216,6 @@ protected String[] getOutputColumnNames() { @NotNull protected abstract Map> getOutputColumns(); - /** - * Whether this operator supports windows (is rolling operator) - * - * @return true if the operator is windowed, false if cumulative - */ - @NotNull - protected boolean getIsWindowed() { - return isWindowed; - } - /** * Indicate that the operation should start tracking previous values for ticking updates. */ diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java index 8bc564f0ac6..042bee96db1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/BooleanFillByOperator.java @@ -57,7 +57,7 @@ public BooleanFillByOperator(@NotNull final MatchPair fillPair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection, false); + super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java index d9b64c53afd..1ef6a5c15c3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ByteFillByOperator.java @@ -50,7 +50,7 @@ public ByteFillByOperator(@NotNull final MatchPair fillPair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection, false); + super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java index 5a2f2b56864..780def8b4b6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java @@ -45,7 +45,7 @@ public CharFillByOperator(@NotNull final MatchPair fillPair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection, false); + super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java index ed2ff529cbc..bf738831ab0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/DoubleFillByOperator.java @@ -50,7 +50,7 @@ public DoubleFillByOperator(@NotNull final MatchPair fillPair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection, false); + super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java index 7610c7b1877..9c15c947f5d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/FloatFillByOperator.java @@ -50,7 +50,7 @@ public FloatFillByOperator(@NotNull final MatchPair fillPair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection, false); + super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java index 018823d839e..d0b57de8c03 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/IntFillByOperator.java @@ -50,7 +50,7 @@ public IntFillByOperator(@NotNull final MatchPair fillPair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection, false); + super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java index 9b50252377e..4d8a2dd54c4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/LongFillByOperator.java @@ -59,7 +59,7 @@ public LongFillByOperator(@NotNull final MatchPair fillPair, ,@NotNull final Class type // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection, false); + super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection); // region constructor this.type = type; // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java index ebd54d0441b..20d98b45b1b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ObjectFillByOperator.java @@ -52,7 +52,7 @@ public ObjectFillByOperator(@NotNull final MatchPair fillPair, , final Class colType // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection, false, colType); + super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection, colType); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java index 6e0d28c0e96..5b61aec06ea 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/ShortFillByOperator.java @@ -50,7 +50,7 @@ public ShortFillByOperator(@NotNull final MatchPair fillPair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection, false); + super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java index eae12354867..f8c457e1212 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java @@ -147,12 +147,11 @@ public void close() { */ public BaseByteUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final RowRedirection rowRedirection, - final boolean isWindowed + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - this(pair, affectingColumns, rowRedirection, null, 0, 0, isWindowed); + this(pair, affectingColumns, rowRedirection, null, 0, 0, false); } /** @@ -164,7 +163,9 @@ public BaseByteUpdateByOperator(@NotNull final MatchPair pair, * @param rowRedirection the {@link RowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. - * @param reverseWindowScaleUnits the smoothing window for the operator. If no {@code timestampColumnName} is provided, this + * @param reverseWindowScaleUnits the reverse window for the operator. If no {@code timestampColumnName} is provided, this + * is measured in ticks, otherwise it is measured in nanoseconds. + * @param forwardWindowScaleUnits the forward window for the operator. If no {@code timestampColumnName} is provided, this * is measured in ticks, otherwise it is measured in nanoseconds. */ public BaseByteUpdateByOperator(@NotNull final MatchPair pair, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java index 48fa2dcb883..b47fb977421 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java @@ -136,12 +136,11 @@ public void close() { */ public BaseCharUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final RowRedirection rowRedirection, - final boolean isWindowed + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - this(pair, affectingColumns, rowRedirection, null, 0, 0, isWindowed); + this(pair, affectingColumns, rowRedirection, null, 0, 0, false); } /** @@ -153,7 +152,9 @@ public BaseCharUpdateByOperator(@NotNull final MatchPair pair, * @param rowRedirection the {@link RowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. - * @param reverseWindowScaleUnits the smoothing window for the operator. If no {@code timestampColumnName} is provided, this + * @param reverseWindowScaleUnits the reverse window for the operator. If no {@code timestampColumnName} is provided, this + * is measured in ticks, otherwise it is measured in nanoseconds. + * @param forwardWindowScaleUnits the forward window for the operator. If no {@code timestampColumnName} is provided, this * is measured in ticks, otherwise it is measured in nanoseconds. */ public BaseCharUpdateByOperator(@NotNull final MatchPair pair, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java index b56dcf7dc65..965ab174640 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java @@ -141,12 +141,11 @@ public void close() { */ public BaseDoubleUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final RowRedirection rowRedirection, - final boolean isWindowed + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - this(pair, affectingColumns, rowRedirection, null, 0, 0, isWindowed); + this(pair, affectingColumns, rowRedirection, null, 0, 0, false); } /** @@ -158,7 +157,9 @@ public BaseDoubleUpdateByOperator(@NotNull final MatchPair pair, * @param rowRedirection the {@link RowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. - * @param reverseWindowScaleUnits the smoothing window for the operator. If no {@code timestampColumnName} is provided, this + * @param reverseWindowScaleUnits the reverse window for the operator. If no {@code timestampColumnName} is provided, this + * is measured in ticks, otherwise it is measured in nanoseconds. + * @param forwardWindowScaleUnits the forward window for the operator. If no {@code timestampColumnName} is provided, this * is measured in ticks, otherwise it is measured in nanoseconds. */ public BaseDoubleUpdateByOperator(@NotNull final MatchPair pair, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java index 76667ba6be2..c9303ea0e02 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java @@ -141,12 +141,11 @@ public void close() { */ public BaseFloatUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final RowRedirection rowRedirection, - final boolean isWindowed + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - this(pair, affectingColumns, rowRedirection, null, 0, 0, isWindowed); + this(pair, affectingColumns, rowRedirection, null, 0, 0, false); } /** @@ -158,7 +157,9 @@ public BaseFloatUpdateByOperator(@NotNull final MatchPair pair, * @param rowRedirection the {@link RowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. - * @param reverseWindowScaleUnits the smoothing window for the operator. If no {@code timestampColumnName} is provided, this + * @param reverseWindowScaleUnits the reverse window for the operator. If no {@code timestampColumnName} is provided, this + * is measured in ticks, otherwise it is measured in nanoseconds. + * @param forwardWindowScaleUnits the forward window for the operator. If no {@code timestampColumnName} is provided, this * is measured in ticks, otherwise it is measured in nanoseconds. */ public BaseFloatUpdateByOperator(@NotNull final MatchPair pair, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java index 99458bbbfc3..3cb2adea89f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java @@ -141,12 +141,11 @@ public void close() { */ public BaseIntUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final RowRedirection rowRedirection, - final boolean isWindowed + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - this(pair, affectingColumns, rowRedirection, null, 0, 0, isWindowed); + this(pair, affectingColumns, rowRedirection, null, 0, 0, false); } /** @@ -158,7 +157,9 @@ public BaseIntUpdateByOperator(@NotNull final MatchPair pair, * @param rowRedirection the {@link RowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. - * @param reverseWindowScaleUnits the smoothing window for the operator. If no {@code timestampColumnName} is provided, this + * @param reverseWindowScaleUnits the reverse window for the operator. If no {@code timestampColumnName} is provided, this + * is measured in ticks, otherwise it is measured in nanoseconds. + * @param forwardWindowScaleUnits the forward window for the operator. If no {@code timestampColumnName} is provided, this * is measured in ticks, otherwise it is measured in nanoseconds. */ public BaseIntUpdateByOperator(@NotNull final MatchPair pair, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java index 4de5666546b..3c3dca78e0b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java @@ -141,12 +141,11 @@ public void close() { */ public BaseLongUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final RowRedirection rowRedirection, - final boolean isWindowed + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - this(pair, affectingColumns, rowRedirection, null, 0, 0, isWindowed); + this(pair, affectingColumns, rowRedirection, null, 0, 0, false); } /** @@ -158,7 +157,9 @@ public BaseLongUpdateByOperator(@NotNull final MatchPair pair, * @param rowRedirection the {@link RowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. - * @param reverseWindowScaleUnits the smoothing window for the operator. If no {@code timestampColumnName} is provided, this + * @param reverseWindowScaleUnits the reverse window for the operator. If no {@code timestampColumnName} is provided, this + * is measured in ticks, otherwise it is measured in nanoseconds. + * @param forwardWindowScaleUnits the forward window for the operator. If no {@code timestampColumnName} is provided, this * is measured in ticks, otherwise it is measured in nanoseconds. */ public BaseLongUpdateByOperator(@NotNull final MatchPair pair, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java index 036fdf5bd00..ca4ebab7de7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectBinaryOperator.java @@ -44,9 +44,8 @@ public void reset() { public BaseObjectBinaryOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, @Nullable final RowRedirection rowRedirection, - final boolean isWindowed, @NotNull final Class type) { - super(pair, affectingColumns, rowRedirection, isWindowed, type); + super(pair, affectingColumns, rowRedirection, type); } protected abstract T doOperation(T bucketCurVal, T chunkCurVal); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java index f7d6338e6d4..489ddf39ad7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java @@ -144,13 +144,12 @@ public void close() { */ public BaseObjectUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final RowRedirection rowRedirection, - final boolean isWindowed + @Nullable final RowRedirection rowRedirection // region extra-constructor-args , final Class colType // endregion extra-constructor-args ) { - this(pair, affectingColumns, rowRedirection, null, 0, 0, isWindowed, colType); + this(pair, affectingColumns, rowRedirection, null, 0, 0, false, colType); } /** @@ -162,7 +161,9 @@ public BaseObjectUpdateByOperator(@NotNull final MatchPair pair, * @param rowRedirection the {@link RowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. - * @param reverseWindowScaleUnits the smoothing window for the operator. If no {@code timestampColumnName} is provided, this + * @param reverseWindowScaleUnits the reverse window for the operator. If no {@code timestampColumnName} is provided, this + * is measured in ticks, otherwise it is measured in nanoseconds. + * @param forwardWindowScaleUnits the forward window for the operator. If no {@code timestampColumnName} is provided, this * is measured in ticks, otherwise it is measured in nanoseconds. */ public BaseObjectUpdateByOperator(@NotNull final MatchPair pair, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java index e8337859382..a9482fba484 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java @@ -141,12 +141,11 @@ public void close() { */ public BaseShortUpdateByOperator(@NotNull final MatchPair pair, @NotNull final String[] affectingColumns, - @Nullable final RowRedirection rowRedirection, - final boolean isWindowed + @Nullable final RowRedirection rowRedirection // region extra-constructor-args // endregion extra-constructor-args ) { - this(pair, affectingColumns, rowRedirection, null, 0, 0, isWindowed); + this(pair, affectingColumns, rowRedirection, null, 0, 0, false); } /** @@ -158,7 +157,9 @@ public BaseShortUpdateByOperator(@NotNull final MatchPair pair, * @param rowRedirection the {@link RowRedirection} for the output column * @param timestampColumnName an optional timestamp column. If this is null, it will be assumed time is measured in * integer ticks. - * @param reverseWindowScaleUnits the smoothing window for the operator. If no {@code timestampColumnName} is provided, this + * @param reverseWindowScaleUnits the reverse window for the operator. If no {@code timestampColumnName} is provided, this + * is measured in ticks, otherwise it is measured in nanoseconds. + * @param forwardWindowScaleUnits the forward window for the operator. If no {@code timestampColumnName} is provided, this * is measured in ticks, otherwise it is measured in nanoseconds. */ public BaseShortUpdateByOperator(@NotNull final MatchPair pair, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java index aaac21ecb23..8deae513dc1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ByteCumMinMaxOperator.java @@ -59,7 +59,7 @@ public ByteCumMinMaxOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection, false); + super(pair, new String[] { pair.rightColumn }, rowRedirection); this.isMax = isMax; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ComparableCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ComparableCumMinMaxOperator.java index 1a1e0b1c139..b3f7c962bd3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ComparableCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ComparableCumMinMaxOperator.java @@ -13,7 +13,7 @@ public ComparableCumMinMaxOperator(@NotNull final MatchPair inputPair, final boolean isMax, @Nullable final RowRedirection rowRedirection, final Class colType) { - super(inputPair, new String[] {inputPair.rightColumn}, rowRedirection, false, colType); + super(inputPair, new String[] {inputPair.rightColumn}, rowRedirection, colType); this.isMax = isMax; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java index ab9492de8e9..777732c2b98 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/DoubleCumMinMaxOperator.java @@ -59,7 +59,7 @@ public DoubleCumMinMaxOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection, false); + super(pair, new String[] { pair.rightColumn }, rowRedirection); this.isMax = isMax; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java index be0f0fb0eb2..b8f5c708639 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/FloatCumMinMaxOperator.java @@ -59,7 +59,7 @@ public FloatCumMinMaxOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection, false); + super(pair, new String[] { pair.rightColumn }, rowRedirection); this.isMax = isMax; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java index edec0a79ee6..26042c82bca 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/IntCumMinMaxOperator.java @@ -59,7 +59,7 @@ public IntCumMinMaxOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection, false); + super(pair, new String[] { pair.rightColumn }, rowRedirection); this.isMax = isMax; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java index c195d2b339a..5aa25777665 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/LongCumMinMaxOperator.java @@ -68,7 +68,7 @@ public LongCumMinMaxOperator(@NotNull final MatchPair pair, ,@NotNull final Class type // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection, false); + super(pair, new String[] { pair.rightColumn }, rowRedirection); this.isMax = isMax; // region constructor this.type = type; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java index ae789326766..24a5f407d2e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/minmax/ShortCumMinMaxOperator.java @@ -54,7 +54,7 @@ public ShortCumMinMaxOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection, false); + super(pair, new String[] { pair.rightColumn }, rowRedirection); this.isMax = isMax; // region constructor // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java index 6f8586d6be8..a204c2a98c0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigDecimalCumProdOperator.java @@ -50,7 +50,7 @@ public void reset() { public BigDecimalCumProdOperator(@NotNull final MatchPair inputPair, @Nullable final RowRedirection rowRedirection, @NotNull final MathContext mathContext) { - super(inputPair, new String[] {inputPair.rightColumn}, rowRedirection, false, BigDecimal.class); + super(inputPair, new String[] {inputPair.rightColumn}, rowRedirection, BigDecimal.class); this.mathContext = mathContext; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java index e3593a9c0dc..b4450199098 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/BigIntegerCumProdOperator.java @@ -45,7 +45,7 @@ public void reset() { public BigIntegerCumProdOperator(@NotNull final MatchPair inputPair, @Nullable final RowRedirection rowRedirection) { - super(inputPair, new String[] {inputPair.rightColumn}, rowRedirection, false, BigInteger.class); + super(inputPair, new String[] {inputPair.rightColumn}, rowRedirection, BigInteger.class); } @NotNull diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java index 2d7bfa3a497..fba66b78429 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ByteCumProdOperator.java @@ -52,7 +52,7 @@ public ByteCumProdOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection, false); + super(pair, new String[] { pair.rightColumn }, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java index 0536746c630..890e47e21f0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/DoubleCumProdOperator.java @@ -51,7 +51,7 @@ public DoubleCumProdOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection, false); + super(pair, new String[] { pair.rightColumn }, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java index 7ed159d88e3..a04dea385a6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/FloatCumProdOperator.java @@ -46,7 +46,7 @@ public FloatCumProdOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection, false); + super(pair, new String[] { pair.rightColumn }, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java index 1d817163813..8aa7535dc40 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/IntCumProdOperator.java @@ -52,7 +52,7 @@ public IntCumProdOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection, false); + super(pair, new String[] { pair.rightColumn }, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java index 9bb7bb15345..4c8fae1487f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/LongCumProdOperator.java @@ -52,7 +52,7 @@ public LongCumProdOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection, false); + super(pair, new String[] { pair.rightColumn }, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java index 1e952539ad5..5eabaaa2a95 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/prod/ShortCumProdOperator.java @@ -47,7 +47,7 @@ public ShortCumProdOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection, false); + super(pair, new String[] { pair.rightColumn }, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java index 93131b8e368..ec195f16aa1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigDecimalCumSumOperator.java @@ -52,7 +52,7 @@ public void push(long key, int pos, int count) { public BigDecimalCumSumOperator(@NotNull final MatchPair inputPair, @Nullable final RowRedirection rowRedirection, @NotNull final MathContext mathContext) { - super(inputPair, new String[] {inputPair.rightColumn}, rowRedirection, false, BigDecimal.class); + super(inputPair, new String[] {inputPair.rightColumn}, rowRedirection, BigDecimal.class); this.mathContext = mathContext; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java index 8f993cabeec..8cab2aa2eed 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/BigIntegerCumSumOperator.java @@ -46,7 +46,7 @@ public void push(long key, int pos, int count) { public BigIntegerCumSumOperator(@NotNull final MatchPair inputPair, @Nullable final RowRedirection rowRedirection) { - super(inputPair, new String[] {inputPair.rightColumn}, rowRedirection, false, BigInteger.class); + super(inputPair, new String[] {inputPair.rightColumn}, rowRedirection, BigInteger.class); } @NotNull diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java index 4fe201c173c..13b5b19db40 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ByteCumSumOperator.java @@ -57,7 +57,7 @@ public ByteCumSumOperator(@NotNull final MatchPair pair, ,final byte nullValue // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection, false); + super(pair, new String[] { pair.rightColumn }, rowRedirection); // region constructor this.nullValue = nullValue; // endregion constructor diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java index 65e0d2ea78c..57ef9fccc02 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/DoubleCumSumOperator.java @@ -54,7 +54,7 @@ public DoubleCumSumOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection, false); + super(pair, new String[] { pair.rightColumn }, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java index e11752bedce..a5b2bc9fcdf 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/FloatCumSumOperator.java @@ -49,7 +49,7 @@ public FloatCumSumOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection, false); + super(pair, new String[] { pair.rightColumn }, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java index df9d84c24eb..adbac668892 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/IntCumSumOperator.java @@ -55,7 +55,7 @@ public IntCumSumOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection, false); + super(pair, new String[] { pair.rightColumn }, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java index 84d2a8dd9f4..afc757ec429 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/LongCumSumOperator.java @@ -55,7 +55,7 @@ public LongCumSumOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection, false); + super(pair, new String[] { pair.rightColumn }, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java index c17d9e17d12..2521a7de3e4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/sum/ShortCumSumOperator.java @@ -50,7 +50,7 @@ public ShortCumSumOperator(@NotNull final MatchPair pair, // region extra-constructor-args // endregion extra-constructor-args ) { - super(pair, new String[] { pair.rightColumn }, rowRedirection, false); + super(pair, new String[] { pair.rightColumn }, rowRedirection); // region constructor // endregion constructor } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java index 0f1ae43af5f..6a060cf3530 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestUpdateByGeneral.java @@ -36,7 +36,6 @@ public class TestUpdateByGeneral extends BaseUpdateByTest { public void testMixedAppendOnlyZeroKey() { for (int size = 10; size <= 10000; size *= 10) { for (int seed = 10; seed < 20; seed++) { - System.out.println("Stating test: size " + size + ", seed " + seed); doTestTicking(seed > 15, false, true, 20, size, seed); } } @@ -46,7 +45,6 @@ public void testMixedAppendOnlyZeroKey() { public void testMixedAppendOnlyBucketed() { for (int size = 10; size <= 10000; size *= 10) { for (int seed = 10; seed < 20; seed++) { - System.out.println("Stating test: size " + size + ", seed " + seed); doTestTicking(seed > 15, true, true, 20, size, seed); } } @@ -56,7 +54,6 @@ public void testMixedAppendOnlyBucketed() { public void testMixedGeneralZeroKey() { for (int size = 10; size <= 10000; size *= 10) { for (int seed = 10; seed < 20; seed++) { - System.out.println("Stating test: size " + size + ", seed " + seed); doTestTicking(seed > 15, false, false, 20, size, seed); } } @@ -66,7 +63,6 @@ public void testMixedGeneralZeroKey() { public void testMixedGeneralBucketed() { for (int size = 10; size <= 10000; size *= 10) { for (int seed = 10; seed < 20; seed++) { - System.out.println("Stating test: size " + size + ", seed " + seed); doTestTicking(seed > 15, true, false, 20, size, seed); } } @@ -134,7 +130,6 @@ protected Table e() { }; for (int step = 0; step < steps; step++) { - System.out.println(" beginning step " + step); try { if (appendOnly) { UpdateGraphProcessor.DEFAULT.runWithinUnitTestCycle(() -> { diff --git a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java index 860dcbbe7c8..b63a82da3e4 100644 --- a/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java +++ b/replication/static/src/main/java/io/deephaven/replicators/ReplicateUpdateBy.java @@ -34,8 +34,8 @@ public static void main(String[] args) throws IOException { String objectResult = ReplicatePrimitiveCode.charToObject( "engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/fill/CharFillByOperator.java"); fixupStandardObject(objectResult, "ObjectFillByOperator", false, - "super\\(fillPair, new String\\[\\] \\{ fillPair.rightColumn \\}, rowRedirection, false\\);", - "super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection, false, colType);", + "super\\(fillPair, new String\\[\\] \\{ fillPair.rightColumn \\}, rowRedirection\\);", + "super(fillPair, new String[] { fillPair.rightColumn }, rowRedirection, colType);", " BaseObjectUpdateByOperator", " BaseObjectUpdateByOperator", "public ObjectChunk Date: Tue, 31 Jan 2023 16:40:39 -0800 Subject: [PATCH 116/123] Merge-related bug fix --- .../table/impl/updateby/BucketedPartitionedUpdateByManager.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java index 2ab7c144c6d..98270f2bf33 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java @@ -42,7 +42,6 @@ class BucketedPartitionedUpdateByManager extends UpdateBy { /** * Perform a bucketed updateBy using {@code byColumns} as the keys * - * @param description the operation description * @param operators the operations to perform * @param windows the unique windows for this UpdateBy * @param inputSources the primitive input sources @@ -55,7 +54,6 @@ class BucketedPartitionedUpdateByManager extends UpdateBy { * @param control the control object. */ protected BucketedPartitionedUpdateByManager( - @NotNull final String description, @NotNull final UpdateByOperator[] operators, @NotNull final UpdateByWindow[] windows, @NotNull final ColumnSource[] inputSources, From ff5ef49f40a6ab6e2ed0eb0e3f17eff7e399bd46 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 31 Jan 2023 17:54:24 -0800 Subject: [PATCH 117/123] Added TODO and linked the created tickets --- .../java/io/deephaven/engine/table/impl/updateby/UpdateBy.java | 3 +++ .../main/java/io/deephaven/client/impl/UpdateByBuilder.java | 2 +- .../java/io/deephaven/client/impl/UpdateBySpecBuilderTest.java | 2 +- 3 files changed, 5 insertions(+), 2 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java index 1c079a6f821..a98ac9a966b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java @@ -1110,6 +1110,9 @@ public boolean equalKey(UpdateByOperator updateByOperator, String.join(", ", problems) + "}"); } + // TODO: test whether the source is static and that UpdateBy call uses only cumulative operators. In this + // case, we can use an optimized manager and a single pass through the column sources (DHC #3393) + return LivenessScopeStack.computeEnclosed(() -> { final BucketedPartitionedUpdateByManager bm = new BucketedPartitionedUpdateByManager( opArr, diff --git a/java-client/session/src/main/java/io/deephaven/client/impl/UpdateByBuilder.java b/java-client/session/src/main/java/io/deephaven/client/impl/UpdateByBuilder.java index 9905ca77f87..0b8ca03efd3 100644 --- a/java-client/session/src/main/java/io/deephaven/client/impl/UpdateByBuilder.java +++ b/java-client/session/src/main/java/io/deephaven/client/impl/UpdateByBuilder.java @@ -152,7 +152,7 @@ public UpdateByColumn.UpdateBySpec visit(CumProdSpec p) { .build(); } - // TODO: add this correctly to `table.proto` + // TODO: add this correctly to `table.proto` (DHC #3392) @Override public UpdateByColumn.UpdateBySpec visit(RollingSumSpec rs) { return null; diff --git a/java-client/session/src/test/java/io/deephaven/client/impl/UpdateBySpecBuilderTest.java b/java-client/session/src/test/java/io/deephaven/client/impl/UpdateBySpecBuilderTest.java index 35445cb3dd3..fbe0c767236 100644 --- a/java-client/session/src/test/java/io/deephaven/client/impl/UpdateBySpecBuilderTest.java +++ b/java-client/session/src/test/java/io/deephaven/client/impl/UpdateBySpecBuilderTest.java @@ -68,7 +68,7 @@ public UpdateByColumn.UpdateBySpec visit(CumProdSpec p) { .build(); } - // TODO: add this properly + // TODO: add this correctly (DHC #3392) @Override public UpdateByColumn.UpdateBySpec visit(RollingSumSpec p) { return null; From 51c484d751a6e009e103b1ba474eec23b9f1e5c8 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 31 Jan 2023 18:38:10 -0800 Subject: [PATCH 118/123] TestTableValidator changes --- .../java/io/deephaven/engine/table/impl/TestTableValidator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestTableValidator.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestTableValidator.java index 595aeee465c..aea80fe8d4d 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestTableValidator.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestTableValidator.java @@ -36,7 +36,7 @@ public void testValidator() { final EvalNugget[] en = new EvalNugget[] { new EvalNugget() { public Table e() { - return queryTable; + return queryTable.copy(); } }, }; From dd37fe5de1088a1784a173e34ee551725d4dd075 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 31 Jan 2023 18:42:02 -0800 Subject: [PATCH 119/123] Spotless fix --- .../io/deephaven/engine/table/impl/updateby/UpdateBy.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java index a98ac9a966b..3b69b62c96a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateBy.java @@ -1110,8 +1110,8 @@ public boolean equalKey(UpdateByOperator updateByOperator, String.join(", ", problems) + "}"); } - // TODO: test whether the source is static and that UpdateBy call uses only cumulative operators. In this - // case, we can use an optimized manager and a single pass through the column sources (DHC #3393) + // TODO: test whether the source is static and that UpdateBy call uses only cumulative operators. In this + // case, we can use an optimized manager and a single pass through the column sources (DHC #3393) return LivenessScopeStack.computeEnclosed(() -> { final BucketedPartitionedUpdateByManager bm = new BucketedPartitionedUpdateByManager( From 8f3f559f14aecb52cca8e5ef2fd28126a2fcab03 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 1 Feb 2023 09:01:59 -0800 Subject: [PATCH 120/123] Change to EvalNugget to pass unchanged table references --- .../java/io/deephaven/engine/table/impl/TestTableValidator.java | 2 +- .../src/main/java/io/deephaven/engine/testutil/EvalNugget.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestTableValidator.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestTableValidator.java index aea80fe8d4d..595aeee465c 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestTableValidator.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestTableValidator.java @@ -36,7 +36,7 @@ public void testValidator() { final EvalNugget[] en = new EvalNugget[] { new EvalNugget() { public Table e() { - return queryTable.copy(); + return queryTable; } }, }; diff --git a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/EvalNugget.java b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/EvalNugget.java index b2aca26a2eb..52cf4a688d6 100644 --- a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/EvalNugget.java +++ b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/EvalNugget.java @@ -171,7 +171,7 @@ public void show() { @Override public void releaseRecomputed() { - if (recomputedTable != null) { + if (recomputedTable != null && recomputedTable != originalValue) { if (recomputedTable.tryRetainReference()) { recomputedTable.dropReference(); throw new IllegalStateException("Recomputed table " + recomputedTable + " is still live upon release"); From b9a7cceed37d1f1b77e30203421c7a16985c9385 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 1 Feb 2023 14:34:24 -0800 Subject: [PATCH 121/123] Supressed exceptions in EngineCleanup --- .../io/deephaven/engine/testutil/junit4/EngineCleanup.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/junit4/EngineCleanup.java b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/junit4/EngineCleanup.java index 0effeccf513..794dc6720d4 100644 --- a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/junit4/EngineCleanup.java +++ b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/junit4/EngineCleanup.java @@ -45,10 +45,8 @@ public Statement apply(Statement statement, Description description) { @Override public void evaluate() throws Throwable { setUp(); - try { + try (final AutoCloseable ignored = () -> tearDown()) { statement.evaluate(); - } finally { - tearDown(); } } }; From 761b04440261b6de2ee29065234875052510c7b6 Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Wed, 1 Feb 2023 23:18:37 -0500 Subject: [PATCH 122/123] LivenessScopeStack.computeEnclosed and LivenessScopeStack.computeArrayEnclosed should ensure reachability to make sure referents can't escape the LivenessScope.release. --- .../io/deephaven/engine/liveness/LivenessScopeStack.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/engine/updategraph/src/main/java/io/deephaven/engine/liveness/LivenessScopeStack.java b/engine/updategraph/src/main/java/io/deephaven/engine/liveness/LivenessScopeStack.java index 2ec973cb83c..59296179820 100644 --- a/engine/updategraph/src/main/java/io/deephaven/engine/liveness/LivenessScopeStack.java +++ b/engine/updategraph/src/main/java/io/deephaven/engine/liveness/LivenessScopeStack.java @@ -173,7 +173,9 @@ public static RESULT_TYPE computeEnclosed @NotNull final BooleanSupplier shouldEnclose, @NotNull final Predicate shouldManageResult) { final LivenessManager enclosingLivenessManager = LivenessScopeStack.peek(); - try (final SafeCloseable ignored = shouldEnclose.getAsBoolean() ? LivenessScopeStack.open() : null) { + try (final SafeCloseable ignored = shouldEnclose.getAsBoolean() + ? LivenessScopeStack.open(new LivenessScope(true), true) + : null) { final RESULT_TYPE result = computation.get(); if (shouldManageResult.test(result)) { enclosingLivenessManager.manage(result); @@ -199,7 +201,9 @@ public static RESULT_TYPE[] computeArrayE @NotNull final BooleanSupplier shouldEnclose, @NotNull final Predicate shouldManageResult) { final LivenessManager enclosingLivenessManager = LivenessScopeStack.peek(); - try (final SafeCloseable ignored = shouldEnclose.getAsBoolean() ? LivenessScopeStack.open() : null) { + try (final SafeCloseable ignored = shouldEnclose.getAsBoolean() + ? LivenessScopeStack.open(new LivenessScope(true), true) + : null) { final RESULT_TYPE[] results = computation.get(); for (final RESULT_TYPE result : results) { if (shouldManageResult.test(result)) { From 93bec77483b19d61b59acb15afaa11e3a71c85e7 Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Wed, 1 Feb 2023 23:18:43 -0500 Subject: [PATCH 123/123] Refactor BucketedPartitionedUpdateByManager a little to put all the "if source.isRefreshing()" code in one place, and to ensure proper proactive liveness cleanup for the PartitionedTables. --- .../BucketedPartitionedUpdateByManager.java | 63 +++++++++---------- 1 file changed, 31 insertions(+), 32 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java index 98270f2bf33..13cbba3bef0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/BucketedPartitionedUpdateByManager.java @@ -3,6 +3,7 @@ import io.deephaven.api.ColumnName; import io.deephaven.api.updateby.UpdateByControl; import io.deephaven.base.verify.Assert; +import io.deephaven.engine.liveness.LivenessScopeStack; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.rowset.RowSetShiftData; import io.deephaven.engine.table.*; @@ -10,6 +11,7 @@ import io.deephaven.engine.table.impl.QueryTable; import io.deephaven.engine.table.impl.TableUpdateImpl; import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.updategraph.DynamicNode; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -71,7 +73,32 @@ protected BucketedPartitionedUpdateByManager( final String[] byColumnNames = byColumns.stream().map(ColumnName::name).toArray(String[]::new); - final PartitionedTable pt; + final Table transformedTable = LivenessScopeStack.computeEnclosed(() -> { + final PartitionedTable partitioned = source.partitionedAggBy(List.of(), true, null, byColumnNames); + final PartitionedTable transformed = partitioned.transform(t -> { + final long firstSourceRowKey = t.getRowSet().firstRowKey(); + final String bucketDescription = BucketedPartitionedUpdateByManager.this + "-bucket-" + + Arrays.stream(byColumnNames) + .map(bcn -> Objects.toString(t.getColumnSource(bcn).get(firstSourceRowKey))) + .collect(Collectors.joining(", ", "[", "]")); + UpdateByBucketHelper bucket = new UpdateByBucketHelper( + bucketDescription, + (QueryTable) t, + windows, + resultSources, + timestampColumnName, + control, + this::onBucketFailure); + // add this to the bucket list + synchronized (buckets) { + buckets.offer(bucket); + } + // return the table + return bucket.result; + }); + return transformed.table(); + }, source::isRefreshing, DynamicNode::isRefreshing); + if (source.isRefreshing()) { // this is a refreshing source, we will need a listener sourceListener = newUpdateByListener(); @@ -84,45 +111,17 @@ protected BucketedPartitionedUpdateByManager( op.createInputModifiedColumnSet(source); op.createOutputModifiedColumnSet(result); } - pt = source.partitionedAggBy(List.of(), true, null, byColumnNames); // make the source->result transformer from only the columns in the source that are present in result mcsTransformer = source.newModifiedColumnSetTransformer(result, preservedColumns); - } else { - sourceListener = null; - mcsTransformer = null; - pt = source.partitionedAggBy(List.of(), true, null, byColumnNames); - } - final PartitionedTable transformed = pt.transform(t -> { - final long firstSourceRowKey = t.getRowSet().firstRowKey(); - final String bucketDescription = BucketedPartitionedUpdateByManager.this + "-bucket-" + - Arrays.stream(byColumnNames) - .map(bcn -> Objects.toString(t.getColumnSource(bcn).get(firstSourceRowKey))) - .collect(Collectors.joining(", ", "[", "]")); - UpdateByBucketHelper bucket = new UpdateByBucketHelper( - bucketDescription, - (QueryTable) t, - windows, - resultSources, - timestampColumnName, - control, - this::onBucketFailure); - - // add this to the bucket list - synchronized (buckets) { - buckets.offer(bucket); - } - // return the table - return bucket.result; - }); - - if (source.isRefreshing()) { - final Table transformedTable = transformed.table(); + // we also need to monitor for failures in bucketing or transformation transformFailureListener = new TransformFailureListener(transformedTable); transformedTable.addUpdateListener(transformFailureListener); result.addParentReference(transformFailureListener); } else { + sourceListener = null; + mcsTransformer = null; transformFailureListener = null; }