Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Port of DH-13185: fix FunctionalColumn #4235

Merged
merged 2 commits into from
Jul 27, 2023
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -176,7 +176,7 @@ public void fillPrevChunk(@NotNull FillContext fillContext,
@NotNull final WritableChunk<? super Values> destination,
@NotNull final RowSequence rowSequence) {
final FunctionalColumnFillContext ctx = (FunctionalColumnFillContext) fillContext;
ctx.chunkFiller.fillByIndices(this, rowSequence, destination);
ctx.chunkFiller.fillPrevByIndices(this, rowSequence, destination);
}
}, false);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -172,7 +172,7 @@ public void fillPrevChunk(@NotNull FillContext fillContext,
@NotNull final WritableChunk<? super Values> destination,
@NotNull final RowSequence rowSequence) {
final FunctionalColumnFillContext ctx = (FunctionalColumnFillContext) fillContext;
ctx.chunkFiller.fillByIndices(this, rowSequence, destination);
ctx.chunkFiller.fillPrevByIndices(this, rowSequence, destination);
}
}, false);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -5,13 +5,31 @@

import io.deephaven.base.SleepUtil;
import io.deephaven.engine.context.ExecutionContext;
import io.deephaven.engine.rowset.RowSetFactory;
import io.deephaven.engine.table.impl.QueryTable;
import io.deephaven.engine.table.impl.select.FunctionalColumn;
import io.deephaven.engine.testutil.ControlledUpdateGraph;
import io.deephaven.engine.testutil.testcase.RefreshingTableTestCase;
import io.deephaven.engine.updategraph.LogicalClock;
import io.deephaven.engine.updategraph.UpdateGraph;
import io.deephaven.engine.util.TableTools;
import io.deephaven.util.SafeCloseable;
import io.deephaven.util.thread.NamingThreadFactory;
import org.apache.commons.lang3.mutable.MutableLong;

import java.util.BitSet;
import java.util.List;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;

import static io.deephaven.engine.testutil.TstUtils.addToTable;
import static io.deephaven.engine.testutil.TstUtils.assertTableEquals;
import static io.deephaven.engine.testutil.TstUtils.i;
import static io.deephaven.engine.testutil.TstUtils.testRefreshingTable;
import static io.deephaven.engine.util.TableTools.intCol;
import static io.deephaven.engine.util.TableTools.stringCol;

public class TestConstructSnapshot extends RefreshingTableTestCase {

public void testClockChange() throws InterruptedException {
Expand Down Expand Up @@ -64,4 +82,74 @@ public UpdateGraph getUpdateGraph() {
t2.join();
assertEquals(1, changed.longValue());
}

public void testUsePrevSnapshot() throws ExecutionException, InterruptedException {
final ExecutorService executor = Executors.newSingleThreadExecutor(
new NamingThreadFactory(TestConstructSnapshot.class, "TestConstructSnapshot Executor"));

final QueryTable table = testRefreshingTable(i(1000).toTracking(), intCol("I", 10));
FunctionalColumn<Integer, String> plusOneColumn =
new FunctionalColumn<>("I", Integer.class, "S2", String.class, (Integer i) -> Integer.toString(i + 1));
final QueryTable functionalTable = (QueryTable) table.updateView(List.of(plusOneColumn));

TableTools.show(functionalTable);

final BitSet oneBit = new BitSet();
oneBit.set(0);
final BitSet twoBits = new BitSet();
twoBits.set(0);
twoBits.set(1);
final InitialSnapshot initialSnapshot = ConstructSnapshot.constructInitialSnapshotInPositionSpace(table, table,
oneBit, RowSetFactory.fromRange(0, 10));
final InitialSnapshot funcSnapshot = ConstructSnapshot.constructInitialSnapshotInPositionSpace(functionalTable,
functionalTable, twoBits, RowSetFactory.fromRange(0, 10));

final InitialSnapshotTable initialSnapshotTable =
InitialSnapshotTable.setupInitialSnapshotTable(table.getDefinition(), initialSnapshot);
TableTools.showWithRowSet(initialSnapshotTable);

final InitialSnapshotTable funcSnapshotTable =
InitialSnapshotTable.setupInitialSnapshotTable(functionalTable.getDefinition(), funcSnapshot);
TableTools.showWithRowSet(funcSnapshotTable);

assertTableEquals(TableTools.newTable(intCol("I", 10)), initialSnapshotTable);
assertTableEquals(TableTools.newTable(intCol("I", 10), stringCol("S2", "11")), funcSnapshotTable);

ExecutionContext.getContext().getUpdateGraph().<ControlledUpdateGraph>cast().startCycleForUnitTests();
addToTable(table, i(1000), intCol("I", 20));
final InitialSnapshot initialSnapshot2 = executor.submit(() -> ConstructSnapshot
.constructInitialSnapshotInPositionSpace(table, table, oneBit, RowSetFactory.fromRange(0, 10))).get();
final InitialSnapshot funcSnapshot2 =
executor.submit(() -> ConstructSnapshot.constructInitialSnapshotInPositionSpace(functionalTable,
functionalTable, twoBits, RowSetFactory.fromRange(0, 10))).get();
table.notifyListeners(i(), i(), i(1000));

while (ExecutionContext.getContext().getUpdateGraph().<ControlledUpdateGraph>cast()
.flushOneNotificationForUnitTests());

final InitialSnapshot initialSnapshot3 = executor.submit(() -> ConstructSnapshot
.constructInitialSnapshotInPositionSpace(table, table, oneBit, RowSetFactory.fromRange(0, 10))).get();
final InitialSnapshot funcSnapshot3 =
executor.submit(() -> ConstructSnapshot.constructInitialSnapshotInPositionSpace(functionalTable,
functionalTable, twoBits, RowSetFactory.fromRange(0, 10))).get();
ExecutionContext.getContext().getUpdateGraph().<ControlledUpdateGraph>cast().completeCycleForUnitTests();

final InitialSnapshotTable initialSnapshotTable2 =
InitialSnapshotTable.setupInitialSnapshotTable(table.getDefinition(), initialSnapshot2);
final InitialSnapshotTable initialSnapshotTable3 =
InitialSnapshotTable.setupInitialSnapshotTable(table.getDefinition(), initialSnapshot3);

assertTableEquals(TableTools.newTable(intCol("I", 10)), initialSnapshotTable2);
assertTableEquals(TableTools.newTable(intCol("I", 20)), initialSnapshotTable3);

final InitialSnapshotTable funcSnapshotTable2 =
InitialSnapshotTable.setupInitialSnapshotTable(functionalTable.getDefinition(), funcSnapshot2);
final InitialSnapshotTable funcSnapshotTable3 =
InitialSnapshotTable.setupInitialSnapshotTable(functionalTable.getDefinition(), funcSnapshot3);

assertTableEquals(TableTools.newTable(intCol("I", 10), stringCol("S2", "11")), funcSnapshotTable2);
assertTableEquals(TableTools.newTable(intCol("I", 20), stringCol("S2", "21")), funcSnapshotTable3);

executor.shutdownNow();
}
}
Loading