From c0bdf60cda6fe40f6ccc27eef7c3266ad1aa20f0 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 9 Feb 2022 10:03:30 -0800 Subject: [PATCH 01/20] work in progress --- .../table/impl/remote/ConstructSnapshot.java | 8 +++++--- .../deephaven/client/impl/BarrageSnapshot.java | 8 ++++++++ .../client/impl/BarrageSnapshotImpl.java | 9 ++++++++- .../client/impl/BarrageSubscription.java | 8 ++++++++ .../client/impl/BarrageSubscriptionImpl.java | 11 +++++++++-- .../deephaven/server/arrow/ArrowFlightUtil.java | 10 ++++++++-- .../server/barrage/BarrageMessageProducer.java | 12 +++++++++--- .../server/uri/BarrageTableResolver.java | 16 ++++++++-------- .../barrage/BarrageMessageRoundTripTest.java | 2 +- 9 files changed, 64 insertions(+), 20 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/remote/ConstructSnapshot.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/remote/ConstructSnapshot.java index 1338cd262cd..e230d612916 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/remote/ConstructSnapshot.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/remote/ConstructSnapshot.java @@ -539,9 +539,10 @@ public static BarrageMessage constructBackplaneSnapshot(final Object logIdentity public static BarrageMessage constructBackplaneSnapshotInPositionSpace(final Object logIdentityObject, final BaseTable table, @Nullable final BitSet columnsToSerialize, - @Nullable final RowSet positionsToSnapshot) { + @Nullable final RowSet positionsToSnapshot, + @Nullable final RowSet reversePositionsToSnapshot) { return constructBackplaneSnapshotInPositionSpace(logIdentityObject, table, columnsToSerialize, - positionsToSnapshot, makeSnapshotControl(false, table)); + positionsToSnapshot, reversePositionsToSnapshot, makeSnapshotControl(false, table)); } /** @@ -559,6 +560,7 @@ public static BarrageMessage constructBackplaneSnapshotInPositionSpace(final Obj @NotNull final BaseTable table, @Nullable final BitSet columnsToSerialize, @Nullable final RowSet positionsToSnapshot, + @Nullable final RowSet reversePositionsToSnapshot, @NotNull final SnapshotControl control) { final BarrageMessage snapshot = new BarrageMessage(); @@ -567,7 +569,7 @@ public static BarrageMessage constructBackplaneSnapshotInPositionSpace(final Obj final SnapshotFunction doSnapshot = (usePrev, beforeClockValue) -> { final RowSet keysToSnapshot; - if (positionsToSnapshot == null) { + if (positionsToSnapshot == null && reversePositionsToSnapshot == null) { keysToSnapshot = null; } else if (usePrev) { try (final RowSet prevRowSet = table.getRowSet().copyPrev()) { diff --git a/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSnapshot.java b/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSnapshot.java index 80f4cb66265..b2be6faef01 100644 --- a/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSnapshot.java +++ b/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSnapshot.java @@ -53,4 +53,12 @@ BarrageSnapshot snapshot(TableSpec tableSpec, BarrageSnapshotOptions options) * @return the {@code BarrageTable} */ BarrageTable partialTable(RowSet viewport, BitSet columns) throws InterruptedException; + + /** + * Request a partial snapshot of the data limited by viewport or column set and populate a {@link BarrageTable} with + * the data that is received. Allows the viewport to be reversed. + * + * @return the {@code BarrageTable} + */ + BarrageTable partialTable(RowSet viewport, BitSet columns, boolean reverseViewport) throws InterruptedException; } diff --git a/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSnapshotImpl.java b/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSnapshotImpl.java index 7719131a134..fc3f9896cbd 100644 --- a/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSnapshotImpl.java +++ b/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSnapshotImpl.java @@ -152,6 +152,11 @@ public BarrageTable entireTable() throws InterruptedException { @Override public synchronized BarrageTable partialTable(RowSet viewport, BitSet columns) throws InterruptedException { + return partialTable(viewport, columns, false); + } + + @Override + public synchronized BarrageTable partialTable(RowSet viewport, BitSet columns, boolean reverseViewport) throws InterruptedException { // notify user when connection has already been used and closed if (prevUsed) { throw new UnsupportedOperationException("Snapshot object already used"); @@ -178,7 +183,7 @@ public synchronized BarrageTable partialTable(RowSet viewport, BitSet columns) t // Send the snapshot request: observer.onNext(FlightData.newBuilder() - .setAppMetadata(ByteStringAccess.wrap(makeRequestInternal(viewport, columns, options))) + .setAppMetadata(ByteStringAccess.wrap(makeRequestInternal(viewport, columns, reverseViewport, options))) .build()); observer.onCompleted(); @@ -255,6 +260,7 @@ public LogOutput append(final LogOutput logOutput) { private ByteBuffer makeRequestInternal( @Nullable final RowSet viewport, @Nullable final BitSet columns, + boolean reverseViewport, @Nullable BarrageSnapshotOptions options) { final FlatBufferBuilder metadata = new FlatBufferBuilder(); @@ -278,6 +284,7 @@ private ByteBuffer makeRequestInternal( BarrageSnapshotRequest.addViewport(metadata, vpOffset); BarrageSnapshotRequest.addSnapshotOptions(metadata, optOffset); BarrageSnapshotRequest.addTicket(metadata, ticOffset); + BarrageSnapshotRequest.addReverseViewport(metadata, reverseViewport); metadata.finish(BarrageSnapshotRequest.endBarrageSnapshotRequest(metadata)); final FlatBufferBuilder wrapper = new FlatBufferBuilder(); diff --git a/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSubscription.java b/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSubscription.java index 4cd1cd23cc7..b14fcce994a 100644 --- a/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSubscription.java +++ b/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSubscription.java @@ -56,4 +56,12 @@ BarrageSubscription subscribe(TableSpec tableSpec, BarrageSubscriptionOptions op */ BarrageTable partialTable(RowSet viewport, BitSet columns) throws InterruptedException; + /** + * Request a partial subscription of the data limited by viewport or column set and populate a {@link BarrageTable} + * with the data that is received. Allows the viewport to be reversed. + * + * @return the {@code BarrageTable} + */ + BarrageTable partialTable(RowSet viewport, BitSet columns, boolean reverseViewport) throws InterruptedException; + } diff --git a/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSubscriptionImpl.java b/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSubscriptionImpl.java index c77a43e0982..32baff4e0c8 100644 --- a/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSubscriptionImpl.java +++ b/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSubscriptionImpl.java @@ -134,11 +134,16 @@ public void onCompleted() { @Override public BarrageTable entireTable() { - return partialTable(null, null); + return partialTable(null, null, false); } @Override public synchronized BarrageTable partialTable(RowSet viewport, BitSet columns) { + return partialTable(viewport, columns, false); + } + + @Override + public synchronized BarrageTable partialTable(RowSet viewport, BitSet columns, boolean reverseViewport) { if (!connected) { throw new UncheckedDeephavenException( this + " is no longer an active subscription and cannot be retained further"); @@ -146,7 +151,7 @@ public synchronized BarrageTable partialTable(RowSet viewport, BitSet columns) { if (!subscribed) { // Send the initial subscription: observer.onNext(FlightData.newBuilder() - .setAppMetadata(ByteStringAccess.wrap(makeRequestInternal(viewport, columns, options))) + .setAppMetadata(ByteStringAccess.wrap(makeRequestInternal(viewport, columns, reverseViewport, options))) .build()); subscribed = true; } @@ -192,6 +197,7 @@ public LogOutput append(final LogOutput logOutput) { private ByteBuffer makeRequestInternal( @Nullable final RowSet viewport, @Nullable final BitSet columns, + boolean reverseViewport, @Nullable BarrageSubscriptionOptions options) { final FlatBufferBuilder metadata = new FlatBufferBuilder(); @@ -215,6 +221,7 @@ private ByteBuffer makeRequestInternal( BarrageSubscriptionRequest.addViewport(metadata, vpOffset); BarrageSubscriptionRequest.addSubscriptionOptions(metadata, optOffset); BarrageSubscriptionRequest.addTicket(metadata, ticOffset); + BarrageSubscriptionRequest.addReverseViewport(metadata, reverseViewport); metadata.finish(BarrageSubscriptionRequest.endBarrageSubscriptionRequest(metadata)); final FlatBufferBuilder wrapper = new FlatBufferBuilder(); diff --git a/server/src/main/java/io/deephaven/server/arrow/ArrowFlightUtil.java b/server/src/main/java/io/deephaven/server/arrow/ArrowFlightUtil.java index 7c92dfce22b..cbb3a259614 100644 --- a/server/src/main/java/io/deephaven/server/arrow/ArrowFlightUtil.java +++ b/server/src/main/java/io/deephaven/server/arrow/ArrowFlightUtil.java @@ -468,11 +468,15 @@ public void handleMessage(BarrageProtoUtil.MessageInfo message) { hasColumns ? BitSet.valueOf(snapshotRequest.columnsAsByteBuffer()) : null; final boolean hasViewport = snapshotRequest.viewportVector() != null; - final RowSet viewport = + RowSet viewport = hasViewport ? BarrageProtoUtil.toRowSet(snapshotRequest.viewportAsByteBuffer()) : null; + final boolean reverseViewport = snapshotRequest.reverseViewport(); + + // do we need to reverse the viewport here + // get ourselves some data! final BarrageMessage msg = ConstructSnapshot.constructBackplaneSnapshotInPositionSpace(this, @@ -604,7 +608,9 @@ private synchronized void onExportResolved(final SessionState.ExportObject listener, final BarrageSubscriptionOptions options, final BitSet subscribedColumns, - final @Nullable RowSet initialViewport) { + final @Nullable RowSet initialViewport, + final boolean reverseViewport) { this.options = options; this.listener = listener; this.logPrefix = "Sub{" + Integer.toHexString(System.identityHashCode(listener)) + "}: "; @@ -466,6 +470,7 @@ private Subscription(final StreamObserver listener, this.subscribedColumns = new BitSet(); this.pendingColumns = subscribedColumns; this.pendingViewport = initialViewport; + this.reverseViewport = reverseViewport; } public boolean isViewport() { @@ -484,7 +489,8 @@ public boolean isViewport() { public void addSubscription(final StreamObserver listener, final BarrageSubscriptionOptions options, final @Nullable BitSet columnsToSubscribe, - final @Nullable RowSet initialViewport) { + final @Nullable RowSet initialViewport, + final boolean reverseViewport) { synchronized (this) { final boolean hasSubscription = activeSubscriptions.stream().anyMatch(item -> item.listener == listener) || pendingSubscriptions.stream().anyMatch(item -> item.listener == listener); @@ -500,7 +506,7 @@ public void addSubscription(final StreamObserver listener, } else { cols = (BitSet) columnsToSubscribe.clone(); } - final Subscription subscription = new Subscription(listener, options, cols, initialViewport); + final Subscription subscription = new Subscription(listener, options, cols, initialViewport, reverseViewport); log.info().append(logPrefix) .append(subscription.logPrefix) diff --git a/server/src/main/java/io/deephaven/server/uri/BarrageTableResolver.java b/server/src/main/java/io/deephaven/server/uri/BarrageTableResolver.java index 8041efc53a1..8386b4d13fa 100644 --- a/server/src/main/java/io/deephaven/server/uri/BarrageTableResolver.java +++ b/server/src/main/java/io/deephaven/server/uri/BarrageTableResolver.java @@ -139,9 +139,9 @@ public Table subscribe(DeephavenTarget target, TableSpec table, BarrageSubscript * @param table the table spec * @return the subscribed table */ - public Table subscribe(String targetUri, TableSpec table, RowSet viewport, BitSet columns) + public Table subscribe(String targetUri, TableSpec table, RowSet viewport, BitSet columns, boolean reverseViewport) throws TableHandleException, InterruptedException { - return subscribe(DeephavenTarget.of(URI.create(targetUri)), table, SUB_OPTIONS, viewport, columns); + return subscribe(DeephavenTarget.of(URI.create(targetUri)), table, SUB_OPTIONS, viewport, columns, reverseViewport); } /** @@ -153,11 +153,11 @@ public Table subscribe(String targetUri, TableSpec table, RowSet viewport, BitSe * @return the subscribed table */ public Table subscribe(DeephavenTarget target, TableSpec table, BarrageSubscriptionOptions options, RowSet viewport, - BitSet columns) + BitSet columns, boolean reverseViewport) throws TableHandleException, InterruptedException { final BarrageSession session = session(target); final BarrageSubscription sub = session.subscribe(table, options); - return sub.partialTable(viewport, columns); + return sub.partialTable(viewport, columns, reverseViewport); } /** @@ -205,9 +205,9 @@ public Table snapshot(DeephavenTarget target, TableSpec table, BarrageSnapshotOp * @param table the table spec * @return the table to snapshot */ - public Table snapshot(String targetUri, TableSpec table, RowSet viewport, BitSet columns) + public Table snapshot(String targetUri, TableSpec table, RowSet viewport, BitSet columns, boolean reverseViewport) throws TableHandleException, InterruptedException { - return snapshot(DeephavenTarget.of(URI.create(targetUri)), table, SNAP_OPTIONS, viewport, columns); + return snapshot(DeephavenTarget.of(URI.create(targetUri)), table, SNAP_OPTIONS, viewport, columns, reverseViewport); } /** @@ -219,11 +219,11 @@ public Table snapshot(String targetUri, TableSpec table, RowSet viewport, BitSet * @return the table to snapshot */ public Table snapshot(DeephavenTarget target, TableSpec table, BarrageSnapshotOptions options, RowSet viewport, - BitSet columns) + BitSet columns, boolean reverseViewport) throws TableHandleException, InterruptedException { final BarrageSession session = session(target); final BarrageSnapshot snap = session.snapshot(table, options); - return snap.partialTable(viewport, columns); + return snap.partialTable(viewport, columns, reverseViewport); } diff --git a/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java b/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java index f1b424702dc..dbdba773384 100644 --- a/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java +++ b/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java @@ -220,7 +220,7 @@ public void doSubscribe() { .useDeephavenNulls(useDeephavenNulls) .build(); barrageMessageProducer.addSubscription(dummyObserver, options, subscribedColumns, - viewport == null ? null : viewport.copy()); + viewport == null ? null : viewport.copy(), false); } public void validate(final String msg, QueryTable expected) { From 5c68ccf87ad07b0b1a8202b973314730f80c0913 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Thu, 10 Feb 2022 13:00:30 -0800 Subject: [PATCH 02/20] builds, untested but shows potential impl --- .../io/deephaven/engine/rowset/RowSet.java | 8 ++++ .../rowset/impl/WritableRowSetImpl.java | 15 +++++- .../table/impl/remote/ConstructSnapshot.java | 37 +++++++++++---- .../server/arrow/ArrowFlightUtil.java | 18 +++++--- .../barrage/BarrageMessageProducer.java | 46 +++++++++++++++---- .../server/uri/BarrageTableResolver.java | 6 ++- 6 files changed, 101 insertions(+), 29 deletions(-) diff --git a/engine/rowset/src/main/java/io/deephaven/engine/rowset/RowSet.java b/engine/rowset/src/main/java/io/deephaven/engine/rowset/RowSet.java index adaa57e3d17..0fd15e520d5 100644 --- a/engine/rowset/src/main/java/io/deephaven/engine/rowset/RowSet.java +++ b/engine/rowset/src/main/java/io/deephaven/engine/rowset/RowSet.java @@ -435,6 +435,14 @@ public long binarySearchValue(TargetComparator targetComparator, int direction) */ WritableRowSet subSetForPositions(RowSet posRowSet); + /** + * Get a subset of this RowSet according to the supplied set of row positions in {@code posRowSet}. + * + * @param posRowSet The RowSet of position-based ranges to extract. + * @return A new RowSet, containing values at the locations in the provided RowSet. + */ + WritableRowSet subSetForReversePositions(RowSet posRowSet); + /** * Returns the row key at the given row position. * diff --git a/engine/rowset/src/main/java/io/deephaven/engine/rowset/impl/WritableRowSetImpl.java b/engine/rowset/src/main/java/io/deephaven/engine/rowset/impl/WritableRowSetImpl.java index 01514e5c7ae..0c80c756af1 100644 --- a/engine/rowset/src/main/java/io/deephaven/engine/rowset/impl/WritableRowSetImpl.java +++ b/engine/rowset/src/main/java/io/deephaven/engine/rowset/impl/WritableRowSetImpl.java @@ -23,8 +23,7 @@ import java.io.IOException; import java.io.ObjectInput; import java.io.ObjectOutput; -import java.util.Objects; -import java.util.PrimitiveIterator; +import java.util.*; import java.util.function.LongConsumer; public class WritableRowSetImpl extends RowSequenceAsChunkImpl implements WritableRowSet, Externalizable { @@ -358,6 +357,18 @@ public final WritableRowSet subSetForPositions(RowSet posRowSet) { return builder.build(); } + @Override + public final WritableRowSet subSetForReversePositions(RowSet posRowSet) { + final RowSetBuilderRandom builder = RowSetFactory.builderRandom(); + + posRowSet.forEachRowKeyRange((start, end) -> { + builder.addRange(this.size() - end - 1, this.size() - start - 1); + return true; + }); + + return subSetForPositions(builder.build()); + } + @Override public final long get(final long rowPosition) { return innerSet.ixGet(rowPosition); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/remote/ConstructSnapshot.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/remote/ConstructSnapshot.java index e230d612916..e37d707dc13 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/remote/ConstructSnapshot.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/remote/ConstructSnapshot.java @@ -11,6 +11,7 @@ import io.deephaven.configuration.Configuration; import io.deephaven.datastructures.util.CollectionUtil; import io.deephaven.engine.rowset.RowSetShiftData; +import io.deephaven.engine.rowset.WritableRowSet; import io.deephaven.engine.table.SharedContext; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; @@ -522,7 +523,7 @@ static InitialSnapshot constructInitialSnapshotInPositionSpace(final Object logI */ public static BarrageMessage constructBackplaneSnapshot(final Object logIdentityObject, final BaseTable table) { - return constructBackplaneSnapshotInPositionSpace(logIdentityObject, table, null, null); + return constructBackplaneSnapshotInPositionSpace(logIdentityObject, table, null, null, null); } /** @@ -568,15 +569,33 @@ public static BarrageMessage constructBackplaneSnapshotInPositionSpace(final Obj snapshot.shifted = RowSetShiftData.EMPTY; final SnapshotFunction doSnapshot = (usePrev, beforeClockValue) -> { - final RowSet keysToSnapshot; - if (positionsToSnapshot == null && reversePositionsToSnapshot == null) { - keysToSnapshot = null; - } else if (usePrev) { - try (final RowSet prevRowSet = table.getRowSet().copyPrev()) { - keysToSnapshot = prevRowSet.subSetForPositions(positionsToSnapshot); + RowSet keysToSnapshot = null; + if (positionsToSnapshot != null || reversePositionsToSnapshot != null) { + if (usePrev) { + // perform actions on the previous rowset + try (final RowSet prevRowSet = table.getRowSet().copyPrev()) { + if (positionsToSnapshot != null && reversePositionsToSnapshot != null) { + // union both key sets from forward and reverse viewports + keysToSnapshot = prevRowSet.subSetForPositions(positionsToSnapshot); + keysToSnapshot.union(prevRowSet.subSetForReversePositions(reversePositionsToSnapshot)); + } else if (positionsToSnapshot != null) { + keysToSnapshot = prevRowSet.subSetForPositions(positionsToSnapshot); + } else { + keysToSnapshot = prevRowSet.subSetForReversePositions(reversePositionsToSnapshot); + } + } + } else { + // perform actions on the current rowset + if (positionsToSnapshot != null && reversePositionsToSnapshot != null) { + // union both key sets from forward and reverse viewports + keysToSnapshot = table.getRowSet().subSetForPositions(positionsToSnapshot); + keysToSnapshot.union(table.getRowSet().subSetForReversePositions(reversePositionsToSnapshot)); + } else if (positionsToSnapshot != null) { + keysToSnapshot = table.getRowSet().subSetForPositions(positionsToSnapshot); + } else { + keysToSnapshot = table.getRowSet().subSetForReversePositions(reversePositionsToSnapshot); + } } - } else { - keysToSnapshot = table.getRowSet().subSetForPositions(positionsToSnapshot); } return serializeAllTable(usePrev, snapshot, table, logIdentityObject, columnsToSerialize, keysToSnapshot); }; diff --git a/server/src/main/java/io/deephaven/server/arrow/ArrowFlightUtil.java b/server/src/main/java/io/deephaven/server/arrow/ArrowFlightUtil.java index cbb3a259614..4201c698863 100644 --- a/server/src/main/java/io/deephaven/server/arrow/ArrowFlightUtil.java +++ b/server/src/main/java/io/deephaven/server/arrow/ArrowFlightUtil.java @@ -475,12 +475,15 @@ public void handleMessage(BarrageProtoUtil.MessageInfo message) { final boolean reverseViewport = snapshotRequest.reverseViewport(); - // do we need to reverse the viewport here - - // get ourselves some data! - final BarrageMessage msg = - ConstructSnapshot.constructBackplaneSnapshotInPositionSpace(this, - table, columns, viewport); + // get ourselves some data (reversing viewport as instructed) + final BarrageMessage msg; + if (reverseViewport) { + msg = ConstructSnapshot.constructBackplaneSnapshotInPositionSpace(this, table, + columns, null, viewport); + } else { + msg = ConstructSnapshot.constructBackplaneSnapshotInPositionSpace(this, table, + columns, viewport, null); + } msg.modColumnData = ZERO_MOD_COLUMNS; // no mod column data // translate the viewport to keyspace and make the call @@ -610,7 +613,8 @@ private synchronized void onExportResolved(final SessionState.ExportObject listener, } else { cols = (BitSet) columnsToSubscribe.clone(); } - final Subscription subscription = new Subscription(listener, options, cols, initialViewport, reverseViewport); + final Subscription subscription = + new Subscription(listener, options, cols, initialViewport, reverseViewport); log.info().append(logPrefix) .append(subscription.logPrefix) @@ -938,6 +941,8 @@ private void updateSubscriptionsSnapshotAndPropagate() { boolean firstSubscription = false; BitSet snapshotColumns = null; RowSetBuilderRandom snapshotRows = null; + RowSetBuilderRandom reverseSnapshotRows = null; + List updatedSubscriptions = null; // first, we take out any new subscriptions (under the lock) @@ -962,6 +967,7 @@ private void updateSubscriptionsSnapshotAndPropagate() { needsSnapshot = true; snapshotColumns = new BitSet(); snapshotRows = RowSetFactory.builderRandom(); + reverseSnapshotRows = RowSetFactory.builderRandom(); } subscription.hasPendingUpdate = false; @@ -982,7 +988,12 @@ private void updateSubscriptionsSnapshotAndPropagate() { subscription.snapshotViewport = subscription.pendingViewport; subscription.pendingViewport = null; if (!needsFullSnapshot) { - snapshotRows.addRowSet(subscription.snapshotViewport); + // track forward and reverse viewport rows separately + if (subscription.reverseViewport) { + reverseSnapshotRows.addRowSet(subscription.snapshotViewport); + } else { + snapshotRows.addRowSet(subscription.snapshotViewport); + } } } @@ -994,11 +1005,13 @@ private void updateSubscriptionsSnapshotAndPropagate() { needsFullSnapshot = true; } } - } + } // end updatedSubscriptions loop boolean haveViewport = false; postSnapshotColumns.clear(); + final RowSetBuilderRandom postSnapshotViewportBuilder = RowSetFactory.builderRandom(); + final RowSetBuilderRandom postSnapshotReverseViewportBuilder = RowSetFactory.builderRandom(); for (int i = 0; i < activeSubscriptions.size(); ++i) { final Subscription sub = activeSubscriptions.get(i); @@ -1015,13 +1028,20 @@ private void updateSubscriptionsSnapshotAndPropagate() { if (sub.isViewport()) { haveViewport = true; - postSnapshotViewportBuilder - .addRowSet(sub.snapshotViewport != null ? sub.snapshotViewport : sub.viewport); + // handle forward and reverse snapshots separately + if (sub.reverseViewport) { + postSnapshotReverseViewportBuilder + .addRowSet(sub.snapshotViewport != null ? sub.snapshotViewport : sub.viewport); + } else { + postSnapshotViewportBuilder + .addRowSet(sub.snapshotViewport != null ? sub.snapshotViewport : sub.viewport); + } } postSnapshotColumns.or(sub.snapshotColumns != null ? sub.snapshotColumns : sub.subscribedColumns); } postSnapshotViewport = haveViewport ? postSnapshotViewportBuilder.build() : null; + postSnapshotReverseViewport = haveViewport ? postSnapshotReverseViewportBuilder.build() : null; if (!needsSnapshot) { // i.e. We have only removed subscriptions; we can update this state immediately. @@ -1037,9 +1057,11 @@ private void updateSubscriptionsSnapshotAndPropagate() { // then we spend the effort to take a snapshot if (needsSnapshot) { - try (final RowSet snapshotRowSet = snapshotRows.build()) { + try (final RowSet snapshotRowSet = snapshotRows.build(); + final RowSet reverseSnapshotRowSet = reverseSnapshotRows.build()) { snapshot = - getSnapshot(updatedSubscriptions, snapshotColumns, needsFullSnapshot ? null : snapshotRowSet); + getSnapshot(updatedSubscriptions, snapshotColumns, needsFullSnapshot ? null : snapshotRowSet, + needsFullSnapshot ? null : reverseSnapshotRowSet); } } @@ -1574,7 +1596,11 @@ private void promoteSnapshotToActive() { this.activeViewport.close(); } this.activeViewport = this.postSnapshotViewport; + this.activeReverseViewport = this.postSnapshotReverseViewport; + + // clean up the temporary objects this.postSnapshotViewport = null; + this.postSnapshotReverseViewport = null; // Pre-condition: activeObjectColumns == objectColumns & activeColumns this.objectColumnsToClear.or(postSnapshotColumns); @@ -1654,7 +1680,8 @@ public boolean snapshotCompletedConsistently(final long afterClockValue, final b @VisibleForTesting BarrageMessage getSnapshot(final List snapshotSubscriptions, final BitSet columnsToSnapshot, - final RowSet positionsToSnapshot) { + final RowSet positionsToSnapshot, + final RowSet reversePositionsToSnapshot) { if (onGetSnapshot != null) { onGetSnapshot.run(); } @@ -1663,7 +1690,8 @@ BarrageMessage getSnapshot(final List snapshotSubscriptions, // TODO: Let notification-indifferent use cases skip notification test final SnapshotControl snapshotControl = new SnapshotControl(snapshotSubscriptions); return ConstructSnapshot.constructBackplaneSnapshotInPositionSpace( - this, parent, columnsToSnapshot, positionsToSnapshot, snapshotControl); + this, parent, columnsToSnapshot, positionsToSnapshot, reversePositionsToSnapshot, + snapshotControl); } //////////////////////////////////////////////////// diff --git a/server/src/main/java/io/deephaven/server/uri/BarrageTableResolver.java b/server/src/main/java/io/deephaven/server/uri/BarrageTableResolver.java index 8386b4d13fa..2a05b0d5e74 100644 --- a/server/src/main/java/io/deephaven/server/uri/BarrageTableResolver.java +++ b/server/src/main/java/io/deephaven/server/uri/BarrageTableResolver.java @@ -141,7 +141,8 @@ public Table subscribe(DeephavenTarget target, TableSpec table, BarrageSubscript */ public Table subscribe(String targetUri, TableSpec table, RowSet viewport, BitSet columns, boolean reverseViewport) throws TableHandleException, InterruptedException { - return subscribe(DeephavenTarget.of(URI.create(targetUri)), table, SUB_OPTIONS, viewport, columns, reverseViewport); + return subscribe(DeephavenTarget.of(URI.create(targetUri)), table, SUB_OPTIONS, viewport, columns, + reverseViewport); } /** @@ -207,7 +208,8 @@ public Table snapshot(DeephavenTarget target, TableSpec table, BarrageSnapshotOp */ public Table snapshot(String targetUri, TableSpec table, RowSet viewport, BitSet columns, boolean reverseViewport) throws TableHandleException, InterruptedException { - return snapshot(DeephavenTarget.of(URI.create(targetUri)), table, SNAP_OPTIONS, viewport, columns, reverseViewport); + return snapshot(DeephavenTarget.of(URI.create(targetUri)), table, SNAP_OPTIONS, viewport, columns, + reverseViewport); } /** From b580a0f6de9add30e866c19b3b2ebbb4c2b25f3b Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Mon, 14 Feb 2022 13:11:25 -0800 Subject: [PATCH 03/20] WIP commit --- .../io/deephaven/engine/rowset/RowSet.java | 12 +++- .../rowset/impl/WritableRowSetImpl.java | 12 +++- .../table/impl/remote/ConstructSnapshot.java | 61 +++++++++---------- .../table/impl/util/BarrageMessage.java | 1 + .../barrage/table/BarrageTable.java | 12 ++-- .../barrage/util/BarrageStreamReader.java | 2 + .../client/examples/SnapshotExampleBase.java | 29 +++++++++ .../client/examples/SubscribeExampleBase.java | 7 ++- .../server/arrow/ArrowFlightUtil.java | 6 +- .../barrage/BarrageMessageProducer.java | 39 ++++++++---- .../barrage/BarrageStreamGenerator.java | 24 ++++++-- .../server/uri/BarrageTableResolver.java | 24 ++++++++ 12 files changed, 170 insertions(+), 59 deletions(-) diff --git a/engine/rowset/src/main/java/io/deephaven/engine/rowset/RowSet.java b/engine/rowset/src/main/java/io/deephaven/engine/rowset/RowSet.java index 0fd15e520d5..edd84f3833c 100644 --- a/engine/rowset/src/main/java/io/deephaven/engine/rowset/RowSet.java +++ b/engine/rowset/src/main/java/io/deephaven/engine/rowset/RowSet.java @@ -431,9 +431,10 @@ public long binarySearchValue(TargetComparator targetComparator, int direction) * Get a subset of this RowSet according to the supplied set of row positions in {@code posRowSet}. * * @param posRowSet The RowSet of position-based ranges to extract. + * @param reversed use a reversed view of the table (index 0 is last row of table) * @return A new RowSet, containing values at the locations in the provided RowSet. */ - WritableRowSet subSetForPositions(RowSet posRowSet); + WritableRowSet subSetForPositions(RowSet posRowSet, boolean reversed); /** * Get a subset of this RowSet according to the supplied set of row positions in {@code posRowSet}. @@ -441,6 +442,15 @@ public long binarySearchValue(TargetComparator targetComparator, int direction) * @param posRowSet The RowSet of position-based ranges to extract. * @return A new RowSet, containing values at the locations in the provided RowSet. */ + WritableRowSet subSetForPositions(RowSet posRowSet); + + /** + * Get a subset of this RowSet according to the supplied set of row positions in {@code posRowSet} in reversed + * position space (index 0 is last row of table) + * + * @param posRowSet The RowSet of position-based ranges to extract. + * @return A new RowSet, containing values at the locations in the provided RowSet. + */ WritableRowSet subSetForReversePositions(RowSet posRowSet); /** diff --git a/engine/rowset/src/main/java/io/deephaven/engine/rowset/impl/WritableRowSetImpl.java b/engine/rowset/src/main/java/io/deephaven/engine/rowset/impl/WritableRowSetImpl.java index 0c80c756af1..e1941ddd224 100644 --- a/engine/rowset/src/main/java/io/deephaven/engine/rowset/impl/WritableRowSetImpl.java +++ b/engine/rowset/src/main/java/io/deephaven/engine/rowset/impl/WritableRowSetImpl.java @@ -333,6 +333,14 @@ public final WritableRowSet subSetByKeyRange(final long startKey, final long end return new WritableRowSetImpl(innerSet.ixSubindexByKeyOnNew(startKey, endKey)); } + @Override + public final WritableRowSet subSetForPositions(RowSet posRowSet, boolean reversed) { + if (reversed) { + return subSetForReversePositions(posRowSet); + } + return subSetForPositions(posRowSet); + } + @Override public final WritableRowSet subSetForPositions(RowSet posRowSet) { final MutableLong currentOffset = new MutableLong(); @@ -361,8 +369,10 @@ public final WritableRowSet subSetForPositions(RowSet posRowSet) { public final WritableRowSet subSetForReversePositions(RowSet posRowSet) { final RowSetBuilderRandom builder = RowSetFactory.builderRandom(); + long lastIndex = size() - 1; + posRowSet.forEachRowKeyRange((start, end) -> { - builder.addRange(this.size() - end - 1, this.size() - start - 1); + builder.addRange(lastIndex - end, lastIndex - start); return true; }); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/remote/ConstructSnapshot.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/remote/ConstructSnapshot.java index e37d707dc13..39cf54174ed 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/remote/ConstructSnapshot.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/remote/ConstructSnapshot.java @@ -10,11 +10,8 @@ import io.deephaven.base.verify.Assert; import io.deephaven.configuration.Configuration; import io.deephaven.datastructures.util.CollectionUtil; -import io.deephaven.engine.rowset.RowSetShiftData; -import io.deephaven.engine.rowset.WritableRowSet; +import io.deephaven.engine.rowset.*; import io.deephaven.engine.table.SharedContext; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.updategraph.UpdateGraphProcessor; import io.deephaven.engine.table.impl.ShiftObliviousInstrumentedListener; import io.deephaven.engine.table.impl.sources.ReinterpretUtils; @@ -569,33 +566,31 @@ public static BarrageMessage constructBackplaneSnapshotInPositionSpace(final Obj snapshot.shifted = RowSetShiftData.EMPTY; final SnapshotFunction doSnapshot = (usePrev, beforeClockValue) -> { - RowSet keysToSnapshot = null; - if (positionsToSnapshot != null || reversePositionsToSnapshot != null) { + final RowSet keysToSnapshot; + if (positionsToSnapshot == null && reversePositionsToSnapshot == null) { + keysToSnapshot = null; + } else { + final RowSetBuilderRandom keyBuilder = RowSetFactory.builderRandom(); if (usePrev) { // perform actions on the previous rowset try (final RowSet prevRowSet = table.getRowSet().copyPrev()) { - if (positionsToSnapshot != null && reversePositionsToSnapshot != null) { - // union both key sets from forward and reverse viewports - keysToSnapshot = prevRowSet.subSetForPositions(positionsToSnapshot); - keysToSnapshot.union(prevRowSet.subSetForReversePositions(reversePositionsToSnapshot)); - } else if (positionsToSnapshot != null) { - keysToSnapshot = prevRowSet.subSetForPositions(positionsToSnapshot); - } else { - keysToSnapshot = prevRowSet.subSetForReversePositions(reversePositionsToSnapshot); + if (positionsToSnapshot != null) { + keyBuilder.addRowSet(prevRowSet.subSetForPositions(positionsToSnapshot)); + } + if (reversePositionsToSnapshot != null) { + keyBuilder.addRowSet(prevRowSet.subSetForReversePositions(reversePositionsToSnapshot)); } } } else { // perform actions on the current rowset - if (positionsToSnapshot != null && reversePositionsToSnapshot != null) { - // union both key sets from forward and reverse viewports - keysToSnapshot = table.getRowSet().subSetForPositions(positionsToSnapshot); - keysToSnapshot.union(table.getRowSet().subSetForReversePositions(reversePositionsToSnapshot)); - } else if (positionsToSnapshot != null) { - keysToSnapshot = table.getRowSet().subSetForPositions(positionsToSnapshot); - } else { - keysToSnapshot = table.getRowSet().subSetForReversePositions(reversePositionsToSnapshot); + if (positionsToSnapshot != null) { + keyBuilder.addRowSet(table.getRowSet().subSetForPositions(positionsToSnapshot)); + } + if (reversePositionsToSnapshot != null) { + keyBuilder.addRowSet(table.getRowSet().subSetForReversePositions(reversePositionsToSnapshot)); } } + keysToSnapshot = keyBuilder.build(); } return serializeAllTable(usePrev, snapshot, table, logIdentityObject, columnsToSerialize, keysToSnapshot); }; @@ -1289,20 +1284,20 @@ public static boolean serializeAllTable(boolean usePrev, * {@link #constructBackplaneSnapshot} for simple use cases or {@link #callDataSnapshotFunction} for more advanced * uses. * - * @param usePrev Use previous values? - * @param snapshot The snapshot to populate - * @param logIdentityObject an object for use with log() messages + * @param usePrev Use previous values? + * @param snapshot The snapshot to populate + * @param logIdentityObject an object for use with log() messages * @param columnsToSerialize A {@link BitSet} of columns to include, null for all - * @param keysToSnapshot A RowSet of keys within the table to include, null for all - * + * @param keysToSnapshot A RowSet of keys within the table to include, null for all * @return true if the snapshot was computed with an unchanged clock, false otherwise. */ public static boolean serializeAllTable(final boolean usePrev, - final BarrageMessage snapshot, - final BaseTable table, - final Object logIdentityObject, - final BitSet columnsToSerialize, - final RowSet keysToSnapshot) { + final BarrageMessage snapshot, + final BaseTable table, + final Object logIdentityObject, + final BitSet columnsToSerialize, + final RowSet keysToSnapshot) { + snapshot.rowsAdded = (usePrev ? table.getRowSet().copyPrev() : table.getRowSet()).copy(); snapshot.rowsRemoved = RowSetFactory.empty(); snapshot.addColumnData = new BarrageMessage.AddColumnData[table.getColumnSources().size()]; @@ -1322,7 +1317,7 @@ public static boolean serializeAllTable(final boolean usePrev, final String[] columnSources = sourceMap.keySet().toArray(CollectionUtil.ZERO_LENGTH_STRING_ARRAY); try (final SharedContext sharedContext = - (columnSources.length > 1) ? SharedContext.makeSharedContext() : null) { + (columnSources.length > 1) ? SharedContext.makeSharedContext() : null) { for (int ii = 0; ii < columnSources.length; ++ii) { if (concurrentAttemptInconsistent()) { final LogEntry logEntry = log.info().append(System.identityHashCode(logIdentityObject)) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/BarrageMessage.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/BarrageMessage.java index a5bcf3eeff0..ef44222cc91 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/BarrageMessage.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/BarrageMessage.java @@ -43,6 +43,7 @@ public static class AddColumnData { public boolean isSnapshot; public RowSet snapshotRowSet; + public boolean snapshotReverseRowSet; public BitSet snapshotColumns; public RowSet rowsAdded; diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/table/BarrageTable.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/table/BarrageTable.java index 45e8a4c6a02..bdf6e7ec07e 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/table/BarrageTable.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/table/BarrageTable.java @@ -86,6 +86,7 @@ public class BarrageTable extends QueryTable implements BarrageMessage.Listener, * re-send data that the client should already have within the existing viewport. */ private RowSet serverViewport; + private boolean serverReverseViewport; private BitSet serverColumns; @@ -198,8 +199,7 @@ public void handleBarrageError(Throwable t) { enqueueError(t); } - private UpdateCoalescer processUpdate(final BarrageMessage update, - final UpdateCoalescer coalescer) { + private UpdateCoalescer processUpdate(final BarrageMessage update, final UpdateCoalescer coalescer) { if (DEBUG_ENABLED) { saveForDebugging(update); @@ -223,6 +223,7 @@ private UpdateCoalescer processUpdate(final BarrageMessage update, if (update.isSnapshot) { serverViewport = update.snapshotRowSet == null ? null : update.snapshotRowSet.copy(); + serverReverseViewport = update.snapshotReverseRowSet; serverColumns = update.snapshotColumns == null ? null : (BitSet) update.snapshotColumns.clone(); } @@ -233,7 +234,9 @@ private UpdateCoalescer processUpdate(final BarrageMessage update, try (final RowSet currRowsFromPrev = currentRowSet.copy(); final WritableRowSet populatedRows = - (serverViewport != null ? currentRowSet.subSetForPositions(serverViewport) : null)) { + (serverViewport != null + ? currentRowSet.subSetForPositions(serverViewport, serverReverseViewport) + : null)) { // removes currentRowSet.remove(update.rowsRemoved); @@ -307,7 +310,8 @@ private UpdateCoalescer processUpdate(final BarrageMessage update, // remove all data outside of our viewport if (serverViewport != null) { - try (final RowSet newPopulated = currentRowSet.subSetForPositions(serverViewport)) { + try (final RowSet newPopulated = + currentRowSet.subSetForPositions(serverViewport, serverReverseViewport)) { populatedRows.remove(newPopulated); freeRows(populatedRows); } diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/BarrageStreamReader.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/BarrageStreamReader.java index dbe71bed4cd..e176cd130cb 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/BarrageStreamReader.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/BarrageStreamReader.java @@ -81,6 +81,8 @@ public BarrageMessage safelyParseFrom(final StreamReaderOptions options, msg = new BarrageMessage(); msg.isSnapshot = metadata.isSnapshot(); + msg.snapshotReverseRowSet = metadata.effectiveReverseViewport(); + numAddBatchesRemaining = metadata.numAddBatches(); numModBatchesRemaining = metadata.numModBatches(); if (numAddBatchesRemaining > 1 || numModBatchesRemaining > 1) { diff --git a/java-client/barrage-examples/src/main/java/io/deephaven/client/examples/SnapshotExampleBase.java b/java-client/barrage-examples/src/main/java/io/deephaven/client/examples/SnapshotExampleBase.java index 042c8d25d73..cd6e900fccf 100644 --- a/java-client/barrage-examples/src/main/java/io/deephaven/client/examples/SnapshotExampleBase.java +++ b/java-client/barrage-examples/src/main/java/io/deephaven/client/examples/SnapshotExampleBase.java @@ -106,6 +106,35 @@ protected void execute(final BarrageSession client) throws Exception { TableTools.show(table); } + // example #6 - reverse viewport, all columns + try (final TableHandle handle = manager.executeLogic(logic()); + final RowSet viewport = RowSetFactory.flat(5); // range inclusive + + final BarrageSnapshot snapshot = client.snapshot(handle, options)) { + + // expect this to block until all reading complete + final BarrageTable table = snapshot.partialTable(viewport, null, true); + + System.out.println("Table info: rows = " + table.size() + ", cols = " + table.getColumns().length); + TableTools.show(table); + } + + // example #7 - reverse viewport, some columns + try (final TableHandle handle = manager.executeLogic(logic()); + final RowSet viewport = RowSetFactory.flat(5); // range inclusive + + final BarrageSnapshot snapshot = client.snapshot(handle, options)) { + + final BitSet columns = new BitSet(); + columns.set(0, 2); // range not inclusive (sets bits 0-1) + + // expect this to block until all reading complete + final BarrageTable table = snapshot.partialTable(viewport, columns, true); + + System.out.println("Table info: rows = " + table.size() + ", cols = " + table.getColumns().length); + TableTools.show(table); + } + System.out.println("End of Snapshot examples"); } diff --git a/java-client/barrage-examples/src/main/java/io/deephaven/client/examples/SubscribeExampleBase.java b/java-client/barrage-examples/src/main/java/io/deephaven/client/examples/SubscribeExampleBase.java index 978711be4c7..5472d6f5645 100644 --- a/java-client/barrage-examples/src/main/java/io/deephaven/client/examples/SubscribeExampleBase.java +++ b/java-client/barrage-examples/src/main/java/io/deephaven/client/examples/SubscribeExampleBase.java @@ -8,6 +8,7 @@ import io.deephaven.client.impl.BarrageSubscription; import io.deephaven.client.impl.TableHandle; import io.deephaven.client.impl.TableHandleManager; +import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.impl.InstrumentedTableUpdateListener; import io.deephaven.extensions.barrage.BarrageSubscriptionOptions; @@ -42,7 +43,9 @@ protected void execute(final BarrageSession client) throws Exception { try (final TableHandle handle = manager.executeLogic(logic()); final BarrageSubscription subscription = client.subscribe(handle, options)) { - final BarrageTable table = subscription.entireTable(); +// final BarrageTable table = subscription.entireTable(); + final BarrageTable table = subscription.partialTable(RowSetFactory.flat(10), null, true); + final CountDownLatch countDownLatch = new CountDownLatch(1); table.listenForUpdates(new InstrumentedTableUpdateListener("example-listener") { @@ -57,6 +60,8 @@ protected void onFailureInternal(final Throwable originalException, final Entry public void onUpdate(final TableUpdate upstream) { System.out.println("Received table update:"); System.out.println(upstream); + + System.out.println(" Table size: " + table. } }); diff --git a/server/src/main/java/io/deephaven/server/arrow/ArrowFlightUtil.java b/server/src/main/java/io/deephaven/server/arrow/ArrowFlightUtil.java index 4201c698863..5491beaeffb 100644 --- a/server/src/main/java/io/deephaven/server/arrow/ArrowFlightUtil.java +++ b/server/src/main/java/io/deephaven/server/arrow/ArrowFlightUtil.java @@ -489,10 +489,12 @@ public void handleMessage(BarrageProtoUtil.MessageInfo message) { // translate the viewport to keyspace and make the call try (final BarrageStreamGenerator bsg = new BarrageStreamGenerator(msg); final RowSet keySpaceViewport = - hasViewport ? msg.rowsAdded.subSetForPositions(viewport) : null) { + hasViewport + ? msg.rowsAdded.subSetForPositions(viewport, reverseViewport) + : null) { listener.onNext( bsg.getSnapshotView(DEFAULT_SNAPSHOT_DESER_OPTIONS, viewport, - keySpaceViewport, columns)); + reverseViewport, keySpaceViewport, columns)); } listener.onCompleted(); diff --git a/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java b/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java index 0848035ad4c..56679c6fdac 100644 --- a/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java +++ b/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java @@ -9,7 +9,6 @@ import dagger.assisted.Assisted; import dagger.assisted.AssistedFactory; import dagger.assisted.AssistedInject; -import io.deephaven.barrage.flatbuf.BarrageSubscriptionRequest; import io.deephaven.base.formatters.FormatBitSet; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.LongChunk; @@ -154,12 +153,13 @@ interface Factory { * @param options serialization options for this specific view * @param isInitialSnapshot indicates whether or not this is the first snapshot for the listener * @param viewport is the position-space viewport + * @param reverseViewport is the viewport reversed (relative to end of table instead of beginning) * @param keyspaceViewport is the key-space viewport * @param subscribedColumns are the columns subscribed for this view * @return a MessageView filtered by the subscription properties that can be sent to that subscriber */ MessageView getSubView(BarrageSubscriptionOptions options, boolean isInitialSnapshot, @Nullable RowSet viewport, - @Nullable RowSet keyspaceViewport, BitSet subscribedColumns); + boolean reverseViewport, @Nullable RowSet keyspaceViewport, BitSet subscribedColumns); /** * Obtain a Full-Snapshot View of this StreamGenerator that can be sent to a single requestor. @@ -174,11 +174,11 @@ MessageView getSubView(BarrageSubscriptionOptions options, boolean isInitialSnap * * @param options serialization options for this specific view * @param viewport is the position-space viewport - * @param keyspaceViewport is the key-space viewport + * @param reverseViewport is the viewport reversed (relative to end of table instead of beginning) * @param snapshotColumns are the columns included for this view * @return a MessageView filtered by the snapshot properties that can be sent to that requestor */ - MessageView getSnapshotView(BarrageSnapshotOptions options, @Nullable RowSet viewport, + MessageView getSnapshotView(BarrageSnapshotOptions options, @Nullable RowSet viewport, boolean reverseViewport, @Nullable RowSet keyspaceViewport, BitSet snapshotColumns); } @@ -698,8 +698,15 @@ private void enqueueUpdate(final TableUpdate upstream) { if (numFullSubscriptions > 0) { addsToRecord = upstream.added().copy(); modsToRecord = upstream.modified().copy(); - } else if (activeViewport != null) { - try (final WritableRowSet deltaViewport = rowSet.subSetForPositions(activeViewport)) { + } else if (activeViewport != null || activeReverseViewport != null) { + // build the combined position-space viewport (from forward and reverse) + try (final WritableRowSet deltaViewport = RowSetFactory.builderRandom().build()) { + if (activeViewport != null) { + deltaViewport.insert(rowSet.subSetForPositions(activeViewport)); + } + if (activeReverseViewport != null) { + deltaViewport.insert(rowSet.subSetForReversePositions(activeReverseViewport)); + } addsToRecord = deltaViewport.intersect(upstream.added()); modsToRecord = deltaViewport.intersect(upstream.modified()); } @@ -1115,7 +1122,6 @@ private void updateSubscriptionsSnapshotAndPropagate() { objectColumnsToClear.and(activeColumns); } - nextFreeDeltaKey = 0; for (final Delta delta : pendingDeltas) { delta.close(); @@ -1179,9 +1185,12 @@ private void propagateToSubscribers(final BarrageMessage message, final RowSet p : subscription.subscribedColumns; try (final RowSet clientView = - subscription.isViewport() ? propRowSetForMessage.subSetForPositions(vp) : null) { + subscription.isViewport() + ? propRowSetForMessage.subSetForPositions(vp, subscription.reverseViewport) + : null) { subscription.listener - .onNext(generator.getSubView(subscription.options, false, vp, clientView, cols)); + .onNext(generator.getSubView(subscription.options, false, vp, subscription.reverseViewport, + clientView, cols)); } catch (final Exception e) { try { subscription.listener.onError(GrpcUtil.securelyWrapError(log, e)); @@ -1236,7 +1245,9 @@ private void propagateSnapshotForSubscription(final Subscription subscription, final boolean isViewport = subscription.viewport != null; try (final RowSet keySpaceViewport = - isViewport ? snapshotGenerator.getMessage().rowsAdded.subSetForPositions(subscription.viewport) + isViewport + ? snapshotGenerator.getMessage().rowsAdded + .subSetForPositions(subscription.viewport, subscription.reverseViewport) : null) { if (subscription.pendingInitialSnapshot) { // Send schema metadata to this new client. @@ -1247,7 +1258,8 @@ private void propagateSnapshotForSubscription(final Subscription subscription, subscription.listener .onNext(snapshotGenerator.getSubView(subscription.options, subscription.pendingInitialSnapshot, - subscription.viewport, keySpaceViewport, subscription.subscribedColumns)); + subscription.viewport, subscription.reverseViewport, keySpaceViewport, + subscription.subscribedColumns)); } catch (final Exception e) { GrpcUtil.safelyExecute(() -> subscription.listener.onError(GrpcUtil.securelyWrapError(log, e))); removeSubscription(subscription.listener); @@ -1595,10 +1607,13 @@ private void promoteSnapshotToActive() { if (this.activeViewport != null) { this.activeViewport.close(); } + if (this.activeReverseViewport != null) { + this.activeReverseViewport.close(); + } + this.activeViewport = this.postSnapshotViewport; this.activeReverseViewport = this.postSnapshotReverseViewport; - // clean up the temporary objects this.postSnapshotViewport = null; this.postSnapshotReverseViewport = null; diff --git a/server/src/main/java/io/deephaven/server/barrage/BarrageStreamGenerator.java b/server/src/main/java/io/deephaven/server/barrage/BarrageStreamGenerator.java index 61d6644da37..3e2e80d18e4 100644 --- a/server/src/main/java/io/deephaven/server/barrage/BarrageStreamGenerator.java +++ b/server/src/main/java/io/deephaven/server/barrage/BarrageStreamGenerator.java @@ -190,6 +190,7 @@ public void close() { * @param options serialization options for this specific view * @param isInitialSnapshot indicates whether or not this is the first snapshot for the listener * @param viewport is the position-space viewport + * @param reverseViewport is the viewport reversed (relative to end of table instead of beginning) * @param keyspaceViewport is the key-space viewport * @param subscribedColumns are the columns subscribed for this view * @return a MessageView filtered by the subscription properties that can be sent to that subscriber @@ -198,9 +199,11 @@ public void close() { public SubView getSubView(final BarrageSubscriptionOptions options, final boolean isInitialSnapshot, @Nullable final RowSet viewport, + boolean reverseViewport, @Nullable final RowSet keyspaceViewport, @Nullable final BitSet subscribedColumns) { - return new SubView(this, options, isInitialSnapshot, viewport, keyspaceViewport, subscribedColumns); + return new SubView(this, options, isInitialSnapshot, viewport, reverseViewport, keyspaceViewport, + subscribedColumns); } /** @@ -212,7 +215,7 @@ public SubView getSubView(final BarrageSubscriptionOptions options, */ @Override public SubView getSubView(BarrageSubscriptionOptions options, boolean isInitialSnapshot) { - return getSubView(options, isInitialSnapshot, null, null, null); + return getSubView(options, isInitialSnapshot, null, false, null, null); } public static class SubView implements View { @@ -220,6 +223,7 @@ public static class SubView implements View { public final BarrageSubscriptionOptions options; public final boolean isInitialSnapshot; public final RowSet viewport; + public final boolean reverseViewport; public final RowSet keyspaceViewport; public final BitSet subscribedColumns; public final boolean hasAddBatch; @@ -229,12 +233,14 @@ public SubView(final BarrageStreamGenerator generator, final BarrageSubscriptionOptions options, final boolean isInitialSnapshot, @Nullable final RowSet viewport, + final boolean reverseViewport, @Nullable final RowSet keyspaceViewport, @Nullable final BitSet subscribedColumns) { this.generator = generator; this.options = options; this.isInitialSnapshot = isInitialSnapshot; this.viewport = viewport; + this.reverseViewport = reverseViewport; this.keyspaceViewport = keyspaceViewport; this.subscribedColumns = subscribedColumns; this.hasModBatch = generator.doesSubViewHaveMods(this); @@ -273,6 +279,7 @@ public final RowSet keyspaceViewport() { * * @param options serialization options for this specific view * @param viewport is the position-space viewport + * @param reverseViewport is the viewport reversed (relative to end of table instead of beginning) * @param keyspaceViewport is the key-space viewport * @param snapshotColumns are the columns subscribed for this view * @return a MessageView filtered by the snapshot properties that can be sent to that subscriber @@ -280,9 +287,10 @@ public final RowSet keyspaceViewport() { @Override public SnapshotView getSnapshotView(final BarrageSnapshotOptions options, @Nullable final RowSet viewport, + final boolean reverseViewport, @Nullable final RowSet keyspaceViewport, @Nullable final BitSet snapshotColumns) { - return new SnapshotView(this, options, viewport, keyspaceViewport, snapshotColumns); + return new SnapshotView(this, options, viewport, reverseViewport, keyspaceViewport, snapshotColumns); } /** @@ -293,13 +301,14 @@ public SnapshotView getSnapshotView(final BarrageSnapshotOptions options, */ @Override public SnapshotView getSnapshotView(BarrageSnapshotOptions options) { - return getSnapshotView(options, null, null, null); + return getSnapshotView(options, null, false, null, null); } public static class SnapshotView implements View { public final BarrageStreamGenerator generator; public final BarrageSnapshotOptions options; public final RowSet viewport; + public final boolean reverseViewport; public final RowSet keyspaceViewport; public final BitSet subscribedColumns; public final boolean hasAddBatch; @@ -308,11 +317,14 @@ public static class SnapshotView implements View { public SnapshotView(final BarrageStreamGenerator generator, final BarrageSnapshotOptions options, @Nullable final RowSet viewport, + final boolean reverseViewport, @Nullable final RowSet keyspaceViewport, @Nullable final BitSet subscribedColumns) { this.generator = generator; this.options = options; this.viewport = viewport; + this.reverseViewport = reverseViewport; + this.keyspaceViewport = keyspaceViewport; this.subscribedColumns = subscribedColumns; this.hasModBatch = false; @@ -501,6 +513,7 @@ private long appendAddColumns(final View view, final Consumer addStream, final ChunkInputStreamGenerator.FieldNodeListener fieldNodeListener, final ChunkInputStreamGenerator.BufferListener bufferListener) throws IOException { + // Added Chunk Data: final RowSet myAddedOffsets; if (view.isViewport()) { @@ -524,7 +537,7 @@ private long appendAddColumns(final View view, // Add the drainable last as it is allowed to immediately close a row set the visitors need addStream.accept(drainableColumn); } - return rowsAdded.original.size(); + return myAddedOffsets == null ? rowsAdded.original.size() : myAddedOffsets.size(); } private long appendModColumns(final View view, @@ -632,6 +645,7 @@ private ByteBuffer getSubscriptionMetadata(final SubView view) throws IOExceptio BarrageUpdateMetadata.addLastSeq(metadata, lastSeq); BarrageUpdateMetadata.addEffectiveViewport(metadata, effectiveViewportOffset); BarrageUpdateMetadata.addEffectiveColumnSet(metadata, effectiveColumnSetOffset); + BarrageUpdateMetadata.addEffectiveReverseViewport(metadata, view.reverseViewport); BarrageUpdateMetadata.addAddedRows(metadata, rowsAddedOffset); BarrageUpdateMetadata.addRemovedRows(metadata, rowsRemovedOffset); BarrageUpdateMetadata.addShiftData(metadata, shiftDataOffset); diff --git a/server/src/main/java/io/deephaven/server/uri/BarrageTableResolver.java b/server/src/main/java/io/deephaven/server/uri/BarrageTableResolver.java index 2a05b0d5e74..70d7ee60925 100644 --- a/server/src/main/java/io/deephaven/server/uri/BarrageTableResolver.java +++ b/server/src/main/java/io/deephaven/server/uri/BarrageTableResolver.java @@ -132,6 +132,18 @@ public Table subscribe(DeephavenTarget target, TableSpec table, BarrageSubscript return sub.entireTable(); } + /** + * Create a partial table subscription to the {@code table} via the {@code targetUri}. Uses {@link #SUB_OPTIONS}. + * + * @param targetUri the target URI + * @param table the table spec + * @return the subscribed table + */ + public Table subscribe(String targetUri, TableSpec table, RowSet viewport, BitSet columns) + throws TableHandleException, InterruptedException { + return subscribe(DeephavenTarget.of(URI.create(targetUri)), table, SUB_OPTIONS, viewport, columns, false); + } + /** * Create a partial table subscription to the {@code table} via the {@code targetUri}. Uses {@link #SUB_OPTIONS}. * @@ -199,6 +211,18 @@ public Table snapshot(DeephavenTarget target, TableSpec table, BarrageSnapshotOp return snap.entireTable(); } + /** + * Create a partial table snapshot to the {@code table} via the {@code targetUri}. Uses {@link #SNAP_OPTIONS}. + * + * @param targetUri the target URI + * @param table the table spec + * @return the table to snapshot + */ + public Table snapshot(String targetUri, TableSpec table, RowSet viewport, BitSet columns) + throws TableHandleException, InterruptedException { + return snapshot(DeephavenTarget.of(URI.create(targetUri)), table, SNAP_OPTIONS, viewport, columns, false); + } + /** * Create a partial table snapshot to the {@code table} via the {@code targetUri}. Uses {@link #SNAP_OPTIONS}. * From 372f61a9208c4ab3314819f72182ff4cd677ebb2 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Mon, 14 Feb 2022 14:36:04 -0800 Subject: [PATCH 04/20] tested worker to worker for snapshot & subscription --- .../io/deephaven/client/examples/SubscribeExampleBase.java | 2 -- .../io/deephaven/server/barrage/BarrageStreamGenerator.java | 4 ++-- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/java-client/barrage-examples/src/main/java/io/deephaven/client/examples/SubscribeExampleBase.java b/java-client/barrage-examples/src/main/java/io/deephaven/client/examples/SubscribeExampleBase.java index 5472d6f5645..a7497043483 100644 --- a/java-client/barrage-examples/src/main/java/io/deephaven/client/examples/SubscribeExampleBase.java +++ b/java-client/barrage-examples/src/main/java/io/deephaven/client/examples/SubscribeExampleBase.java @@ -60,8 +60,6 @@ protected void onFailureInternal(final Throwable originalException, final Entry public void onUpdate(final TableUpdate upstream) { System.out.println("Received table update:"); System.out.println(upstream); - - System.out.println(" Table size: " + table. } }); diff --git a/server/src/main/java/io/deephaven/server/barrage/BarrageStreamGenerator.java b/server/src/main/java/io/deephaven/server/barrage/BarrageStreamGenerator.java index 3e2e80d18e4..f01be8b3fc1 100644 --- a/server/src/main/java/io/deephaven/server/barrage/BarrageStreamGenerator.java +++ b/server/src/main/java/io/deephaven/server/barrage/BarrageStreamGenerator.java @@ -199,7 +199,7 @@ public void close() { public SubView getSubView(final BarrageSubscriptionOptions options, final boolean isInitialSnapshot, @Nullable final RowSet viewport, - boolean reverseViewport, + final boolean reverseViewport, @Nullable final RowSet keyspaceViewport, @Nullable final BitSet subscribedColumns) { return new SubView(this, options, isInitialSnapshot, viewport, reverseViewport, keyspaceViewport, @@ -537,7 +537,7 @@ private long appendAddColumns(final View view, // Add the drainable last as it is allowed to immediately close a row set the visitors need addStream.accept(drainableColumn); } - return myAddedOffsets == null ? rowsAdded.original.size() : myAddedOffsets.size(); + return rowsAdded.original.size(); } private long appendModColumns(final View view, From 7829fdc3e61ae6f9e7a3da65eedf4bee2152479f Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 18 Feb 2022 07:53:24 -0800 Subject: [PATCH 05/20] addressing review comments --- .../io/deephaven/engine/rowset/RowSet.java | 29 +++++---- .../rowset/impl/WritableRowSetImpl.java | 59 ++++++++++++++----- .../table/impl/remote/ConstructSnapshot.java | 35 +++++------ .../table/impl/util/BarrageMessage.java | 2 +- extensions/barrage/build.gradle | 2 +- .../barrage/table/BarrageTable.java | 2 +- .../barrage/util/BarrageStreamReader.java | 2 +- .../client/examples/SubscribeExampleBase.java | 2 +- .../barrage/BarrageMessageProducer.java | 19 ++++-- .../server/uri/BarrageTableResolver.java | 16 +++++ 10 files changed, 110 insertions(+), 58 deletions(-) diff --git a/engine/rowset/src/main/java/io/deephaven/engine/rowset/RowSet.java b/engine/rowset/src/main/java/io/deephaven/engine/rowset/RowSet.java index edd84f3833c..ded13ee707c 100644 --- a/engine/rowset/src/main/java/io/deephaven/engine/rowset/RowSet.java +++ b/engine/rowset/src/main/java/io/deephaven/engine/rowset/RowSet.java @@ -428,30 +428,33 @@ public long binarySearchValue(TargetComparator targetComparator, int direction) WritableRowSet subSetByKeyRange(long startKey, long endKey); /** - * Get a subset of this RowSet according to the supplied set of row positions in {@code posRowSet}. + * Get a subset of this RowSet according to the supplied sequence of row positions in {@code posRowSequence}. * - * @param posRowSet The RowSet of position-based ranges to extract. - * @param reversed use a reversed view of the table (index 0 is last row of table) - * @return A new RowSet, containing values at the locations in the provided RowSet. + * @param posRowSequence The {@link RowSequence} of positions ranges to get (as in {@link #get(long)}) + * @param reversed Whether to treat {@code posRowSet} as offsets relative to {@link #size()} rather than {@code 0} + * @return A new RowSet, containing the row keys from this RowSet at the row positions in {@code posRowSequence} */ - WritableRowSet subSetForPositions(RowSet posRowSet, boolean reversed); + WritableRowSet subSetForPositions(RowSequence posRowSequence, boolean reversed); /** - * Get a subset of this RowSet according to the supplied set of row positions in {@code posRowSet}. + * Get a subset of this RowSet according to the supplied sequence of row positions in {@code posRowSequence}. * - * @param posRowSet The RowSet of position-based ranges to extract. + * @param posRowSequence The {@link RowSequence} of position-based ranges to extract. * @return A new RowSet, containing values at the locations in the provided RowSet. */ - WritableRowSet subSetForPositions(RowSet posRowSet); + WritableRowSet subSetForPositions(RowSequence posRowSequence); + + /** - * Get a subset of this RowSet according to the supplied set of row positions in {@code posRowSet} in reversed - * position space (index 0 is last row of table) + * Get a subset of this RowSet according to the supplied sequence of row positions relative to {@link #size()} in + * {@code posRowSequence}. * - * @param posRowSet The RowSet of position-based ranges to extract. - * @return A new RowSet, containing values at the locations in the provided RowSet. + * @param posRowSequence The {@link RowSequence} of positions ranges to get (as in {@link #get(long)}) + * @return A new RowSet, containing the row keys from this RowSet at the row positions in {@code posRowSequence} */ - WritableRowSet subSetForReversePositions(RowSet posRowSet); + + WritableRowSet subSetForReversePositions(RowSequence posRowSequence); /** * Returns the row key at the given row position. diff --git a/engine/rowset/src/main/java/io/deephaven/engine/rowset/impl/WritableRowSetImpl.java b/engine/rowset/src/main/java/io/deephaven/engine/rowset/impl/WritableRowSetImpl.java index e1941ddd224..8c0662b8317 100644 --- a/engine/rowset/src/main/java/io/deephaven/engine/rowset/impl/WritableRowSetImpl.java +++ b/engine/rowset/src/main/java/io/deephaven/engine/rowset/impl/WritableRowSetImpl.java @@ -334,29 +334,30 @@ public final WritableRowSet subSetByKeyRange(final long startKey, final long end } @Override - public final WritableRowSet subSetForPositions(RowSet posRowSet, boolean reversed) { + public final WritableRowSet subSetForPositions(RowSequence posRowSequence, boolean reversed) { if (reversed) { - return subSetForReversePositions(posRowSet); + return subSetForReversePositions(posRowSequence); } - return subSetForPositions(posRowSet); + return subSetForPositions(posRowSequence); } @Override - public final WritableRowSet subSetForPositions(RowSet posRowSet) { + public final WritableRowSet subSetForPositions(RowSequence positions) { + if (positions.isEmpty()) { + return RowSetFactory.empty(); + } final MutableLong currentOffset = new MutableLong(); final RowSequence.Iterator iter = getRowSequenceIterator(); final RowSetBuilderSequential builder = RowSetFactory.builderSequential(); - posRowSet.forEachRowKeyRange((start, end) -> { + positions.forEachRowKeyRange((start, end) -> { if (currentOffset.longValue() < start) { // skip items until the beginning of this range iter.getNextRowSequenceWithLength(start - currentOffset.longValue()); currentOffset.setValue(start); } - if (!iter.hasMore()) { return false; } - iter.getNextRowSequenceWithLength(end + 1 - currentOffset.longValue()) .forAllRowKeyRanges(builder::appendRange); currentOffset.setValue(end + 1); @@ -366,17 +367,45 @@ public final WritableRowSet subSetForPositions(RowSet posRowSet) { } @Override - public final WritableRowSet subSetForReversePositions(RowSet posRowSet) { - final RowSetBuilderRandom builder = RowSetFactory.builderRandom(); + public final WritableRowSet subSetForReversePositions(RowSequence positions) { + if (positions.isEmpty()) { + return RowSetFactory.empty(); + } + + final long lastRowPosition = size() - 1; + if (positions.size() == positions.lastRowKey() - positions.firstRowKey() + 1) { + // We have a single range in the input sequence + final long forwardEnd = lastRowPosition - positions.firstRowKey(); + if (forwardEnd < 0) { + // The single range does not overlap with the available positions at all + return RowSetFactory.empty(); + } + // Clamp the single range end to 0 + final long forwardStart = Math.max(lastRowPosition - positions.lastRowKey(), 0); + try (final RowSequence forwardPositions = RowSequenceFactory.forRange(forwardStart, forwardEnd)) { + return subSetForPositions(forwardPositions); + } + } - long lastIndex = size() - 1; + // We have some non-trivial input sequence + final RowSetBuilderRandom builder = RowSetFactory.builderRandom(); + positions.forEachRowKeyRange((start, end) -> { + final long forwardEnd = lastRowPosition - start; + if (forwardEnd < 0) { + // This range does not overlap with the available positions at all, and thus neither can subsequent + // ranges that are offset further from the lastRowPosition. + return false; + } + // Clamp the range end to 0 + final long forwardStart = Math.max(lastRowPosition - end, 0); + builder.addRange(forwardStart, forwardEnd); - posRowSet.forEachRowKeyRange((start, end) -> { - builder.addRange(lastIndex - end, lastIndex - start); - return true; + // Continue iff subsequent ranges may overlap the available positions + return forwardStart != 0; }); - - return subSetForPositions(builder.build()); + try (final RowSequence forwardPositions = builder.build()) { + return subSetForPositions(forwardPositions); + } } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/remote/ConstructSnapshot.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/remote/ConstructSnapshot.java index 39cf54174ed..c78d46f6c97 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/remote/ConstructSnapshot.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/remote/ConstructSnapshot.java @@ -570,29 +570,26 @@ public static BarrageMessage constructBackplaneSnapshotInPositionSpace(final Obj if (positionsToSnapshot == null && reversePositionsToSnapshot == null) { keysToSnapshot = null; } else { - final RowSetBuilderRandom keyBuilder = RowSetFactory.builderRandom(); - if (usePrev) { - // perform actions on the previous rowset - try (final RowSet prevRowSet = table.getRowSet().copyPrev()) { - if (positionsToSnapshot != null) { - keyBuilder.addRowSet(prevRowSet.subSetForPositions(positionsToSnapshot)); + final RowSet rowSetToUse = usePrev ? table.getRowSet().copyPrev() : table.getRowSet(); + try (final SafeCloseable ignored = usePrev ? rowSetToUse : null) { + final WritableRowSet forwardKeys = positionsToSnapshot == null ? null : + rowSetToUse.subSetForPositions(positionsToSnapshot); + final RowSet reverseKeys = reversePositionsToSnapshot == null ? null : + rowSetToUse.subSetForReversePositions(reversePositionsToSnapshot); + if (forwardKeys != null) { + if (reverseKeys != null) { + forwardKeys.insert(reverseKeys); + reverseKeys.close(); } - if (reversePositionsToSnapshot != null) { - keyBuilder.addRowSet(prevRowSet.subSetForReversePositions(reversePositionsToSnapshot)); - } - } - } else { - // perform actions on the current rowset - if (positionsToSnapshot != null) { - keyBuilder.addRowSet(table.getRowSet().subSetForPositions(positionsToSnapshot)); - } - if (reversePositionsToSnapshot != null) { - keyBuilder.addRowSet(table.getRowSet().subSetForReversePositions(reversePositionsToSnapshot)); + keysToSnapshot = forwardKeys; + } else { + keysToSnapshot = reverseKeys; } } - keysToSnapshot = keyBuilder.build(); } - return serializeAllTable(usePrev, snapshot, table, logIdentityObject, columnsToSerialize, keysToSnapshot); + try (final RowSet ignored = keysToSnapshot) { + return serializeAllTable(usePrev, snapshot, table, logIdentityObject, columnsToSerialize, keysToSnapshot); + } }; snapshot.step = callDataSnapshotFunction(System.identityHashCode(logIdentityObject), control, doSnapshot); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/BarrageMessage.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/BarrageMessage.java index ef44222cc91..f12dce7ba51 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/BarrageMessage.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/BarrageMessage.java @@ -43,7 +43,7 @@ public static class AddColumnData { public boolean isSnapshot; public RowSet snapshotRowSet; - public boolean snapshotReverseRowSet; + public boolean snapshotRowSetIsReversed; public BitSet snapshotColumns; public RowSet rowsAdded; diff --git a/extensions/barrage/build.gradle b/extensions/barrage/build.gradle index 95229c77a69..dcd00db20e8 100644 --- a/extensions/barrage/build.gradle +++ b/extensions/barrage/build.gradle @@ -15,7 +15,7 @@ dependencies { api project(':engine-table') implementation project(':proto:proto-backplane-grpc-flight') implementation project(':log-factory') - api "io.deephaven.barrage:barrage-format:0.4.0" + api "io.deephaven.barrage:barrage-format:0.4.1" Classpaths.inheritFlatbuffer(project, 'implementation') diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/table/BarrageTable.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/table/BarrageTable.java index bdf6e7ec07e..1a446578e06 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/table/BarrageTable.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/table/BarrageTable.java @@ -223,7 +223,7 @@ private UpdateCoalescer processUpdate(final BarrageMessage update, final UpdateC if (update.isSnapshot) { serverViewport = update.snapshotRowSet == null ? null : update.snapshotRowSet.copy(); - serverReverseViewport = update.snapshotReverseRowSet; + serverReverseViewport = update.snapshotRowSetIsReversed; serverColumns = update.snapshotColumns == null ? null : (BitSet) update.snapshotColumns.clone(); } diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/BarrageStreamReader.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/BarrageStreamReader.java index e176cd130cb..60cb74c16b0 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/BarrageStreamReader.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/BarrageStreamReader.java @@ -81,7 +81,7 @@ public BarrageMessage safelyParseFrom(final StreamReaderOptions options, msg = new BarrageMessage(); msg.isSnapshot = metadata.isSnapshot(); - msg.snapshotReverseRowSet = metadata.effectiveReverseViewport(); + msg.snapshotRowSetIsReversed = metadata.effectiveReverseViewport(); numAddBatchesRemaining = metadata.numAddBatches(); numModBatchesRemaining = metadata.numModBatches(); diff --git a/java-client/barrage-examples/src/main/java/io/deephaven/client/examples/SubscribeExampleBase.java b/java-client/barrage-examples/src/main/java/io/deephaven/client/examples/SubscribeExampleBase.java index a7497043483..3dd66cb5188 100644 --- a/java-client/barrage-examples/src/main/java/io/deephaven/client/examples/SubscribeExampleBase.java +++ b/java-client/barrage-examples/src/main/java/io/deephaven/client/examples/SubscribeExampleBase.java @@ -43,7 +43,7 @@ protected void execute(final BarrageSession client) throws Exception { try (final TableHandle handle = manager.executeLogic(logic()); final BarrageSubscription subscription = client.subscribe(handle, options)) { -// final BarrageTable table = subscription.entireTable(); + final BarrageTable table = subscription.partialTable(RowSetFactory.flat(10), null, true); final CountDownLatch countDownLatch = new CountDownLatch(1); diff --git a/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java b/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java index 56679c6fdac..9657f5cb2fa 100644 --- a/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java +++ b/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java @@ -700,13 +700,20 @@ private void enqueueUpdate(final TableUpdate upstream) { modsToRecord = upstream.modified().copy(); } else if (activeViewport != null || activeReverseViewport != null) { // build the combined position-space viewport (from forward and reverse) - try (final WritableRowSet deltaViewport = RowSetFactory.builderRandom().build()) { - if (activeViewport != null) { - deltaViewport.insert(rowSet.subSetForPositions(activeViewport)); - } - if (activeReverseViewport != null) { - deltaViewport.insert(rowSet.subSetForReversePositions(activeReverseViewport)); + try (final WritableRowSet forwardDeltaViewport = activeViewport == null ? null : + rowSet.subSetForPositions(activeViewport); + final WritableRowSet reverseDeltaViewport = activeReverseViewport == null ? null : + rowSet.subSetForReversePositions(activeReverseViewport)) { + final RowSet deltaViewport; + if (forwardDeltaViewport != null) { + if (reverseDeltaViewport != null) { + forwardDeltaViewport.insert(reverseDeltaViewport); + } + deltaViewport = forwardDeltaViewport; + } else { + deltaViewport = reverseDeltaViewport; } + addsToRecord = deltaViewport.intersect(upstream.added()); modsToRecord = deltaViewport.intersect(upstream.modified()); } diff --git a/server/src/main/java/io/deephaven/server/uri/BarrageTableResolver.java b/server/src/main/java/io/deephaven/server/uri/BarrageTableResolver.java index 70d7ee60925..a1f11a7ddef 100644 --- a/server/src/main/java/io/deephaven/server/uri/BarrageTableResolver.java +++ b/server/src/main/java/io/deephaven/server/uri/BarrageTableResolver.java @@ -137,6 +137,8 @@ public Table subscribe(DeephavenTarget target, TableSpec table, BarrageSubscript * * @param targetUri the target URI * @param table the table spec + * @param viewport the position-space viewport to use for the subscription + * @param columns the columns to include in the subscription * @return the subscribed table */ public Table subscribe(String targetUri, TableSpec table, RowSet viewport, BitSet columns) @@ -149,6 +151,9 @@ public Table subscribe(String targetUri, TableSpec table, RowSet viewport, BitSe * * @param targetUri the target URI * @param table the table spec + * @param viewport the position-space viewport to use for the subscription + * @param columns the columns to include in the subscription + * @param reverseViewport Whether to treat {@code posRowSet} as offsets from to {@link #size()} rather than {@code 0} * @return the subscribed table */ public Table subscribe(String targetUri, TableSpec table, RowSet viewport, BitSet columns, boolean reverseViewport) @@ -163,6 +168,9 @@ public Table subscribe(String targetUri, TableSpec table, RowSet viewport, BitSe * @param target the target * @param table the table * @param options the options + * @param viewport the position-space viewport to use for the subscription + * @param columns the columns to include in the subscription + * @param reverseViewport Whether to treat {@code posRowSet} as offsets from to {@link #size()} rather than {@code 0} * * @return the subscribed table */ public Table subscribe(DeephavenTarget target, TableSpec table, BarrageSubscriptionOptions options, RowSet viewport, @@ -216,6 +224,8 @@ public Table snapshot(DeephavenTarget target, TableSpec table, BarrageSnapshotOp * * @param targetUri the target URI * @param table the table spec + * @param viewport the position-space viewport to use for the snapshot + * @param columns the columns to include in the snapshot * @return the table to snapshot */ public Table snapshot(String targetUri, TableSpec table, RowSet viewport, BitSet columns) @@ -228,6 +238,9 @@ public Table snapshot(String targetUri, TableSpec table, RowSet viewport, BitSet * * @param targetUri the target URI * @param table the table spec + * @param viewport the position-space viewport to use for the snapshot + * @param columns the columns to include in the snapshot + * @param reverseViewport Whether to treat {@code posRowSet} as offsets from to {@link #size()} rather than {@code 0} * @return the table to snapshot */ public Table snapshot(String targetUri, TableSpec table, RowSet viewport, BitSet columns, boolean reverseViewport) @@ -242,6 +255,9 @@ public Table snapshot(String targetUri, TableSpec table, RowSet viewport, BitSet * @param target the target * @param table the table * @param options the options + * @param viewport the position-space viewport to use for the snapshot + * @param columns the columns to include in the snapshot + * @param reverseViewport Whether to treat {@code posRowSet} as offsets from to {@link #size()} rather than {@code 0} * @return the table to snapshot */ public Table snapshot(DeephavenTarget target, TableSpec table, BarrageSnapshotOptions options, RowSet viewport, From 791f1fcfb12d9f17fedf295fc77a4765fd8d8f34 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 18 Feb 2022 08:02:34 -0800 Subject: [PATCH 06/20] spotless checks --- .../main/java/io/deephaven/engine/rowset/RowSet.java | 3 --- .../deephaven/client/impl/BarrageSnapshotImpl.java | 3 ++- .../client/impl/BarrageSubscriptionImpl.java | 3 ++- .../server/barrage/BarrageMessageProducer.java | 8 ++++---- .../deephaven/server/uri/BarrageTableResolver.java | 12 ++++++++---- 5 files changed, 16 insertions(+), 13 deletions(-) diff --git a/engine/rowset/src/main/java/io/deephaven/engine/rowset/RowSet.java b/engine/rowset/src/main/java/io/deephaven/engine/rowset/RowSet.java index ded13ee707c..7eb2e7dafc6 100644 --- a/engine/rowset/src/main/java/io/deephaven/engine/rowset/RowSet.java +++ b/engine/rowset/src/main/java/io/deephaven/engine/rowset/RowSet.java @@ -444,8 +444,6 @@ public long binarySearchValue(TargetComparator targetComparator, int direction) */ WritableRowSet subSetForPositions(RowSequence posRowSequence); - - /** * Get a subset of this RowSet according to the supplied sequence of row positions relative to {@link #size()} in * {@code posRowSequence}. @@ -453,7 +451,6 @@ public long binarySearchValue(TargetComparator targetComparator, int direction) * @param posRowSequence The {@link RowSequence} of positions ranges to get (as in {@link #get(long)}) * @return A new RowSet, containing the row keys from this RowSet at the row positions in {@code posRowSequence} */ - WritableRowSet subSetForReversePositions(RowSequence posRowSequence); /** diff --git a/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSnapshotImpl.java b/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSnapshotImpl.java index fc3f9896cbd..e1425b8a97f 100644 --- a/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSnapshotImpl.java +++ b/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSnapshotImpl.java @@ -156,7 +156,8 @@ public synchronized BarrageTable partialTable(RowSet viewport, BitSet columns) t } @Override - public synchronized BarrageTable partialTable(RowSet viewport, BitSet columns, boolean reverseViewport) throws InterruptedException { + public synchronized BarrageTable partialTable(RowSet viewport, BitSet columns, boolean reverseViewport) + throws InterruptedException { // notify user when connection has already been used and closed if (prevUsed) { throw new UnsupportedOperationException("Snapshot object already used"); diff --git a/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSubscriptionImpl.java b/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSubscriptionImpl.java index 32baff4e0c8..58987a17891 100644 --- a/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSubscriptionImpl.java +++ b/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSubscriptionImpl.java @@ -151,7 +151,8 @@ public synchronized BarrageTable partialTable(RowSet viewport, BitSet columns, b if (!subscribed) { // Send the initial subscription: observer.onNext(FlightData.newBuilder() - .setAppMetadata(ByteStringAccess.wrap(makeRequestInternal(viewport, columns, reverseViewport, options))) + .setAppMetadata( + ByteStringAccess.wrap(makeRequestInternal(viewport, columns, reverseViewport, options))) .build()); subscribed = true; } diff --git a/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java b/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java index 9657f5cb2fa..ae392a3a158 100644 --- a/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java +++ b/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java @@ -700,10 +700,10 @@ private void enqueueUpdate(final TableUpdate upstream) { modsToRecord = upstream.modified().copy(); } else if (activeViewport != null || activeReverseViewport != null) { // build the combined position-space viewport (from forward and reverse) - try (final WritableRowSet forwardDeltaViewport = activeViewport == null ? null : - rowSet.subSetForPositions(activeViewport); - final WritableRowSet reverseDeltaViewport = activeReverseViewport == null ? null : - rowSet.subSetForReversePositions(activeReverseViewport)) { + try (final WritableRowSet forwardDeltaViewport = + activeViewport == null ? null : rowSet.subSetForPositions(activeViewport); + final WritableRowSet reverseDeltaViewport = activeReverseViewport == null ? null + : rowSet.subSetForReversePositions(activeReverseViewport)) { final RowSet deltaViewport; if (forwardDeltaViewport != null) { if (reverseDeltaViewport != null) { diff --git a/server/src/main/java/io/deephaven/server/uri/BarrageTableResolver.java b/server/src/main/java/io/deephaven/server/uri/BarrageTableResolver.java index a1f11a7ddef..fe891892ad8 100644 --- a/server/src/main/java/io/deephaven/server/uri/BarrageTableResolver.java +++ b/server/src/main/java/io/deephaven/server/uri/BarrageTableResolver.java @@ -153,7 +153,8 @@ public Table subscribe(String targetUri, TableSpec table, RowSet viewport, BitSe * @param table the table spec * @param viewport the position-space viewport to use for the subscription * @param columns the columns to include in the subscription - * @param reverseViewport Whether to treat {@code posRowSet} as offsets from to {@link #size()} rather than {@code 0} + * @param reverseViewport Whether to treat {@code posRowSet} as offsets from to {@link #size()} rather than + * {@code 0} * @return the subscribed table */ public Table subscribe(String targetUri, TableSpec table, RowSet viewport, BitSet columns, boolean reverseViewport) @@ -170,7 +171,8 @@ public Table subscribe(String targetUri, TableSpec table, RowSet viewport, BitSe * @param options the options * @param viewport the position-space viewport to use for the subscription * @param columns the columns to include in the subscription - * @param reverseViewport Whether to treat {@code posRowSet} as offsets from to {@link #size()} rather than {@code 0} * + * @param reverseViewport Whether to treat {@code posRowSet} as offsets from to {@link #size()} rather than + * {@code 0} * * @return the subscribed table */ public Table subscribe(DeephavenTarget target, TableSpec table, BarrageSubscriptionOptions options, RowSet viewport, @@ -240,7 +242,8 @@ public Table snapshot(String targetUri, TableSpec table, RowSet viewport, BitSet * @param table the table spec * @param viewport the position-space viewport to use for the snapshot * @param columns the columns to include in the snapshot - * @param reverseViewport Whether to treat {@code posRowSet} as offsets from to {@link #size()} rather than {@code 0} + * @param reverseViewport Whether to treat {@code posRowSet} as offsets from to {@link #size()} rather than + * {@code 0} * @return the table to snapshot */ public Table snapshot(String targetUri, TableSpec table, RowSet viewport, BitSet columns, boolean reverseViewport) @@ -257,7 +260,8 @@ public Table snapshot(String targetUri, TableSpec table, RowSet viewport, BitSet * @param options the options * @param viewport the position-space viewport to use for the snapshot * @param columns the columns to include in the snapshot - * @param reverseViewport Whether to treat {@code posRowSet} as offsets from to {@link #size()} rather than {@code 0} + * @param reverseViewport Whether to treat {@code posRowSet} as offsets from to {@link #size()} rather than + * {@code 0} * @return the table to snapshot */ public Table snapshot(DeephavenTarget target, TableSpec table, BarrageSnapshotOptions options, RowSet viewport, From 96f49930ff786291757d2c5fd9c07049d164b4c8 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 18 Feb 2022 11:02:28 -0800 Subject: [PATCH 07/20] added reverseViewport to test utilities, existing tests pass --- .../client/impl/BarrageSnapshot.java | 8 ++++ .../client/impl/BarrageSubscription.java | 8 ++++ .../barrage/BarrageMessageProducer.java | 12 ++++++ .../barrage/BarrageMessageRoundTripTest.java | 43 +++++++++++++++---- 4 files changed, 62 insertions(+), 9 deletions(-) diff --git a/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSnapshot.java b/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSnapshot.java index b2be6faef01..1e62f67444a 100644 --- a/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSnapshot.java +++ b/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSnapshot.java @@ -50,6 +50,9 @@ BarrageSnapshot snapshot(TableSpec tableSpec, BarrageSnapshotOptions options) * Request a partial snapshot of the data limited by viewport or column set and populate a {@link BarrageTable} with * the data that is received. * + * @param viewport the position-space viewport to use for the snapshot + * @param columns the columns to include in the snapshot + * * @return the {@code BarrageTable} */ BarrageTable partialTable(RowSet viewport, BitSet columns) throws InterruptedException; @@ -58,6 +61,11 @@ BarrageSnapshot snapshot(TableSpec tableSpec, BarrageSnapshotOptions options) * Request a partial snapshot of the data limited by viewport or column set and populate a {@link BarrageTable} with * the data that is received. Allows the viewport to be reversed. * + * @param viewport the position-space viewport to use for the snapshot + * @param columns the columns to include in the snapshot + * @param reverseViewport Whether to treat {@code posRowSet} as offsets from to {@link #size()} rather than + * {@code 0} + * * @return the {@code BarrageTable} */ BarrageTable partialTable(RowSet viewport, BitSet columns, boolean reverseViewport) throws InterruptedException; diff --git a/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSubscription.java b/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSubscription.java index b14fcce994a..9f78b3a3096 100644 --- a/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSubscription.java +++ b/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSubscription.java @@ -52,6 +52,9 @@ BarrageSubscription subscribe(TableSpec tableSpec, BarrageSubscriptionOptions op * Request a partial subscription of the data limited by viewport or column set and populate a {@link BarrageTable} * with the data that is received. * + * @param viewport the position-space viewport to use for the subscription + * @param columns the columns to include in the subscription + * * @return the {@code BarrageTable} */ BarrageTable partialTable(RowSet viewport, BitSet columns) throws InterruptedException; @@ -60,6 +63,11 @@ BarrageSubscription subscribe(TableSpec tableSpec, BarrageSubscriptionOptions op * Request a partial subscription of the data limited by viewport or column set and populate a {@link BarrageTable} * with the data that is received. Allows the viewport to be reversed. * + * @param viewport the position-space viewport to use for the subscription + * @param columns the columns to include in the subscription + * @param reverseViewport Whether to treat {@code posRowSet} as offsets from to {@link #size()} rather than + * {@code 0} * + * * @return the {@code BarrageTable} */ BarrageTable partialTable(RowSet viewport, BitSet columns, boolean reverseViewport) throws InterruptedException; diff --git a/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java b/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java index ae392a3a158..b31d6d33a96 100644 --- a/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java +++ b/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java @@ -565,11 +565,17 @@ public boolean updateSubscription(final StreamObserver listener, public boolean updateViewport(final StreamObserver listener, final RowSet newViewport) { + return updateViewport(listener, newViewport, false); + } + + public boolean updateViewport(final StreamObserver listener, final RowSet newViewport, + final boolean newReverseViewport) { return findAndUpdateSubscription(listener, sub -> { if (sub.pendingViewport != null) { sub.pendingViewport.close(); } sub.pendingViewport = newViewport.copy(); + sub.reverseViewport = newReverseViewport; if (sub.pendingColumns == null) { sub.pendingColumns = (BitSet) sub.subscribedColumns.clone(); } @@ -580,11 +586,17 @@ public boolean updateViewport(final StreamObserver listener, public boolean updateViewportAndColumns(final StreamObserver listener, final RowSet newViewport, final BitSet columnsToSubscribe) { + return updateViewportAndColumns(listener, newViewport, columnsToSubscribe); + } + + public boolean updateViewportAndColumns(final StreamObserver listener, final RowSet newViewport, + final BitSet columnsToSubscribe, final boolean newReverseViewport) { return findAndUpdateSubscription(listener, sub -> { if (sub.pendingViewport != null) { sub.pendingViewport.close(); } sub.pendingViewport = newViewport.copy(); + sub.reverseViewport = newReverseViewport; sub.pendingColumns = (BitSet) columnsToSubscribe.clone(); log.info().append(logPrefix).append(sub.logPrefix) .append("scheduling update immediately, for viewport and column updates.").endl(); diff --git a/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java b/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java index dbdba773384..7064146a2eb 100644 --- a/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java +++ b/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java @@ -153,6 +153,8 @@ public void onFailureInternal(Throwable originalException, Entry sourceEntry) { private class RemoteClient { private RowSet viewport; + private boolean reverseViewport; + private BitSet subscribedColumns; private final String name; @@ -175,13 +177,15 @@ private class RemoteClient { RemoteClient(final RowSet viewport, final BitSet subscribedColumns, final BarrageMessageProducer barrageMessageProducer, final String name) { - this(viewport, subscribedColumns, barrageMessageProducer, name, false); + // assume a forward viewport when not specified + this(viewport, subscribedColumns, barrageMessageProducer, name, false, false); } RemoteClient(final RowSet viewport, final BitSet subscribedColumns, - final BarrageMessageProducer barrageMessageProducer, - final String name, final boolean deferSubscription) { + final BarrageMessageProducer barrageMessageProducer, + final String name, final boolean reverseViewport, final boolean deferSubscription) { this.viewport = viewport; + this.reverseViewport = reverseViewport; this.subscribedColumns = subscribedColumns; this.name = name; this.barrageMessageProducer = barrageMessageProducer; @@ -220,7 +224,7 @@ public void doSubscribe() { .useDeephavenNulls(useDeephavenNulls) .build(); barrageMessageProducer.addSubscription(dummyObserver, options, subscribedColumns, - viewport == null ? null : viewport.copy(), false); + viewport == null ? null : viewport.copy(), reverseViewport); } public void validate(final String msg, QueryTable expected) { @@ -232,8 +236,8 @@ public void validate(final String msg, QueryTable expected) { QueryTable toCheck = barrageTable; if (viewport != null) { - expected = expected.getSubTable(expected.getRowSet().subSetForPositions(viewport).toTracking()); - toCheck = toCheck.getSubTable(toCheck.getRowSet().subSetForPositions(viewport).toTracking()); + expected = expected.getSubTable(expected.getRowSet().subSetForPositions(viewport, reverseViewport).toTracking()); + toCheck = toCheck.getSubTable(toCheck.getRowSet().subSetForPositions(viewport, reverseViewport).toTracking()); } if (subscribedColumns.cardinality() != expected.getColumns().length) { final List columns = new ArrayList<>(); @@ -301,8 +305,15 @@ public void flushEventsToReplicatedTable() { } public void setViewport(final RowSet newViewport) { + // assume a forward viewport when not specified + setViewport(newViewport, false); + } + + public void setViewport(final RowSet newViewport, final boolean newReverseViewport) { viewport = newViewport; - barrageMessageProducer.updateViewport(dummyObserver, viewport); + reverseViewport = newReverseViewport; + + barrageMessageProducer.updateViewport(dummyObserver, viewport, reverseViewport); } public void setSubscribedColumns(final BitSet newColumns) { @@ -311,6 +322,12 @@ public void setSubscribedColumns(final BitSet newColumns) { } public void setViewportAndColumns(final RowSet newViewport, final BitSet newColumns) { + // assume a forward viewport when not specified + setViewportAndColumns(newViewport, newColumns, false); + } + + public void setViewportAndColumns(final RowSet newViewport, final BitSet newColumns, + final boolean newReverseViewport) { viewport = newViewport; subscribedColumns = newColumns; barrageMessageProducer.updateViewportAndColumns(dummyObserver, viewport, subscribedColumns); @@ -366,7 +383,13 @@ public void flushClientEvents() { } public RemoteClient newClient(final RowSet viewport, final BitSet subscribedColumns, final String name) { - clients.add(new RemoteClient(viewport, subscribedColumns, barrageMessageProducer, name)); + // assume a forward viewport when not specified + return newClient(viewport, subscribedColumns, false, name); + } + + public RemoteClient newClient(final RowSet viewport, final BitSet subscribedColumns, + final boolean reverseViewport, final String name) { + clients.add(new RemoteClient(viewport, subscribedColumns, barrageMessageProducer, name, reverseViewport,false)); return clients.get(clients.size() - 1); } @@ -501,6 +524,8 @@ void createNuggetsForTableMaker(final Supplier makeTable) { nugget.newClient(RowSetFactory.fromRange(size / 2, size * 3L / 4), subscribedColumns, "floating"); + //nugget.newClient(RowSetFactory.fromRange(0, size / 10), subscribedColumns, true, "reverse header"); + final RowSetBuilderSequential swissIndexBuilder = RowSetFactory.builderSequential(); final long rangeSize = Math.max(1, size / 20); for (long nr = 1; nr < 20; nr += 2) { @@ -896,7 +921,7 @@ public void onGetSnapshot() { final boolean deferSubscription = true; nugget.clients.add(new RemoteClient( RowSetFactory.fromRange(size / 5, 2 * size / 5), - columns, nugget.barrageMessageProducer, "sub-changer", deferSubscription)); + columns, nugget.barrageMessageProducer, "sub-changer", false, deferSubscription)); } }.runTest(); From cfe2306880126af592aaa997c9a3be6fae6b672a Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 22 Feb 2022 11:35:17 -0800 Subject: [PATCH 08/20] most tests passing (not swiss reverse) --- .../barrage/BarrageMessageProducer.java | 71 +++++++++++++++---- .../barrage/BarrageMessageRoundTripTest.java | 15 ++-- 2 files changed, 67 insertions(+), 19 deletions(-) diff --git a/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java b/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java index b31d6d33a96..0409bebf455 100644 --- a/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java +++ b/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java @@ -25,6 +25,7 @@ import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.rowset.TrackingRowSet; import io.deephaven.engine.rowset.WritableRowSet; +import io.deephaven.engine.rowset.impl.AdaptiveRowSetBuilderRandom; import io.deephaven.engine.table.ChunkSink; import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.ColumnSource; @@ -739,7 +740,7 @@ private void enqueueUpdate(final TableUpdate upstream) { // Note: viewports are in position space, inserted and removed rows may cause the keyspace for a given viewport // to shift. Let's compute which rows are being scoped into view. If current RowSet is empty, we have nothing to // store. If prev RowSet is empty, all rows are new and are already in addsToRecord. - if (activeViewport != null && (upstream.added().isNonempty() || upstream.removed().isNonempty()) + if ((activeViewport != null || activeReverseViewport != null) && (upstream.added().isNonempty() || upstream.removed().isNonempty()) && rowSet.isNonempty() && rowSet.sizePrev() > 0) { final RowSetBuilderRandom scopedViewBuilder = RowSetFactory.builderRandom(); @@ -752,12 +753,40 @@ private void enqueueUpdate(final TableUpdate upstream) { final ShiftInversionHelper inverter = new ShiftInversionHelper(upstream.shifted()); +// System.out.println("scoped: step=" + this.lastIndexClockStep); +// System.out.println(" upstream=" + upstream); + sub.viewport.forAllRowKeyRanges((posStart, posEnd) -> { + final long localStart, localEnd; + +// System.out.println(" rowSet.size()=" + rowSet.size() + ", prevRowSet.size()=" + prevRowSet.size()); + + // handle reverse viewports + if (sub.reverseViewport) { + // reset positions to be relative to the final position of this table + final long lastRowPosition = rowSet.size() - 1; + + localStart = Math.max(lastRowPosition - posEnd, 0); + localEnd = lastRowPosition - posStart; + + if (localEnd < 0) { + // This range does not overlap with the available positions at all + return; + } + } else { + localStart = posStart; + localEnd = posEnd; + } + +// System.out.println(" localStart=" + localStart + ", localEnd=" + localEnd); + // Note: we already know that both rowSet and prevRowSet are non-empty. final long currKeyStart = - inverter.mapToPrevKeyspace(rowSet.get(Math.min(posStart, rowSet.size() - 1)), false); + inverter.mapToPrevKeyspace(rowSet.get(Math.min(localStart, rowSet.size() - 1)), false); final long currKeyEnd = - inverter.mapToPrevKeyspace(rowSet.get(Math.min(posEnd, rowSet.size() - 1)), true); + inverter.mapToPrevKeyspace(rowSet.get(Math.min(localEnd, rowSet.size() - 1)), true); + +// System.out.println(" currKeyStart=" + currKeyStart + ", currKeyEnd=" + currKeyEnd); // if our current viewport includes no previous values this range may be empty if (currKeyEnd < currKeyStart) { @@ -765,16 +794,23 @@ private void enqueueUpdate(final TableUpdate upstream) { } final long prevKeyStart = - posStart >= prevRowSet.size() ? prevRowSet.lastRowKey() + 1 : prevRowSet.get(posStart); - final long prevKeyEnd = prevRowSet.get(Math.min(posEnd, prevRowSet.size() - 1)); + localStart >= prevRowSet.size() ? prevRowSet.lastRowKey() + 1 : prevRowSet.get(localStart); + final long prevKeyEnd = prevRowSet.get(Math.min(localEnd, prevRowSet.size() - 1)); - // Note: we already know that scoped rows must touch viewport boundaries - if (currKeyStart < prevKeyStart) { - scopedViewBuilder.addRange(currKeyStart, Math.min(prevKeyStart - 1, currKeyEnd)); - } - if (currKeyEnd > prevKeyEnd) { - scopedViewBuilder.addRange(Math.max(prevKeyEnd + 1, currKeyStart), currKeyEnd); - } +// System.out.println(" prevKeyStart=" + prevKeyStart + ", prevKeyEnd=" + prevKeyEnd); + + final long minStart = Math.min(currKeyStart, prevKeyStart); + final long maxEnd = Math.max(currKeyEnd, prevKeyEnd); + + scopedViewBuilder.addRange(minStart, maxEnd); + +// // Note: we already know that scoped rows must touch viewport boundaries +// if (currKeyStart < prevKeyStart) { +// scopedViewBuilder.addRange(currKeyStart, Math.min(prevKeyStart - 1, currKeyEnd)); +// } +// if (currKeyEnd > prevKeyEnd) { +// scopedViewBuilder.addRange(Math.max(prevKeyEnd + 1, currKeyStart), currKeyEnd); +// } }); } } @@ -783,6 +819,8 @@ private void enqueueUpdate(final TableUpdate upstream) { upstream.shifted().apply(scoped); // we built scoped rows in prev-keyspace scoped.retain(rowSet); // we only record valid rows addsToRecord.insert(scoped); + +// System.out.println(" scoped=" + scoped); } } @@ -1324,9 +1362,11 @@ private BarrageMessage aggregateUpdatesInRange(final int startDelta, final int e modColumnSet = firstDelta.modifiedColumns; downstream.rowsAdded = firstDelta.update.added().copy(); + downstream.rowsRemoved = firstDelta.update.removed().copy(); downstream.shifted = firstDelta.update.shifted(); downstream.rowsIncluded = firstDelta.recordedAdds.copy(); + downstream.addColumnData = new BarrageMessage.AddColumnData[sourceColumns.length]; downstream.modColumnData = new BarrageMessage.ModColumnData[sourceColumns.length]; @@ -1630,8 +1670,11 @@ private void promoteSnapshotToActive() { this.activeReverseViewport.close(); } - this.activeViewport = this.postSnapshotViewport; - this.activeReverseViewport = this.postSnapshotReverseViewport; + this.activeViewport = this.postSnapshotViewport == null || this.postSnapshotViewport.isEmpty() ? null : + this.postSnapshotViewport; + + this.activeReverseViewport = this.postSnapshotReverseViewport == null || this.postSnapshotReverseViewport.isEmpty() ? null : + this.postSnapshotReverseViewport; this.postSnapshotViewport = null; this.postSnapshotReverseViewport = null; diff --git a/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java b/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java index 7064146a2eb..60f04b14c07 100644 --- a/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java +++ b/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java @@ -41,6 +41,7 @@ import io.deephaven.server.arrow.ArrowModule; import io.deephaven.server.util.Scheduler; import io.deephaven.server.util.TestControlledScheduler; +import io.deephaven.tablelogger.Row; import io.deephaven.test.types.OutOfBandTest; import io.deephaven.time.DateTimeUtils; import io.deephaven.util.annotations.ReferentialIntegrity; @@ -387,9 +388,8 @@ public RemoteClient newClient(final RowSet viewport, final BitSet subscribedColu return newClient(viewport, subscribedColumns, false, name); } - public RemoteClient newClient(final RowSet viewport, final BitSet subscribedColumns, - final boolean reverseViewport, final String name) { - clients.add(new RemoteClient(viewport, subscribedColumns, barrageMessageProducer, name, reverseViewport,false)); + public RemoteClient newClient(final RowSet viewport, final BitSet subscribedColumns, final boolean reverseViewport, final String name) { + clients.add(new RemoteClient(viewport, subscribedColumns, barrageMessageProducer, name, reverseViewport, false)); return clients.get(clients.size() - 1); } @@ -518,20 +518,25 @@ void createNuggetsForTableMaker(final Supplier
makeTable) { final BitSet subscribedColumns = new BitSet(); subscribedColumns.set(0, nugget.originalTable.getColumns().length); + nugget.newClient(null, subscribedColumns, "full"); nugget.newClient(RowSetFactory.fromRange(0, size / 10), subscribedColumns, "header"); nugget.newClient(RowSetFactory.fromRange(size / 2, size * 3L / 4), subscribedColumns, "floating"); - //nugget.newClient(RowSetFactory.fromRange(0, size / 10), subscribedColumns, true, "reverse header"); + nugget.newClient(RowSetFactory.fromRange(0, size / 10), subscribedColumns, true, "footer"); + nugget.newClient(RowSetFactory.fromRange(size / 2, size * 3L / 4), subscribedColumns, true, "reverse floating"); final RowSetBuilderSequential swissIndexBuilder = RowSetFactory.builderSequential(); final long rangeSize = Math.max(1, size / 20); for (long nr = 1; nr < 20; nr += 2) { swissIndexBuilder.appendRange(nr * rangeSize, (nr + 1) * rangeSize - 1); } - nugget.newClient(swissIndexBuilder.build(), subscribedColumns, "swiss viewport"); + + nugget.newClient(swissIndexBuilder.build(), subscribedColumns, "swiss"); + +// nugget.newClient(swissIndexBuilder.build(), subscribedColumns, true, "reverse swiss"); } } From bcc18029bb6f17c1929128042a95726de1bc9386 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 22 Feb 2022 16:40:38 -0800 Subject: [PATCH 09/20] added reverse viewport to unit tests, all passing --- .../table/impl/util/ShiftInversionHelper.java | 30 +++++-- .../barrage/BarrageMessageProducer.java | 78 +++++++++---------- .../barrage/BarrageMessageRoundTripTest.java | 20 ++--- 3 files changed, 72 insertions(+), 56 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/ShiftInversionHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/ShiftInversionHelper.java index 3b397507916..fef5a049e50 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/ShiftInversionHelper.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/ShiftInversionHelper.java @@ -10,31 +10,47 @@ */ public class ShiftInversionHelper { - final RowSetShiftData shifted; + final private RowSetShiftData shifted; + final private boolean reverseOrder; - private int destShiftIdx = 0; + private int destShiftIdx; public ShiftInversionHelper(final RowSetShiftData shifted) { + // if not specified, assume forward viewport ordering + this(shifted, false); + } + + public ShiftInversionHelper(final RowSetShiftData shifted, final boolean reverseOrder) { this.shifted = shifted; + this.reverseOrder = reverseOrder; + this.destShiftIdx = reverseOrder ? shifted.size() : 0; } private void advanceDestShiftIdx(final long destKey) { Assert.geq(destKey, "destKey", 0); - destShiftIdx = (int) binarySearch(destShiftIdx, shifted.size(), innerShiftIdx -> { - long destEnd = shifted.getEndRange((int) innerShiftIdx) + shifted.getShiftDelta((int) innerShiftIdx); - // due to destKey's expected range, we know this subtraction will not overflow - return destEnd - destKey; - }); + destShiftIdx = (int) binarySearch( + reverseOrder ? 0 : destShiftIdx, + reverseOrder ? destShiftIdx : shifted.size(), + innerShiftIdx -> { + long destEnd = shifted.getEndRange((int) innerShiftIdx) + shifted.getShiftDelta((int) innerShiftIdx); + // due to destKey's expected range, we know this subtraction will not overflow + return destEnd - destKey; + }); } /** * Converts post-keyspace key to pre-keyspace key. It expects to be invoked in ascending key order. */ public long mapToPrevKeyspace(final long key, final boolean isEnd) { + if (shifted.empty()) { + return key; + } + advanceDestShiftIdx(key); final long retval; final int idx = destShiftIdx; + if (idx < shifted.size() && shifted.getBeginRange(idx) + shifted.getShiftDelta(idx) <= key) { // inside of a destination shift; this is easy to map to prev retval = key - shifted.getShiftDelta(idx); diff --git a/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java b/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java index 0409bebf455..461cb767bfb 100644 --- a/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java +++ b/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java @@ -751,19 +751,14 @@ private void enqueueUpdate(final TableUpdate upstream) { continue; } - final ShiftInversionHelper inverter = new ShiftInversionHelper(upstream.shifted()); - -// System.out.println("scoped: step=" + this.lastIndexClockStep); -// System.out.println(" upstream=" + upstream); + final ShiftInversionHelper inverter = new ShiftInversionHelper(upstream.shifted(), sub.reverseViewport); sub.viewport.forAllRowKeyRanges((posStart, posEnd) -> { final long localStart, localEnd; -// System.out.println(" rowSet.size()=" + rowSet.size() + ", prevRowSet.size()=" + prevRowSet.size()); - // handle reverse viewports if (sub.reverseViewport) { - // reset positions to be relative to the final position of this table + // compute positions to be relative to the final position of rowSet final long lastRowPosition = rowSet.size() - 1; localStart = Math.max(lastRowPosition - posEnd, 0); @@ -778,39 +773,52 @@ private void enqueueUpdate(final TableUpdate upstream) { localEnd = posEnd; } -// System.out.println(" localStart=" + localStart + ", localEnd=" + localEnd); - // Note: we already know that both rowSet and prevRowSet are non-empty. - final long currKeyStart = - inverter.mapToPrevKeyspace(rowSet.get(Math.min(localStart, rowSet.size() - 1)), false); - final long currKeyEnd = - inverter.mapToPrevKeyspace(rowSet.get(Math.min(localEnd, rowSet.size() - 1)), true); - -// System.out.println(" currKeyStart=" + currKeyStart + ", currKeyEnd=" + currKeyEnd); + final long currKeyStart, currKeyEnd; + if (sub.reverseViewport) { + // using the reverse ShiftHelper, must pass `key` in descending order + currKeyEnd = + inverter.mapToPrevKeyspace(rowSet.get(Math.min(localEnd, rowSet.size() - 1)), true); + currKeyStart = + inverter.mapToPrevKeyspace(rowSet.get(Math.min(localStart, rowSet.size() - 1)), false); + } else { + // using the forward ShiftHelper, must pass `key` in ascending order + currKeyStart = + inverter.mapToPrevKeyspace(rowSet.get(Math.min(localStart, rowSet.size() - 1)), false); + currKeyEnd = + inverter.mapToPrevKeyspace(rowSet.get(Math.min(localEnd, rowSet.size() - 1)), true); + } // if our current viewport includes no previous values this range may be empty if (currKeyEnd < currKeyStart) { return; } - final long prevKeyStart = - localStart >= prevRowSet.size() ? prevRowSet.lastRowKey() + 1 : prevRowSet.get(localStart); - final long prevKeyEnd = prevRowSet.get(Math.min(localEnd, prevRowSet.size() - 1)); + final long prevKeyStart, prevKeyEnd; -// System.out.println(" prevKeyStart=" + prevKeyStart + ", prevKeyEnd=" + prevKeyEnd); + if (sub.reverseViewport) { + // compute positions to be relative to the final position of prevRowSet + final long lastPrevRowPosition = prevRowSet.size() - 1; - final long minStart = Math.min(currKeyStart, prevKeyStart); - final long maxEnd = Math.max(currKeyEnd, prevKeyEnd); + long prevStart = Math.max(lastPrevRowPosition - posEnd, 0); + long prevEnd = lastPrevRowPosition - posStart; - scopedViewBuilder.addRange(minStart, maxEnd); + prevKeyStart = + prevStart >= prevRowSet.size() ? prevRowSet.lastRowKey() + 1 : prevRowSet.get(prevStart); + prevKeyEnd = prevRowSet.get(Math.min(prevEnd, prevRowSet.size() - 1)); + } else { + prevKeyStart = + localStart >= prevRowSet.size() ? prevRowSet.lastRowKey() + 1 : prevRowSet.get(localStart); + prevKeyEnd = prevRowSet.get(Math.min(localEnd, prevRowSet.size() - 1)); + } -// // Note: we already know that scoped rows must touch viewport boundaries -// if (currKeyStart < prevKeyStart) { -// scopedViewBuilder.addRange(currKeyStart, Math.min(prevKeyStart - 1, currKeyEnd)); -// } -// if (currKeyEnd > prevKeyEnd) { -// scopedViewBuilder.addRange(Math.max(prevKeyEnd + 1, currKeyStart), currKeyEnd); -// } + // Note: we already know that scoped rows must touch viewport boundaries + if (currKeyStart < prevKeyStart) { + scopedViewBuilder.addRange(currKeyStart, Math.min(prevKeyStart - 1, currKeyEnd)); + } + if (currKeyEnd > prevKeyEnd) { + scopedViewBuilder.addRange(Math.max(prevKeyEnd + 1, currKeyStart), currKeyEnd); + } }); } } @@ -819,8 +827,6 @@ private void enqueueUpdate(final TableUpdate upstream) { upstream.shifted().apply(scoped); // we built scoped rows in prev-keyspace scoped.retain(rowSet); // we only record valid rows addsToRecord.insert(scoped); - -// System.out.println(" scoped=" + scoped); } } @@ -868,14 +874,8 @@ private void enqueueUpdate(final TableUpdate upstream) { final BiConsumer recordRows = (keysToAdd, columnsToRecord) -> { try (final RowSequence.Iterator rsIt = keysToAdd.getRowSequenceIterator()) { while (rsIt.hasMore()) { - final RowSequence srcKeys = rsIt.getNextRowSequenceWithLength(DELTA_CHUNK_SIZE); // NB: This - // will - // never - // return - // more - // keys - // than - // deltaChunkSize + // NB: This will never return more keys than deltaChunkSize + final RowSequence srcKeys = rsIt.getNextRowSequenceWithLength(DELTA_CHUNK_SIZE); try (final RowSequence dstKeys = RowSequenceFactory.forRange(nextFreeDeltaKey, nextFreeDeltaKey + srcKeys.size() - 1)) { diff --git a/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java b/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java index 60f04b14c07..f0904bc84cb 100644 --- a/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java +++ b/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java @@ -519,14 +519,14 @@ void createNuggetsForTableMaker(final Supplier
makeTable) { final BitSet subscribedColumns = new BitSet(); subscribedColumns.set(0, nugget.originalTable.getColumns().length); - nugget.newClient(null, subscribedColumns, "full"); - - nugget.newClient(RowSetFactory.fromRange(0, size / 10), subscribedColumns, "header"); - nugget.newClient(RowSetFactory.fromRange(size / 2, size * 3L / 4), subscribedColumns, - "floating"); - - nugget.newClient(RowSetFactory.fromRange(0, size / 10), subscribedColumns, true, "footer"); - nugget.newClient(RowSetFactory.fromRange(size / 2, size * 3L / 4), subscribedColumns, true, "reverse floating"); +// nugget.newClient(null, subscribedColumns, "full"); +// +// nugget.newClient(RowSetFactory.fromRange(0, size / 10), subscribedColumns, "header"); +// nugget.newClient(RowSetFactory.fromRange(size / 2, size * 3L / 4), subscribedColumns, +// "floating"); +// +// nugget.newClient(RowSetFactory.fromRange(0, size / 10), subscribedColumns, true, "footer"); +// nugget.newClient(RowSetFactory.fromRange(size / 2, size * 3L / 4), subscribedColumns, true, "reverse floating"); final RowSetBuilderSequential swissIndexBuilder = RowSetFactory.builderSequential(); final long rangeSize = Math.max(1, size / 20); @@ -534,9 +534,9 @@ void createNuggetsForTableMaker(final Supplier
makeTable) { swissIndexBuilder.appendRange(nr * rangeSize, (nr + 1) * rangeSize - 1); } - nugget.newClient(swissIndexBuilder.build(), subscribedColumns, "swiss"); +// nugget.newClient(swissIndexBuilder.build(), subscribedColumns, "swiss"); -// nugget.newClient(swissIndexBuilder.build(), subscribedColumns, true, "reverse swiss"); + nugget.newClient(swissIndexBuilder.build(), subscribedColumns, true, "reverse swiss"); } } From a1b3ea7e90daf35ed3460f588c160a1fd2b2c3fb Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 22 Feb 2022 16:43:13 -0800 Subject: [PATCH 10/20] spotless checks completed --- .../barrage/BarrageMessageProducer.java | 31 ++++++++++------ .../barrage/BarrageMessageRoundTripTest.java | 37 +++++++++++-------- 2 files changed, 40 insertions(+), 28 deletions(-) diff --git a/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java b/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java index 461cb767bfb..cc4b799f281 100644 --- a/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java +++ b/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java @@ -570,7 +570,7 @@ public boolean updateViewport(final StreamObserver listener, } public boolean updateViewport(final StreamObserver listener, final RowSet newViewport, - final boolean newReverseViewport) { + final boolean newReverseViewport) { return findAndUpdateSubscription(listener, sub -> { if (sub.pendingViewport != null) { sub.pendingViewport.close(); @@ -591,7 +591,7 @@ public boolean updateViewportAndColumns(final StreamObserver listen } public boolean updateViewportAndColumns(final StreamObserver listener, final RowSet newViewport, - final BitSet columnsToSubscribe, final boolean newReverseViewport) { + final BitSet columnsToSubscribe, final boolean newReverseViewport) { return findAndUpdateSubscription(listener, sub -> { if (sub.pendingViewport != null) { sub.pendingViewport.close(); @@ -740,7 +740,8 @@ private void enqueueUpdate(final TableUpdate upstream) { // Note: viewports are in position space, inserted and removed rows may cause the keyspace for a given viewport // to shift. Let's compute which rows are being scoped into view. If current RowSet is empty, we have nothing to // store. If prev RowSet is empty, all rows are new and are already in addsToRecord. - if ((activeViewport != null || activeReverseViewport != null) && (upstream.added().isNonempty() || upstream.removed().isNonempty()) + if ((activeViewport != null || activeReverseViewport != null) + && (upstream.added().isNonempty() || upstream.removed().isNonempty()) && rowSet.isNonempty() && rowSet.sizePrev() > 0) { final RowSetBuilderRandom scopedViewBuilder = RowSetFactory.builderRandom(); @@ -751,7 +752,8 @@ private void enqueueUpdate(final TableUpdate upstream) { continue; } - final ShiftInversionHelper inverter = new ShiftInversionHelper(upstream.shifted(), sub.reverseViewport); + final ShiftInversionHelper inverter = + new ShiftInversionHelper(upstream.shifted(), sub.reverseViewport); sub.viewport.forAllRowKeyRanges((posStart, posEnd) -> { final long localStart, localEnd; @@ -780,11 +782,13 @@ private void enqueueUpdate(final TableUpdate upstream) { currKeyEnd = inverter.mapToPrevKeyspace(rowSet.get(Math.min(localEnd, rowSet.size() - 1)), true); currKeyStart = - inverter.mapToPrevKeyspace(rowSet.get(Math.min(localStart, rowSet.size() - 1)), false); + inverter.mapToPrevKeyspace(rowSet.get(Math.min(localStart, rowSet.size() - 1)), + false); } else { // using the forward ShiftHelper, must pass `key` in ascending order currKeyStart = - inverter.mapToPrevKeyspace(rowSet.get(Math.min(localStart, rowSet.size() - 1)), false); + inverter.mapToPrevKeyspace(rowSet.get(Math.min(localStart, rowSet.size() - 1)), + false); currKeyEnd = inverter.mapToPrevKeyspace(rowSet.get(Math.min(localEnd, rowSet.size() - 1)), true); } @@ -804,11 +808,13 @@ private void enqueueUpdate(final TableUpdate upstream) { long prevEnd = lastPrevRowPosition - posStart; prevKeyStart = - prevStart >= prevRowSet.size() ? prevRowSet.lastRowKey() + 1 : prevRowSet.get(prevStart); + prevStart >= prevRowSet.size() ? prevRowSet.lastRowKey() + 1 + : prevRowSet.get(prevStart); prevKeyEnd = prevRowSet.get(Math.min(prevEnd, prevRowSet.size() - 1)); } else { prevKeyStart = - localStart >= prevRowSet.size() ? prevRowSet.lastRowKey() + 1 : prevRowSet.get(localStart); + localStart >= prevRowSet.size() ? prevRowSet.lastRowKey() + 1 + : prevRowSet.get(localStart); prevKeyEnd = prevRowSet.get(Math.min(localEnd, prevRowSet.size() - 1)); } @@ -1670,11 +1676,12 @@ private void promoteSnapshotToActive() { this.activeReverseViewport.close(); } - this.activeViewport = this.postSnapshotViewport == null || this.postSnapshotViewport.isEmpty() ? null : - this.postSnapshotViewport; + this.activeViewport = this.postSnapshotViewport == null || this.postSnapshotViewport.isEmpty() ? null + : this.postSnapshotViewport; - this.activeReverseViewport = this.postSnapshotReverseViewport == null || this.postSnapshotReverseViewport.isEmpty() ? null : - this.postSnapshotReverseViewport; + this.activeReverseViewport = + this.postSnapshotReverseViewport == null || this.postSnapshotReverseViewport.isEmpty() ? null + : this.postSnapshotReverseViewport; this.postSnapshotViewport = null; this.postSnapshotReverseViewport = null; diff --git a/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java b/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java index f0904bc84cb..63044b22cff 100644 --- a/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java +++ b/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java @@ -183,8 +183,8 @@ private class RemoteClient { } RemoteClient(final RowSet viewport, final BitSet subscribedColumns, - final BarrageMessageProducer barrageMessageProducer, - final String name, final boolean reverseViewport, final boolean deferSubscription) { + final BarrageMessageProducer barrageMessageProducer, + final String name, final boolean reverseViewport, final boolean deferSubscription) { this.viewport = viewport; this.reverseViewport = reverseViewport; this.subscribedColumns = subscribedColumns; @@ -237,8 +237,10 @@ public void validate(final String msg, QueryTable expected) { QueryTable toCheck = barrageTable; if (viewport != null) { - expected = expected.getSubTable(expected.getRowSet().subSetForPositions(viewport, reverseViewport).toTracking()); - toCheck = toCheck.getSubTable(toCheck.getRowSet().subSetForPositions(viewport, reverseViewport).toTracking()); + expected = expected + .getSubTable(expected.getRowSet().subSetForPositions(viewport, reverseViewport).toTracking()); + toCheck = toCheck + .getSubTable(toCheck.getRowSet().subSetForPositions(viewport, reverseViewport).toTracking()); } if (subscribedColumns.cardinality() != expected.getColumns().length) { final List columns = new ArrayList<>(); @@ -328,7 +330,7 @@ public void setViewportAndColumns(final RowSet newViewport, final BitSet newColu } public void setViewportAndColumns(final RowSet newViewport, final BitSet newColumns, - final boolean newReverseViewport) { + final boolean newReverseViewport) { viewport = newViewport; subscribedColumns = newColumns; barrageMessageProducer.updateViewportAndColumns(dummyObserver, viewport, subscribedColumns); @@ -388,8 +390,10 @@ public RemoteClient newClient(final RowSet viewport, final BitSet subscribedColu return newClient(viewport, subscribedColumns, false, name); } - public RemoteClient newClient(final RowSet viewport, final BitSet subscribedColumns, final boolean reverseViewport, final String name) { - clients.add(new RemoteClient(viewport, subscribedColumns, barrageMessageProducer, name, reverseViewport, false)); + public RemoteClient newClient(final RowSet viewport, final BitSet subscribedColumns, + final boolean reverseViewport, final String name) { + clients.add(new RemoteClient(viewport, subscribedColumns, barrageMessageProducer, name, reverseViewport, + false)); return clients.get(clients.size() - 1); } @@ -519,14 +523,15 @@ void createNuggetsForTableMaker(final Supplier
makeTable) { final BitSet subscribedColumns = new BitSet(); subscribedColumns.set(0, nugget.originalTable.getColumns().length); -// nugget.newClient(null, subscribedColumns, "full"); -// -// nugget.newClient(RowSetFactory.fromRange(0, size / 10), subscribedColumns, "header"); -// nugget.newClient(RowSetFactory.fromRange(size / 2, size * 3L / 4), subscribedColumns, -// "floating"); -// -// nugget.newClient(RowSetFactory.fromRange(0, size / 10), subscribedColumns, true, "footer"); -// nugget.newClient(RowSetFactory.fromRange(size / 2, size * 3L / 4), subscribedColumns, true, "reverse floating"); + // nugget.newClient(null, subscribedColumns, "full"); + // + // nugget.newClient(RowSetFactory.fromRange(0, size / 10), subscribedColumns, "header"); + // nugget.newClient(RowSetFactory.fromRange(size / 2, size * 3L / 4), subscribedColumns, + // "floating"); + // + // nugget.newClient(RowSetFactory.fromRange(0, size / 10), subscribedColumns, true, "footer"); + // nugget.newClient(RowSetFactory.fromRange(size / 2, size * 3L / 4), subscribedColumns, true, "reverse + // floating"); final RowSetBuilderSequential swissIndexBuilder = RowSetFactory.builderSequential(); final long rangeSize = Math.max(1, size / 20); @@ -534,7 +539,7 @@ void createNuggetsForTableMaker(final Supplier
makeTable) { swissIndexBuilder.appendRange(nr * rangeSize, (nr + 1) * rangeSize - 1); } -// nugget.newClient(swissIndexBuilder.build(), subscribedColumns, "swiss"); + // nugget.newClient(swissIndexBuilder.build(), subscribedColumns, "swiss"); nugget.newClient(swissIndexBuilder.build(), subscribedColumns, true, "reverse swiss"); } From c7174e6623f4a120ff3a17eef8fca27454112e0b Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 23 Feb 2022 07:18:57 -0800 Subject: [PATCH 11/20] passing all existing tests --- .../barrage/BarrageMessageProducer.java | 48 +++++-------------- .../barrage/BarrageMessageRoundTripTest.java | 39 ++++++++++----- 2 files changed, 40 insertions(+), 47 deletions(-) diff --git a/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java b/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java index cc4b799f281..0f1d2ba8a6a 100644 --- a/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java +++ b/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java @@ -18,28 +18,9 @@ import io.deephaven.configuration.Configuration; import io.deephaven.engine.liveness.LivenessArtifact; import io.deephaven.engine.liveness.LivenessReferent; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.RowSequenceFactory; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.rowset.RowSetBuilderRandom; -import io.deephaven.engine.rowset.RowSetFactory; -import io.deephaven.engine.rowset.TrackingRowSet; -import io.deephaven.engine.rowset.WritableRowSet; -import io.deephaven.engine.rowset.impl.AdaptiveRowSetBuilderRandom; -import io.deephaven.engine.table.ChunkSink; -import io.deephaven.engine.table.ChunkSource; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.ModifiedColumnSet; -import io.deephaven.engine.table.SharedContext; -import io.deephaven.engine.table.TableDefinition; -import io.deephaven.engine.table.TableUpdate; -import io.deephaven.engine.table.WritableColumnSource; -import io.deephaven.engine.table.impl.BaseTable; -import io.deephaven.engine.table.impl.InstrumentedTableUpdateListener; -import io.deephaven.engine.table.impl.MemoizedOperationKey; -import io.deephaven.engine.table.impl.NotificationStepReceiver; -import io.deephaven.engine.table.impl.QueryTable; -import io.deephaven.engine.table.impl.TableUpdateImpl; +import io.deephaven.engine.rowset.*; +import io.deephaven.engine.table.*; +import io.deephaven.engine.table.impl.*; import io.deephaven.engine.table.impl.remote.ConstructSnapshot; import io.deephaven.engine.table.impl.sources.ArrayBackedColumnSource; import io.deephaven.engine.table.impl.sources.FillUnordered; @@ -66,13 +47,7 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.BitSet; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; +import java.util.*; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.ReentrantLock; import java.util.function.BiConsumer; @@ -456,10 +431,11 @@ private class Subscription { boolean hasPendingUpdate = false; // is this subscription in our pending list? boolean pendingInitialSnapshot = true; // do we need to send the initial snapshot? RowSet pendingViewport; // if an update is pending this is our new viewport + boolean pendingReverseViewport = false; // is the pending viewport reversed (indexed from end of set) BitSet pendingColumns; // if an update is pending this is our new column subscription set RowSet snapshotViewport = null; // captured viewport during snapshot portion of propagation job BitSet snapshotColumns = null; // captured column during snapshot portion of propagation job - boolean reverseViewport = false; // treat the provided viewport as reversed (from end of set not beginning) + boolean reverseViewport = false; // treat the provided viewport as reversed (indexed from end of set) private Subscription(final StreamObserver listener, final BarrageSubscriptionOptions options, @@ -473,7 +449,7 @@ private Subscription(final StreamObserver listener, this.subscribedColumns = new BitSet(); this.pendingColumns = subscribedColumns; this.pendingViewport = initialViewport; - this.reverseViewport = reverseViewport; + this.pendingReverseViewport = this.reverseViewport = reverseViewport; } public boolean isViewport() { @@ -576,7 +552,7 @@ public boolean updateViewport(final StreamObserver listener, final sub.pendingViewport.close(); } sub.pendingViewport = newViewport.copy(); - sub.reverseViewport = newReverseViewport; + sub.pendingReverseViewport = newReverseViewport; if (sub.pendingColumns == null) { sub.pendingColumns = (BitSet) sub.subscribedColumns.clone(); } @@ -597,7 +573,7 @@ public boolean updateViewportAndColumns(final StreamObserver listen sub.pendingViewport.close(); } sub.pendingViewport = newViewport.copy(); - sub.reverseViewport = newReverseViewport; + sub.pendingReverseViewport = newReverseViewport; sub.pendingColumns = (BitSet) columnsToSubscribe.clone(); log.info().append(logPrefix).append(sub.logPrefix) .append("scheduling update immediately, for viewport and column updates.").endl(); @@ -814,8 +790,8 @@ private void enqueueUpdate(final TableUpdate upstream) { } else { prevKeyStart = localStart >= prevRowSet.size() ? prevRowSet.lastRowKey() + 1 - : prevRowSet.get(localStart); - prevKeyEnd = prevRowSet.get(Math.min(localEnd, prevRowSet.size() - 1)); + : prevRowSet.get(posStart); + prevKeyEnd = prevRowSet.get(Math.min(posEnd, prevRowSet.size() - 1)); } // Note: we already know that scoped rows must touch viewport boundaries @@ -1075,6 +1051,8 @@ private void updateSubscriptionsSnapshotAndPropagate() { needsFullSnapshot = true; } } + + subscription.reverseViewport = subscription.pendingReverseViewport; } // end updatedSubscriptions loop boolean haveViewport = false; diff --git a/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java b/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java index 63044b22cff..31ed707e234 100644 --- a/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java +++ b/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java @@ -332,6 +332,7 @@ public void setViewportAndColumns(final RowSet newViewport, final BitSet newColu public void setViewportAndColumns(final RowSet newViewport, final BitSet newColumns, final boolean newReverseViewport) { viewport = newViewport; + reverseViewport = newReverseViewport; subscribedColumns = newColumns; barrageMessageProducer.updateViewportAndColumns(dummyObserver, viewport, subscribedColumns); } @@ -499,13 +500,27 @@ void createNuggetsForTableMaker(final Supplier
makeTable) { RowSetFactory.fromRange(size / 2, size * 3 / 4), subscribedColumns, "floating"); + nuggets.add(new RemoteNugget(makeTable)); + nuggets.get(nuggets.size() - 1).newClient( + RowSetFactory.fromRange(0, size / 10), + subscribedColumns, true, "footer"); + nuggets.add(new RemoteNugget(makeTable)); + nuggets.get(nuggets.size() - 1).newClient( + RowSetFactory.fromRange(size / 2, size * 3L / 4), + subscribedColumns, true, "reverse floating"); + final RowSetBuilderSequential swissIndexBuilder = RowSetFactory.builderSequential(); final long rangeSize = Math.max(1, size / 20); for (long nr = 1; nr < 20; nr += 2) { swissIndexBuilder.appendRange(nr * rangeSize, (nr + 1) * rangeSize - 1); } + + nuggets.add(new RemoteNugget(makeTable)); + nuggets.get(nuggets.size() - 1).newClient(swissIndexBuilder.build(), subscribedColumns, "swiss"); + + final RemoteNugget nugget = new RemoteNugget(makeTable); - nugget.newClient(swissIndexBuilder.build(), subscribedColumns, "swiss viewport"); + nugget.newClient(swissIndexBuilder.build(), subscribedColumns, true, "reverse swiss"); nuggets.add(nugget); } } @@ -523,15 +538,15 @@ void createNuggetsForTableMaker(final Supplier
makeTable) { final BitSet subscribedColumns = new BitSet(); subscribedColumns.set(0, nugget.originalTable.getColumns().length); - // nugget.newClient(null, subscribedColumns, "full"); - // - // nugget.newClient(RowSetFactory.fromRange(0, size / 10), subscribedColumns, "header"); - // nugget.newClient(RowSetFactory.fromRange(size / 2, size * 3L / 4), subscribedColumns, - // "floating"); - // - // nugget.newClient(RowSetFactory.fromRange(0, size / 10), subscribedColumns, true, "footer"); - // nugget.newClient(RowSetFactory.fromRange(size / 2, size * 3L / 4), subscribedColumns, true, "reverse - // floating"); + nugget.newClient(null, subscribedColumns, "full"); + + nugget.newClient(RowSetFactory.fromRange(0, size / 10), subscribedColumns, "header"); + nugget.newClient(RowSetFactory.fromRange(size / 2, size * 3L / 4), subscribedColumns, + "floating"); + + nugget.newClient(RowSetFactory.fromRange(0, size / 10), subscribedColumns, true, "footer"); + nugget.newClient(RowSetFactory.fromRange(size / 2, size * 3L / 4), subscribedColumns, true, + "reverse floating"); final RowSetBuilderSequential swissIndexBuilder = RowSetFactory.builderSequential(); final long rangeSize = Math.max(1, size / 20); @@ -539,7 +554,7 @@ void createNuggetsForTableMaker(final Supplier
makeTable) { swissIndexBuilder.appendRange(nr * rangeSize, (nr + 1) * rangeSize - 1); } - // nugget.newClient(swissIndexBuilder.build(), subscribedColumns, "swiss"); + nugget.newClient(swissIndexBuilder.build(), subscribedColumns, "swiss"); nugget.newClient(swissIndexBuilder.build(), subscribedColumns, true, "reverse swiss"); } @@ -1013,7 +1028,7 @@ void maybeChangeSub(final int step, final int rt, final int pt) { final RemoteClient client = nugget.clients.get(nugget.clients.size() - 1); final int firstKey = random.nextInt(size); client.setViewport(RowSetFactory.fromRange(firstKey, - firstKey + random.nextInt(size - firstKey))); + firstKey + random.nextInt(size - firstKey)), client.reverseViewport); } } }.runTest(); From 11278176fbcf3cfcec6867dff3a2d94024b270cc Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 23 Feb 2022 07:36:55 -0800 Subject: [PATCH 12/20] small change --- .../deephaven/server/barrage/BarrageMessageRoundTripTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java b/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java index 31ed707e234..216c2849633 100644 --- a/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java +++ b/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java @@ -850,7 +850,8 @@ void maybeChangeSub(final int step, final int rt, final int pt) { final RemoteClient client = nugget.clients.get(nugget.clients.size() - 1); final WritableRowSet viewport = client.viewport.copy(); viewport.shiftInPlace(Math.max(size / 25, 1)); - client.setViewport(viewport); + + client.setViewport(viewport, client.reverseViewport); } } }.runTest(); From 5ae45534fd0c2df01166c392f1df389d8bb5d995 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 23 Feb 2022 12:09:05 -0800 Subject: [PATCH 13/20] new tests for , corrections to --- .../rowset/impl/WritableRowSetImplTest.java | 117 ++++++++++++++++++ .../barrage/BarrageMessageRoundTripTest.java | 8 +- 2 files changed, 122 insertions(+), 3 deletions(-) diff --git a/engine/rowset/src/test/java/io/deephaven/engine/rowset/impl/WritableRowSetImplTest.java b/engine/rowset/src/test/java/io/deephaven/engine/rowset/impl/WritableRowSetImplTest.java index d432b923464..ccfdac7504f 100644 --- a/engine/rowset/src/test/java/io/deephaven/engine/rowset/impl/WritableRowSetImplTest.java +++ b/engine/rowset/src/test/java/io/deephaven/engine/rowset/impl/WritableRowSetImplTest.java @@ -14,6 +14,7 @@ import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeyRanges; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.rowset.impl.rsp.container.MutableInteger; import io.deephaven.util.datastructures.LongRangeIterator; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableLongChunk; @@ -3250,4 +3251,120 @@ public void testRemoveSrFromRspFullBlockRegression2() { final OrderedLongSet result = rb.ixRemove(sr); assertEquals(card - sr.getCardinality(), result.ixCardinality()); } + + public void testSubSetForPositions() { + final RowSetBuilderSequential b = RowSetFactory.builderSequential(); + final long[] vs = new long[] {3, 4, 5, 8, 10, 12, 29, 31, 44, 45, 46, 59, 60, 61, 72, 65537, 65539, 65536 * 3, + 65536 * 3 + 5}; + for (long v : vs) { + b.appendKey(v); + } + final RowSet ix = b.build(); + final long sz = ix.size(); + + // test the empty ranges + try (final RowSet empty = RowSetFactory.empty()) { + assertEquals("empty range, fwd", ix.subSetForPositions(empty, false).size(), 0); + assertEquals("empty range, rev", ix.subSetForPositions(empty, true).size(), 0); + } + + final int EXTRA_RANGE = 10; + + // single range, deliberately allow some to be outside the upper boundary + for (int start = 0; start < vs.length; start++) { + for (int end = start; end <= vs.length + EXTRA_RANGE; end++) { + + int coercedEnd = Math.min(end, vs.length - 1); + int expectedSize = coercedEnd - start + 1; // rowset ranges are inclusive + + String m = "single range, s=" + start + ", end=" + end; + try (final RowSequence rs = RowSetFactory.fromRange(start, end)) { + try (final RowSet ret = ix.subSetForPositions(rs, false)) { + // verify the size is correct + assertEquals(m + ", fwd: size check", expectedSize, ret.size()); + for (int i = 0; i < ret.size(); i++) { + int idx = start + i; + assertEquals(m + ", fwd: i=" + i, vs[idx], ret.get(i)); + } + } + // now test the reversed functionality + try (final RowSet ret = ix.subSetForPositions(rs, true)) { + // verify the size is correct + assertEquals(m + ", rev: size check", expectedSize, ret.size()); + int lastPos = vs.length - 1; + for (int i = 0; i < ret.size(); i++) { + int idx = lastPos - coercedEnd + i; + assertEquals(m + ", rev: i=" + i, vs[idx], ret.get(i)); + } + } + } + } + } + + // complex ranges, deliberately allow some to be outside the upper boundary + for (int rangeSize = 1; rangeSize < vs.length + EXTRA_RANGE + 1 / 2; rangeSize++) { + String m = "complex range, rangeSize=" + rangeSize; + + final RowSetBuilderSequential rb = RowSetFactory.builderSequential(); + + int expectedSize = 0; + + // create ranges of these sizes, skipping 1 value each time + long idx = 0; + while (idx < vs.length + EXTRA_RANGE) { + long s = idx; long e = idx + rangeSize - 1; + + // compute the expected size + if (s < vs.length) { + expectedSize += Math.min(e, vs.length - 1) - s + 1; + } + + rb.appendRange(idx, e); // range is inclusive + idx += rangeSize + 1; // skip a value + } + + try (final RowSequence rs = rb.build()) { + try (final RowSet ret = ix.subSetForPositions(rs, false)) { + // verify the size is correct + assertEquals(m + ", fwd: size check", expectedSize, ret.size()); + + // keep track of the index of the working set + final MutableInteger retIndex = new MutableInteger(0); + + rs.forEachRowKeyRange((final long start, final long end) -> { + for (long i = start; i <= end; i++) { + int index = (int)i; + if (index >= 0 && index < vs.length) { + assertEquals(m + ", fwd: i=" + index, vs[index], ret.get(retIndex.value)); + retIndex.value++; + } + } + return true; + }); + } + + // now test the reversed functionality + try (final RowSet ret = ix.subSetForPositions(rs, true)) { + // verify the size is correct + assertEquals(m + ", rev: size check", expectedSize, ret.size()); + + // keep track of the index of the working set + final MutableInteger retIndex = new MutableInteger(0); + final int lastPos = vs.length - 1; + + rs.forEachRowKeyRange((final long start, final long end) -> { + // translate into reversed positions + for (long i = start; i <= end; i++) { + int index = lastPos - (int)i; + if (index >= 0 && index < vs.length) { + assertEquals(m + ", fwd: i=" + index, vs[index], ret.get(ret.size() - retIndex.value - 1)); + retIndex.value++; + } + } + return true; + }); + } + } + } + } } diff --git a/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java b/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java index 216c2849633..8fede7867fa 100644 --- a/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java +++ b/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java @@ -851,6 +851,7 @@ void maybeChangeSub(final int step, final int rt, final int pt) { final WritableRowSet viewport = client.viewport.copy(); viewport.shiftInPlace(Math.max(size / 25, 1)); + // maintain viewport direction in this test client.setViewport(viewport, client.reverseViewport); } } @@ -977,8 +978,7 @@ public void createNuggets() { columns.set(0, 4); nugget.clients.add( new RemoteClient( - RowSetFactory.fromRange(size / 5, - 3 * size / 5), + RowSetFactory.fromRange(size / 5, 3 * size / 5), columns, nugget.barrageMessageProducer, "sub-changer")); } } @@ -992,7 +992,9 @@ void maybeChangeSub(final int step, final int rt, final int pt) { final RemoteClient client = nugget.clients.get(nugget.clients.size() - 1); final WritableRowSet viewport = client.viewport.copy(); viewport.shiftInPlace(size / 5); - client.setViewport(viewport); + + // maintain viewport direction in this test + client.setViewport(viewport, client.reverseViewport); } } }.runTest(); From f06eaf484be91a6acaa1bb76808c855b16c323c6 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 23 Feb 2022 14:26:09 -0800 Subject: [PATCH 14/20] forward & reverse viewport transition tests added --- .../client/impl/BarrageSnapshot.java | 3 +- .../client/impl/BarrageSubscription.java | 3 +- .../barrage/BarrageMessageProducer.java | 13 +++-- .../server/uri/BarrageTableResolver.java | 12 ++-- .../barrage/BarrageMessageRoundTripTest.java | 57 +++++++++++++++++-- 5 files changed, 67 insertions(+), 21 deletions(-) diff --git a/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSnapshot.java b/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSnapshot.java index 1e62f67444a..a20ca1bb6eb 100644 --- a/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSnapshot.java +++ b/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSnapshot.java @@ -63,8 +63,7 @@ BarrageSnapshot snapshot(TableSpec tableSpec, BarrageSnapshotOptions options) * * @param viewport the position-space viewport to use for the snapshot * @param columns the columns to include in the snapshot - * @param reverseViewport Whether to treat {@code posRowSet} as offsets from to {@link #size()} rather than - * {@code 0} + * @param reverseViewport Whether to treat {@code posRowSet} as offsets from {@link #size()} rather than {@code 0} * * @return the {@code BarrageTable} */ diff --git a/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSubscription.java b/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSubscription.java index 9f78b3a3096..a3569db77c6 100644 --- a/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSubscription.java +++ b/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSubscription.java @@ -65,8 +65,7 @@ BarrageSubscription subscribe(TableSpec tableSpec, BarrageSubscriptionOptions op * * @param viewport the position-space viewport to use for the subscription * @param columns the columns to include in the subscription - * @param reverseViewport Whether to treat {@code posRowSet} as offsets from to {@link #size()} rather than - * {@code 0} * + * @param reverseViewport Whether to treat {@code posRowSet} as offsets from {@link #size()} rather than {@code 0} * * @return the {@code BarrageTable} */ diff --git a/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java b/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java index 0f1d2ba8a6a..6ba5a7657ed 100644 --- a/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java +++ b/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java @@ -425,17 +425,19 @@ private class Subscription { RowSet viewport; // active viewport BitSet subscribedColumns; // active subscription columns + boolean reverseViewport = false; // is the active viewport reversed (indexed from end of table) boolean isActive = false; // is this subscription in our active list? boolean pendingDelete = false; // is this subscription deleted as far as the client is concerned? boolean hasPendingUpdate = false; // is this subscription in our pending list? boolean pendingInitialSnapshot = true; // do we need to send the initial snapshot? RowSet pendingViewport; // if an update is pending this is our new viewport - boolean pendingReverseViewport = false; // is the pending viewport reversed (indexed from end of set) + boolean pendingReverseViewport; // is the pending viewport reversed (indexed from end of table) BitSet pendingColumns; // if an update is pending this is our new column subscription set + RowSet snapshotViewport = null; // captured viewport during snapshot portion of propagation job BitSet snapshotColumns = null; // captured column during snapshot portion of propagation job - boolean reverseViewport = false; // treat the provided viewport as reversed (indexed from end of set) + boolean snapshotReverseViewport = false; // captured setting during snapshot portion of propagation job private Subscription(final StreamObserver listener, final BarrageSubscriptionOptions options, @@ -1035,7 +1037,7 @@ private void updateSubscriptionsSnapshotAndPropagate() { subscription.pendingViewport = null; if (!needsFullSnapshot) { // track forward and reverse viewport rows separately - if (subscription.reverseViewport) { + if (subscription.pendingReverseViewport) { reverseSnapshotRows.addRowSet(subscription.snapshotViewport); } else { snapshotRows.addRowSet(subscription.snapshotViewport); @@ -1052,7 +1054,7 @@ private void updateSubscriptionsSnapshotAndPropagate() { } } - subscription.reverseViewport = subscription.pendingReverseViewport; + subscription.snapshotReverseViewport = subscription.pendingReverseViewport; } // end updatedSubscriptions loop boolean haveViewport = false; @@ -1077,7 +1079,7 @@ private void updateSubscriptionsSnapshotAndPropagate() { if (sub.isViewport()) { haveViewport = true; // handle forward and reverse snapshots separately - if (sub.reverseViewport) { + if (sub.snapshotReverseViewport) { postSnapshotReverseViewportBuilder .addRowSet(sub.snapshotViewport != null ? sub.snapshotViewport : sub.viewport); } else { @@ -1634,6 +1636,7 @@ private void flipSnapshotStateForSubscriptions(final List subscrip if (subscription.snapshotViewport != null) { final RowSet tmp = subscription.viewport; subscription.viewport = subscription.snapshotViewport; + subscription.reverseViewport = subscription.snapshotReverseViewport; subscription.snapshotViewport = tmp; } if (subscription.snapshotColumns != null) { diff --git a/server/src/main/java/io/deephaven/server/uri/BarrageTableResolver.java b/server/src/main/java/io/deephaven/server/uri/BarrageTableResolver.java index fe891892ad8..857e57cd909 100644 --- a/server/src/main/java/io/deephaven/server/uri/BarrageTableResolver.java +++ b/server/src/main/java/io/deephaven/server/uri/BarrageTableResolver.java @@ -153,8 +153,7 @@ public Table subscribe(String targetUri, TableSpec table, RowSet viewport, BitSe * @param table the table spec * @param viewport the position-space viewport to use for the subscription * @param columns the columns to include in the subscription - * @param reverseViewport Whether to treat {@code posRowSet} as offsets from to {@link #size()} rather than - * {@code 0} + * @param reverseViewport Whether to treat {@code posRowSet} as offsets from {@link #size()} rather than {@code 0} * @return the subscribed table */ public Table subscribe(String targetUri, TableSpec table, RowSet viewport, BitSet columns, boolean reverseViewport) @@ -171,8 +170,7 @@ public Table subscribe(String targetUri, TableSpec table, RowSet viewport, BitSe * @param options the options * @param viewport the position-space viewport to use for the subscription * @param columns the columns to include in the subscription - * @param reverseViewport Whether to treat {@code posRowSet} as offsets from to {@link #size()} rather than - * {@code 0} * + * @param reverseViewport Whether to treat {@code posRowSet} as offsets from {@link #size()} rather than {@code 0} * @return the subscribed table */ public Table subscribe(DeephavenTarget target, TableSpec table, BarrageSubscriptionOptions options, RowSet viewport, @@ -242,8 +240,7 @@ public Table snapshot(String targetUri, TableSpec table, RowSet viewport, BitSet * @param table the table spec * @param viewport the position-space viewport to use for the snapshot * @param columns the columns to include in the snapshot - * @param reverseViewport Whether to treat {@code posRowSet} as offsets from to {@link #size()} rather than - * {@code 0} + * @param reverseViewport Whether to treat {@code posRowSet} as offsets from {@link #size()} rather than {@code 0} * @return the table to snapshot */ public Table snapshot(String targetUri, TableSpec table, RowSet viewport, BitSet columns, boolean reverseViewport) @@ -260,8 +257,7 @@ public Table snapshot(String targetUri, TableSpec table, RowSet viewport, BitSet * @param options the options * @param viewport the position-space viewport to use for the snapshot * @param columns the columns to include in the snapshot - * @param reverseViewport Whether to treat {@code posRowSet} as offsets from to {@link #size()} rather than - * {@code 0} + * @param reverseViewport Whether to treat {@code posRowSet} as offsets from {@link #size()} rather than {@code 0} * @return the table to snapshot */ public Table snapshot(DeephavenTarget target, TableSpec table, BarrageSnapshotOptions options, RowSet viewport, diff --git a/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java b/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java index 8fede7867fa..fdb58246edb 100644 --- a/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java +++ b/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java @@ -514,13 +514,14 @@ void createNuggetsForTableMaker(final Supplier
makeTable) { for (long nr = 1; nr < 20; nr += 2) { swissIndexBuilder.appendRange(nr * rangeSize, (nr + 1) * rangeSize - 1); } + final RowSet rs = swissIndexBuilder.build(); nuggets.add(new RemoteNugget(makeTable)); - nuggets.get(nuggets.size() - 1).newClient(swissIndexBuilder.build(), subscribedColumns, "swiss"); + nuggets.get(nuggets.size() - 1).newClient(rs, subscribedColumns, "swiss"); final RemoteNugget nugget = new RemoteNugget(makeTable); - nugget.newClient(swissIndexBuilder.build(), subscribedColumns, true, "reverse swiss"); + nugget.newClient(rs.copy(), subscribedColumns, true, "reverse swiss"); nuggets.add(nugget); } } @@ -554,9 +555,10 @@ void createNuggetsForTableMaker(final Supplier
makeTable) { swissIndexBuilder.appendRange(nr * rangeSize, (nr + 1) * rangeSize - 1); } - nugget.newClient(swissIndexBuilder.build(), subscribedColumns, "swiss"); + final RowSet rs = swissIndexBuilder.build(); + nugget.newClient(rs, subscribedColumns, "swiss"); - nugget.newClient(swissIndexBuilder.build(), subscribedColumns, true, "reverse swiss"); + nugget.newClient(rs.copy(), subscribedColumns, true, "reverse swiss"); } } @@ -863,6 +865,53 @@ void maybeChangeSub(final int step, final int rt, final int pt) { } } + public void testViewportDirectionChange() { + for (final int size : new int[] {10, 100}) { + for (final int numProducerCoalesce : new int[] {1, 4}) { + for (final int numConsumerCoalesce : new int[] {1, 4}) { + for (int subProducerCoalesce = + 0; subProducerCoalesce < numProducerCoalesce; ++subProducerCoalesce) { + for (int subConsumerCoalesce = + 0; subConsumerCoalesce < numConsumerCoalesce; ++subConsumerCoalesce) { + final int finalSubProducerCoalesce = 0; + final int finalSubConsumerCoalesce = 1; + new SubscriptionChangingHelper(numProducerCoalesce, numConsumerCoalesce, size, 0, + new MutableInt(25)) { + @Override + void createNuggetsForTableMaker(final Supplier
makeTable) { + final RemoteNugget nugget = new RemoteNugget(makeTable); + nuggets.add(nugget); + + final BitSet columns = new BitSet(); + columns.set(0, 4); + nugget.clients.add( + new RemoteClient(RowSetFactory.fromRange(0, size / 5), + columns, nugget.barrageMessageProducer, "sub-changer")); + } + + void maybeChangeSub(final int step, final int rt, final int pt) { + if (step % 2 != 0 || rt != finalSubConsumerCoalesce + || pt != finalSubProducerCoalesce) { + return; + } + + for (final RemoteNugget nugget : nuggets) { + final RemoteClient client = nugget.clients.get(nugget.clients.size() - 1); + final WritableRowSet viewport = client.viewport.copy(); + viewport.shiftInPlace(Math.max(size / 25, 1)); + + // alternate viewport direction with every call to this function + client.setViewport(viewport, !client.reverseViewport); + } + } + }.runTest(); + } + } + } + } + } + } + public void testOverlappedColumnSubsChange() { for (final int size : new int[] {10, 100, 1000}) { for (final int numProducerCoalesce : new int[] {1, 4}) { From 006d6f94260f1eb790c11b21bd64745c2b21b4fb Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 23 Feb 2022 14:39:28 -0800 Subject: [PATCH 15/20] addressing PR comments --- .../barrage/BarrageMessageProducer.java | 23 ++++++++----------- 1 file changed, 10 insertions(+), 13 deletions(-) diff --git a/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java b/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java index 6ba5a7657ed..c098f5606cb 100644 --- a/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java +++ b/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java @@ -776,25 +776,22 @@ private void enqueueUpdate(final TableUpdate upstream) { return; } - final long prevKeyStart, prevKeyEnd; - + final long prevStart; + final long prevEnd; if (sub.reverseViewport) { // compute positions to be relative to the final position of prevRowSet final long lastPrevRowPosition = prevRowSet.size() - 1; - long prevStart = Math.max(lastPrevRowPosition - posEnd, 0); - long prevEnd = lastPrevRowPosition - posStart; - - prevKeyStart = - prevStart >= prevRowSet.size() ? prevRowSet.lastRowKey() + 1 - : prevRowSet.get(prevStart); - prevKeyEnd = prevRowSet.get(Math.min(prevEnd, prevRowSet.size() - 1)); + prevStart = Math.max(lastPrevRowPosition - posEnd, 0); + prevEnd = Math.max(lastPrevRowPosition - posStart, 0); } else { - prevKeyStart = - localStart >= prevRowSet.size() ? prevRowSet.lastRowKey() + 1 - : prevRowSet.get(posStart); - prevKeyEnd = prevRowSet.get(Math.min(posEnd, prevRowSet.size() - 1)); + prevStart = localStart; + prevEnd = localEnd; } + final long prevKeyStart = + prevStart >= prevRowSet.size() ? prevRowSet.lastRowKey() + 1 : prevRowSet.get(prevStart); + final long prevKeyEnd = prevRowSet.get(Math.min(prevEnd, prevRowSet.size() - 1)); + // Note: we already know that scoped rows must touch viewport boundaries if (currKeyStart < prevKeyStart) { From cd80cb052578539e893c49a97d3397eaeee343c4 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 23 Feb 2022 16:33:11 -0800 Subject: [PATCH 16/20] all tests pass --- .../rowset/impl/WritableRowSetImplTest.java | 10 ++++++---- .../table/impl/util/ShiftInversionHelper.java | 4 ++-- .../client/examples/SubscribeExampleBase.java | 18 +++++++++++++++++- .../server/barrage/BarrageMessageProducer.java | 2 +- 4 files changed, 26 insertions(+), 8 deletions(-) diff --git a/engine/rowset/src/test/java/io/deephaven/engine/rowset/impl/WritableRowSetImplTest.java b/engine/rowset/src/test/java/io/deephaven/engine/rowset/impl/WritableRowSetImplTest.java index ccfdac7504f..48ea01d67b0 100644 --- a/engine/rowset/src/test/java/io/deephaven/engine/rowset/impl/WritableRowSetImplTest.java +++ b/engine/rowset/src/test/java/io/deephaven/engine/rowset/impl/WritableRowSetImplTest.java @@ -3312,7 +3312,8 @@ public void testSubSetForPositions() { // create ranges of these sizes, skipping 1 value each time long idx = 0; while (idx < vs.length + EXTRA_RANGE) { - long s = idx; long e = idx + rangeSize - 1; + long s = idx; + long e = idx + rangeSize - 1; // compute the expected size if (s < vs.length) { @@ -3333,7 +3334,7 @@ public void testSubSetForPositions() { rs.forEachRowKeyRange((final long start, final long end) -> { for (long i = start; i <= end; i++) { - int index = (int)i; + int index = (int) i; if (index >= 0 && index < vs.length) { assertEquals(m + ", fwd: i=" + index, vs[index], ret.get(retIndex.value)); retIndex.value++; @@ -3355,9 +3356,10 @@ public void testSubSetForPositions() { rs.forEachRowKeyRange((final long start, final long end) -> { // translate into reversed positions for (long i = start; i <= end; i++) { - int index = lastPos - (int)i; + int index = lastPos - (int) i; if (index >= 0 && index < vs.length) { - assertEquals(m + ", fwd: i=" + index, vs[index], ret.get(ret.size() - retIndex.value - 1)); + assertEquals(m + ", fwd: i=" + index, vs[index], + ret.get(ret.size() - retIndex.value - 1)); retIndex.value++; } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/ShiftInversionHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/ShiftInversionHelper.java index fef5a049e50..e44fa995e69 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/ShiftInversionHelper.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/ShiftInversionHelper.java @@ -10,8 +10,8 @@ */ public class ShiftInversionHelper { - final private RowSetShiftData shifted; - final private boolean reverseOrder; + private final RowSetShiftData shifted; + private final boolean reverseOrder; private int destShiftIdx; diff --git a/java-client/barrage-examples/src/main/java/io/deephaven/client/examples/SubscribeExampleBase.java b/java-client/barrage-examples/src/main/java/io/deephaven/client/examples/SubscribeExampleBase.java index 3dd66cb5188..663e310143a 100644 --- a/java-client/barrage-examples/src/main/java/io/deephaven/client/examples/SubscribeExampleBase.java +++ b/java-client/barrage-examples/src/main/java/io/deephaven/client/examples/SubscribeExampleBase.java @@ -26,6 +26,12 @@ static class Mode { @CommandLine.Option(names = {"-s", "--serial"}, required = true, description = "Serial mode") boolean serial; + + @CommandLine.Option(names = {"--tail"}, required = false, description = "Tail viewport size") + int tailSize = 0; + + @CommandLine.Option(names = {"--head"}, required = false, description = "Header viewport size") + int headerSize = 0; } @CommandLine.ArgGroup(exclusive = true) @@ -44,7 +50,17 @@ protected void execute(final BarrageSession client) throws Exception { try (final TableHandle handle = manager.executeLogic(logic()); final BarrageSubscription subscription = client.subscribe(handle, options)) { - final BarrageTable table = subscription.partialTable(RowSetFactory.flat(10), null, true); + final BarrageTable table; + if (mode != null && mode.headerSize > 0) { + // create a table subscription with forward viewport of the specified size + table = subscription.partialTable(RowSetFactory.flat(mode.headerSize), null, true); + } else if (mode != null && mode.tailSize > 0) { + // create a table subscription with reverse viewport of the specified size + table = subscription.partialTable(RowSetFactory.flat(mode.tailSize), null, true); + } else { + // create a table subscription of the entire table + table = subscription.entireTable(); + } final CountDownLatch countDownLatch = new CountDownLatch(1); diff --git a/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java b/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java index c098f5606cb..0a11972cf16 100644 --- a/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java +++ b/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java @@ -783,7 +783,7 @@ private void enqueueUpdate(final TableUpdate upstream) { final long lastPrevRowPosition = prevRowSet.size() - 1; prevStart = Math.max(lastPrevRowPosition - posEnd, 0); - prevEnd = Math.max(lastPrevRowPosition - posStart, 0); + prevEnd = lastPrevRowPosition - posStart; } else { prevStart = localStart; prevEnd = localEnd; From 4728f4872dedfd232a4c4ea60d641e7517a1f1b7 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Thu, 24 Feb 2022 06:58:36 -0800 Subject: [PATCH 17/20] more informative comments on scoping --- .../server/barrage/BarrageMessageProducer.java | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java b/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java index 0a11972cf16..426bc9f8616 100644 --- a/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java +++ b/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java @@ -779,19 +779,25 @@ private void enqueueUpdate(final TableUpdate upstream) { final long prevStart; final long prevEnd; if (sub.reverseViewport) { - // compute positions to be relative to the final position of prevRowSet final long lastPrevRowPosition = prevRowSet.size() - 1; prevStart = Math.max(lastPrevRowPosition - posEnd, 0); - prevEnd = lastPrevRowPosition - posStart; + prevEnd = lastPrevRowPosition - posStart; // this can be left of the prev rowset (i.e. <0) } else { prevStart = localStart; - prevEnd = localEnd; + prevEnd = localEnd; // this can be right of the prev rowset (i.e. >= size()) } + + // get the key that represents the start of the viewport in the prev rowset key space or + // prevRowSet.lastRowKey() + 1 if the start is past the end of prev rowset final long prevKeyStart = - prevStart >= prevRowSet.size() ? prevRowSet.lastRowKey() + 1 : prevRowSet.get(prevStart); - final long prevKeyEnd = prevRowSet.get(Math.min(prevEnd, prevRowSet.size() - 1)); + prevStart >= prevRowSet.size() ? prevRowSet.lastRowKey() + 1 + : prevRowSet.get(prevStart); + // get the key that represents the end of the viewport in the prev rowset key space or + // -1 if the end is before the beginning of prev rowset + final long prevKeyEnd = + prevEnd < 0 ? -1 : prevRowSet.get(Math.min(prevEnd, prevRowSet.size() - 1)); // Note: we already know that scoped rows must touch viewport boundaries if (currKeyStart < prevKeyStart) { From a8c3fd3247cf3067de067184e3033dee66645135 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Thu, 24 Feb 2022 07:04:21 -0800 Subject: [PATCH 18/20] spotless cleanup --- .../table/impl/remote/ConstructSnapshot.java | 31 ++++++++++--------- .../table/impl/util/ShiftInversionHelper.java | 3 +- 2 files changed, 18 insertions(+), 16 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/remote/ConstructSnapshot.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/remote/ConstructSnapshot.java index c78d46f6c97..9111f450f75 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/remote/ConstructSnapshot.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/remote/ConstructSnapshot.java @@ -572,10 +572,10 @@ public static BarrageMessage constructBackplaneSnapshotInPositionSpace(final Obj } else { final RowSet rowSetToUse = usePrev ? table.getRowSet().copyPrev() : table.getRowSet(); try (final SafeCloseable ignored = usePrev ? rowSetToUse : null) { - final WritableRowSet forwardKeys = positionsToSnapshot == null ? null : - rowSetToUse.subSetForPositions(positionsToSnapshot); - final RowSet reverseKeys = reversePositionsToSnapshot == null ? null : - rowSetToUse.subSetForReversePositions(reversePositionsToSnapshot); + final WritableRowSet forwardKeys = + positionsToSnapshot == null ? null : rowSetToUse.subSetForPositions(positionsToSnapshot); + final RowSet reverseKeys = reversePositionsToSnapshot == null ? null + : rowSetToUse.subSetForReversePositions(reversePositionsToSnapshot); if (forwardKeys != null) { if (reverseKeys != null) { forwardKeys.insert(reverseKeys); @@ -588,7 +588,8 @@ public static BarrageMessage constructBackplaneSnapshotInPositionSpace(final Obj } } try (final RowSet ignored = keysToSnapshot) { - return serializeAllTable(usePrev, snapshot, table, logIdentityObject, columnsToSerialize, keysToSnapshot); + return serializeAllTable(usePrev, snapshot, table, logIdentityObject, columnsToSerialize, + keysToSnapshot); } }; @@ -1281,19 +1282,19 @@ public static boolean serializeAllTable(boolean usePrev, * {@link #constructBackplaneSnapshot} for simple use cases or {@link #callDataSnapshotFunction} for more advanced * uses. * - * @param usePrev Use previous values? - * @param snapshot The snapshot to populate - * @param logIdentityObject an object for use with log() messages + * @param usePrev Use previous values? + * @param snapshot The snapshot to populate + * @param logIdentityObject an object for use with log() messages * @param columnsToSerialize A {@link BitSet} of columns to include, null for all - * @param keysToSnapshot A RowSet of keys within the table to include, null for all + * @param keysToSnapshot A RowSet of keys within the table to include, null for all * @return true if the snapshot was computed with an unchanged clock, false otherwise. */ public static boolean serializeAllTable(final boolean usePrev, - final BarrageMessage snapshot, - final BaseTable table, - final Object logIdentityObject, - final BitSet columnsToSerialize, - final RowSet keysToSnapshot) { + final BarrageMessage snapshot, + final BaseTable table, + final Object logIdentityObject, + final BitSet columnsToSerialize, + final RowSet keysToSnapshot) { snapshot.rowsAdded = (usePrev ? table.getRowSet().copyPrev() : table.getRowSet()).copy(); snapshot.rowsRemoved = RowSetFactory.empty(); @@ -1314,7 +1315,7 @@ public static boolean serializeAllTable(final boolean usePrev, final String[] columnSources = sourceMap.keySet().toArray(CollectionUtil.ZERO_LENGTH_STRING_ARRAY); try (final SharedContext sharedContext = - (columnSources.length > 1) ? SharedContext.makeSharedContext() : null) { + (columnSources.length > 1) ? SharedContext.makeSharedContext() : null) { for (int ii = 0; ii < columnSources.length; ++ii) { if (concurrentAttemptInconsistent()) { final LogEntry logEntry = log.info().append(System.identityHashCode(logIdentityObject)) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/ShiftInversionHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/ShiftInversionHelper.java index e44fa995e69..a3b2aab7d8e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/ShiftInversionHelper.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/ShiftInversionHelper.java @@ -32,7 +32,8 @@ private void advanceDestShiftIdx(final long destKey) { reverseOrder ? 0 : destShiftIdx, reverseOrder ? destShiftIdx : shifted.size(), innerShiftIdx -> { - long destEnd = shifted.getEndRange((int) innerShiftIdx) + shifted.getShiftDelta((int) innerShiftIdx); + long destEnd = + shifted.getEndRange((int) innerShiftIdx) + shifted.getShiftDelta((int) innerShiftIdx); // due to destKey's expected range, we know this subtraction will not overflow return destEnd - destKey; }); From cef5ef03dfd3a09c6fb4296420b4b6aa3f75c83e Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Thu, 24 Feb 2022 07:13:59 -0800 Subject: [PATCH 19/20] formatting changes --- .../io/deephaven/client/examples/SnapshotExampleBase.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/java-client/barrage-examples/src/main/java/io/deephaven/client/examples/SnapshotExampleBase.java b/java-client/barrage-examples/src/main/java/io/deephaven/client/examples/SnapshotExampleBase.java index cd6e900fccf..6487b3e0d16 100644 --- a/java-client/barrage-examples/src/main/java/io/deephaven/client/examples/SnapshotExampleBase.java +++ b/java-client/barrage-examples/src/main/java/io/deephaven/client/examples/SnapshotExampleBase.java @@ -108,9 +108,9 @@ protected void execute(final BarrageSession client) throws Exception { // example #6 - reverse viewport, all columns try (final TableHandle handle = manager.executeLogic(logic()); - final RowSet viewport = RowSetFactory.flat(5); // range inclusive + final RowSet viewport = RowSetFactory.flat(5); // range inclusive - final BarrageSnapshot snapshot = client.snapshot(handle, options)) { + final BarrageSnapshot snapshot = client.snapshot(handle, options)) { // expect this to block until all reading complete final BarrageTable table = snapshot.partialTable(viewport, null, true); @@ -121,9 +121,9 @@ protected void execute(final BarrageSession client) throws Exception { // example #7 - reverse viewport, some columns try (final TableHandle handle = manager.executeLogic(logic()); - final RowSet viewport = RowSetFactory.flat(5); // range inclusive + final RowSet viewport = RowSetFactory.flat(5); // range inclusive - final BarrageSnapshot snapshot = client.snapshot(handle, options)) { + final BarrageSnapshot snapshot = client.snapshot(handle, options)) { final BitSet columns = new BitSet(); columns.set(0, 2); // range not inclusive (sets bits 0-1) From 516f53ee239a7e7c22fcae69d01c6b78e9acbc2a Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Mon, 28 Feb 2022 11:06:39 -0800 Subject: [PATCH 20/20] addressed PR comments --- .../client/examples/SubscribeExampleBase.java | 20 +++++++++---------- .../server/uri/BarrageTableResolver.java | 8 ++++---- 2 files changed, 14 insertions(+), 14 deletions(-) diff --git a/java-client/barrage-examples/src/main/java/io/deephaven/client/examples/SubscribeExampleBase.java b/java-client/barrage-examples/src/main/java/io/deephaven/client/examples/SubscribeExampleBase.java index 663e310143a..ca5d442c08e 100644 --- a/java-client/barrage-examples/src/main/java/io/deephaven/client/examples/SubscribeExampleBase.java +++ b/java-client/barrage-examples/src/main/java/io/deephaven/client/examples/SubscribeExampleBase.java @@ -20,18 +20,18 @@ abstract class SubscribeExampleBase extends BarrageClientExampleBase { + @CommandLine.Option(names = {"--tail"}, required = false, description = "Tail viewport size") + int tailSize = 0; + + @CommandLine.Option(names = {"--head"}, required = false, description = "Header viewport size") + int headerSize = 0; + static class Mode { @CommandLine.Option(names = {"-b", "--batch"}, required = true, description = "Batch mode") boolean batch; @CommandLine.Option(names = {"-s", "--serial"}, required = true, description = "Serial mode") boolean serial; - - @CommandLine.Option(names = {"--tail"}, required = false, description = "Tail viewport size") - int tailSize = 0; - - @CommandLine.Option(names = {"--head"}, required = false, description = "Header viewport size") - int headerSize = 0; } @CommandLine.ArgGroup(exclusive = true) @@ -51,12 +51,12 @@ protected void execute(final BarrageSession client) throws Exception { final BarrageSubscription subscription = client.subscribe(handle, options)) { final BarrageTable table; - if (mode != null && mode.headerSize > 0) { + if (headerSize > 0) { // create a table subscription with forward viewport of the specified size - table = subscription.partialTable(RowSetFactory.flat(mode.headerSize), null, true); - } else if (mode != null && mode.tailSize > 0) { + table = subscription.partialTable(RowSetFactory.flat(headerSize), null, true); + } else if (tailSize > 0) { // create a table subscription with reverse viewport of the specified size - table = subscription.partialTable(RowSetFactory.flat(mode.tailSize), null, true); + table = subscription.partialTable(RowSetFactory.flat(tailSize), null, true); } else { // create a table subscription of the entire table table = subscription.entireTable(); diff --git a/server/src/main/java/io/deephaven/server/uri/BarrageTableResolver.java b/server/src/main/java/io/deephaven/server/uri/BarrageTableResolver.java index 857e57cd909..8b59ab94dd2 100644 --- a/server/src/main/java/io/deephaven/server/uri/BarrageTableResolver.java +++ b/server/src/main/java/io/deephaven/server/uri/BarrageTableResolver.java @@ -153,7 +153,7 @@ public Table subscribe(String targetUri, TableSpec table, RowSet viewport, BitSe * @param table the table spec * @param viewport the position-space viewport to use for the subscription * @param columns the columns to include in the subscription - * @param reverseViewport Whether to treat {@code posRowSet} as offsets from {@link #size()} rather than {@code 0} + * @param reverseViewport Whether to treat {@code viewport} as offsets from {@link #size()} rather than {@code 0} * @return the subscribed table */ public Table subscribe(String targetUri, TableSpec table, RowSet viewport, BitSet columns, boolean reverseViewport) @@ -170,7 +170,7 @@ public Table subscribe(String targetUri, TableSpec table, RowSet viewport, BitSe * @param options the options * @param viewport the position-space viewport to use for the subscription * @param columns the columns to include in the subscription - * @param reverseViewport Whether to treat {@code posRowSet} as offsets from {@link #size()} rather than {@code 0} + * @param reverseViewport Whether to treat {@code viewport} as offsets from {@link #size()} rather than {@code 0} * @return the subscribed table */ public Table subscribe(DeephavenTarget target, TableSpec table, BarrageSubscriptionOptions options, RowSet viewport, @@ -240,7 +240,7 @@ public Table snapshot(String targetUri, TableSpec table, RowSet viewport, BitSet * @param table the table spec * @param viewport the position-space viewport to use for the snapshot * @param columns the columns to include in the snapshot - * @param reverseViewport Whether to treat {@code posRowSet} as offsets from {@link #size()} rather than {@code 0} + * @param reverseViewport Whether to treat {@code viewport} as offsets from {@link #size()} rather than {@code 0} * @return the table to snapshot */ public Table snapshot(String targetUri, TableSpec table, RowSet viewport, BitSet columns, boolean reverseViewport) @@ -257,7 +257,7 @@ public Table snapshot(String targetUri, TableSpec table, RowSet viewport, BitSet * @param options the options * @param viewport the position-space viewport to use for the snapshot * @param columns the columns to include in the snapshot - * @param reverseViewport Whether to treat {@code posRowSet} as offsets from {@link #size()} rather than {@code 0} + * @param reverseViewport Whether to treat {@code viewport} as offsets from {@link #size()} rather than {@code 0} * @return the table to snapshot */ public Table snapshot(DeephavenTarget target, TableSpec table, BarrageSnapshotOptions options, RowSet viewport,